From e435748473d33a578625597faaadb5c0a1db08cb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Mar 2023 18:13:30 +0100 Subject: [PATCH 001/308] Update ubuntu_ami_for_ci.sh --- tests/ci/worker/ubuntu_ami_for_ci.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/worker/ubuntu_ami_for_ci.sh b/tests/ci/worker/ubuntu_ami_for_ci.sh index a43eac828c6..58797502419 100644 --- a/tests/ci/worker/ubuntu_ami_for_ci.sh +++ b/tests/ci/worker/ubuntu_ami_for_ci.sh @@ -75,6 +75,9 @@ cat < /etc/docker/daemon.json } EOT +# Increase the limit on number of virtual memory mappings to aviod 'Cannot mmap' error +sysctl -w vm.max_map_count=2097152 + systemctl restart docker # buildx builder is user-specific From 47cd46f590dfc3660e17b91a4ce8f5755734f2f9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Mar 2023 18:19:19 +0100 Subject: [PATCH 002/308] Update tests/ci/worker/ubuntu_ami_for_ci.sh Co-authored-by: Mikhail f. Shiryaev --- tests/ci/worker/ubuntu_ami_for_ci.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/ubuntu_ami_for_ci.sh b/tests/ci/worker/ubuntu_ami_for_ci.sh index 58797502419..f8d1645a54d 100644 --- a/tests/ci/worker/ubuntu_ami_for_ci.sh +++ b/tests/ci/worker/ubuntu_ami_for_ci.sh @@ -76,7 +76,7 @@ cat < /etc/docker/daemon.json EOT # Increase the limit on number of virtual memory mappings to aviod 'Cannot mmap' error -sysctl -w vm.max_map_count=2097152 +echo "vm.max_map_count = 2097152" > /etc/sysctl.d/01-increase-map-counts.conf systemctl restart docker From 067bfb884406d05722edec2d4fef82c2b65844ec Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 17:47:14 +0000 Subject: [PATCH 003/308] Add new function generateRandomStructure --- .../functions/other-functions.md | 70 ++++ .../sql-reference/table-functions/generate.md | 14 + src/DataTypes/NestedUtils.cpp | 15 +- src/DataTypes/NestedUtils.h | 3 + src/Functions/generateRandomStructure.cpp | 343 ++++++++++++++++++ src/Storages/StorageGenerateRandom.cpp | 4 +- .../TableFunctionGenerateRandom.cpp | 6 +- .../02584_compressor_codecs.reference | 14 + .../0_stateless/02584_compressor_codecs.sh | 34 ++ .../02586_generate_random_structure.reference | 4 + .../02586_generate_random_structure.sql | 10 + 11 files changed, 512 insertions(+), 5 deletions(-) create mode 100644 src/Functions/generateRandomStructure.cpp create mode 100644 tests/queries/0_stateless/02584_compressor_codecs.reference create mode 100755 tests/queries/0_stateless/02584_compressor_codecs.sh create mode 100644 tests/queries/0_stateless/02586_generate_random_structure.reference create mode 100644 tests/queries/0_stateless/02586_generate_random_structure.sql diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 7146484361e..2f46df07b0a 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2609,3 +2609,73 @@ Result: │ 286 │ └──────────────────────────┘ ``` + +## generateRandomStructure + +Generates random table structure in a format `column1_name column1_type, column2_name column2_type, ...`. + +**Syntax** + +``` sql +generateRandomStructure([number_of_columns, seed]) +``` + +**Arguments** + +- `number_of_columns` — The desired number of columns in the result table structure. If set to 0, the number of columns will be random from 1 to 128. Default value - 0. +- `seed` - Random seed to produce stable results. If seed is not specified, it is randomly generated. + +All arguments must be constant. + +**Returned value** + +- Randomly generated table structure. + +Type: [String](../../sql-reference/data-types/string.md). + +**Examples** + +Query: + +``` sql +SELECT generateRandomStructure() +``` + +Result: + +``` text +┌─generateRandomStructure()─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ c1 Decimal32(5), c2 Date, c3 Tuple(LowCardinality(String), Int128, UInt64, UInt16, UInt8, IPv6), c4 Array(UInt128), c5 UInt32, c6 IPv4, c7 Decimal256(64), c8 Decimal128(3), c9 UInt256, c10 UInt64, c11 DateTime │ +└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT generateRandomStructure(1) +``` + +Result: + +``` text +┌─generateRandomStructure(1)─┐ +│ c1 Map(UInt256, UInt16) │ +└────────────────────────────┘ +``` + +Query: + +``` sql +SELECT generateRandomStructure(0, 11) +``` + +Result: + +``` text +┌─generateRandomStructure(0, 11)──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ c1 Date32, c2 String, c3 IPv6, c4 DateTime, c5 UInt16, c6 Tuple(e1 UInt32, e2 Date, e3 Date, e4 IPv6, e5 Nested(e1 DateTime, e2 FixedString(110), e3 Int256, e4 Array(Decimal64(4)), e5 Decimal128(18), e6 Enum16('v0' = 0, 'v1' = 1, 'v2' = 2, 'v3' = 3, 'v4' = 4)), e6 DateTime64(4)), c7 DateTime, c8 DateTime64(6), c9 Bool │ +└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables. + diff --git a/docs/en/sql-reference/table-functions/generate.md b/docs/en/sql-reference/table-functions/generate.md index b53ccdd42b5..7e39786c256 100644 --- a/docs/en/sql-reference/table-functions/generate.md +++ b/docs/en/sql-reference/table-functions/generate.md @@ -53,5 +53,19 @@ SELECT * FROM random; └──────────────────────────────┴──────────────┴────────────────────────────────────────────────────────────────────┘ ``` +In combination with [generateRandomStructure](../../sql-reference/functions/other-functions.md#generateRandomStructure): + +```sql +SELECT * FROM generateRandom(generateRandomStructure(3, 24), 24) LIMIT 3; +``` + +```text +┌─────────────────────────c1─┬─────c2─┬───────────────────c3─┬───────────────────────────────────────c4─┐ +│ 2085-07-05 23:48:43.345759 │ -20656 │ 1632406185424686785 │ -210464718903845545171230673454802.15238 │ +│ 1971-07-17 16:32:36.390777 │ -27071 │ -1553021742787219162 │ 1095158319964381336405161704296125.08074 │ +│ 2024-02-19 13:14:32.902513 │ 24913 │ 7727442383333447640 │ 1090748832613398997057187200834127.07109 │ +└────────────────────────────┴────────┴──────────────────────┴──────────────────────────────────────────┘ +``` + ## Related content - Blog: [Generating random data in ClickHouse](https://clickhouse.com/blog/generating-random-test-distribution-data-for-clickhouse) diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index f029ac6ba27..9ee803c4235 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -71,7 +71,7 @@ std::string extractTableName(const std::string & nested_name) } -Block flatten(const Block & block) +static Block flattenImpl(const Block & block, bool flatten_named_tuple) { Block res; @@ -114,7 +114,7 @@ Block flatten(const Block & block) else res.insert(elem); } - else if (const DataTypeTuple * type_tuple = typeid_cast(elem.type.get())) + else if (const DataTypeTuple * type_tuple = typeid_cast(elem.type.get()); type_tuple && flatten_named_tuple) { if (type_tuple->haveExplicitNames()) { @@ -143,6 +143,17 @@ Block flatten(const Block & block) return res; } +Block flatten(const Block & block) +{ + return flattenImpl(block, true); +} + + +Block flattenArrayOfTuples(const Block & block) +{ + return flattenImpl(block, false); +} + namespace { diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index 90fdd683493..e009ceb18fe 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -23,6 +23,9 @@ namespace Nested /// 2) For an Array with named Tuple element column, a Array(Tuple(x ..., y ..., ...)), replace it with multiple Array Columns, a.x ..., a.y ..., ... Block flatten(const Block & block); + /// Same as flatten but only for Array with named Tuple element column. + Block flattenArrayOfTuples(const Block & block); + /// Collect Array columns in a form of `column_name.element_name` to single Array(Tuple(...)) column. NamesAndTypesList collect(const NamesAndTypesList & names_and_types); diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp new file mode 100644 index 00000000000..c27c8428cb0 --- /dev/null +++ b/src/Functions/generateRandomStructure.cpp @@ -0,0 +1,343 @@ +#include "config.h" + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; +} + +class FunctionGenerateRandomStructure : public IFunction +{ +private: + enum class SimpleTypes + { + Int8, + UInt8, + Bool, + Int16, + UInt16, + Int32, + UInt32, + Int64, + UInt64, + Int128, + UInt128, + Int256, + UInt256, + Float32, + Float64, + DateTime64, + Decimal32, + Decimal64, + Decimal128, + Decimal256, + Date, + Date32, + DateTime, + String, + FixedString, + Enum8, + Enum16, + IPv4, + IPv6, + }; + + enum class ComplexTypes + { + Nullable, + LowCardinality, + Array, + Tuple, + Map, + Nested, + }; + + enum class MapKeyTypes + { + Int8, + UInt8, + Bool, + Int16, + UInt16, + Int32, + UInt32, + Int64, + UInt64, + Int128, + UInt128, + Int256, + UInt256, + Date, + Date32, + DateTime, + String, + FixedString, + }; + + static constexpr size_t MAX_NUMBER_OF_COLUMNS = 128; + static constexpr size_t MAX_TUPLE_ELEMENTS = 16; + static constexpr size_t MAX_DATETIME64_PRECISION = 9; + static constexpr size_t MAX_DECIMAL32_PRECISION = 9; + static constexpr size_t MAX_DECIMAL64_PRECISION = 18; + static constexpr size_t MAX_DECIMAL128_PRECISION = 38; + static constexpr size_t MAX_DECIMAL256_PRECISION = 76; + static constexpr size_t MAX_DEPTH = 32; + +public: + static constexpr auto name = "generateRandomStructure"; + + static FunctionPtr create(ContextPtr /*context*/) + { + return std::make_shared(); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isVariadic() const override { return true; } + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() > 2) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 0, 1 or 2.", + getName(), arguments.size()); + + if (arguments.size() > 1 && !isUnsignedInteger(arguments[0])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the first argument of function {}, expected unsigned integer", + arguments[0]->getName(), + getName()); + } + + if (arguments.size() > 2 && !isUnsignedInteger(arguments[1])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the second argument of function {}, expected unsigned integer", + arguments[1]->getName(), + getName()); + } + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + size_t seed = randomSeed(); + size_t number_of_columns = 0; + + if (!arguments.empty()) + { + const auto & first_arg = arguments[0]; + + if (!isUnsignedInteger(first_arg.type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the first argument of function {}, expected unsigned integer", + first_arg.type->getName(), + getName()); + + number_of_columns = first_arg.column->getUInt(0); + if (number_of_columns > MAX_NUMBER_OF_COLUMNS) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Maximum allowed number of columns is {}, got {}", MAX_NUMBER_OF_COLUMNS, number_of_columns); + + if (arguments.size() == 2) + { + const auto & second_arg = arguments[1]; + + if (!isUnsignedInteger(second_arg.type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the second argument of function {}, expected unsigned integer", + second_arg.type->getName(), + getName()); + + seed = second_arg.column->getUInt(0); + } + } + + pcg64 rng(seed); + if (number_of_columns == 0) + number_of_columns = generateNumberOfColumns(rng); + + auto col_res = ColumnString::create(); + String generated_structure = ""; + for (size_t i = 0; i != number_of_columns; ++i) + { + if (i != 0) + generated_structure += ", "; + auto type = generateRandomType(rng); + generated_structure += "c" + std::to_string(i + 1) + " " + type; + } + col_res->insert(generated_structure); + return ColumnConst::create(std::move(col_res), input_rows_count); + } + +private: + + size_t generateNumberOfColumns(pcg64 & rng) const + { + return rng() % MAX_NUMBER_OF_COLUMNS + 1; + } + + String generateRandomType(pcg64 & rng, bool allow_complex_types = true, size_t depth = 0) const + { + constexpr size_t simple_types_size = magic_enum::enum_count(); + constexpr size_t complex_types_size = magic_enum::enum_count(); + size_t type_index; + if (allow_complex_types) + type_index = rng() % (simple_types_size + complex_types_size); + else + type_index = rng() % simple_types_size; + + if (type_index < simple_types_size) + { + auto type = magic_enum::enum_value(type_index); + switch (type) + { + case SimpleTypes::FixedString: + return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; + case SimpleTypes::DateTime64: + return "DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")"; + case SimpleTypes::Decimal32: + return "Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")"; + case SimpleTypes::Decimal64: + return "Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")"; + case SimpleTypes::Decimal128: + return "Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")"; + case SimpleTypes::Decimal256: + return "Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")"; + case SimpleTypes::Enum8: + return "Enum8(" + generateEnumValues(rng) + ")"; + case SimpleTypes::Enum16: + return "Enum16(" + generateEnumValues(rng) + ")"; + default: + return String(magic_enum::enum_name(type)); + } + } + + auto complex_type = magic_enum::enum_value(type_index - simple_types_size); + switch (complex_type) + { + case ComplexTypes::LowCardinality: + return "LowCardinality(" + generateLowCardinalityNestedType(rng) + ")"; + case ComplexTypes::Nullable: + return "Nullable(" + generateRandomType(rng, false, depth + 1) + ")"; + case ComplexTypes::Array: + return "Array(" + generateRandomType(rng, true, depth + 1) + ")"; + case ComplexTypes::Map: + return "Map(" + generateMapKeyType(rng) + ", " + generateRandomType(rng, true, depth + 1) + ")"; + case ComplexTypes::Tuple: + { + size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; + bool named_tuple = rng() % 2; + String tuple_type = "Tuple("; + for (size_t i = 0; i != elements; ++i) + { + if (i != 0) + tuple_type += ", "; + if (named_tuple) + tuple_type += "e" + std::to_string(i + 1) + " "; + tuple_type += generateRandomType(rng, true, depth + 1); + } + return tuple_type + ")"; + } + case ComplexTypes::Nested: + { + size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; + String nested_type = "Nested("; + for (size_t i = 0; i != elements; ++i) + { + if (i != 0) + nested_type += ", "; + nested_type += "e" + std::to_string(i + 1) + " " + generateRandomType(rng, true, depth + 1); + } + return nested_type + ")"; + } + } + } + + String generateMapKeyType(pcg64 & rng) const + { + constexpr size_t map_keys_types_size = magic_enum::enum_count(); + auto type = magic_enum::enum_value(rng() % map_keys_types_size); + if (type == MapKeyTypes::FixedString) + return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; + return String(magic_enum::enum_name(type)); + } + + String generateLowCardinalityNestedType(pcg64 & rng) const + { + /// Support only String and FixedString. + String nested_type; + if (rng() % 2) + nested_type = "String"; + else + nested_type = "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; + return rng() % 2 ? nested_type : "Nullable(" + nested_type + ")"; + } + + String generateEnumValues(pcg64 & rng) const + { + /// Don't generate big enums, because it will lead to really big strings + /// and slowness of this function, and it can lead to `Max query size exceeded` + /// while using this function with generateRandom. + ssize_t num_values = rng() % 16 + 1; + String result; + for (ssize_t i = 0; i != num_values; ++i) + { + if (i != 0) + result += ", "; + result += "'v" + std::to_string(i) + "' = " + std::to_string(i); + } + return result; + } +}; + + +REGISTER_FUNCTION(GenerateRandomStructure) +{ + factory.registerFunction( + { + R"( +Generates a random table structure. +This function takes an optional constant argument, the number of column in the result structure. +If argument is now specified, the number of columns is random. The maximum number of columns is 1024. +The function returns a value of type String. +)", + Documentation::Examples{ + {"random", "SELECT generateRandomStructure()"}, + {"with specified number of arguments", "SELECT generateRandomStructure(10)"}}, + Documentation::Categories{"Random"} + }, + FunctionFactory::CaseSensitive); +} + +} diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 601306bd1bf..f616313a595 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -422,7 +422,7 @@ class GenerateSource : public ISource { public: GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_, ContextPtr context_) - : ISource(Nested::flatten(prepareBlockToFill(block_header_))) + : ISource(Nested::flattenArrayOfTuples(prepareBlockToFill(block_header_))) , block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_) , block_to_fill(std::move(block_header_)), rng(random_seed_), context(context_) {} @@ -437,7 +437,7 @@ protected: for (const auto & elem : block_to_fill) columns.emplace_back(fillColumnWithRandomData(elem.type, block_size, max_array_length, max_string_length, rng, context)); - columns = Nested::flatten(block_to_fill.cloneWithColumns(columns)).getColumns(); + columns = Nested::flattenArrayOfTuples(block_to_fill.cloneWithColumns(columns)).getColumns(); return {std::move(columns), block_size}; } diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 5f1a13d8857..12cbda334a3 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include "registerTableFunctions.h" @@ -28,7 +29,7 @@ namespace ErrorCodes extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } -void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, ContextPtr /*context*/) +void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, ContextPtr context) { ASTs & args_func = ast_function->children; @@ -45,6 +46,9 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co "Table function '{}' requires at most four arguments: " " structure, [random_seed, max_string_length, max_array_length].", getName()); + /// Allow constant expression for structure argument, it can be generated using generateRandomStructure function. + args[0] = evaluateConstantExpressionAsLiteral(args[0], context); + // All the arguments must be literals. for (const auto & arg : args) { diff --git a/tests/queries/0_stateless/02584_compressor_codecs.reference b/tests/queries/0_stateless/02584_compressor_codecs.reference new file mode 100644 index 00000000000..23751ef6c1f --- /dev/null +++ b/tests/queries/0_stateless/02584_compressor_codecs.reference @@ -0,0 +1,14 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02584_compressor_codecs.sh b/tests/queries/0_stateless/02584_compressor_codecs.sh new file mode 100755 index 00000000000..930d101466b --- /dev/null +++ b/tests/queries/0_stateless/02584_compressor_codecs.sh @@ -0,0 +1,34 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "Hello, World!" > 02584_test_data + +$CLICKHOUSE_COMPRESSOR --codec 'Delta' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; +$CLICKHOUSE_COMPRESSOR --codec 'Delta(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'Delta([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'Delta(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; + +$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; +$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; + +$CLICKHOUSE_COMPRESSOR --codec 'Gorilla' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; +$CLICKHOUSE_COMPRESSOR --codec 'Gorilla(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'Gorilla([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'Gorilla(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; + +$CLICKHOUSE_COMPRESSOR --codec 'FPC' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; +$CLICKHOUSE_COMPRESSOR --codec 'FPC(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; +$CLICKHOUSE_COMPRESSOR --codec 'FPC(5, 1)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'FPC([1,2,3])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'FPC(5, 4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; + + +$CLICKHOUSE_COMPRESSOR --codec 'T64' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_SYNTAX_FOR_CODEC_TYPE"; + +rm 02584_test_data 02584_test_out + diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference new file mode 100644 index 00000000000..335c5807c35 --- /dev/null +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -0,0 +1,4 @@ +c1 Int256, c2 Bool, c3 Int16, c4 Map(Int64, Array(Bool)), c5 Enum16(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4, \'v5\' = 5, \'v6\' = 6, \'v7\' = 7, \'v8\' = 8, \'v9\' = 9, \'v10\' = 10) +String +Const(String) +2085-07-05 23:48:43.345759 10105 1535011673144902513 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql new file mode 100644 index 00000000000..c67196569af --- /dev/null +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -0,0 +1,10 @@ +select generateRandomStructure(5, 42); +select toTypeName(generateRandomStructure(5, 42)); +select toColumnTypeName(generateRandomStructure(5, 42)); +SELECT * FROM generateRandom(generateRandomStructure(3, 24), 24) LIMIT 1; + +select generateRandomStructure(5, 42, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select generateRandomStructure('5'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(materialize(5), 42); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, materialize(42)); -- {serverError ILLEGAL_COLUMN} From e5f6ced3d7d87a11e38f1a1499544fc7c2ad2e05 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 18:01:37 +0000 Subject: [PATCH 004/308] Fix style --- src/Functions/generateRandomStructure.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index c27c8428cb0..11b6f7877e1 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -18,7 +18,6 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_COLUMN; extern const int BAD_ARGUMENTS; } @@ -242,7 +241,7 @@ private: return String(magic_enum::enum_name(type)); } } - + auto complex_type = magic_enum::enum_value(type_index - simple_types_size); switch (complex_type) { From 66eb06d8399144b40ac94a4b0944bb15b37ce2c9 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 20:15:32 +0000 Subject: [PATCH 005/308] Better --- .../functions/other-functions.md | 23 +- src/Functions/generateRandomStructure.cpp | 340 ++++++++++++------ .../02586_generate_random_structure.reference | 7 +- .../02586_generate_random_structure.sql | 13 +- 4 files changed, 267 insertions(+), 116 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 2f46df07b0a..5f6b6e5687d 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2617,13 +2617,15 @@ Generates random table structure in a format `column1_name column1_type, column2 **Syntax** ``` sql -generateRandomStructure([number_of_columns, seed]) +generateRandomStructure([number_of_columns, seed, allow_big_numbers, allow_enums]) ``` **Arguments** -- `number_of_columns` — The desired number of columns in the result table structure. If set to 0, the number of columns will be random from 1 to 128. Default value - 0. -- `seed` - Random seed to produce stable results. If seed is not specified, it is randomly generated. +- `number_of_columns` — The desired number of columns in the result table structure. If set to 0 or `Null`, the number of columns will be random from 1 to 128. Default value: `Null`. +- `seed` - Random seed to produce stable results. If seed is not specified or set to `Null`, it is randomly generated. +- `allow_big_numbers` - Indicates if big number types (`Int128/UInt128/Int256/UInt256/Decimal128/Decinal256`) can be generated. Default value: true. +- `allow_enums` - Indicates if enum types can be generated. Default - true. All arguments must be constant. @@ -2666,7 +2668,7 @@ Result: Query: ``` sql -SELECT generateRandomStructure(0, 11) +SELECT generateRandomStructure(Null, 11) ``` Result: @@ -2677,5 +2679,18 @@ Result: └─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` +``` sql +SELECT generateRandomStructure(6, Null, false, false) +``` + +Result: + +``` text +┌─generateRandomStructure(6, NULL, false, false)───────────────────────────────────────────────────────┐ +│ c1 Float32, c2 Tuple(DateTime), c3 UInt8, c4 UInt16, c5 Int64, c6 Array(Map(FixedString(108), Date)) │ +└──────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + + This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables. diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index 11b6f7877e1..f95b4a279de 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -24,7 +24,7 @@ namespace ErrorCodes class FunctionGenerateRandomStructure : public IFunction { private: - enum class SimpleTypes + enum class Type { Int8, UInt8, @@ -35,30 +35,26 @@ private: UInt32, Int64, UInt64, - Int128, - UInt128, - Int256, - UInt256, Float32, Float64, DateTime64, Decimal32, Decimal64, - Decimal128, - Decimal256, Date, Date32, DateTime, String, FixedString, - Enum8, - Enum16, IPv4, IPv6, - }; - - enum class ComplexTypes - { + Int128, + UInt128, + Int256, + UInt256, + Decimal128, + Decimal256, + Enum8, + Enum16, Nullable, LowCardinality, Array, @@ -66,27 +62,74 @@ private: Map, Nested, }; - - enum class MapKeyTypes + + static constexpr std::array simple_types { - Int8, - UInt8, - Bool, - Int16, - UInt16, - Int32, - UInt32, - Int64, - UInt64, - Int128, - UInt128, - Int256, - UInt256, - Date, - Date32, - DateTime, - String, - FixedString, + Type::Int8, + Type::UInt8, + Type::Bool, + Type::Int16, + Type::UInt16, + Type::Int32, + Type::UInt32, + Type::Int64, + Type::UInt64, + Type::Float32, + Type::Float64, + Type::DateTime64, + Type::Decimal32, + Type::Decimal64, + Type::Date, + Type::Date32, + Type::DateTime, + Type::String, + Type::FixedString, + Type::IPv4, + Type::IPv6, + }; + + static constexpr std::array big_number_types + { + Type::Int128, + Type::UInt128, + Type::Int256, + Type::UInt256, + Type::Decimal128, + Type::Decimal256, + }; + + static constexpr std::array enum_types + { + Type::Enum8, + Type::Enum16, + }; + + static constexpr std::array complex_types + { + Type::Nullable, + Type::LowCardinality, + Type::Array, + Type::Tuple, + Type::Map, + Type::Nested, + }; + + static constexpr std::array map_key_types + { + Type::Int8, + Type::UInt8, + Type::Bool, + Type::Int16, + Type::UInt16, + Type::Int32, + Type::UInt32, + Type::Int64, + Type::UInt64, + Type::Date, + Type::Date32, + Type::DateTime, + Type::String, + Type::FixedString, }; static constexpr size_t MAX_NUMBER_OF_COLUMNS = 128; @@ -114,48 +157,66 @@ public: bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1, 2, 3}; } + bool useDefaultImplementationForConstants() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments.size() > 2) + if (arguments.size() > 4) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 0, 1 or 2.", + "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 4", getName(), arguments.size()); - if (arguments.size() > 1 && !isUnsignedInteger(arguments[0])) + if (!arguments.empty() && !isUnsignedInteger(arguments[0]) && !arguments[0]->onlyNull()) { throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the first argument of function {}, expected unsigned integer", + "Illegal type {} of the first argument of function {}, expected unsigned integer or Null", arguments[0]->getName(), getName()); } - if (arguments.size() > 2 && !isUnsignedInteger(arguments[1])) + if (arguments.size() > 1 && !isUnsignedInteger(arguments[1]) && !arguments[1]->onlyNull()) { throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second argument of function {}, expected unsigned integer", + "Illegal type {} of the second argument of function {}, expected unsigned integer or Null", arguments[1]->getName(), getName()); } + if (arguments.size() > 2 && !isUInt8(arguments[2])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the third argument of function {}, expected UInt8", + arguments[2]->getName(), + getName()); + } + + if (arguments.size() > 3 && !isUInt8(arguments[3])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the fourth argument of function {}, expected UInt8", + arguments[3]->getName(), + getName()); + } + return std::make_shared(); } - bool useDefaultImplementationForConstants() const override { return false; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { size_t seed = randomSeed(); size_t number_of_columns = 0; - if (!arguments.empty()) + if (!arguments.empty() && !arguments[0].column->onlyNull()) { const auto & first_arg = arguments[0]; - if (!isUnsignedInteger(first_arg.type)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, @@ -165,21 +226,55 @@ public: number_of_columns = first_arg.column->getUInt(0); if (number_of_columns > MAX_NUMBER_OF_COLUMNS) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Maximum allowed number of columns is {}, got {}", MAX_NUMBER_OF_COLUMNS, number_of_columns); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Maximum allowed number of columns is {}, got {}", + MAX_NUMBER_OF_COLUMNS, + number_of_columns); + } - if (arguments.size() == 2) - { - const auto & second_arg = arguments[1]; + if (arguments.size() > 1 && !arguments[1].column->onlyNull()) + { + const auto & second_arg = arguments[1]; - if (!isUnsignedInteger(second_arg.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second argument of function {}, expected unsigned integer", - second_arg.type->getName(), - getName()); + if (!isUnsignedInteger(second_arg.type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the second argument of function {}, expected unsigned integer", + second_arg.type->getName(), + getName()); - seed = second_arg.column->getUInt(0); - } + seed = second_arg.column->getUInt(0); + } + + bool allow_big_numbers = true; + if (arguments.size() > 2) + { + const auto & third_arg = arguments[2]; + + if (!isUInt8(third_arg.type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the second argument of function {}, expected UInt8", + third_arg.type->getName(), + getName()); + + allow_big_numbers = third_arg.column->getBool(0); + } + + bool allow_enums = true; + if (arguments.size() > 3) + { + const auto & fourth_arg = arguments[3]; + + if (!isUInt8(fourth_arg.type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the fourth argument of function {}, expected UInt8", + fourth_arg.type->getName(), + getName()); + + allow_enums = fourth_arg.column->getBool(0); } pcg64 rng(seed); @@ -192,7 +287,7 @@ public: { if (i != 0) generated_structure += ", "; - auto type = generateRandomType(rng); + auto type = generateRandomType(rng, allow_big_numbers, allow_enums); generated_structure += "c" + std::to_string(i + 1) + " " + type; } col_res->insert(generated_structure); @@ -205,55 +300,55 @@ private: { return rng() % MAX_NUMBER_OF_COLUMNS + 1; } - - String generateRandomType(pcg64 & rng, bool allow_complex_types = true, size_t depth = 0) const + + String generateRandomType(pcg64 & rng, bool allow_big_numbers, bool allow_enums) const { - constexpr size_t simple_types_size = magic_enum::enum_count(); - constexpr size_t complex_types_size = magic_enum::enum_count(); - size_t type_index; - if (allow_complex_types) - type_index = rng() % (simple_types_size + complex_types_size); - else - type_index = rng() % simple_types_size; - - if (type_index < simple_types_size) + if (allow_big_numbers) { - auto type = magic_enum::enum_value(type_index); - switch (type) - { - case SimpleTypes::FixedString: - return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - case SimpleTypes::DateTime64: - return "DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")"; - case SimpleTypes::Decimal32: - return "Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")"; - case SimpleTypes::Decimal64: - return "Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")"; - case SimpleTypes::Decimal128: - return "Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")"; - case SimpleTypes::Decimal256: - return "Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")"; - case SimpleTypes::Enum8: - return "Enum8(" + generateEnumValues(rng) + ")"; - case SimpleTypes::Enum16: - return "Enum16(" + generateEnumValues(rng) + ")"; - default: - return String(magic_enum::enum_name(type)); - } + if (allow_enums) + return generateRandomTypeImpl(rng); + return generateRandomTypeImpl(rng); } - auto complex_type = magic_enum::enum_value(type_index - simple_types_size); - switch (complex_type) + if (allow_enums) + return generateRandomTypeImpl(rng); + return generateRandomTypeImpl(rng); + } + + + template + String generateRandomTypeImpl(pcg64 & rng, size_t depth = 0) const + { + constexpr auto all_types = getAllTypes(); + auto type = all_types[rng() % all_types.size()]; + + switch (type) { - case ComplexTypes::LowCardinality: + case Type::FixedString: + return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; + case Type::DateTime64: + return "DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")"; + case Type::Decimal32: + return "Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")"; + case Type::Decimal64: + return "Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")"; + case Type::Decimal128: + return "Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")"; + case Type::Decimal256: + return "Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")"; + case Type::Enum8: + return "Enum8(" + generateEnumValues(rng) + ")"; + case Type::Enum16: + return "Enum16(" + generateEnumValues(rng) + ")"; + case Type::LowCardinality: return "LowCardinality(" + generateLowCardinalityNestedType(rng) + ")"; - case ComplexTypes::Nullable: - return "Nullable(" + generateRandomType(rng, false, depth + 1) + ")"; - case ComplexTypes::Array: - return "Array(" + generateRandomType(rng, true, depth + 1) + ")"; - case ComplexTypes::Map: - return "Map(" + generateMapKeyType(rng) + ", " + generateRandomType(rng, true, depth + 1) + ")"; - case ComplexTypes::Tuple: + case Type::Nullable: + return "Nullable(" + generateRandomTypeImpl(rng, depth + 1) + ")"; + case Type::Array: + return "Array(" + generateRandomTypeImpl(rng, depth + 1) + ")"; + case Type::Map: + return "Map(" + generateMapKeyType(rng) + ", " + generateRandomTypeImpl(rng, depth + 1) + ")"; + case Type::Tuple: { size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; bool named_tuple = rng() % 2; @@ -264,11 +359,11 @@ private: tuple_type += ", "; if (named_tuple) tuple_type += "e" + std::to_string(i + 1) + " "; - tuple_type += generateRandomType(rng, true, depth + 1); + tuple_type += generateRandomTypeImpl(rng, depth + 1); } return tuple_type + ")"; } - case ComplexTypes::Nested: + case Type::Nested: { size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; String nested_type = "Nested("; @@ -276,20 +371,21 @@ private: { if (i != 0) nested_type += ", "; - nested_type += "e" + std::to_string(i + 1) + " " + generateRandomType(rng, true, depth + 1); + nested_type += "e" + std::to_string(i + 1) + " " + generateRandomTypeImpl(rng, depth + 1); } return nested_type + ")"; } + default: + return String(magic_enum::enum_name(type)); } } String generateMapKeyType(pcg64 & rng) const { - constexpr size_t map_keys_types_size = magic_enum::enum_count(); - auto type = magic_enum::enum_value(rng() % map_keys_types_size); - if (type == MapKeyTypes::FixedString) + auto type = map_key_types[rng() % map_key_types.size()]; + if (type == Type::FixedString) return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - return String(magic_enum::enum_name(type)); + return String(magic_enum::enum_name(type)); } String generateLowCardinalityNestedType(pcg64 & rng) const @@ -318,6 +414,36 @@ private: } return result; } + + template + static constexpr auto getAllTypes() + { + constexpr size_t result_size = simple_types.size() + big_number_types.size() * allow_big_numbers + enum_types.size() * allow_enums + complex_types.size() * allow_complex_types; + std::array result; + size_t index = 0; + for (size_t i = 0; i != simple_types.size(); ++i, ++index) + result[index] = simple_types[i]; + + if constexpr (allow_big_numbers) + { + for (size_t i = 0; i != big_number_types.size(); ++i, ++index) + result[index] = big_number_types[i]; + } + + if constexpr (allow_enums) + { + for (size_t i = 0; i != enum_types.size(); ++i, ++index) + result[index] = enum_types[i]; + } + + if constexpr (allow_complex_types) + { + for (size_t i = 0; i != complex_types.size(); ++i, ++index) + result[index] = complex_types[i]; + } + + return result; + } }; diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index 335c5807c35..181b7f0ec6f 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -1,4 +1,7 @@ -c1 Int256, c2 Bool, c3 Int16, c4 Map(Int64, Array(Bool)), c5 Enum16(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4, \'v5\' = 5, \'v6\' = 6, \'v7\' = 7, \'v8\' = 8, \'v9\' = 9, \'v10\' = 10) +c1 DateTime64(1), c2 Int16, c3 Map(Int64, Array(Bool)), c4 Decimal256(30), c5 Int128 +c1 Date, c2 Float64, c3 DateTime, c4 Map(Int64, DateTime64(4)), c5 Nested(e1 LowCardinality(String), e2 UInt32, e3 Enum16(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4, \'v5\' = 5, \'v6\' = 6, \'v7\' = 7), e4 Float32, e5 Date32, e6 Int64, e7 Decimal64(0), e8 UInt16, e9 Date32, e10 Int64, e11 DateTime64(0)) +c1 Nested(e1 Int64, e2 Int16, e3 Map(Int16, LowCardinality(Nullable(String))), e4 UInt8, e5 Nested(e1 Array(Nullable(Decimal64(12))), e2 DateTime64(1), e3 UInt64, e4 FixedString(61), e5 Decimal64(13), e6 UInt8), e6 Int8), c2 DateTime64(5), c3 IPv4, c4 String, c5 String +c1 DateTime64(1), c2 IPv4, c3 Nullable(Decimal128(37)), c4 UInt128, c5 Date String Const(String) -2085-07-05 23:48:43.345759 10105 1535011673144902513 +2106-02-02 121 17265 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql index c67196569af..b524f6a5ff1 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.sql +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -1,10 +1,17 @@ select generateRandomStructure(5, 42); +select generateRandomStructure(5, 42, false); +select generateRandomStructure(5, 42, false, false); +select generateRandomStructure(5, 42, true, false); select toTypeName(generateRandomStructure(5, 42)); select toColumnTypeName(generateRandomStructure(5, 42)); SELECT * FROM generateRandom(generateRandomStructure(3, 24), 24) LIMIT 1; -select generateRandomStructure(5, 42, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select generateRandomStructure(5, 42, false, false, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} select generateRandomStructure('5'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(materialize(5), 42); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, materialize(42)); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(5, 42, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(materialize(5), 42, false, false); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, materialize(42), false, false); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, materialize(false), false); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} From 0d430de54fef53e61f821d8b4bae684c816d4f2a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 20:18:17 +0000 Subject: [PATCH 006/308] Better --- src/Functions/generateRandomStructure.cpp | 31 ----------------------- 1 file changed, 31 deletions(-) diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index f95b4a279de..3b42fd99fb4 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -217,13 +217,6 @@ public: if (!arguments.empty() && !arguments[0].column->onlyNull()) { const auto & first_arg = arguments[0]; - if (!isUnsignedInteger(first_arg.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the first argument of function {}, expected unsigned integer", - first_arg.type->getName(), - getName()); - number_of_columns = first_arg.column->getUInt(0); if (number_of_columns > MAX_NUMBER_OF_COLUMNS) throw Exception( @@ -236,14 +229,6 @@ public: if (arguments.size() > 1 && !arguments[1].column->onlyNull()) { const auto & second_arg = arguments[1]; - - if (!isUnsignedInteger(second_arg.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second argument of function {}, expected unsigned integer", - second_arg.type->getName(), - getName()); - seed = second_arg.column->getUInt(0); } @@ -251,14 +236,6 @@ public: if (arguments.size() > 2) { const auto & third_arg = arguments[2]; - - if (!isUInt8(third_arg.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second argument of function {}, expected UInt8", - third_arg.type->getName(), - getName()); - allow_big_numbers = third_arg.column->getBool(0); } @@ -266,14 +243,6 @@ public: if (arguments.size() > 3) { const auto & fourth_arg = arguments[3]; - - if (!isUInt8(fourth_arg.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the fourth argument of function {}, expected UInt8", - fourth_arg.type->getName(), - getName()); - allow_enums = fourth_arg.column->getBool(0); } From 746d12e7ccf4780d6887aa3f961605b7f313b77b Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 20:19:28 +0000 Subject: [PATCH 007/308] Remove wrong tests --- .../02584_compressor_codecs.reference | 14 -------- .../0_stateless/02584_compressor_codecs.sh | 34 ------------------- 2 files changed, 48 deletions(-) delete mode 100644 tests/queries/0_stateless/02584_compressor_codecs.reference delete mode 100755 tests/queries/0_stateless/02584_compressor_codecs.sh diff --git a/tests/queries/0_stateless/02584_compressor_codecs.reference b/tests/queries/0_stateless/02584_compressor_codecs.reference deleted file mode 100644 index 23751ef6c1f..00000000000 --- a/tests/queries/0_stateless/02584_compressor_codecs.reference +++ /dev/null @@ -1,14 +0,0 @@ -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/02584_compressor_codecs.sh b/tests/queries/0_stateless/02584_compressor_codecs.sh deleted file mode 100755 index 930d101466b..00000000000 --- a/tests/queries/0_stateless/02584_compressor_codecs.sh +++ /dev/null @@ -1,34 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -echo "Hello, World!" > 02584_test_data - -$CLICKHOUSE_COMPRESSOR --codec 'Delta' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; -$CLICKHOUSE_COMPRESSOR --codec 'Delta(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'Delta([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'Delta(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; - -$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; -$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; - -$CLICKHOUSE_COMPRESSOR --codec 'Gorilla' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; -$CLICKHOUSE_COMPRESSOR --codec 'Gorilla(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'Gorilla([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'Gorilla(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; - -$CLICKHOUSE_COMPRESSOR --codec 'FPC' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; -$CLICKHOUSE_COMPRESSOR --codec 'FPC(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; -$CLICKHOUSE_COMPRESSOR --codec 'FPC(5, 1)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'FPC([1,2,3])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'FPC(5, 4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; - - -$CLICKHOUSE_COMPRESSOR --codec 'T64' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_SYNTAX_FOR_CODEC_TYPE"; - -rm 02584_test_data 02584_test_out - From 48d701f0e70a10807d64eb041dcdc17562cf6731 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 20:27:14 +0000 Subject: [PATCH 008/308] Better docs --- src/Functions/generateRandomStructure.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index 3b42fd99fb4..0fb9ef0f2f3 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -422,13 +422,19 @@ REGISTER_FUNCTION(GenerateRandomStructure) { R"( Generates a random table structure. -This function takes an optional constant argument, the number of column in the result structure. -If argument is now specified, the number of columns is random. The maximum number of columns is 1024. +This function takes 4 optional constant arguments: the number of column in the result structure (random by default), +random seed (random by default), flag that indicates if big number types can be used (true by default), +flag that indicates if enum types can be used (true by default). +The maximum number of columns is 128. The function returns a value of type String. )", Documentation::Examples{ {"random", "SELECT generateRandomStructure()"}, - {"with specified number of arguments", "SELECT generateRandomStructure(10)"}}, + {"with specified number of arguments", "SELECT generateRandomStructure(10)"}, + {"with specified seed", "SELECT generateRandomStructure(10, 42)"}, + {"without big number types", "SELECT generateRandomStructure(10, NULL, false)"}, + {"without enum types", "SELECT generateRandomStructure(10, NULL, false, false)"}, + }, Documentation::Categories{"Random"} }, FunctionFactory::CaseSensitive); From f9d9b1ee2379b608a3f13bb6e0087e2e4b292ba4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 10 Mar 2023 16:16:28 +0000 Subject: [PATCH 009/308] Add more options --- .../functions/other-functions.md | 19 +- src/Functions/generateRandomStructure.cpp | 262 +++++++++++------- .../02586_generate_random_structure.reference | 14 +- .../02586_generate_random_structure.sql | 16 +- 4 files changed, 201 insertions(+), 110 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 5f6b6e5687d..b4664f75e67 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2617,7 +2617,7 @@ Generates random table structure in a format `column1_name column1_type, column2 **Syntax** ``` sql -generateRandomStructure([number_of_columns, seed, allow_big_numbers, allow_enums]) +generateRandomStructure([number_of_columns, seed, allow_big_numbers, allow_enums, allow_decimals, allow_ip, allow_only_string_map_keys]) ``` **Arguments** @@ -2625,7 +2625,10 @@ generateRandomStructure([number_of_columns, seed, allow_big_numbers, allow_enums - `number_of_columns` — The desired number of columns in the result table structure. If set to 0 or `Null`, the number of columns will be random from 1 to 128. Default value: `Null`. - `seed` - Random seed to produce stable results. If seed is not specified or set to `Null`, it is randomly generated. - `allow_big_numbers` - Indicates if big number types (`Int128/UInt128/Int256/UInt256/Decimal128/Decinal256`) can be generated. Default value: true. -- `allow_enums` - Indicates if enum types can be generated. Default - true. +- `allow_enums` - Indicates if enum types (`Enum8/Enum16`) can be generated. Default - true. +- `allow_decimals` - Indicates if decimal types (`Decimal(P, S)`) can be generated. Default - true. +- `allow_ip` - Indicates if ip types (`IPv4/IPv6`) can be generated. Default - true. +- `allow_only_string_map_keys` - Indicates if Map key type can be only `String/FixedString`. Default - false. All arguments must be constant. @@ -2691,6 +2694,18 @@ Result: └──────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` +``` sql +SELECT generateRandomStructure(6, Null, false, false, false, false, true) +``` + +Result: + +``` text +┌─generateRandomStructure(6, NULL, false, false, false, false, true)─────────────────────────────────────────────────┐ +│ c1 String, c2 UInt32, c3 Int32, c4 Int32, c5 Tuple(LowCardinality(Nullable(FixedString(101))), UInt8), c6 DateTime │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables. diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index 0fb9ef0f2f3..e6766e731b2 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -62,8 +62,8 @@ private: Map, Nested, }; - - static constexpr std::array simple_types + + static constexpr std::array simple_types { Type::Int8, Type::UInt8, @@ -76,24 +76,30 @@ private: Type::UInt64, Type::Float32, Type::Float64, - Type::DateTime64, - Type::Decimal32, - Type::Decimal64, Type::Date, Type::Date32, Type::DateTime, Type::String, Type::FixedString, - Type::IPv4, - Type::IPv6, }; - static constexpr std::array big_number_types + static constexpr std::array big_integer_types { Type::Int128, Type::UInt128, Type::Int256, Type::UInt256, + }; + + static constexpr std::array decimal_types + { + Type::DateTime64, + Type::Decimal32, + Type::Decimal64, + }; + + static constexpr std::array big_decimal_types + { Type::Decimal128, Type::Decimal256, }; @@ -104,6 +110,12 @@ private: Type::Enum16, }; + static constexpr std::array ip_types + { + Type::IPv4, + Type::IPv6, + }; + static constexpr std::array complex_types { Type::Nullable, @@ -132,6 +144,12 @@ private: Type::FixedString, }; + static constexpr std::array map_key_string_types + { + Type::String, + Type::FixedString + }; + static constexpr size_t MAX_NUMBER_OF_COLUMNS = 128; static constexpr size_t MAX_TUPLE_ELEMENTS = 16; static constexpr size_t MAX_DATETIME64_PRECISION = 9; @@ -157,53 +175,48 @@ public: bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1, 2, 3}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1, 2, 3, 4, 5, 6}; } bool useDefaultImplementationForConstants() const override { return false; } bool useDefaultImplementationForNulls() const override { return false; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments.size() > 4) + if (arguments.size() > 7) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 4", + "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 7", getName(), arguments.size()); - if (!arguments.empty() && !isUnsignedInteger(arguments[0]) && !arguments[0]->onlyNull()) + for (size_t i = 0; i != 2; ++i) { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the first argument of function {}, expected unsigned integer or Null", - arguments[0]->getName(), - getName()); + if (arguments.size() == i) + break; + + if (!isUnsignedInteger(arguments[i]) && !arguments[i]->onlyNull()) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the {} argument of function {}, expected unsigned integer or Null", + i + 1, + arguments[i]->getName(), + getName()); + } } - if (arguments.size() > 1 && !isUnsignedInteger(arguments[1]) && !arguments[1]->onlyNull()) + for (size_t i = 2; i != 7; ++i) { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second argument of function {}, expected unsigned integer or Null", - arguments[1]->getName(), - getName()); - } + if (arguments.size() <= i) + break; - if (arguments.size() > 2 && !isUInt8(arguments[2])) - { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the third argument of function {}, expected UInt8", - arguments[2]->getName(), - getName()); - } - - if (arguments.size() > 3 && !isUInt8(arguments[3])) - { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the fourth argument of function {}, expected UInt8", - arguments[3]->getName(), - getName()); + if (!isUInt8(arguments[i])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the {} argument of function {}, expected UInt8", + i + 1, + arguments[i]->getName(), + getName()); + } } return std::make_shared(); @@ -216,8 +229,7 @@ public: if (!arguments.empty() && !arguments[0].column->onlyNull()) { - const auto & first_arg = arguments[0]; - number_of_columns = first_arg.column->getUInt(0); + number_of_columns = arguments[0].column->getUInt(0); if (number_of_columns > MAX_NUMBER_OF_COLUMNS) throw Exception( ErrorCodes::BAD_ARGUMENTS, @@ -227,36 +239,39 @@ public: } if (arguments.size() > 1 && !arguments[1].column->onlyNull()) - { - const auto & second_arg = arguments[1]; - seed = second_arg.column->getUInt(0); - } + seed = arguments[1].column->getUInt(0); bool allow_big_numbers = true; if (arguments.size() > 2) - { - const auto & third_arg = arguments[2]; - allow_big_numbers = third_arg.column->getBool(0); - } + allow_big_numbers = arguments[2].column->getBool(0); bool allow_enums = true; if (arguments.size() > 3) - { - const auto & fourth_arg = arguments[3]; - allow_enums = fourth_arg.column->getBool(0); - } + allow_enums = arguments[3].column->getBool(0); + + bool allow_decimals = true; + if (arguments.size() > 4) + allow_decimals = arguments[4].column->getBool(0); + + bool allow_ip = true; + if (arguments.size() > 5) + allow_ip = arguments[5].column->getBool(0); + + bool only_string_map_key = false; + if (arguments.size() > 6) + only_string_map_key = arguments[6].column->getBool(0); pcg64 rng(seed); if (number_of_columns == 0) number_of_columns = generateNumberOfColumns(rng); auto col_res = ColumnString::create(); - String generated_structure = ""; + String generated_structure; for (size_t i = 0; i != number_of_columns; ++i) { if (i != 0) generated_structure += ", "; - auto type = generateRandomType(rng, allow_big_numbers, allow_enums); + auto type = generateRandomType(rng, allow_big_numbers, allow_enums, allow_decimals, allow_ip, only_string_map_key); generated_structure += "c" + std::to_string(i + 1) + " " + type; } col_res->insert(generated_structure); @@ -269,26 +284,37 @@ private: { return rng() % MAX_NUMBER_OF_COLUMNS + 1; } - - String generateRandomType(pcg64 & rng, bool allow_big_numbers, bool allow_enums) const + + /// Helper struct to call generateRandomTypeImpl with lots of bool template arguments without writing big if/else over all bool variables. + template + struct Dispatcher { - if (allow_big_numbers) + static auto call(const FunctionGenerateRandomStructure * f, pcg64 & rng) { - if (allow_enums) - return generateRandomTypeImpl(rng); - return generateRandomTypeImpl(rng); + return f->generateRandomTypeImpl(rng); } - if (allow_enums) - return generateRandomTypeImpl(rng); - return generateRandomTypeImpl(rng); - } - + template + static auto call(const FunctionGenerateRandomStructure * f, pcg64 & rng, bool b, Args1... ar1) + { + if (b) + return Dispatcher::call(f, rng, ar1...); + else + return Dispatcher::call(f, rng, ar1...); + } - template + friend FunctionGenerateRandomStructure; + }; + + String generateRandomType(pcg64 & rng, bool allow_big_numbers, bool allow_enums, bool allow_decimals, bool allow_ip, bool allow_only_string_map_keys) const + { + return Dispatcher<>::call(this, rng, allow_big_numbers, allow_enums, allow_decimals, allow_ip, allow_only_string_map_keys, true); + } + + template String generateRandomTypeImpl(pcg64 & rng, size_t depth = 0) const { - constexpr auto all_types = getAllTypes(); + constexpr auto all_types = getAllTypes(); auto type = all_types[rng() % all_types.size()]; switch (type) @@ -312,11 +338,21 @@ private: case Type::LowCardinality: return "LowCardinality(" + generateLowCardinalityNestedType(rng) + ")"; case Type::Nullable: - return "Nullable(" + generateRandomTypeImpl(rng, depth + 1) + ")"; + { + auto nested_type = generateRandomTypeImpl(rng, depth + 1); + return "Nullable(" + nested_type + ")"; + } case Type::Array: - return "Array(" + generateRandomTypeImpl(rng, depth + 1) + ")"; + { + auto nested_type = generateRandomTypeImpl(rng, depth + 1); + return "Array(" + nested_type + ")"; + } case Type::Map: - return "Map(" + generateMapKeyType(rng) + ", " + generateRandomTypeImpl(rng, depth + 1) + ")"; + { + auto key_type = generateMapKeyType(rng); + auto value_type = generateRandomTypeImpl(rng, depth + 1); + return "Map(" + key_type + ", " + value_type + ")"; + } case Type::Tuple: { size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; @@ -328,7 +364,7 @@ private: tuple_type += ", "; if (named_tuple) tuple_type += "e" + std::to_string(i + 1) + " "; - tuple_type += generateRandomTypeImpl(rng, depth + 1); + tuple_type += generateRandomTypeImpl(rng, depth + 1); } return tuple_type + ")"; } @@ -340,7 +376,8 @@ private: { if (i != 0) nested_type += ", "; - nested_type += "e" + std::to_string(i + 1) + " " + generateRandomTypeImpl(rng, depth + 1); + auto element_type = generateRandomTypeImpl(rng, depth + 1); + nested_type += "e" + std::to_string(i + 1) + " " + element_type; } return nested_type + ")"; } @@ -349,9 +386,15 @@ private: } } + template String generateMapKeyType(pcg64 & rng) const { - auto type = map_key_types[rng() % map_key_types.size()]; + Type type; + if constexpr (allow_only_string_map_keys) + type = map_key_string_types[rng() % map_key_string_types.size()]; + else + type = map_key_types[rng() % map_key_types.size()]; + if (type == Type::FixedString) return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; return String(magic_enum::enum_name(type)); @@ -384,33 +427,42 @@ private: return result; } - template + template static constexpr auto getAllTypes() { - constexpr size_t result_size = simple_types.size() + big_number_types.size() * allow_big_numbers + enum_types.size() * allow_enums + complex_types.size() * allow_complex_types; + constexpr size_t big_integer_types_size = big_integer_types.size() * allow_big_numbers; + constexpr size_t enum_types_size = enum_types.size() * allow_enums; + constexpr size_t decimal_types_size = decimal_types.size() * allow_decimals; + constexpr size_t big_decimal_types_size = big_decimal_types.size() * allow_big_numbers * allow_decimals; + constexpr size_t ip_types_size = ip_types.size() * allow_ip; + constexpr size_t complex_types_size = complex_types.size() * allow_complex_types; + + constexpr size_t result_size = simple_types.size() + big_integer_types_size + enum_types_size + decimal_types_size + + big_decimal_types_size + ip_types_size + complex_types_size; std::array result; size_t index = 0; + for (size_t i = 0; i != simple_types.size(); ++i, ++index) result[index] = simple_types[i]; - - if constexpr (allow_big_numbers) - { - for (size_t i = 0; i != big_number_types.size(); ++i, ++index) - result[index] = big_number_types[i]; - } - - if constexpr (allow_enums) - { - for (size_t i = 0; i != enum_types.size(); ++i, ++index) - result[index] = enum_types[i]; - } - - if constexpr (allow_complex_types) - { - for (size_t i = 0; i != complex_types.size(); ++i, ++index) - result[index] = complex_types[i]; - } - + + for (size_t i = 0; i != big_integer_types_size; ++i, ++index) + result[index] = big_integer_types[i]; + + for (size_t i = 0; i != enum_types_size; ++i, ++index) + result[index] = enum_types[i]; + + for (size_t i = 0; i != decimal_types_size; ++i, ++index) + result[index] = decimal_types[i]; + + for (size_t i = 0; i != big_decimal_types_size; ++i, ++index) + result[index] = big_decimal_types[i]; + + for (size_t i = 0; i != ip_types_size; ++i, ++index) + result[index] = ip_types[i]; + + for (size_t i = 0; i != complex_types_size; ++i, ++index) + result[index] = complex_types[i]; + return result; } }; @@ -422,9 +474,14 @@ REGISTER_FUNCTION(GenerateRandomStructure) { R"( Generates a random table structure. -This function takes 4 optional constant arguments: the number of column in the result structure (random by default), -random seed (random by default), flag that indicates if big number types can be used (true by default), -flag that indicates if enum types can be used (true by default). +This function takes 4 optional constant arguments: +1) the number of column in the result structure (random by default) +2) random seed (random by default) +3) flag that indicates if big number types can be used (true by default) +4) flag that indicates if enum types can be used (true by default) +5) flag that indicates if decimal types can be used (true by default) +6) flag that indicates if ip types (IPv4, IPv6) can be used (true by default) +7) flag that indicates if map keys should be only String or FixedString (false by default) The maximum number of columns is 128. The function returns a value of type String. )", @@ -433,7 +490,10 @@ The function returns a value of type String. {"with specified number of arguments", "SELECT generateRandomStructure(10)"}, {"with specified seed", "SELECT generateRandomStructure(10, 42)"}, {"without big number types", "SELECT generateRandomStructure(10, NULL, false)"}, - {"without enum types", "SELECT generateRandomStructure(10, NULL, false, false)"}, + {"without enum types", "SELECT generateRandomStructure(10, NULL, true, false)"}, + {"without decimal types", "SELECT generateRandomStructure(10, NULL, true, true, false)"}, + {"without ip types", "SELECT generateRandomStructure(10, NULL, true, true, true, false)"}, + {"with only string mak key types", "SELECT generateRandomStructure(10, NULL, true, true, true, true, true)"}, }, Documentation::Categories{"Random"} }, diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index 181b7f0ec6f..76d89828071 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -1,7 +1,11 @@ -c1 DateTime64(1), c2 Int16, c3 Map(Int64, Array(Bool)), c4 Decimal256(30), c5 Int128 -c1 Date, c2 Float64, c3 DateTime, c4 Map(Int64, DateTime64(4)), c5 Nested(e1 LowCardinality(String), e2 UInt32, e3 Enum16(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4, \'v5\' = 5, \'v6\' = 6, \'v7\' = 7), e4 Float32, e5 Date32, e6 Int64, e7 Decimal64(0), e8 UInt16, e9 Date32, e10 Int64, e11 DateTime64(0)) -c1 Nested(e1 Int64, e2 Int16, e3 Map(Int16, LowCardinality(Nullable(String))), e4 UInt8, e5 Nested(e1 Array(Nullable(Decimal64(12))), e2 DateTime64(1), e3 UInt64, e4 FixedString(61), e5 Decimal64(13), e6 UInt8), e6 Int8), c2 DateTime64(5), c3 IPv4, c4 String, c5 String -c1 DateTime64(1), c2 IPv4, c3 Nullable(Decimal128(37)), c4 UInt128, c5 Date +c1 Date, c2 Bool, c3 Int16, c4 Map(Int64, Array(Bool)), c5 Decimal256(30) +c1 String, c2 Float64, c3 Enum8(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4), c4 UInt64, c5 Date +c1 Nested(e1 Int64, e2 Int16, e3 Map(Int16, LowCardinality(Nullable(String))), e4 UInt8, e5 Nested(e1 Array(Nullable(DateTime)), e2 Nullable(Bool), e3 UInt8, e4 UInt64, e5 Decimal64(6), e6 DateTime), e6 LowCardinality(Nullable(String))), c2 Date, c3 Int32, c4 IPv4, c5 Decimal32(8) +c1 Date, c2 UInt16, c3 UInt256, c4 Nullable(IPv4), c5 Nullable(Decimal64(17)) +c1 Array(Int64), c2 Map(String, LowCardinality(String)), c3 Date, c4 Map(Int64, UInt128), c5 UInt8 +c1 Date, c2 UInt16, c3 UInt256, c4 Nullable(Decimal128(37)), c5 DateTime64(8) +c1 Date, c2 Bool, c3 Int16, c4 Map(FixedString(120), Bool), c5 Decimal256(30) +c1 String, c2 Float64, c3 Enum8(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4), c4 UInt64, c5 Date String Const(String) -2106-02-02 121 17265 +1977-07-28 true 5389 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql index b524f6a5ff1..061fbc24219 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.sql +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -2,16 +2,28 @@ select generateRandomStructure(5, 42); select generateRandomStructure(5, 42, false); select generateRandomStructure(5, 42, false, false); select generateRandomStructure(5, 42, true, false); +select generateRandomStructure(5, 42, true, true, false); +select generateRandomStructure(5, 42, true, true, true, false); +select generateRandomStructure(5, 42, true, true, true, true, true); +select generateRandomStructure(5, 42, false, true, true); select toTypeName(generateRandomStructure(5, 42)); select toColumnTypeName(generateRandomStructure(5, 42)); -SELECT * FROM generateRandom(generateRandomStructure(3, 24), 24) LIMIT 1; +SELECT * FROM generateRandom(generateRandomStructure(3, 42), 42) LIMIT 1; -select generateRandomStructure(5, 42, false, false, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select generateRandomStructure(5, 42, false, false, false, false, true, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} select generateRandomStructure('5'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} select generateRandomStructure(5, 42, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} select generateRandomStructure(5, 42, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(5, 42, false, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(5, 42, false, false, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(5, 42, false, false, false, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} select generateRandomStructure(materialize(5), 42, false, false); -- {serverError ILLEGAL_COLUMN} select generateRandomStructure(5, materialize(42), false, false); -- {serverError ILLEGAL_COLUMN} select generateRandomStructure(5, 42, materialize(false), false); -- {serverError ILLEGAL_COLUMN} select generateRandomStructure(5, 42, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, false, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, false, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, false, false, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} + From 74bb7625a33d959e6bf93f918c8998ff3425faf6 Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Sun, 12 Mar 2023 18:09:49 +0800 Subject: [PATCH 010/308] allow empty column names in CSVWithNames/TSVWithNames --- src/Interpreters/TreeRewriter.cpp | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 4c134e175dc..66a08257a46 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -919,6 +919,26 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select RequiredSourceColumnsVisitor::Data columns_context; columns_context.visit_index_hint = visit_index_hint; + + if (auto * t = query->as()) + { + LOG_INFO(&Poco::Logger::get("TreeRewriter"), "lmx_00 enter"); + NameSet select_columns; + std::vector empty_name; + auto & select_query = *t; + for (size_t i = 0; i < select_query.select()->children.size(); i++) { + auto node = select_query.select()->children[i]; + if (auto* identifier = node->as()) { + LOG_INFO(&Poco::Logger::get("TreeRewriter"), "lmx_1 {}", identifier->name()); + if (identifier->name().empty()) { + empty_name.push_back(i); + select_query.select()->children.erase(select_query.select()->children.begin()+i); + } else { + select_columns.insert(identifier->name()); + } + } + } + } RequiredSourceColumnsVisitor(columns_context).visit(query); NameSet source_column_names; From 0ef6aea8280c3f8e7606dcb8e6d4b2186adabb68 Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Sun, 12 Mar 2023 18:22:10 +0800 Subject: [PATCH 011/308] fix --- src/Interpreters/TreeRewriter.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 66a08257a46..1ac8b62c11c 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -922,19 +922,12 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (auto * t = query->as()) { - LOG_INFO(&Poco::Logger::get("TreeRewriter"), "lmx_00 enter"); - NameSet select_columns; - std::vector empty_name; auto & select_query = *t; for (size_t i = 0; i < select_query.select()->children.size(); i++) { auto node = select_query.select()->children[i]; if (auto* identifier = node->as()) { - LOG_INFO(&Poco::Logger::get("TreeRewriter"), "lmx_1 {}", identifier->name()); if (identifier->name().empty()) { - empty_name.push_back(i); select_query.select()->children.erase(select_query.select()->children.begin()+i); - } else { - select_columns.insert(identifier->name()); } } } From b78e9dcc05d247d1ce9b88456e1edd808cc79c25 Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Sun, 12 Mar 2023 22:45:25 +0800 Subject: [PATCH 012/308] fix style --- src/Interpreters/TreeRewriter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 1ac8b62c11c..81da53756ec 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -923,9 +923,9 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (auto * t = query->as()) { auto & select_query = *t; - for (size_t i = 0; i < select_query.select()->children.size(); i++) { + for (size_t i = 0; i < select_query.select()->children.size(); ++i) { auto node = select_query.select()->children[i]; - if (auto* identifier = node->as()) { + if (auto * identifier = node->as()) { if (identifier->name().empty()) { select_query.select()->children.erase(select_query.select()->children.begin()+i); } From 76f4ca0e8e60052801f24e5b3ff659b750cef04e Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Sun, 12 Mar 2023 23:14:56 +0800 Subject: [PATCH 013/308] fix style --- src/Interpreters/TreeRewriter.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 81da53756ec..95d277b490d 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -926,9 +926,8 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select for (size_t i = 0; i < select_query.select()->children.size(); ++i) { auto node = select_query.select()->children[i]; if (auto * identifier = node->as()) { - if (identifier->name().empty()) { + if (identifier->name().empty()) select_query.select()->children.erase(select_query.select()->children.begin()+i); - } } } } From 48a1f1c838ae57e04ec8fb1f84dcd50ca2649908 Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Sun, 12 Mar 2023 23:49:23 +0800 Subject: [PATCH 014/308] fix style --- src/Interpreters/TreeRewriter.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 95d277b490d..83886dd1cb0 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -923,13 +923,10 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (auto * t = query->as()) { auto & select_query = *t; - for (size_t i = 0; i < select_query.select()->children.size(); ++i) { - auto node = select_query.select()->children[i]; - if (auto * identifier = node->as()) { + for (size_t i = 0; i < select_query.select()->children.size(); ++i) + if (auto * identifier = select_query.select()->children[i]->as()) if (identifier->name().empty()) select_query.select()->children.erase(select_query.select()->children.begin()+i); - } - } } RequiredSourceColumnsVisitor(columns_context).visit(query); From c93202cca4cec2e83c51cb6b3cb56dc820965caa Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 9 Mar 2023 21:23:57 -0500 Subject: [PATCH 015/308] Keeper Client MVP --- programs/CMakeLists.txt | 21 +++ programs/config_tools.h.in | 1 + programs/keeper-client/CMakeLists.txt | 9 + programs/keeper-client/KeeperClient.cpp | 224 ++++++++++++++++++++++++ programs/keeper-client/KeeperClient.h | 44 +++++ programs/main.cpp | 6 + src/Client/ClientBase.cpp | 8 - src/Client/ClientBase.h | 8 + 8 files changed, 313 insertions(+), 8 deletions(-) create mode 100644 programs/keeper-client/CMakeLists.txt create mode 100644 programs/keeper-client/KeeperClient.cpp create mode 100644 programs/keeper-client/KeeperClient.h diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 47017a94cb5..c00d1f5349f 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -55,6 +55,8 @@ option (ENABLE_CLICKHOUSE_KEEPER "ClickHouse alternative to ZooKeeper" ${ENABLE_ option (ENABLE_CLICKHOUSE_KEEPER_CONVERTER "Util allows to convert ZooKeeper logs and snapshots into clickhouse-keeper snapshot" ${ENABLE_CLICKHOUSE_ALL}) +option (ENABLE_CLICKHOUSE_KEEPER_CLIENT "ClickHouse Keeper Client" ${ENABLE_CLICKHOUSE_ALL}) + option (ENABLE_CLICKHOUSE_SU "A tool similar to 'su'" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_DISKS "A tool to manage disks" ${ENABLE_CLICKHOUSE_ALL}) @@ -169,6 +171,13 @@ else() message(STATUS "ClickHouse keeper-converter mode: OFF") endif() +if (ENABLE_CLICKHOUSE_KEEPER_CLIENT) + message(STATUS "ClickHouse keeper-client mode: ON") +else() + message(STATUS "ClickHouse keeper-client mode: OFF") +endif() + + if (ENABLE_CLICKHOUSE_DISKS) message(STATUS "Clickhouse disks mode: ON") else() @@ -237,6 +246,10 @@ if (ENABLE_CLICKHOUSE_KEEPER_CONVERTER) add_subdirectory (keeper-converter) endif() +if (ENABLE_CLICKHOUSE_KEEPER_CLIENT) + add_subdirectory (keeper-client) +endif() + if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) add_subdirectory (odbc-bridge) endif () @@ -301,6 +314,9 @@ endif() if (ENABLE_CLICKHOUSE_KEEPER_CONVERTER) clickhouse_target_link_split_lib(clickhouse keeper-converter) endif() +if (ENABLE_CLICKHOUSE_KEEPER_CLIENT) + clickhouse_target_link_split_lib(clickhouse keeper-client) +endif() if (ENABLE_CLICKHOUSE_INSTALL) clickhouse_target_link_split_lib(clickhouse install) endif () @@ -392,6 +408,11 @@ if (ENABLE_CLICKHOUSE_KEEPER_CONVERTER) install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-keeper-converter" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) list(APPEND CLICKHOUSE_BUNDLE clickhouse-keeper-converter) endif () +if (ENABLE_CLICKHOUSE_KEEPER_CLIENT) + add_custom_target (clickhouse-keeper-client ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-keeper-client DEPENDS clickhouse) + install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-keeper-client" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) + list(APPEND CLICKHOUSE_BUNDLE clickhouse-keeper-client) +endif () if (ENABLE_CLICKHOUSE_DISKS) add_custom_target (clickhouse-disks ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-disks DEPENDS clickhouse) install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-disks" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) diff --git a/programs/config_tools.h.in b/programs/config_tools.h.in index 30444e8c84e..65ef3ca762b 100644 --- a/programs/config_tools.h.in +++ b/programs/config_tools.h.in @@ -17,6 +17,7 @@ #cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE #cmakedefine01 ENABLE_CLICKHOUSE_LIBRARY_BRIDGE #cmakedefine01 ENABLE_CLICKHOUSE_KEEPER +#cmakedefine01 ENABLE_CLICKHOUSE_KEEPER_CLIENT #cmakedefine01 ENABLE_CLICKHOUSE_KEEPER_CONVERTER #cmakedefine01 ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER #cmakedefine01 ENABLE_CLICKHOUSE_SU diff --git a/programs/keeper-client/CMakeLists.txt b/programs/keeper-client/CMakeLists.txt new file mode 100644 index 00000000000..06055d6d820 --- /dev/null +++ b/programs/keeper-client/CMakeLists.txt @@ -0,0 +1,9 @@ +set (CLICKHOUSE_KEEPER_CLIENT_SOURCES KeeperClient.cpp) + +set (CLICKHOUSE_KEEPER_CLIENT_LINK + PRIVATE + boost::program_options + dbms +) + +clickhouse_program_add(keeper-client) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp new file mode 100644 index 00000000000..05011f064fb --- /dev/null +++ b/programs/keeper-client/KeeperClient.cpp @@ -0,0 +1,224 @@ +#include "KeeperClient.h" +#include +#include +#include +#include +#include +#include + + +namespace po = boost::program_options; +namespace fs = std::filesystem; + +namespace DB +{ + +String KeeperClient::getAbsolutePath(const String & relative) +{ + String result; + if (relative.starts_with('/')) + result = fs::weakly_canonical(relative); + else + result = fs::weakly_canonical(cwd / relative); + + if (result.ends_with('/') && result.size() > 1) + result.pop_back(); + + return result; +} + +void KeeperClient::loadCommands(std::vector> && new_commands) +{ + for (auto & [name, args_count, callback] : new_commands) { + commands.insert({{name, args_count}, callback}); + suggest.addWords({name}); + } +} + +void KeeperClient::defineOptions(Poco::Util::OptionSet & options) +{ + Poco::Util::Application::defineOptions(options); + + options.addOption( + Poco::Util::Option("help", "h", "show help and exit") + .binding("help")); + + options.addOption( + Poco::Util::Option("connection-timeout", "", "set connection timeout in seconds. default 10s.") + .argument("connection-timeout") + .binding("connection-timeout")); + + options.addOption( + Poco::Util::Option("session-timeout", "", "set session timeout in seconds. default 10s.") + .argument("session-timeout") + .binding("session-timeout")); + + options.addOption( + Poco::Util::Option("operation-timeout", "", "set operation timeout in seconds. default 10s.") + .argument("operation-timeout") + .binding("operation-timeout")); + + options.addOption( + Poco::Util::Option("history-file", "", "set path of history file. default `~/.keeper-client-history`") + .argument("history-file") + .binding("history-file")); +} + +void KeeperClient::initialize(Poco::Util::Application & /* self */) +{ + loadCommands({ + {"set", 2, [](KeeperClient * client, const std::vector & args) { + client->zookeeper->set(client->getAbsolutePath(args[1]), args[2]); + }}, + + {"create", 2, [](KeeperClient * client, const std::vector & args) { + client->zookeeper->create(client->getAbsolutePath(args[1]), args[2], zkutil::CreateMode::Persistent); + }}, + + {"get", 1, [](KeeperClient * client, const std::vector & args) { + std::cout << client->zookeeper->get(client->getAbsolutePath(args[1])) << "\n"; + }}, + + {"ls", 0, [](KeeperClient * client, const std::vector & /* args */) { + auto children = client->zookeeper->getChildren(client->cwd); + for (auto & child : children) + std::cout << child << " "; + std::cout << "\n"; + }}, + + {"ls", 1, [](KeeperClient * client, const std::vector & args) { + auto children = client->zookeeper->getChildren(client->getAbsolutePath(args[1])); + for (auto & child : children) + std::cout << child << " "; + std::cout << "\n"; + }}, + + {"cd", 0, [](KeeperClient * /* client */, const std::vector & /* args */) { + }}, + + {"cd", 1, [](KeeperClient * client, const std::vector & args) { + auto new_path = client->getAbsolutePath(args[1]); + if (!client->zookeeper->exists(new_path)) + std::cerr << "Path " << new_path << " does not exists\n"; + else + client->cwd = new_path; + }}, + + {"rm", 1, [](KeeperClient * client, const std::vector & args) { + client->zookeeper->remove(client->getAbsolutePath(args[1])); + }}, + + {"rmr", 1, [](KeeperClient * client, const std::vector & args) { + client->zookeeper->removeRecursive(client->getAbsolutePath(args[1])); + }}, + }); + + String home_path; + const char * home_path_cstr = getenv("HOME"); // NOLINT(concurrency-mt-unsafe) + if (home_path_cstr) + home_path = home_path_cstr; + + if (config().has("history-file")) + history_file = config().getString("history-file"); + else + history_file = home_path + "/.keeper-client-history"; + + if (!history_file.empty() && !fs::exists(history_file)) + { + try + { + FS::createFile(history_file); + } + catch (const ErrnoException & e) + { + if (e.getErrno() != EEXIST) + throw; + } + } + + EventNotifier::init(); +} + +bool KeeperClient::processQueryText(const String & text) +{ + if (exit_strings.find(text) != exit_strings.end()) + return false; + + std::vector tokens; + boost::algorithm::split(tokens, text, boost::is_any_of(" ")); + + try + { + auto callback = commands.find({tokens[0], tokens.size() - 1}); + if (callback == commands.end()) + std::cerr << "No command found with name " << tokens[0] << " and args count " << tokens.size() - 1 << "\n"; + else + callback->second(this, tokens); + } + catch (Coordination::Exception & err) + { + std::cerr << err.message() << "\n"; + } + return true; +} + +void KeeperClient::runInteractive() +{ + + LineReader::Patterns query_extenders = {"\\"}; + LineReader::Patterns query_delimiters = {}; + + ReplxxLineReader lr(suggest, history_file, false, query_extenders, query_delimiters, {}); + lr.enableBracketedPaste(); + + while (true) + { + auto input = lr.readLine( cwd.string() + " :) ", ":-] "); + if (input.empty()) + break; + + if (!processQueryText(input)) + break; + } +} + +int KeeperClient::main(const std::vector & args) +{ + zkutil::ZooKeeperArgs zk_args(args[0]); + zk_args.connection_timeout_ms = config().getInt("connection-timeout", 10) * 1000; + zk_args.session_timeout_ms = config().getInt("session-timeout", 10) * 1000; + zk_args.operation_timeout_ms = config().getInt("operation-timeout", 10) * 1000; + zookeeper = std::make_unique(zk_args); + + runInteractive(); + + return 0; +} + +} + + +int mainEntryClickHouseKeeperClient(int argc, char ** argv) +{ + try + { + DB::KeeperClient client; + client.init(argc, argv); + return client.run(); + } + catch (const DB::Exception & e) + { + std::cerr << DB::getExceptionMessage(e, false) << std::endl; + return 1; + } + catch (const boost::program_options::error & e) + { + std::cerr << "Bad arguments: " << e.what() << std::endl; + return DB::ErrorCodes::BAD_ARGUMENTS; + } + catch (...) + { + std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; + return 1; + } +} diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h new file mode 100644 index 00000000000..8d96ade7659 --- /dev/null +++ b/programs/keeper-client/KeeperClient.h @@ -0,0 +1,44 @@ +#pragma once + + +#include +#include +#include +#include + + +namespace DB +{ + +class KeeperClient; + +class KeeperClient: public Poco::Util::Application +{ +public: + using Callback = std::function &)>; + + KeeperClient() = default; + + void initialize(Poco::Util::Application & self) override; + + int main(const std::vector & args) override; + + void defineOptions(Poco::Util::OptionSet & options) override; + +protected: + void runInteractive(); + void loadCommands(std::vector> &&); + bool processQueryText(const String & text); + + String getAbsolutePath(const String & relative); + + std::map, Callback> commands; + + String history_file; + LineReader::Suggest suggest; + + zkutil::ZooKeeperPtr zookeeper; + std::filesystem::path cwd = "/"; +}; + +} diff --git a/programs/main.cpp b/programs/main.cpp index 83e64b8c932..9a3ad47a86e 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -62,6 +62,9 @@ int mainEntryClickHouseKeeper(int argc, char ** argv); #if ENABLE_CLICKHOUSE_KEEPER_CONVERTER int mainEntryClickHouseKeeperConverter(int argc, char ** argv); #endif +#if ENABLE_CLICKHOUSE_KEEPER_CLIENT +int mainEntryClickHouseKeeperClient(int argc, char ** argv); +#endif #if ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER int mainEntryClickHouseStaticFilesDiskUploader(int argc, char ** argv); #endif @@ -133,6 +136,9 @@ std::pair clickhouse_applications[] = #if ENABLE_CLICKHOUSE_KEEPER_CONVERTER {"keeper-converter", mainEntryClickHouseKeeperConverter}, #endif +#if ENABLE_CLICKHOUSE_KEEPER_CLIENT + {"keeper-client", mainEntryClickHouseKeeperClient}, +#endif #if ENABLE_CLICKHOUSE_INSTALL {"install", mainEntryClickHouseInstall}, {"start", mainEntryClickHouseStart}, diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 120d273aa62..c931fb426ec 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -86,14 +86,6 @@ namespace CurrentMetrics namespace DB { -static const NameSet exit_strings -{ - "exit", "quit", "logout", "учше", "йгше", "дщпщге", - "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", - "q", "й", "\\q", "\\Q", "\\й", "\\Й", ":q", "Жй" -}; - - namespace ErrorCodes { extern const int BAD_ARGUMENTS; diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 5926f73f51a..566c5aefa04 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -24,6 +24,14 @@ namespace po = boost::program_options; namespace DB { + +static const NameSet exit_strings +{ + "exit", "quit", "logout", "учше", "йгше", "дщпщге", + "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", + "q", "й", "\\q", "\\Q", "\\й", "\\Й", ":q", "Жй" +}; + namespace ErrorCodes { extern const int NOT_IMPLEMENTED; From 9051ddb1743730f4504abdf2209c33a0b03de9b5 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 9 Mar 2023 21:45:58 -0500 Subject: [PATCH 016/308] Style fix --- programs/keeper-client/KeeperClient.cpp | 37 +++++++++++++++++-------- 1 file changed, 26 insertions(+), 11 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 05011f064fb..0a4cdd4286b 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -13,6 +13,11 @@ namespace fs = std::filesystem; namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + String KeeperClient::getAbsolutePath(const String & relative) { String result; @@ -29,7 +34,8 @@ String KeeperClient::getAbsolutePath(const String & relative) void KeeperClient::loadCommands(std::vector> && new_commands) { - for (auto & [name, args_count, callback] : new_commands) { + for (auto & [name, args_count, callback] : new_commands) + { commands.insert({{name, args_count}, callback}); suggest.addWords({name}); } @@ -67,36 +73,43 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) void KeeperClient::initialize(Poco::Util::Application & /* self */) { loadCommands({ - {"set", 2, [](KeeperClient * client, const std::vector & args) { + {"set", 2, [](KeeperClient * client, const std::vector & args) + { client->zookeeper->set(client->getAbsolutePath(args[1]), args[2]); }}, - {"create", 2, [](KeeperClient * client, const std::vector & args) { + {"create", 2, [](KeeperClient * client, const std::vector & args) + { client->zookeeper->create(client->getAbsolutePath(args[1]), args[2], zkutil::CreateMode::Persistent); }}, - {"get", 1, [](KeeperClient * client, const std::vector & args) { + {"get", 1, [](KeeperClient * client, const std::vector & args) + { std::cout << client->zookeeper->get(client->getAbsolutePath(args[1])) << "\n"; }}, - {"ls", 0, [](KeeperClient * client, const std::vector & /* args */) { + {"ls", 0, [](KeeperClient * client, const std::vector & /* args */) + { auto children = client->zookeeper->getChildren(client->cwd); for (auto & child : children) std::cout << child << " "; std::cout << "\n"; }}, - {"ls", 1, [](KeeperClient * client, const std::vector & args) { + {"ls", 1, [](KeeperClient * client, const std::vector & args) + { auto children = client->zookeeper->getChildren(client->getAbsolutePath(args[1])); for (auto & child : children) std::cout << child << " "; std::cout << "\n"; }}, - {"cd", 0, [](KeeperClient * /* client */, const std::vector & /* args */) { + {"cd", 0, [](KeeperClient * /* client */, const std::vector & /* args */) + { }}, - {"cd", 1, [](KeeperClient * client, const std::vector & args) { + {"cd", 1, [](KeeperClient * client, const std::vector & args) + { auto new_path = client->getAbsolutePath(args[1]); if (!client->zookeeper->exists(new_path)) std::cerr << "Path " << new_path << " does not exists\n"; @@ -104,11 +117,13 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) client->cwd = new_path; }}, - {"rm", 1, [](KeeperClient * client, const std::vector & args) { + {"rm", 1, [](KeeperClient * client, const std::vector & args) + { client->zookeeper->remove(client->getAbsolutePath(args[1])); }}, - {"rmr", 1, [](KeeperClient * client, const std::vector & args) { + {"rmr", 1, [](KeeperClient * client, const std::vector & args) + { client->zookeeper->removeRecursive(client->getAbsolutePath(args[1])); }}, }); @@ -173,7 +188,7 @@ void KeeperClient::runInteractive() while (true) { - auto input = lr.readLine( cwd.string() + " :) ", ":-] "); + auto input = lr.readLine(cwd.string() + " :) ", ":-] "); if (input.empty()) break; From dfea87d24888eb3d223ab5c020ff5a3cdd029409 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 10 Mar 2023 23:12:16 -0500 Subject: [PATCH 017/308] Added confirmation for rmr operation. Implemented support for four-letter-word commands. --- programs/keeper-client/KeeperClient.cpp | 73 ++++++++++++++++++++++--- programs/keeper-client/KeeperClient.h | 13 ++++- 2 files changed, 78 insertions(+), 8 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 0a4cdd4286b..752c44bd8f4 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -13,11 +13,47 @@ namespace fs = std::filesystem; namespace DB { +static const NameSet four_letter_word_commands +{ + "ruok", "mntr", "srvr", "stat", "srst", "conf", + "cons", "crst", "envi", "dirs", "isro", "wchs", + "wchc", "wchp", "dump", "csnp", "lgif", "rqld", +}; + namespace ErrorCodes { extern const int BAD_ARGUMENTS; } +String KeeperClient::executeFourLetterCommand(const String & command) +{ + // We need create new socket every time because ZooKeeper forcefully shut down connection after four-letter-word command. + Poco::Net::StreamSocket socket; + socket.connect(Poco::Net::SocketAddress{zk_args.hosts[0]}, zk_args.connection_timeout_ms * 1000); + + socket.setReceiveTimeout(zk_args.operation_timeout_ms * 1000); + socket.setSendTimeout(zk_args.operation_timeout_ms * 1000); + socket.setNoDelay(true); + + ReadBufferFromPocoSocket in(socket); + WriteBufferFromPocoSocket out(socket); + + out.write(command.data(), command.size()); + out.next(); + + String result; + readStringUntilEOF(result, in); + in.next(); + return result; +} + +void KeeperClient::askConfirmation(const String & prompt, std::function && callback) +{ + std::cout << prompt << " Continue?\n"; + need_confirmation = true; + confirmation_callback = callback; +} + String KeeperClient::getAbsolutePath(const String & relative) { String result; @@ -124,7 +160,9 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) {"rmr", 1, [](KeeperClient * client, const std::vector & args) { - client->zookeeper->removeRecursive(client->getAbsolutePath(args[1])); + String path = client->getAbsolutePath(args[1]); + client->askConfirmation("You are going to recursively delete path " + path, + [client, path]{ client->zookeeper->removeRecursive(path); }); }}, }); @@ -164,11 +202,26 @@ bool KeeperClient::processQueryText(const String & text) try { - auto callback = commands.find({tokens[0], tokens.size() - 1}); - if (callback == commands.end()) - std::cerr << "No command found with name " << tokens[0] << " and args count " << tokens.size() - 1 << "\n"; + if (need_confirmation) + { + if (tokens.size() == 1 && (tokens[0] == "y" || tokens[0] == "Y")) + { + need_confirmation = false; + confirmation_callback(); + } + + need_confirmation = false; + } + else if (tokens.size() == 1 && tokens[0].size() == 4 && four_letter_word_commands.find(tokens[0]) != four_letter_word_commands.end()) + std::cout << executeFourLetterCommand(tokens[0]) << "\n"; else - callback->second(this, tokens); + { + auto callback = commands.find({tokens[0], tokens.size() - 1}); + if (callback == commands.end()) + std::cerr << "No command found with name " << tokens[0] << " and args count " << tokens.size() - 1 << "\n"; + else + callback->second(this, tokens); + } } catch (Coordination::Exception & err) { @@ -188,7 +241,13 @@ void KeeperClient::runInteractive() while (true) { - auto input = lr.readLine(cwd.string() + " :) ", ":-] "); + String prompt; + if (need_confirmation) + prompt = "[y/n] "; + else + prompt = cwd.string() + " :) "; + + auto input = lr.readLine(prompt, ":-] "); if (input.empty()) break; @@ -199,7 +258,7 @@ void KeeperClient::runInteractive() int KeeperClient::main(const std::vector & args) { - zkutil::ZooKeeperArgs zk_args(args[0]); + zk_args.hosts = {args[0]}; zk_args.connection_timeout_ms = config().getInt("connection-timeout", 10) * 1000; zk_args.session_timeout_ms = config().getInt("session-timeout", 10) * 1000; zk_args.operation_timeout_ms = config().getInt("operation-timeout", 10) * 1000; diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h index 8d96ade7659..0634d3e4b37 100644 --- a/programs/keeper-client/KeeperClient.h +++ b/programs/keeper-client/KeeperClient.h @@ -3,6 +3,9 @@ #include #include +#include +#include +#include #include #include @@ -27,10 +30,13 @@ public: protected: void runInteractive(); - void loadCommands(std::vector> &&); + void loadCommands(std::vector> && callback); bool processQueryText(const String & text); + String executeFourLetterCommand(const String & command); + String getAbsolutePath(const String & relative); + void askConfirmation(const String & prompt, std::function && callback); std::map, Callback> commands; @@ -38,7 +44,12 @@ protected: LineReader::Suggest suggest; zkutil::ZooKeeperPtr zookeeper; + zkutil::ZooKeeperArgs zk_args; + std::filesystem::path cwd = "/"; + + bool need_confirmation = false; + std::function confirmation_callback; }; } From 72769d468ea55f419286caf417ed985af9e4069b Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 10 Mar 2023 23:15:15 -0500 Subject: [PATCH 018/308] comment fix --- programs/keeper-client/KeeperClient.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 752c44bd8f4..43a9527a3ca 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -27,7 +27,7 @@ namespace ErrorCodes String KeeperClient::executeFourLetterCommand(const String & command) { - // We need create new socket every time because ZooKeeper forcefully shut down connection after four-letter-word command. + /// We need to create a new socket every time because ZooKeeper forcefully shuts down the connection after a four-letter-word command. Poco::Net::StreamSocket socket; socket.connect(Poco::Net::SocketAddress{zk_args.hosts[0]}, zk_args.connection_timeout_ms * 1000); From 18fada7028c7997f25814201ce7c89c05efb9e3d Mon Sep 17 00:00:00 2001 From: pufit Date: Sun, 12 Mar 2023 12:54:42 -0400 Subject: [PATCH 019/308] fix duplication --- programs/keeper-client/KeeperClient.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 43a9527a3ca..74eae042b4c 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -204,13 +204,9 @@ bool KeeperClient::processQueryText(const String & text) { if (need_confirmation) { - if (tokens.size() == 1 && (tokens[0] == "y" || tokens[0] == "Y")) - { - need_confirmation = false; - confirmation_callback(); - } - need_confirmation = false; + if (tokens.size() == 1 && (tokens[0] == "y" || tokens[0] == "Y")) + confirmation_callback(); } else if (tokens.size() == 1 && tokens[0].size() == 4 && four_letter_word_commands.find(tokens[0]) != four_letter_word_commands.end()) std::cout << executeFourLetterCommand(tokens[0]) << "\n"; From 752eab501ce9da992f7f4a1dc3600521c7c65723 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 14 Mar 2023 14:24:36 -0400 Subject: [PATCH 020/308] default host to connect --- programs/keeper-client/KeeperClient.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 74eae042b4c..1f68cbd05bf 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -254,7 +254,11 @@ void KeeperClient::runInteractive() int KeeperClient::main(const std::vector & args) { - zk_args.hosts = {args[0]}; + if (args.empty()) + zk_args.hosts = {"localhost:2181"}; + else + zk_args.hosts = {args[0]}; + zk_args.connection_timeout_ms = config().getInt("connection-timeout", 10) * 1000; zk_args.session_timeout_ms = config().getInt("session-timeout", 10) * 1000; zk_args.operation_timeout_ms = config().getInt("operation-timeout", 10) * 1000; From 7adc442fedba0617f322cdecdaee7b21904bf51a Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 14 Mar 2023 15:32:48 -0400 Subject: [PATCH 021/308] support run in non-interactive mode --- programs/keeper-client/KeeperClient.cpp | 35 ++++++++++++++++++++++--- programs/keeper-client/KeeperClient.h | 1 + 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 1f68cbd05bf..9ed60dd0d4b 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -70,11 +70,15 @@ String KeeperClient::getAbsolutePath(const String & relative) void KeeperClient::loadCommands(std::vector> && new_commands) { - for (auto & [name, args_count, callback] : new_commands) + for (const auto & [name, args_count, callback] : new_commands) { commands.insert({{name, args_count}, callback}); suggest.addWords({name}); } + + for (const auto & command : four_letter_word_commands) { + suggest.addWords({command}); + } } void KeeperClient::defineOptions(Poco::Util::OptionSet & options) @@ -85,6 +89,11 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) Poco::Util::Option("help", "h", "show help and exit") .binding("help")); + options.addOption( + Poco::Util::Option("query", "q", "will execute given query, then exit.") + .argument("query") + .binding("query")); + options.addOption( Poco::Util::Option("connection-timeout", "", "set connection timeout in seconds. default 10s.") .argument("connection-timeout") @@ -192,6 +201,18 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) EventNotifier::init(); } +void KeeperClient::executeQuery(const String & query) +{ + std::vector queries; + boost::algorithm::split(queries, query, boost::is_any_of(";")); + + for (const auto & query_text : queries) + { + if (!query_text.empty()) + processQueryText(query_text); + } +} + bool KeeperClient::processQueryText(const String & text) { if (exit_strings.find(text) != exit_strings.end()) @@ -214,7 +235,12 @@ bool KeeperClient::processQueryText(const String & text) { auto callback = commands.find({tokens[0], tokens.size() - 1}); if (callback == commands.end()) - std::cerr << "No command found with name " << tokens[0] << " and args count " << tokens.size() - 1 << "\n"; + { + if (tokens[0].size() == 4 && tokens.size() == 1) /// Treat it like unrecognized four-letter command + std::cout << executeFourLetterCommand(tokens[0]) << "\n"; + else + std::cerr << "No command found with name " << tokens[0] << " and args count " << tokens.size() - 1 << "\n"; + } else callback->second(this, tokens); } @@ -264,7 +290,10 @@ int KeeperClient::main(const std::vector & args) zk_args.operation_timeout_ms = config().getInt("operation-timeout", 10) * 1000; zookeeper = std::make_unique(zk_args); - runInteractive(); + if (config().has("query")) + executeQuery(config().getString("query")); + else + runInteractive(); return 0; } diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h index 0634d3e4b37..10099b06021 100644 --- a/programs/keeper-client/KeeperClient.h +++ b/programs/keeper-client/KeeperClient.h @@ -32,6 +32,7 @@ protected: void runInteractive(); void loadCommands(std::vector> && callback); bool processQueryText(const String & text); + void executeQuery(const String & query); String executeFourLetterCommand(const String & command); From bfdc2b58b421718550b586fab6806e24a18cc960 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 14 Mar 2023 15:34:08 -0400 Subject: [PATCH 022/308] style fix --- programs/keeper-client/KeeperClient.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 9ed60dd0d4b..c427c6fc8ef 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -203,7 +203,7 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) void KeeperClient::executeQuery(const String & query) { - std::vector queries; + std::vector queries; boost::algorithm::split(queries, query, boost::is_any_of(";")); for (const auto & query_text : queries) @@ -218,7 +218,7 @@ bool KeeperClient::processQueryText(const String & text) if (exit_strings.find(text) != exit_strings.end()) return false; - std::vector tokens; + std::vector tokens; boost::algorithm::split(tokens, text, boost::is_any_of(" ")); try From 7dc6ff02c33b3d84fdaf4ff2f16a74c8b2edbd3e Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 14 Mar 2023 17:50:09 -0400 Subject: [PATCH 023/308] use keeper-client in integration tests --- programs/keeper-client/KeeperClient.cpp | 7 +++++++ tests/integration/helpers/keeper_utils.py | 22 ++++------------------ 2 files changed, 11 insertions(+), 18 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index c427c6fc8ef..10aa4b1dedd 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -113,6 +113,11 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) Poco::Util::Option("history-file", "", "set path of history file. default `~/.keeper-client-history`") .argument("history-file") .binding("history-file")); + + options.addOption( + Poco::Util::Option("log-level", "", "set log level") + .argument("log-level") + .binding("log-level")); } void KeeperClient::initialize(Poco::Util::Application & /* self */) @@ -198,6 +203,8 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) } } + Poco::Logger::root().setLevel(config().getString("log-level", "error")); + EventNotifier::init(); } diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 3b909194b63..c6cd9dfa18a 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -1,26 +1,12 @@ import socket import time - - -def get_keeper_socket(cluster, node, port=9181): - hosts = cluster.get_instance_ip(node.name) - client = socket.socket() - client.settimeout(10) - client.connect((hosts, port)) - return client +from helper.client import CommandRequest def send_4lw_cmd(cluster, node, cmd="ruok", port=9181): - client = None - try: - client = get_keeper_socket(cluster, node, port) - client.send(cmd.encode()) - data = client.recv(100_000) - data = data.decode() - return data - finally: - if client is not None: - client.close() + return CommandRequest( + ["cluster.server_bin_path", "keeper-client", f"{cluster.get_instance_ip(node.name)}:{port}", "-q", cmd] + ).get_answer() NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving requests" From e6d01c617b16117d314dbeea43660ece5c4b54d2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 14 Mar 2023 22:14:43 +0000 Subject: [PATCH 024/308] Automatic style fix --- tests/integration/helpers/keeper_utils.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index c6cd9dfa18a..f9cc2fb29fb 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -5,7 +5,13 @@ from helper.client import CommandRequest def send_4lw_cmd(cluster, node, cmd="ruok", port=9181): return CommandRequest( - ["cluster.server_bin_path", "keeper-client", f"{cluster.get_instance_ip(node.name)}:{port}", "-q", cmd] + [ + "cluster.server_bin_path", + "keeper-client", + f"{cluster.get_instance_ip(node.name)}:{port}", + "-q", + cmd, + ] ).get_answer() From ef9f66e36fed106f59bafaf971a02799fbecb75f Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 14 Mar 2023 18:30:23 -0400 Subject: [PATCH 025/308] style fix --- programs/keeper-client/KeeperClient.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 10aa4b1dedd..54b5cf1d6cf 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -76,9 +76,8 @@ void KeeperClient::loadCommands(std::vector suggest.addWords({name}); } - for (const auto & command : four_letter_word_commands) { + for (const auto & command : four_letter_word_commands) suggest.addWords({command}); - } } void KeeperClient::defineOptions(Poco::Util::OptionSet & options) From 65f2516079f3a2b53af8224078119fc4062b6ef7 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 16 Mar 2023 15:37:06 -0400 Subject: [PATCH 026/308] separate integration tests for keeper-client --- tests/integration/helpers/keeper_utils.py | 28 +++++---- .../test_keeper_client/__init__.py | 0 .../configs/keeper_config.xml | 3 + tests/integration/test_keeper_client/test.py | 57 +++++++++++++++++++ 4 files changed, 78 insertions(+), 10 deletions(-) create mode 100644 tests/integration/test_keeper_client/__init__.py create mode 100644 tests/integration/test_keeper_client/configs/keeper_config.xml create mode 100644 tests/integration/test_keeper_client/test.py diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index f9cc2fb29fb..3b909194b63 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -1,18 +1,26 @@ import socket import time -from helper.client import CommandRequest + + +def get_keeper_socket(cluster, node, port=9181): + hosts = cluster.get_instance_ip(node.name) + client = socket.socket() + client.settimeout(10) + client.connect((hosts, port)) + return client def send_4lw_cmd(cluster, node, cmd="ruok", port=9181): - return CommandRequest( - [ - "cluster.server_bin_path", - "keeper-client", - f"{cluster.get_instance_ip(node.name)}:{port}", - "-q", - cmd, - ] - ).get_answer() + client = None + try: + client = get_keeper_socket(cluster, node, port) + client.send(cmd.encode()) + data = client.recv(100_000) + data = data.decode() + return data + finally: + if client is not None: + client.close() NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving requests" diff --git a/tests/integration/test_keeper_client/__init__.py b/tests/integration/test_keeper_client/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_client/configs/keeper_config.xml b/tests/integration/test_keeper_client/configs/keeper_config.xml new file mode 100644 index 00000000000..7e912283ac0 --- /dev/null +++ b/tests/integration/test_keeper_client/configs/keeper_config.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py new file mode 100644 index 00000000000..64ef62b6243 --- /dev/null +++ b/tests/integration/test_keeper_client/test.py @@ -0,0 +1,57 @@ +import pytest +from helpers.client import CommandRequest +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/keeper_config.xml"], + with_zookeeper=True, + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_base_commands(started_cluster): + _ = started_cluster + + command = CommandRequest( + [ + started_cluster.server_bin_path, + "keeper-client", + f"{cluster.get_instance_ip('zoo1')}:{cluster.zookeeper_port}", + "-q", + "create test_create_zk_node1 testvalue1;create test_create_zk_node_2 testvalue2;get test_create_zk_node1;", + ], + stdin="", + ) + + assert command.get_answer() == "testvalue1\n" + + +def test_four_letter_word_commands(started_cluster): + _ = started_cluster + + command = CommandRequest( + [ + started_cluster.server_bin_path, + "keeper-client", + f"{cluster.get_instance_ip('zoo1')}:{cluster.zookeeper_port}", + "-q", + "ruok", + ], + stdin="", + ) + + assert command.get_answer() == "imok\n" From 3380e467d91ca8797135b860342944b46505efd3 Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 27 Mar 2023 11:12:46 -0400 Subject: [PATCH 027/308] fix typo --- programs/keeper-client/KeeperClient.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h index 10099b06021..50a8b35f6c5 100644 --- a/programs/keeper-client/KeeperClient.h +++ b/programs/keeper-client/KeeperClient.h @@ -30,7 +30,7 @@ public: protected: void runInteractive(); - void loadCommands(std::vector> && callback); + void loadCommands(std::vector> && new_commands); bool processQueryText(const String & text); void executeQuery(const String & query); From 17efdbf6251e51d945dd1df598a6526c465fe99b Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Fri, 31 Mar 2023 21:56:35 +0800 Subject: [PATCH 028/308] change --- src/Formats/ReadSchemaUtils.cpp | 374 +++++++++++++++--------------- src/Interpreters/TreeRewriter.cpp | 9 - 2 files changed, 182 insertions(+), 201 deletions(-) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 653efd4f5c1..f12eb8cb71e 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -9,223 +9,213 @@ #include #include -namespace DB -{ +namespace DB { -namespace ErrorCodes -{ - extern const int EMPTY_DATA_PASSED; - extern const int BAD_ARGUMENTS; - extern const int ONLY_NULLS_WHILE_READING_SCHEMA; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; -} + namespace ErrorCodes { + extern const int EMPTY_DATA_PASSED; + extern const int BAD_ARGUMENTS; + extern const int ONLY_NULLS_WHILE_READING_SCHEMA; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + } -static std::optional getOrderedColumnsList( - const NamesAndTypesList & columns_list, const Names & columns_order_hint) -{ - if (columns_list.size() != columns_order_hint.size()) - return {}; - - std::unordered_map available_columns; - for (const auto & [name, type] : columns_list) - available_columns.emplace(name, type); - - NamesAndTypesList res; - for (const auto & name : columns_order_hint) - { - auto it = available_columns.find(name); - if (it == available_columns.end()) + static std::optional getOrderedColumnsList( + const NamesAndTypesList &columns_list, const Names &columns_order_hint) { + if (columns_list.size() != columns_order_hint.size()) return {}; - res.emplace_back(name, it->second); - } - return res; -} + std::unordered_map available_columns; + for (const auto &[name, type]: columns_list) + available_columns.emplace(name, type); -bool isRetryableSchemaInferenceError(int code) -{ - return code == ErrorCodes::EMPTY_DATA_PASSED || code == ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA; -} + NamesAndTypesList res; + for (const auto &name: columns_order_hint) { + auto it = available_columns.find(name); + if (it == available_columns.end()) + return {}; -ColumnsDescription readSchemaFromFormat( - const String & format_name, - const std::optional & format_settings, - ReadBufferIterator & read_buffer_iterator, - bool retry, - ContextPtr & context, - std::unique_ptr & buf) -{ - NamesAndTypesList names_and_types; - if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name)) - { - auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(format_name, context, format_settings); - try - { - names_and_types = external_schema_reader->readSchema(); - } - catch (Exception & e) - { - e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); - throw; + res.emplace_back(name, it->second); } + return res; } - else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name)) - { - std::string exception_messages; - SchemaReaderPtr schema_reader; - size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; - size_t iterations = 0; - ColumnsDescription cached_columns; - while (true) - { - bool is_eof = false; - try - { - buf = read_buffer_iterator(cached_columns); - if (!buf) - break; - is_eof = buf->eof(); + + bool isRetryableSchemaInferenceError(int code) { + return code == ErrorCodes::EMPTY_DATA_PASSED || code == ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA; + } + + ColumnsDescription readSchemaFromFormat( + const String &format_name, + const std::optional &format_settings, + ReadBufferIterator &read_buffer_iterator, + bool retry, + ContextPtr &context, + std::unique_ptr &buf) { + NamesAndTypesList names_and_types; + if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name)) { + auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(format_name, context, + format_settings); + try { + names_and_types = external_schema_reader->readSchema(); } - catch (Exception & e) - { + catch (Exception &e) { e.addMessage(fmt::format( - "Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); + "Cannot extract table structure from {} format file. You can specify the structure manually", + format_name)); throw; } - catch (...) - { - auto exception_message = getCurrentExceptionMessage(false); - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file:\n{}\nYou can specify the structure manually", - format_name, - exception_message); - } - - ++iterations; - - if (is_eof) - { - auto exception_message = fmt::format("Cannot extract table structure from {} format file, file is empty", format_name); - - if (!retry) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message); - - exception_messages += "\n" + exception_message; - continue; - } - - try - { - schema_reader = FormatFactory::instance().getSchemaReader(format_name, *buf, context, format_settings); - schema_reader->setMaxRowsToRead(max_rows_to_read); - names_and_types = schema_reader->readSchema(); - break; - } - catch (...) - { - auto exception_message = getCurrentExceptionMessage(false); - if (schema_reader) - { - size_t rows_read = schema_reader->getNumRowsRead(); - assert(rows_read <= max_rows_to_read); - max_rows_to_read -= schema_reader->getNumRowsRead(); - if (rows_read != 0 && max_rows_to_read == 0) - { - exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; - if (iterations > 1) - { - exception_messages += "\n" + exception_message; - break; - } - retry = false; - } + } else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name)) { + std::string exception_messages; + SchemaReaderPtr schema_reader; + size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference + : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; + size_t iterations = 0; + ColumnsDescription cached_columns; + while (true) { + bool is_eof = false; + try { + buf = read_buffer_iterator(cached_columns); + if (!buf) + break; + is_eof = buf->eof(); + } + catch (Exception &e) { + e.addMessage(fmt::format( + "Cannot extract table structure from {} format file. You can specify the structure manually", + format_name)); + throw; + } + catch (...) { + auto exception_message = getCurrentExceptionMessage(false); + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file:\n{}\nYou can specify the structure manually", + format_name, + exception_message); } - if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode())) - { - try - { - throw; - } - catch (Exception & e) - { - e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); - throw; - } - catch (...) - { + ++iterations; + + if (is_eof) { + auto exception_message = fmt::format( + "Cannot extract table structure from {} format file, file is empty", format_name); + + if (!retry) throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file. " - "Error: {}. You can specify the structure manually", - format_name, exception_message); - } + "{}. You can specify the structure manually", exception_message); + + exception_messages += "\n" + exception_message; + continue; } - exception_messages += "\n" + exception_message; + try { + schema_reader = FormatFactory::instance().getSchemaReader(format_name, *buf, context, + format_settings); + schema_reader->setMaxRowsToRead(max_rows_to_read); + names_and_types = schema_reader->readSchema(); + break; + } + catch (...) { + auto exception_message = getCurrentExceptionMessage(false); + if (schema_reader) { + size_t rows_read = schema_reader->getNumRowsRead(); + assert(rows_read <= max_rows_to_read); + max_rows_to_read -= schema_reader->getNumRowsRead(); + if (rows_read != 0 && max_rows_to_read == 0) { + exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; + if (iterations > 1) { + exception_messages += "\n" + exception_message; + break; + } + retry = false; + } + } + + if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode())) { + try { + throw; + } + catch (Exception &e) { + e.addMessage(fmt::format( + "Cannot extract table structure from {} format file. You can specify the structure manually", + format_name)); + throw; + } + catch (...) { + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file. " + "Error: {}. You can specify the structure manually", + format_name, exception_message); + } + } + + exception_messages += "\n" + exception_message; + } } - } - if (!cached_columns.empty()) - return cached_columns; + if (!cached_columns.empty()) + return cached_columns; - if (names_and_types.empty()) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "All attempts to extract table structure from files failed. " - "Errors:{}\nYou can specify the structure manually", exception_messages); + if (names_and_types.empty()) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "All attempts to extract table structure from files failed. " + "Errors:{}\nYou can specify the structure manually", exception_messages); - /// If we have "INSERT SELECT" query then try to order - /// columns as they are ordered in table schema for formats - /// without strict column order (like JSON and TSKV). - /// It will allow to execute simple data loading with query - /// "INSERT INTO table SELECT * FROM ..." - const auto & insertion_table = context->getInsertionTable(); - if (!schema_reader->hasStrictOrderOfColumns() && !insertion_table.empty()) - { - auto storage = DatabaseCatalog::instance().getTable(insertion_table, context); - auto metadata = storage->getInMemoryMetadataPtr(); - auto names_in_storage = metadata->getColumns().getNamesOfPhysical(); - auto ordered_list = getOrderedColumnsList(names_and_types, names_in_storage); - if (ordered_list) - names_and_types = *ordered_list; - } + /// If we have "INSERT SELECT" query then try to order + /// columns as they are ordered in table schema for formats + /// without strict column order (like JSON and TSKV). + /// It will allow to execute simple data loading with query + /// "INSERT INTO table SELECT * FROM ..." + const auto &insertion_table = context->getInsertionTable(); + if (!schema_reader->hasStrictOrderOfColumns() && !insertion_table.empty()) { + auto storage = DatabaseCatalog::instance().getTable(insertion_table, context); + auto metadata = storage->getInMemoryMetadataPtr(); + auto names_in_storage = metadata->getColumns().getNamesOfPhysical(); + auto ordered_list = getOrderedColumnsList(names_and_types, names_in_storage); + if (ordered_list) + names_and_types = *ordered_list; + } + } else + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "{} file format doesn't support schema inference. You must specify the structure manually", + format_name); + names_and_types.erase(std::remove_if(names_and_types.begin(), names_and_types.end(), + [](const NameAndTypePair &pair) { return pair.name.empty(); }), + names_and_types.end()); + return ColumnsDescription(names_and_types); } - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "{} file format doesn't support schema inference. You must specify the structure manually", - format_name); - return ColumnsDescription(names_and_types); -} + ColumnsDescription + readSchemaFromFormat(const String &format_name, const std::optional &format_settings, + ReadBufferIterator &read_buffer_iterator, bool retry, ContextPtr &context) { + std::unique_ptr buf_out; + return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out); + } -ColumnsDescription readSchemaFromFormat(const String & format_name, const std::optional & format_settings, ReadBufferIterator & read_buffer_iterator, bool retry, ContextPtr & context) -{ - std::unique_ptr buf_out; - return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out); -} + SchemaCache::Key getKeyForSchemaCache(const String &source, const String &format, + const std::optional &format_settings, + const ContextPtr &context) { + return getKeysForSchemaCache({source}, format, format_settings, context).front(); + } -SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional & format_settings, const ContextPtr & context) -{ - return getKeysForSchemaCache({source}, format, format_settings, context).front(); -} + static SchemaCache::Key + makeSchemaCacheKey(const String &source, const String &format, const String &additional_format_info) { + return SchemaCache::Key{source, format, additional_format_info}; + } -static SchemaCache::Key makeSchemaCacheKey(const String & source, const String & format, const String & additional_format_info) -{ - return SchemaCache::Key{source, format, additional_format_info}; -} - -SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, const String & format, const std::optional & format_settings, const ContextPtr & context) -{ - /// For some formats data schema depends on some settings, so it's possible that - /// two queries to the same source will get two different schemas. To process this - /// case we add some additional information specific for the format to the cache key. - /// For example, for Protobuf format additional information is the path to the schema - /// and message name. - String additional_format_info = FormatFactory::instance().getAdditionalInfoForSchemaCache(format, context, format_settings); - SchemaCache::Keys cache_keys; - cache_keys.reserve(sources.size()); - std::transform(sources.begin(), sources.end(), std::back_inserter(cache_keys), [&](const auto & source){ return makeSchemaCacheKey(source, format, additional_format_info); }); - return cache_keys; -} + SchemaCache::Keys getKeysForSchemaCache(const Strings &sources, const String &format, + const std::optional &format_settings, + const ContextPtr &context) { + /// For some formats data schema depends on some settings, so it's possible that + /// two queries to the same source will get two different schemas. To process this + /// case we add some additional information specific for the format to the cache key. + /// For example, for Protobuf format additional information is the path to the schema + /// and message name. + String additional_format_info = FormatFactory::instance().getAdditionalInfoForSchemaCache(format, context, + format_settings); + SchemaCache::Keys cache_keys; + cache_keys.reserve(sources.size()); + std::transform(sources.begin(), sources.end(), std::back_inserter(cache_keys), + [&](const auto &source) { return makeSchemaCacheKey(source, format, additional_format_info); }); + return cache_keys; + } } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 83886dd1cb0..4c134e175dc 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -919,15 +919,6 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select RequiredSourceColumnsVisitor::Data columns_context; columns_context.visit_index_hint = visit_index_hint; - - if (auto * t = query->as()) - { - auto & select_query = *t; - for (size_t i = 0; i < select_query.select()->children.size(); ++i) - if (auto * identifier = select_query.select()->children[i]->as()) - if (identifier->name().empty()) - select_query.select()->children.erase(select_query.select()->children.begin()+i); - } RequiredSourceColumnsVisitor(columns_context).visit(query); NameSet source_column_names; From 3b756ef0261d33a9c8771cad7906f690db11bf75 Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Fri, 31 Mar 2023 21:58:20 +0800 Subject: [PATCH 029/308] rollback --- src/Formats/ReadSchemaUtils.cpp | 375 ++++++++++++++++---------------- 1 file changed, 193 insertions(+), 182 deletions(-) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index f12eb8cb71e..7f1c3bf3e63 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -9,213 +9,224 @@ #include #include -namespace DB { +namespace DB +{ - namespace ErrorCodes { - extern const int EMPTY_DATA_PASSED; - extern const int BAD_ARGUMENTS; - extern const int ONLY_NULLS_WHILE_READING_SCHEMA; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - } +namespace ErrorCodes +{ + extern const int EMPTY_DATA_PASSED; + extern const int BAD_ARGUMENTS; + extern const int ONLY_NULLS_WHILE_READING_SCHEMA; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; +} - static std::optional getOrderedColumnsList( - const NamesAndTypesList &columns_list, const Names &columns_order_hint) { - if (columns_list.size() != columns_order_hint.size()) +static std::optional getOrderedColumnsList( + const NamesAndTypesList & columns_list, const Names & columns_order_hint) +{ + if (columns_list.size() != columns_order_hint.size()) + return {}; + + std::unordered_map available_columns; + for (const auto & [name, type] : columns_list) + available_columns.emplace(name, type); + + NamesAndTypesList res; + for (const auto & name : columns_order_hint) + { + auto it = available_columns.find(name); + if (it == available_columns.end()) return {}; - std::unordered_map available_columns; - for (const auto &[name, type]: columns_list) - available_columns.emplace(name, type); + res.emplace_back(name, it->second); + } + return res; +} - NamesAndTypesList res; - for (const auto &name: columns_order_hint) { - auto it = available_columns.find(name); - if (it == available_columns.end()) - return {}; +bool isRetryableSchemaInferenceError(int code) +{ + return code == ErrorCodes::EMPTY_DATA_PASSED || code == ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA; +} - res.emplace_back(name, it->second); +ColumnsDescription readSchemaFromFormat( + const String & format_name, + const std::optional & format_settings, + ReadBufferIterator & read_buffer_iterator, + bool retry, + ContextPtr & context, + std::unique_ptr & buf) +{ + NamesAndTypesList names_and_types; + if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name)) + { + auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(format_name, context, format_settings); + try + { + names_and_types = external_schema_reader->readSchema(); + } + catch (Exception & e) + { + e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); + throw; } - return res; } - - bool isRetryableSchemaInferenceError(int code) { - return code == ErrorCodes::EMPTY_DATA_PASSED || code == ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA; - } - - ColumnsDescription readSchemaFromFormat( - const String &format_name, - const std::optional &format_settings, - ReadBufferIterator &read_buffer_iterator, - bool retry, - ContextPtr &context, - std::unique_ptr &buf) { - NamesAndTypesList names_and_types; - if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name)) { - auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(format_name, context, - format_settings); - try { - names_and_types = external_schema_reader->readSchema(); + else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name)) + { + std::string exception_messages; + SchemaReaderPtr schema_reader; + size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; + size_t iterations = 0; + ColumnsDescription cached_columns; + while (true) + { + bool is_eof = false; + try + { + buf = read_buffer_iterator(cached_columns); + if (!buf) + break; + is_eof = buf->eof(); } - catch (Exception &e) { + catch (Exception & e) + { e.addMessage(fmt::format( - "Cannot extract table structure from {} format file. You can specify the structure manually", - format_name)); + "Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); throw; } - } else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name)) { - std::string exception_messages; - SchemaReaderPtr schema_reader; - size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference - : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; - size_t iterations = 0; - ColumnsDescription cached_columns; - while (true) { - bool is_eof = false; - try { - buf = read_buffer_iterator(cached_columns); - if (!buf) - break; - is_eof = buf->eof(); - } - catch (Exception &e) { - e.addMessage(fmt::format( - "Cannot extract table structure from {} format file. You can specify the structure manually", - format_name)); - throw; - } - catch (...) { - auto exception_message = getCurrentExceptionMessage(false); - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file:\n{}\nYou can specify the structure manually", - format_name, - exception_message); + catch (...) + { + auto exception_message = getCurrentExceptionMessage(false); + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file:\n{}\nYou can specify the structure manually", + format_name, + exception_message); + } + + ++iterations; + + if (is_eof) + { + auto exception_message = fmt::format("Cannot extract table structure from {} format file, file is empty", format_name); + + if (!retry) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message); + + exception_messages += "\n" + exception_message; + continue; + } + + try + { + schema_reader = FormatFactory::instance().getSchemaReader(format_name, *buf, context, format_settings); + schema_reader->setMaxRowsToRead(max_rows_to_read); + names_and_types = schema_reader->readSchema(); + break; + } + catch (...) + { + auto exception_message = getCurrentExceptionMessage(false); + if (schema_reader) + { + size_t rows_read = schema_reader->getNumRowsRead(); + assert(rows_read <= max_rows_to_read); + max_rows_to_read -= schema_reader->getNumRowsRead(); + if (rows_read != 0 && max_rows_to_read == 0) + { + exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; + if (iterations > 1) + { + exception_messages += "\n" + exception_message; + break; + } + retry = false; + } } - ++iterations; - - if (is_eof) { - auto exception_message = fmt::format( - "Cannot extract table structure from {} format file, file is empty", format_name); - - if (!retry) + if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode())) + { + try + { + throw; + } + catch (Exception & e) + { + e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); + throw; + } + catch (...) + { throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "{}. You can specify the structure manually", exception_message); - - exception_messages += "\n" + exception_message; - continue; - } - - try { - schema_reader = FormatFactory::instance().getSchemaReader(format_name, *buf, context, - format_settings); - schema_reader->setMaxRowsToRead(max_rows_to_read); - names_and_types = schema_reader->readSchema(); - break; - } - catch (...) { - auto exception_message = getCurrentExceptionMessage(false); - if (schema_reader) { - size_t rows_read = schema_reader->getNumRowsRead(); - assert(rows_read <= max_rows_to_read); - max_rows_to_read -= schema_reader->getNumRowsRead(); - if (rows_read != 0 && max_rows_to_read == 0) { - exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; - if (iterations > 1) { - exception_messages += "\n" + exception_message; - break; - } - retry = false; - } + "Cannot extract table structure from {} format file. " + "Error: {}. You can specify the structure manually", + format_name, exception_message); } - - if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode())) { - try { - throw; - } - catch (Exception &e) { - e.addMessage(fmt::format( - "Cannot extract table structure from {} format file. You can specify the structure manually", - format_name)); - throw; - } - catch (...) { - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file. " - "Error: {}. You can specify the structure manually", - format_name, exception_message); - } - } - - exception_messages += "\n" + exception_message; } + + exception_messages += "\n" + exception_message; } + } - if (!cached_columns.empty()) - return cached_columns; + if (!cached_columns.empty()) + return cached_columns; - if (names_and_types.empty()) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "All attempts to extract table structure from files failed. " - "Errors:{}\nYou can specify the structure manually", exception_messages); + if (names_and_types.empty()) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "All attempts to extract table structure from files failed. " + "Errors:{}\nYou can specify the structure manually", exception_messages); - /// If we have "INSERT SELECT" query then try to order - /// columns as they are ordered in table schema for formats - /// without strict column order (like JSON and TSKV). - /// It will allow to execute simple data loading with query - /// "INSERT INTO table SELECT * FROM ..." - const auto &insertion_table = context->getInsertionTable(); - if (!schema_reader->hasStrictOrderOfColumns() && !insertion_table.empty()) { - auto storage = DatabaseCatalog::instance().getTable(insertion_table, context); - auto metadata = storage->getInMemoryMetadataPtr(); - auto names_in_storage = metadata->getColumns().getNamesOfPhysical(); - auto ordered_list = getOrderedColumnsList(names_and_types, names_in_storage); - if (ordered_list) - names_and_types = *ordered_list; - } - } else - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "{} file format doesn't support schema inference. You must specify the structure manually", - format_name); - names_and_types.erase(std::remove_if(names_and_types.begin(), names_and_types.end(), - [](const NameAndTypePair &pair) { return pair.name.empty(); }), - names_and_types.end()); - return ColumnsDescription(names_and_types); + /// If we have "INSERT SELECT" query then try to order + /// columns as they are ordered in table schema for formats + /// without strict column order (like JSON and TSKV). + /// It will allow to execute simple data loading with query + /// "INSERT INTO table SELECT * FROM ..." + const auto & insertion_table = context->getInsertionTable(); + if (!schema_reader->hasStrictOrderOfColumns() && !insertion_table.empty()) + { + auto storage = DatabaseCatalog::instance().getTable(insertion_table, context); + auto metadata = storage->getInMemoryMetadataPtr(); + auto names_in_storage = metadata->getColumns().getNamesOfPhysical(); + auto ordered_list = getOrderedColumnsList(names_and_types, names_in_storage); + if (ordered_list) + names_and_types = *ordered_list; + } } + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "{} file format doesn't support schema inference. You must specify the structure manually", + format_name); + names_and_types.erase(std::remove_if(names_and_types.begin(), names_and_types.end(), + [](const NameAndTypePair& pair) { return pair.name.empty(); }), names_and_types.end()); + return ColumnsDescription(names_and_types); +} - ColumnsDescription - readSchemaFromFormat(const String &format_name, const std::optional &format_settings, - ReadBufferIterator &read_buffer_iterator, bool retry, ContextPtr &context) { - std::unique_ptr buf_out; - return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out); - } +ColumnsDescription readSchemaFromFormat(const String & format_name, const std::optional & format_settings, ReadBufferIterator & read_buffer_iterator, bool retry, ContextPtr & context) +{ + std::unique_ptr buf_out; + return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out); +} - SchemaCache::Key getKeyForSchemaCache(const String &source, const String &format, - const std::optional &format_settings, - const ContextPtr &context) { - return getKeysForSchemaCache({source}, format, format_settings, context).front(); - } +SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional & format_settings, const ContextPtr & context) +{ + return getKeysForSchemaCache({source}, format, format_settings, context).front(); +} - static SchemaCache::Key - makeSchemaCacheKey(const String &source, const String &format, const String &additional_format_info) { - return SchemaCache::Key{source, format, additional_format_info}; - } +static SchemaCache::Key makeSchemaCacheKey(const String & source, const String & format, const String & additional_format_info) +{ + return SchemaCache::Key{source, format, additional_format_info}; +} - SchemaCache::Keys getKeysForSchemaCache(const Strings &sources, const String &format, - const std::optional &format_settings, - const ContextPtr &context) { - /// For some formats data schema depends on some settings, so it's possible that - /// two queries to the same source will get two different schemas. To process this - /// case we add some additional information specific for the format to the cache key. - /// For example, for Protobuf format additional information is the path to the schema - /// and message name. - String additional_format_info = FormatFactory::instance().getAdditionalInfoForSchemaCache(format, context, - format_settings); - SchemaCache::Keys cache_keys; - cache_keys.reserve(sources.size()); - std::transform(sources.begin(), sources.end(), std::back_inserter(cache_keys), - [&](const auto &source) { return makeSchemaCacheKey(source, format, additional_format_info); }); - return cache_keys; - } +SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, const String & format, const std::optional & format_settings, const ContextPtr & context) +{ + /// For some formats data schema depends on some settings, so it's possible that + /// two queries to the same source will get two different schemas. To process this + /// case we add some additional information specific for the format to the cache key. + /// For example, for Protobuf format additional information is the path to the schema + /// and message name. + String additional_format_info = FormatFactory::instance().getAdditionalInfoForSchemaCache(format, context, format_settings); + SchemaCache::Keys cache_keys; + cache_keys.reserve(sources.size()); + std::transform(sources.begin(), sources.end(), std::back_inserter(cache_keys), [&](const auto & source){ return makeSchemaCacheKey(source, format, additional_format_info); }); + return cache_keys; +} } From b869572a5411ce22519152a9a650d7fa65c8c517 Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Sat, 1 Apr 2023 15:20:26 +0800 Subject: [PATCH 030/308] reformat code --- src/Formats/ReadSchemaUtils.cpp | 81 +++++++++++++++++++++------------ 1 file changed, 51 insertions(+), 30 deletions(-) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 7f1c3bf3e63..d185b938ed6 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -1,13 +1,13 @@ -#include #include -#include -#include #include +#include +#include +#include #include -#include -#include #include +#include #include +#include namespace DB { @@ -20,8 +20,7 @@ namespace ErrorCodes extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } -static std::optional getOrderedColumnsList( - const NamesAndTypesList & columns_list, const Names & columns_order_hint) +static std::optional getOrderedColumnsList(const NamesAndTypesList & columns_list, const Names & columns_order_hint) { if (columns_list.size() != columns_order_hint.size()) return {}; @@ -65,7 +64,8 @@ ColumnsDescription readSchemaFromFormat( } catch (Exception & e) { - e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); + e.addMessage( + fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); throw; } } @@ -73,7 +73,8 @@ ColumnsDescription readSchemaFromFormat( { std::string exception_messages; SchemaReaderPtr schema_reader; - size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; + size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference + : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; size_t iterations = 0; ColumnsDescription cached_columns; while (true) @@ -88,8 +89,8 @@ ColumnsDescription readSchemaFromFormat( } catch (Exception & e) { - e.addMessage(fmt::format( - "Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); + e.addMessage( + fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); throw; } catch (...) @@ -109,7 +110,8 @@ ColumnsDescription readSchemaFromFormat( auto exception_message = fmt::format("Cannot extract table structure from {} format file, file is empty", format_name); if (!retry) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message); + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message); exception_messages += "\n" + exception_message; continue; @@ -132,7 +134,8 @@ ColumnsDescription readSchemaFromFormat( max_rows_to_read -= schema_reader->getNumRowsRead(); if (rows_read != 0 && max_rows_to_read == 0) { - exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; + exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting " + "input_format_max_rows_to_read_for_schema_inference"; if (iterations > 1) { exception_messages += "\n" + exception_message; @@ -150,15 +153,18 @@ ColumnsDescription readSchemaFromFormat( } catch (Exception & e) { - e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); + e.addMessage(fmt::format( + "Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); throw; } catch (...) { - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file. " - "Error: {}. You can specify the structure manually", - format_name, exception_message); + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file. " + "Error: {}. You can specify the structure manually", + format_name, + exception_message); } } @@ -170,9 +176,11 @@ ColumnsDescription readSchemaFromFormat( return cached_columns; if (names_and_types.empty()) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "All attempts to extract table structure from files failed. " - "Errors:{}\nYou can specify the structure manually", exception_messages); + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "All attempts to extract table structure from files failed. " + "Errors:{}\nYou can specify the structure manually", + exception_messages); /// If we have "INSERT SELECT" query then try to order /// columns as they are ordered in table schema for formats @@ -191,21 +199,29 @@ ColumnsDescription readSchemaFromFormat( } } else - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "{} file format doesn't support schema inference. You must specify the structure manually", - format_name); - names_and_types.erase(std::remove_if(names_and_types.begin(), names_and_types.end(), - [](const NameAndTypePair& pair) { return pair.name.empty(); }), names_and_types.end()); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "{} file format doesn't support schema inference. You must specify the structure manually", + format_name); + names_and_types.erase( + std::remove_if(names_and_types.begin(), names_and_types.end(), [](const NameAndTypePair & pair) { return pair.name.empty(); }), + names_and_types.end()); return ColumnsDescription(names_and_types); } -ColumnsDescription readSchemaFromFormat(const String & format_name, const std::optional & format_settings, ReadBufferIterator & read_buffer_iterator, bool retry, ContextPtr & context) +ColumnsDescription readSchemaFromFormat( + const String & format_name, + const std::optional & format_settings, + ReadBufferIterator & read_buffer_iterator, + bool retry, + ContextPtr & context) { std::unique_ptr buf_out; return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out); } -SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional & format_settings, const ContextPtr & context) +SchemaCache::Key getKeyForSchemaCache( + const String & source, const String & format, const std::optional & format_settings, const ContextPtr & context) { return getKeysForSchemaCache({source}, format, format_settings, context).front(); } @@ -215,7 +231,8 @@ static SchemaCache::Key makeSchemaCacheKey(const String & source, const String & return SchemaCache::Key{source, format, additional_format_info}; } -SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, const String & format, const std::optional & format_settings, const ContextPtr & context) +SchemaCache::Keys getKeysForSchemaCache( + const Strings & sources, const String & format, const std::optional & format_settings, const ContextPtr & context) { /// For some formats data schema depends on some settings, so it's possible that /// two queries to the same source will get two different schemas. To process this @@ -225,7 +242,11 @@ SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, const String & String additional_format_info = FormatFactory::instance().getAdditionalInfoForSchemaCache(format, context, format_settings); SchemaCache::Keys cache_keys; cache_keys.reserve(sources.size()); - std::transform(sources.begin(), sources.end(), std::back_inserter(cache_keys), [&](const auto & source){ return makeSchemaCacheKey(source, format, additional_format_info); }); + std::transform( + sources.begin(), + sources.end(), + std::back_inserter(cache_keys), + [&](const auto & source) { return makeSchemaCacheKey(source, format, additional_format_info); }); return cache_keys; } From 892e436046f3f7bb135c5df8b18a1951833dd29f Mon Sep 17 00:00:00 2001 From: pufit Date: Sun, 2 Apr 2023 16:51:10 -0400 Subject: [PATCH 031/308] Move host and port to options --- programs/keeper-client/KeeperClient.cpp | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 54b5cf1d6cf..52a31a388cc 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -85,9 +85,19 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) Poco::Util::Application::defineOptions(options); options.addOption( - Poco::Util::Option("help", "h", "show help and exit") + Poco::Util::Option("help", "", "show help and exit") .binding("help")); + options.addOption( + Poco::Util::Option("host", "h", "server hostname. default `localhost`") + .argument("host") + .binding("host")); + + options.addOption( + Poco::Util::Option("port", "p", "server port. default `2181`") + .argument("port") + .binding("port")); + options.addOption( Poco::Util::Option("query", "q", "will execute given query, then exit.") .argument("query") @@ -284,13 +294,12 @@ void KeeperClient::runInteractive() } } -int KeeperClient::main(const std::vector & args) +int KeeperClient::main(const std::vector & /* args */) { - if (args.empty()) - zk_args.hosts = {"localhost:2181"}; - else - zk_args.hosts = {args[0]}; + auto host = config().getString("host", "localhost"); + auto port = config().getString("port", "2181"); + zk_args.hosts = {host + ":" + port}; zk_args.connection_timeout_ms = config().getInt("connection-timeout", 10) * 1000; zk_args.session_timeout_ms = config().getInt("session-timeout", 10) * 1000; zk_args.operation_timeout_ms = config().getInt("operation-timeout", 10) * 1000; From f087f0e87733eeb2672dc989e792f3c85d462601 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 11 Apr 2023 14:18:16 +0200 Subject: [PATCH 032/308] Update src/Formats/ReadSchemaUtils.cpp --- src/Formats/ReadSchemaUtils.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index d185b938ed6..f80d9ee41d2 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -203,6 +203,7 @@ ColumnsDescription readSchemaFromFormat( ErrorCodes::BAD_ARGUMENTS, "{} file format doesn't support schema inference. You must specify the structure manually", format_name); + /// Some formats like CSVWithNames can contain empty column names. We don't support empty column names and futher processing can fail with an exception. Let's just remove columns with empty names from the structure. names_and_types.erase( std::remove_if(names_and_types.begin(), names_and_types.end(), [](const NameAndTypePair & pair) { return pair.name.empty(); }), names_and_types.end()); From 029c92344b2db5b5129b7adb39eb0b5f09addb42 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Apr 2023 16:57:43 +0200 Subject: [PATCH 033/308] Fix possible terminate called for uncaught exception in Connection --- src/Client/Connection.cpp | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index d39148d3016..3dd78afb79b 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -232,12 +232,27 @@ void Connection::disconnect() maybe_compressed_out = nullptr; in = nullptr; last_input_packet_type.reset(); - out = nullptr; // can write to socket + std::exception_ptr finalize_exception; + try + { + // finalize() can write to socket and throw an exception. + out->finalize(); + } + catch (...) + { + /// Don't throw an exception here, it will leave Connection in invalid state. + finalize_exception = std::current_exception(); + } + out = nullptr; + if (socket) socket->close(); socket = nullptr; connected = false; nonce.reset(); + + if (finalize_exception) + std::rethrow_exception(finalize_exception); } From bc0c431eb77d7844429b70da58169dc49d5de4a7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Apr 2023 17:03:48 +0200 Subject: [PATCH 034/308] Fix possible terminate called for uncaught exception in InterserverIOHTTPHandler::handleRequest --- src/Server/InterserverIOHTTPHandler.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index 6b0cd543053..f7128e7e5a3 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -93,10 +93,13 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe auto write_response = [&](const std::string & message) { - if (response.sent()) - return; - auto & out = *used_output.out; + if (response.sent()) + { + out.finalize(); + return; + } + try { writeString(message, out); @@ -127,7 +130,10 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe catch (Exception & e) { if (e.code() == ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES) + { + used_output.out->finalize(); return; + } response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); From c503f6532c19ed98a119328c89a9d6e282767332 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 24 Apr 2023 15:11:36 +0000 Subject: [PATCH 035/308] Add more finalize() to avoid terminate --- src/Storages/HDFS/StorageHDFS.cpp | 1 + src/Storages/StorageFile.cpp | 1 + src/Storages/StorageS3.cpp | 1 + src/Storages/StorageURL.cpp | 1 + 4 files changed, 4 insertions(+) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index c915213f4ac..f4dd26435b3 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -490,6 +490,7 @@ private: { /// Stop ParallelFormattingOutputFormat correctly. writer.reset(); + write_buf->finalize(); throw; } } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7e5a93c13c1..dafb51509ea 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -928,6 +928,7 @@ private: { /// Stop ParallelFormattingOutputFormat correctly. writer.reset(); + write_buf->finalize(); throw; } } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index bd3e8fe886d..b0fde7c8e02 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -823,6 +823,7 @@ private: { /// Stop ParallelFormattingOutputFormat correctly. writer.reset(); + write_buf->finalize(); throw; } } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index d2df3881c71..00b4c174834 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -442,6 +442,7 @@ void StorageURLSink::finalize() { /// Stop ParallelFormattingOutputFormat correctly. writer.reset(); + write_buf->finalize(); throw; } } From 57801b7a02b5574c1d385deb38d17846780389ac Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Apr 2023 19:06:45 +0200 Subject: [PATCH 036/308] Fix --- src/Client/Connection.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 3dd78afb79b..e328d0c4e43 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -236,7 +236,8 @@ void Connection::disconnect() try { // finalize() can write to socket and throw an exception. - out->finalize(); + if (out) + out->finalize(); } catch (...) { From c693c1bd17b62d9bc1bcb969867721415b4cb0a8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Apr 2023 19:07:30 +0200 Subject: [PATCH 037/308] Fix style --- src/Server/InterserverIOHTTPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index f7128e7e5a3..ea71d954cc0 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -99,7 +99,7 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe out.finalize(); return; } - + try { writeString(message, out); From 8bef8fc1de5acf9910f83b978c8b91768da7f670 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Apr 2023 00:54:28 -0400 Subject: [PATCH 038/308] Parsing Keeper commands via ClickHouse Parser --- programs/keeper-client/CMakeLists.txt | 2 +- programs/keeper-client/Commands.cpp | 184 ++++++++++++++++++++++++ programs/keeper-client/Commands.h | 131 +++++++++++++++++ programs/keeper-client/KeeperClient.cpp | 122 +++++----------- programs/keeper-client/KeeperClient.h | 39 ++--- programs/keeper-client/Parser.cpp | 94 ++++++++++++ programs/keeper-client/Parser.h | 36 +++++ src/Parsers/TokenIterator.cpp | 4 +- src/Parsers/TokenIterator.h | 2 +- src/Parsers/parseQuery.cpp | 5 +- src/Parsers/parseQuery.h | 3 +- 11 files changed, 514 insertions(+), 108 deletions(-) create mode 100644 programs/keeper-client/Commands.cpp create mode 100644 programs/keeper-client/Commands.h create mode 100644 programs/keeper-client/Parser.cpp create mode 100644 programs/keeper-client/Parser.h diff --git a/programs/keeper-client/CMakeLists.txt b/programs/keeper-client/CMakeLists.txt index 06055d6d820..f54892fe559 100644 --- a/programs/keeper-client/CMakeLists.txt +++ b/programs/keeper-client/CMakeLists.txt @@ -1,4 +1,4 @@ -set (CLICKHOUSE_KEEPER_CLIENT_SOURCES KeeperClient.cpp) +set (CLICKHOUSE_KEEPER_CLIENT_SOURCES KeeperClient.cpp Parser.cpp Commands.cpp) set (CLICKHOUSE_KEEPER_CLIENT_LINK PRIVATE diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp new file mode 100644 index 00000000000..a21550e969d --- /dev/null +++ b/programs/keeper-client/Commands.cpp @@ -0,0 +1,184 @@ + +#include "Commands.h" +#include "KeeperClient.h" + + +namespace DB +{ + +bool LSCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return true; + + node->args.push_back(std::move(arg)); + return true; +} + +void LSCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + String path; + if (!query->args.empty()) + path = client->getAbsolutePath(query->args[0].safeGet()); + else + path = client->cwd; + + const auto children = client->zookeeper->getChildren(path); + for (const auto & child : children) + std::cout << child << " "; + std::cout << "\n"; +} + +bool CDCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return true; + + node->args.push_back(std::move(arg)); + return true; +} + +void CDCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + if (!query->args.empty()) + return; + + auto new_path = client->getAbsolutePath(query->args[0].safeGet()); + if (!client->zookeeper->exists(new_path)) + std::cerr << "Path " << new_path << " does not exists\n"; + else + client->cwd = new_path; +} + +bool SetCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + if (!parseKeeperArg(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + ASTPtr version; + if (ParserNumber{}.parse(pos, version, expected)) + node->args.push_back(version->as().value); + + return true; +} + +void SetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + if (query->args.size() == 2) + client->zookeeper->set(client->getAbsolutePath(query->args[0].safeGet()), query->args[1].safeGet()); + else + client->zookeeper->set( + client->getAbsolutePath(query->args[0].safeGet()), + query->args[1].safeGet(), + static_cast(query->args[2].safeGet())); +} + +bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + if (!parseKeeperArg(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + return true; +} + +void CreateCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + client->zookeeper->create( + client->getAbsolutePath(query->args[0].safeGet()), + query->args[1].safeGet(), + zkutil::CreateMode::Persistent); +} + +bool GetCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + return true; +} + +void GetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + std::cout << client->zookeeper->get(client->getAbsolutePath(query->args[0].safeGet())) << "\n"; +} + +bool RMCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + return true; +} + +void RMCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + client->zookeeper->remove(client->getAbsolutePath(query->args[0].safeGet())); +} + +bool RMRCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + return true; +} + +void RMRCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + String path = client->getAbsolutePath(query->args[0].safeGet()); + client->askConfirmation("You are going to recursively delete path " + path, + [client, path]{ client->zookeeper->removeRecursive(path); }); +} + +bool HelpCommand::parse(IParser::Pos & /* pos */, std::shared_ptr & /* node */, Expected & /* expected */) const +{ + return true; +} + +void HelpCommand::execute(const ASTKeeperQuery * /* query */, KeeperClient * /* client */) const +{ + for (const auto & pair : KeeperClient::commands) + std::cout << pair.second->getHelpMessage() << '\n'; +} + +bool FourLetterWordCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + expected.add(pos, "four-letter-word command"); + if (pos->type != TokenType::BareWord) + return false; + + String cmd(pos->begin, pos->end); + if (cmd.size() != 4) + return false; + + ++pos; + node->args.push_back(std::move(cmd)); + return true; +} + +void FourLetterWordCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + std::cout << client->executeFourLetterCommand(query->args[0].safeGet()) << "\n"; +} + +} diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h new file mode 100644 index 00000000000..e4debd53e42 --- /dev/null +++ b/programs/keeper-client/Commands.h @@ -0,0 +1,131 @@ +#pragma once + +#include "Parser.h" + +namespace DB +{ + +class KeeperClient; + +class IKeeperClientCommand +{ +public: + static const String name; + + virtual bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const = 0; + + virtual void execute(const ASTKeeperQuery * query, KeeperClient * client) const = 0; + + virtual String getHelpMessage() const = 0; + + virtual String getName() const = 0; + + virtual ~IKeeperClientCommand() = default; +}; + +using Command = std::shared_ptr; + + +class LSCommand : public IKeeperClientCommand +{ + String getName() const override { return "ls"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "ls [path] -- Lists the nodes for the given path (default: cwd)"; } +}; + +class CDCommand : public IKeeperClientCommand +{ + String getName() const override { return "cd"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "cd [path] -- Change the working path (default `.`)"; } +}; + +class SetCommand : public IKeeperClientCommand +{ + String getName() const override { return "set"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override + { + return "set [version] -- Updates the node's value. Only update if version matches (default: -1)"; + } +}; + +class CreateCommand : public IKeeperClientCommand +{ + String getName() const override { return "create"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "create -- Creates new node"; } +}; + +class GetCommand : public IKeeperClientCommand +{ + String getName() const override { return "get"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "get -- Returns the node's value"; } +}; + +class RMCommand : public IKeeperClientCommand +{ + String getName() const override { return "rm"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "remove -- Remove the node"; } +}; + +class RMRCommand : public IKeeperClientCommand +{ + String getName() const override { return "rmr"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "rmr -- Recursively deletes path. Confirmation required"; } +}; + +class HelpCommand : public IKeeperClientCommand +{ + String getName() const override { return "help"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "help -- Prints this message"; } +}; + +class FourLetterWordCommand : public IKeeperClientCommand +{ + String getName() const override { return "flwc"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "flwc -- Executes four-letter-word command"; } +}; + +} diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 52a31a388cc..92aa822231d 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -1,9 +1,11 @@ #include "KeeperClient.h" +#include "Commands.h" #include #include #include #include #include +#include #include @@ -13,13 +15,6 @@ namespace fs = std::filesystem; namespace DB { -static const NameSet four_letter_word_commands -{ - "ruok", "mntr", "srvr", "stat", "srst", "conf", - "cons", "crst", "envi", "dirs", "isro", "wchs", - "wchc", "wchp", "dump", "csnp", "lgif", "rqld", -}; - namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -54,7 +49,7 @@ void KeeperClient::askConfirmation(const String & prompt, std::function confirmation_callback = callback; } -String KeeperClient::getAbsolutePath(const String & relative) +String KeeperClient::getAbsolutePath(const String & relative) const { String result; if (relative.starts_with('/')) @@ -68,16 +63,20 @@ String KeeperClient::getAbsolutePath(const String & relative) return result; } -void KeeperClient::loadCommands(std::vector> && new_commands) +void KeeperClient::loadCommands(std::vector && new_commands) { - for (const auto & [name, args_count, callback] : new_commands) + std::vector suggestions; + for (const auto & command : new_commands) { - commands.insert({{name, args_count}, callback}); - suggest.addWords({name}); + String name = command->getName(); + commands.insert({name, command}); + suggestions.push_back(std::move(name)); } for (const auto & command : four_letter_word_commands) - suggest.addWords({command}); + suggestions.push_back(command); + + suggest.addWords(std::move(suggestions)); } void KeeperClient::defineOptions(Poco::Util::OptionSet & options) @@ -132,61 +131,15 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) void KeeperClient::initialize(Poco::Util::Application & /* self */) { loadCommands({ - {"set", 2, [](KeeperClient * client, const std::vector & args) - { - client->zookeeper->set(client->getAbsolutePath(args[1]), args[2]); - }}, - - {"create", 2, [](KeeperClient * client, const std::vector & args) - { - client->zookeeper->create(client->getAbsolutePath(args[1]), args[2], zkutil::CreateMode::Persistent); - }}, - - {"get", 1, [](KeeperClient * client, const std::vector & args) - { - std::cout << client->zookeeper->get(client->getAbsolutePath(args[1])) << "\n"; - }}, - - {"ls", 0, [](KeeperClient * client, const std::vector & /* args */) - { - auto children = client->zookeeper->getChildren(client->cwd); - for (auto & child : children) - std::cout << child << " "; - std::cout << "\n"; - }}, - - {"ls", 1, [](KeeperClient * client, const std::vector & args) - { - auto children = client->zookeeper->getChildren(client->getAbsolutePath(args[1])); - for (auto & child : children) - std::cout << child << " "; - std::cout << "\n"; - }}, - - {"cd", 0, [](KeeperClient * /* client */, const std::vector & /* args */) - { - }}, - - {"cd", 1, [](KeeperClient * client, const std::vector & args) - { - auto new_path = client->getAbsolutePath(args[1]); - if (!client->zookeeper->exists(new_path)) - std::cerr << "Path " << new_path << " does not exists\n"; - else - client->cwd = new_path; - }}, - - {"rm", 1, [](KeeperClient * client, const std::vector & args) - { - client->zookeeper->remove(client->getAbsolutePath(args[1])); - }}, - - {"rmr", 1, [](KeeperClient * client, const std::vector & args) - { - String path = client->getAbsolutePath(args[1]); - client->askConfirmation("You are going to recursively delete path " + path, - [client, path]{ client->zookeeper->removeRecursive(path); }); - }}, + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), }); String home_path; @@ -234,32 +187,31 @@ bool KeeperClient::processQueryText(const String & text) if (exit_strings.find(text) != exit_strings.end()) return false; - std::vector tokens; - boost::algorithm::split(tokens, text, boost::is_any_of(" ")); - try { if (need_confirmation) { need_confirmation = false; - if (tokens.size() == 1 && (tokens[0] == "y" || tokens[0] == "Y")) + if (text.size() == 1 && (text == "y" || text == "Y")) confirmation_callback(); + return true; } - else if (tokens.size() == 1 && tokens[0].size() == 4 && four_letter_word_commands.find(tokens[0]) != four_letter_word_commands.end()) - std::cout << executeFourLetterCommand(tokens[0]) << "\n"; - else + + KeeperParser parser; + String message; + const char * begin = text.data(); + ASTPtr res = tryParseQuery(parser, begin, begin + text.size(), message, true, "", false, 0, 0, false); + + if (!res) { - auto callback = commands.find({tokens[0], tokens.size() - 1}); - if (callback == commands.end()) - { - if (tokens[0].size() == 4 && tokens.size() == 1) /// Treat it like unrecognized four-letter command - std::cout << executeFourLetterCommand(tokens[0]) << "\n"; - else - std::cerr << "No command found with name " << tokens[0] << " and args count " << tokens.size() - 1 << "\n"; - } - else - callback->second(this, tokens); + std::cerr << message << "\n"; + return true; } + + auto * query = res->as(); + + auto command = KeeperClient::commands.find(query->command); + command->second->execute(query, this); } catch (Coordination::Exception & err) { diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h index 50a8b35f6c5..0297491bd28 100644 --- a/programs/keeper-client/KeeperClient.h +++ b/programs/keeper-client/KeeperClient.h @@ -1,10 +1,12 @@ #pragma once - +#include "Parser.h" +#include "Commands.h" #include #include #include #include +#include #include #include #include @@ -13,13 +15,16 @@ namespace DB { -class KeeperClient; +static const NameSet four_letter_word_commands + { + "ruok", "mntr", "srvr", "stat", "srst", "conf", + "cons", "crst", "envi", "dirs", "isro", "wchs", + "wchc", "wchp", "dump", "csnp", "lgif", "rqld", + }; class KeeperClient: public Poco::Util::Application { public: - using Callback = std::function &)>; - KeeperClient() = default; void initialize(Poco::Util::Application & self) override; @@ -28,29 +33,31 @@ public: void defineOptions(Poco::Util::OptionSet & options) override; -protected: - void runInteractive(); - void loadCommands(std::vector> && new_commands); - bool processQueryText(const String & text); - void executeQuery(const String & query); + String getAbsolutePath(const String & relative) const; + + void askConfirmation(const String & prompt, std::function && callback); String executeFourLetterCommand(const String & command); - String getAbsolutePath(const String & relative); - void askConfirmation(const String & prompt, std::function && callback); + zkutil::ZooKeeperPtr zookeeper; + std::filesystem::path cwd = "/"; + std::function confirmation_callback; - std::map, Callback> commands; + inline static std::map commands; + +protected: + void runInteractive(); + bool processQueryText(const String & text); + void executeQuery(const String & query); + + void loadCommands(std::vector && new_commands); String history_file; LineReader::Suggest suggest; - zkutil::ZooKeeperPtr zookeeper; zkutil::ZooKeeperArgs zk_args; - std::filesystem::path cwd = "/"; - bool need_confirmation = false; - std::function confirmation_callback; }; } diff --git a/programs/keeper-client/Parser.cpp b/programs/keeper-client/Parser.cpp new file mode 100644 index 00000000000..0f3fc39704e --- /dev/null +++ b/programs/keeper-client/Parser.cpp @@ -0,0 +1,94 @@ +#include "Parser.h" +#include "KeeperClient.h" + + +namespace DB +{ + +bool parseKeeperArg(IParser::Pos & pos, Expected & expected, String & result) +{ + expected.add(pos, getTokenName(TokenType::BareWord)); + + if (pos->type == TokenType::BareWord) + { + result = String(pos->begin, pos->end); + ++pos; + ParserToken{TokenType::Whitespace}.ignore(pos); + return true; + } + + bool status = parseIdentifierOrStringLiteral(pos, expected, result); + ParserToken{TokenType::Whitespace}.ignore(pos); + return status; +} + +bool parseKeeperPath(IParser::Pos & pos, Expected & expected, String & path) +{ + expected.add(pos, "path"); + + if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral) + return parseIdentifierOrStringLiteral(pos, expected, path); + + String result; + while (pos->type == TokenType::BareWord || pos->type == TokenType::Slash || pos->type == TokenType::Dot) + { + result.append(pos->begin, pos->end); + ++pos; + } + ParserToken{TokenType::Whitespace}.ignore(pos); + + if (result.empty()) + return false; + + path = result; + return true; +} + +bool KeeperParser::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + auto query = std::make_shared(); + + for (const auto & pair : KeeperClient::commands) + expected.add(pos, pair.first.data()); + + for (const auto & flwc : four_letter_word_commands) + expected.add(pos, flwc.data()); + + if (pos->type != TokenType::BareWord) + return false; + + String command_name(pos->begin, pos->end); + Command command; + + auto iter = KeeperClient::commands.find(command_name); + if (iter == KeeperClient::commands.end()) + { + if (command_name.size() == 4) + { + /// Treat it like four-letter command + /// Since keeper server can potentially have different version we don't want to match this command with embedded list + command = std::make_shared(); + command_name = command->getName(); + /// We also don't move the position, so the command will be parsed as an argument + } + else + return false; + } + else + { + command = iter->second; + ++pos; + ParserToken{TokenType::Whitespace}.ignore(pos); + } + + query->command = command_name; + if (!command->parse(pos, query, expected)) + return false; + + ParserToken{TokenType::Whitespace}.ignore(pos); + + node = query; + return true; +} + +} diff --git a/programs/keeper-client/Parser.h b/programs/keeper-client/Parser.h new file mode 100644 index 00000000000..57ee6ce4a18 --- /dev/null +++ b/programs/keeper-client/Parser.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ + +bool parseKeeperArg(IParser::Pos & pos, Expected & expected, String & result); + +bool parseKeeperPath(IParser::Pos & pos, Expected & expected, String & path); + + +class ASTKeeperQuery : public IAST +{ +public: + String getID(char) const override { return "KeeperQuery"; } + ASTPtr clone() const override { return std::make_shared(*this); } + + String command; + std::vector args; +}; + +class KeeperParser : public IParserBase +{ +protected: + const char * getName() const override { return "Keeper client query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/TokenIterator.cpp b/src/Parsers/TokenIterator.cpp index 6633ddb9563..6b798f6f576 100644 --- a/src/Parsers/TokenIterator.cpp +++ b/src/Parsers/TokenIterator.cpp @@ -4,7 +4,7 @@ namespace DB { -Tokens::Tokens(const char * begin, const char * end, size_t max_query_size) +Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool skipp_insignificant) { Lexer lexer(begin, end, max_query_size); @@ -13,7 +13,7 @@ Tokens::Tokens(const char * begin, const char * end, size_t max_query_size) { Token token = lexer.nextToken(); stop = token.isEnd() || token.type == TokenType::ErrorMaxQuerySizeExceeded; - if (token.isSignificant()) + if (token.isSignificant() || (!skipp_insignificant && !data.empty() && data.back().isSignificant())) data.emplace_back(std::move(token)); } while (!stop); } diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index c9ac61dfef9..31cb644d879 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -24,7 +24,7 @@ private: std::size_t last_accessed_index = 0; public: - Tokens(const char * begin, const char * end, size_t max_query_size = 0); + Tokens(const char * begin, const char * end, size_t max_query_size = 0, bool skipp_insignificant = true); ALWAYS_INLINE inline const Token & operator[](size_t index) { diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 8d794409f78..9f688f204a2 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -233,10 +233,11 @@ ASTPtr tryParseQuery( const std::string & query_description, bool allow_multi_statements, size_t max_query_size, - size_t max_parser_depth) + size_t max_parser_depth, + bool skipp_insignificant) { const char * query_begin = _out_query_end; - Tokens tokens(query_begin, all_queries_end, max_query_size); + Tokens tokens(query_begin, all_queries_end, max_query_size, skipp_insignificant); /// NOTE: consider use UInt32 for max_parser_depth setting. IParser::Pos token_iterator(tokens, static_cast(max_parser_depth)); diff --git a/src/Parsers/parseQuery.h b/src/Parsers/parseQuery.h index cc077bbdab2..30f43261103 100644 --- a/src/Parsers/parseQuery.h +++ b/src/Parsers/parseQuery.h @@ -18,7 +18,8 @@ ASTPtr tryParseQuery( bool allow_multi_statements, /// If false, check for non-space characters after semicolon and set error message if any. size_t max_query_size, /// If (end - pos) > max_query_size and query is longer than max_query_size then throws "Max query size exceeded". /// Disabled if zero. Is used in order to check query size if buffer can contains data for INSERT query. - size_t max_parser_depth); + size_t max_parser_depth, + bool skipp_insignificant = true); /// If true, lexer will skip all insignificant tokens (e.g. whitespaces) /// Parse query or throw an exception with error message. From a20f8e74a0cbf0b72ee657b3c6c302d16c61952e Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Apr 2023 01:05:51 -0400 Subject: [PATCH 039/308] Fix `cd` command. Style consistency --- programs/keeper-client/Commands.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index a21550e969d..7b7c4670b7d 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -42,7 +42,7 @@ bool CDCommand::parse(IParser::Pos & pos, std::shared_ptr & node void CDCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const { - if (!query->args.empty()) + if (query->args.empty()) return; auto new_path = client->getAbsolutePath(query->args[0].safeGet()); @@ -158,7 +158,7 @@ bool HelpCommand::parse(IParser::Pos & /* pos */, std::shared_ptrgetHelpMessage() << '\n'; + std::cout << pair.second->getHelpMessage() << "\n"; } bool FourLetterWordCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const From bacba6e34735be07a37283224326172aa1e9a71b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 26 Apr 2023 12:18:12 +0200 Subject: [PATCH 040/308] Fix typo --- src/Formats/ReadSchemaUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index f80d9ee41d2..61683b226ee 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -203,7 +203,7 @@ ColumnsDescription readSchemaFromFormat( ErrorCodes::BAD_ARGUMENTS, "{} file format doesn't support schema inference. You must specify the structure manually", format_name); - /// Some formats like CSVWithNames can contain empty column names. We don't support empty column names and futher processing can fail with an exception. Let's just remove columns with empty names from the structure. + /// Some formats like CSVWithNames can contain empty column names. We don't support empty column names and further processing can fail with an exception. Let's just remove columns with empty names from the structure. names_and_types.erase( std::remove_if(names_and_types.begin(), names_and_types.end(), [](const NameAndTypePair & pair) { return pair.name.empty(); }), names_and_types.end()); From 2426c445b0f17a0c98be86463efda8bd552d18de Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Apr 2023 20:10:32 -0400 Subject: [PATCH 041/308] Improve `set` command --- programs/keeper-client/Commands.cpp | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 7b7c4670b7d..05928a0d20b 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -24,8 +24,7 @@ void LSCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con else path = client->cwd; - const auto children = client->zookeeper->getChildren(path); - for (const auto & child : children) + for (const auto & child : client->zookeeper->getChildren(path)) std::cout << child << " "; std::cout << "\n"; } @@ -92,6 +91,19 @@ bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr & return false; node->args.push_back(std::move(arg)); + int mode = zkutil::CreateMode::Persistent; + + if (ParserKeyword{"PERSISTENT"}.ignore(pos, expected)) + mode = zkutil::CreateMode::Persistent; + else if (ParserKeyword{"EPHEMERAL"}.ignore(pos, expected)) + mode = zkutil::CreateMode::Ephemeral; + else if (ParserKeyword{"EPHEMERAL SEQUENTIAL"}.ignore(pos, expected)) + mode = zkutil::CreateMode::EphemeralSequential; + else if (ParserKeyword{"PERSISTENT SEQUENTIAL"}.ignore(pos, expected)) + mode = zkutil::CreateMode::PersistentSequential; + + node->args.push_back(mode); + return true; } @@ -100,7 +112,7 @@ void CreateCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) client->zookeeper->create( client->getAbsolutePath(query->args[0].safeGet()), query->args[1].safeGet(), - zkutil::CreateMode::Persistent); + static_cast(query->args[2].safeGet())); } bool GetCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const From 9204e2e3de85563ecc8d24ac356e608f95880c6b Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Apr 2023 20:11:45 -0400 Subject: [PATCH 042/308] Better completion --- programs/keeper-client/KeeperClient.cpp | 56 +++++++++++++++++++++---- programs/keeper-client/KeeperClient.h | 8 +++- src/Client/LineReader.cpp | 21 ++++++---- src/Client/LineReader.h | 5 +++ 4 files changed, 75 insertions(+), 15 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 92aa822231d..0ea4c4dde28 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -10,7 +10,6 @@ namespace po = boost::program_options; -namespace fs = std::filesystem; namespace DB { @@ -42,6 +41,48 @@ String KeeperClient::executeFourLetterCommand(const String & command) return result; } +std::vector KeeperClient::getCompletions(const String & prefix) const +{ + Tokens tokens(prefix.data(), prefix.data() + prefix.size(), 0, false); + IParser::Pos pos(tokens, 0); + + if (pos->type != TokenType::BareWord) + return registered_commands_and_four_letter_words; + + ++pos; + if (pos->isEnd()) + return registered_commands_and_four_letter_words; + + ParserToken{TokenType::Whitespace}.ignore(pos); + + std::vector result; + String string_path; + Expected expected; + if (!parseKeeperPath(pos, expected, string_path)) + string_path = cwd; + + if (!pos->isEnd()) + return result; + + fs::path path = string_path; + String parent_path; + if (string_path.ends_with("/")) + parent_path = getAbsolutePath(string_path); + else + parent_path = getAbsolutePath(path.parent_path()); + + try + { + for (const auto & child : zookeeper->getChildren(parent_path)) + result.push_back(child); + } + catch (Coordination::Exception &) {} + + std::sort(result.begin(), result.end()); + + return result; +} + void KeeperClient::askConfirmation(const String & prompt, std::function && callback) { std::cout << prompt << " Continue?\n"; @@ -49,7 +90,7 @@ void KeeperClient::askConfirmation(const String & prompt, std::function confirmation_callback = callback; } -String KeeperClient::getAbsolutePath(const String & relative) const +fs::path KeeperClient::getAbsolutePath(const String & relative) const { String result; if (relative.starts_with('/')) @@ -65,18 +106,17 @@ String KeeperClient::getAbsolutePath(const String & relative) const void KeeperClient::loadCommands(std::vector && new_commands) { - std::vector suggestions; for (const auto & command : new_commands) { String name = command->getName(); commands.insert({name, command}); - suggestions.push_back(std::move(name)); + registered_commands_and_four_letter_words.push_back(std::move(name)); } for (const auto & command : four_letter_word_commands) - suggestions.push_back(command); + registered_commands_and_four_letter_words.push_back(command); - suggest.addWords(std::move(suggestions)); + std::sort(registered_commands_and_four_letter_words.begin(), registered_commands_and_four_letter_words.end()); } void KeeperClient::defineOptions(Poco::Util::OptionSet & options) @@ -130,6 +170,9 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) void KeeperClient::initialize(Poco::Util::Application & /* self */) { + suggest.setCompletionsCallback( + [&](const String & prefix, size_t /* prefix_length */) { return getCompletions(prefix); }); + loadCommands({ std::make_shared(), std::make_shared(), @@ -248,7 +291,6 @@ void KeeperClient::runInteractive() int KeeperClient::main(const std::vector & /* args */) { - auto host = config().getString("host", "localhost"); auto port = config().getString("port", "2181"); zk_args.hosts = {host + ":" + port}; diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h index 0297491bd28..e7fa5cdc5fe 100644 --- a/programs/keeper-client/KeeperClient.h +++ b/programs/keeper-client/KeeperClient.h @@ -12,6 +12,8 @@ #include +namespace fs = std::filesystem; + namespace DB { @@ -33,7 +35,7 @@ public: void defineOptions(Poco::Util::OptionSet & options) override; - String getAbsolutePath(const String & relative) const; + fs::path getAbsolutePath(const String & relative) const; void askConfirmation(const String & prompt, std::function && callback); @@ -52,12 +54,16 @@ protected: void loadCommands(std::vector && new_commands); + std::vector getCompletions(const String & prefix) const; + String history_file; LineReader::Suggest suggest; zkutil::ZooKeeperArgs zk_args; bool need_confirmation = false; + + std::vector registered_commands_and_four_letter_words; }; } diff --git a/src/Client/LineReader.cpp b/src/Client/LineReader.cpp index 04b387c9f7d..f6cd7bffef7 100644 --- a/src/Client/LineReader.cpp +++ b/src/Client/LineReader.cpp @@ -81,17 +81,24 @@ replxx::Replxx::completions_t LineReader::Suggest::getCompletions(const String & std::lock_guard lock(mutex); + Words to_search; /// Only perform case sensitive completion when the prefix string contains any uppercase characters if (std::none_of(prefix.begin(), prefix.end(), [](char32_t x) { return iswupper(static_cast(x)); })) - range = std::equal_range( - words_no_case.begin(), words_no_case.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched) - { - return strncasecmp(s.data(), prefix_searched.data(), prefix_length) < 0; - }); + to_search = words_no_case; else - range = std::equal_range(words.begin(), words.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched) + to_search = words; + + if (custom_completions_callback) + { + auto new_words = custom_completions_callback(prefix, prefix_length); + assert(std::is_sorted(new_words.begin(), new_words.end())); + addNewWords(to_search, new_words, std::less{}); + } + + range = std::equal_range( + to_search.begin(), to_search.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched) { - return strncmp(s.data(), prefix_searched.data(), prefix_length) < 0; + return strncasecmp(s.data(), prefix_searched.data(), prefix_length) < 0; }); return replxx::Replxx::completions_t(range.first, range.second); diff --git a/src/Client/LineReader.h b/src/Client/LineReader.h index 321cf41b77e..7d053df1458 100644 --- a/src/Client/LineReader.h +++ b/src/Client/LineReader.h @@ -18,15 +18,20 @@ public: struct Suggest { using Words = std::vector; + using Callback = std::function; /// Get vector for the matched range of words if any. replxx::Replxx::completions_t getCompletions(const String & prefix, size_t prefix_length); void addWords(Words && new_words); + void setCompletionsCallback(Callback && callback) { custom_completions_callback = callback; } + private: Words words TSA_GUARDED_BY(mutex); Words words_no_case TSA_GUARDED_BY(mutex); + Callback custom_completions_callback = nullptr; + std::mutex mutex; }; From d235fdd5722c724fc7824ae0f7336c053b3d7084 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Apr 2023 20:39:33 -0400 Subject: [PATCH 043/308] Produce help message --- programs/keeper-client/KeeperClient.cpp | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 0ea4c4dde28..f38da1b72aa 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -6,11 +6,9 @@ #include #include #include -#include +#include -namespace po = boost::program_options; - namespace DB { @@ -291,6 +289,15 @@ void KeeperClient::runInteractive() int KeeperClient::main(const std::vector & /* args */) { + if (config().hasOption("help")) + { + Poco::Util::HelpFormatter help_formatter(KeeperClient::options()); + auto header_str = fmt::format("{} [OPTION]\n", commandName()); + help_formatter.setHeader(header_str); + help_formatter.format(std::cout); + return 0; + } + auto host = config().getString("host", "localhost"); auto port = config().getString("port", "2181"); zk_args.hosts = {host + ":" + port}; From de03b905709bb6da3196b1b76fcbe60c14652a6e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 27 Apr 2023 17:07:00 +0000 Subject: [PATCH 044/308] impl --- src/Core/Settings.h | 2 + ...chronousReadIndirectBufferFromRemoteFS.cpp | 26 +++++---- ...ynchronousReadIndirectBufferFromRemoteFS.h | 9 ++-- .../IO/CachedOnDiskReadBufferFromFile.cpp | 19 ++++++- src/Disks/IO/ThreadPoolRemoteFSReader.cpp | 2 +- src/Interpreters/Cache/FileCache.cpp | 38 ++++++++++--- src/Interpreters/Cache/FileCache.h | 4 +- src/Interpreters/Cache/FileCacheSettings.cpp | 2 + src/Interpreters/Cache/FileCacheSettings.h | 3 ++ src/Interpreters/Cache/FileCache_fwd.h | 2 +- src/Interpreters/Cache/FileSegment.cpp | 2 + .../FilesystemReadPrefetchesLog.cpp | 8 +-- .../FilesystemReadPrefetchesLog.h | 2 +- src/Interpreters/executeQuery.cpp | 2 + .../tests/gtest_lru_file_cache.cpp | 53 ++++++++++--------- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 3 ++ src/Storages/MergeTree/MergeTreeReadPool.cpp | 42 ++++++++++++--- src/Storages/MergeTree/MergeTreeReadPool.h | 5 +- src/Storages/MergeTree/MergeTreeSource.cpp | 2 + 19 files changed, 160 insertions(+), 66 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7f1fe838b80..465e27b0985 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -645,6 +645,8 @@ class IColumn; M(UInt64, merge_tree_min_rows_for_concurrent_read_for_remote_filesystem, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \ M(UInt64, merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \ M(UInt64, remote_read_min_bytes_for_seek, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes required for remote read (url, s3) to do seek, instead of read with ignore.", 0) \ + M(UInt64, merge_tree_min_bytes_per_task_for_remote_reading, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes to read per task.", 0) \ + M(Bool, merge_tree_use_const_size_tasks_for_remote_reading, true, "Whether to use constant size tasks for reading from a remote table.", 0) \ \ M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ M(Bool, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background. Makes sense only for inserts via HTTP protocol. If wait_for_async_insert is false, INSERT query is processed almost instantly, otherwise client will wait until data will be flushed to table", 0) \ diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 9448bbaf798..1dbe22a431f 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -42,21 +42,25 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; } +static size_t chooseBufferSize(const ReadSettings & settings, size_t file_size) +{ + /// Buffers used for prefetch or pre-download better to have enough size, but not bigger than the whole file. + return std::min(std::max(settings.prefetch_buffer_size, DBMS_DEFAULT_BUFFER_SIZE), file_size); +} AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRemoteFS( - IAsynchronousReader & reader_, - const ReadSettings & settings_, - std::shared_ptr impl_, - size_t min_bytes_for_seek_) - : ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0) + IAsynchronousReader & reader_, + const ReadSettings & settings_, + std::shared_ptr impl_, + size_t min_bytes_for_seek_) + : ReadBufferFromFileBase(chooseBufferSize(settings_, impl_->getFileSize()), nullptr, 0) , read_settings(settings_) , reader(reader_) , base_priority(settings_.priority) , impl(impl_) - , prefetch_buffer(settings_.prefetch_buffer_size) + , prefetch_buffer(chooseBufferSize(settings_, impl->getFileSize())) , min_bytes_for_seek(min_bytes_for_seek_) - , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr - ? CurrentThread::getQueryId() : "") + , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "") , current_reader_id(getRandomASCIIString(8)) #ifndef NDEBUG , log(&Poco::Logger::get("AsynchronousBufferFromRemoteFS")) @@ -135,11 +139,11 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch(int64_t priority) if (!hasPendingDataToRead()) return; - last_prefetch_info.submit_time = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + last_prefetch_info.submit_time = std::chrono::system_clock::now(); last_prefetch_info.priority = priority; /// Prefetch even in case hasPendingData() == true. - chassert(prefetch_buffer.size() == read_settings.prefetch_buffer_size || prefetch_buffer.size() == read_settings.remote_fs_buffer_size); + chassert(prefetch_buffer.size() == chooseBufferSize(read_settings, impl->getFileSize()) || prefetch_buffer.size() == read_settings.remote_fs_buffer_size); prefetch_future = asyncReadInto(prefetch_buffer.data(), prefetch_buffer.size(), priority); ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); } @@ -224,7 +228,7 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() { ProfileEventTimeIncrement watch(ProfileEvents::SynchronousRemoteReadWaitMicroseconds); - chassert(memory.size() == read_settings.prefetch_buffer_size || memory.size() == read_settings.remote_fs_buffer_size); + chassert(memory.size() == chooseBufferSize(read_settings, impl->getFileSize()) || memory.size() == read_settings.remote_fs_buffer_size); std::tie(size, offset) = impl->readInto(memory.data(), memory.size(), file_offset_of_buffer_end, bytes_to_ignore); bytes_to_ignore = 0; diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index 8cb0e2826b4..e72bbe15e5b 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -1,11 +1,12 @@ #pragma once -#include "config.h" -#include +#include +#include #include +#include #include #include -#include +#include "config.h" namespace Poco { class Logger; } @@ -97,7 +98,7 @@ private: struct LastPrefetchInfo { - UInt64 submit_time = 0; + std::chrono::system_clock::time_point submit_time; size_t priority = 0; }; LastPrefetchInfo last_prefetch_info; diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 68efd3f5d78..66a4d51abaa 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -48,7 +48,7 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( bool allow_seeks_after_first_read_, bool use_external_buffer_, std::optional read_until_position_) - : ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0, file_size_) + : ReadBufferFromFileBase(use_external_buffer_ ? 0 : settings_.remote_fs_buffer_size, nullptr, 0, file_size_) #ifndef NDEBUG , log(&Poco::Logger::get("CachedOnDiskReadBufferFromFile(" + source_file_path_ + ")")) #else @@ -120,7 +120,7 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) else { CreateFileSegmentSettings create_settings(is_persistent ? FileSegmentKind::Persistent : FileSegmentKind::Regular); - file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, create_settings)); + file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, file_size.value(), create_settings)); } /** @@ -150,6 +150,8 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm ReadSettings local_read_settings{settings}; /// Do not allow to use asynchronous version of LocalFSReadMethod. local_read_settings.local_fs_method = LocalFSReadMethod::pread; + if (use_external_buffer) + local_read_settings.local_fs_buffer_size = 0; // The buffer will unnecessarily allocate a Memory of size local_fs_buffer_size, which will then // most likely be unused because we're swap()ping our own internal_buffer into @@ -538,6 +540,9 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment) ProfileEvents::FileSegmentPredownloadMicroseconds, predownload_watch.elapsedMicroseconds()); }); + OpenTelemetry::SpanHolder span{ + fmt::format("CachedOnDiskReadBufferFromFile::predownload(key={}, size={})", file_segment->key().toString(), bytes_to_predownload)}; + if (bytes_to_predownload) { /// Consider this case. Some user needed segment [a, b] and downloaded it partially. @@ -806,6 +811,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (current_file_segment_it == file_segments_holder->file_segments.end()) return false; + const size_t original_buffer_size = internal_buffer.size(); + bool implementation_buffer_can_be_reused = false; SCOPE_EXIT({ try @@ -831,6 +838,9 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } } + if (use_external_buffer && initialized) + internal_buffer.resize(original_buffer_size); + chassert(!file_segment->isDownloader()); } catch (...) @@ -857,6 +867,11 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() chassert(!internal_buffer.empty()); + /// We allocate buffers not less than 1M so that s3 requests will not be too small. But the same buffers (members of AsynchronousReadIndirectBufferFromRemoteFS) + /// are used for reading from files. Some of these readings are fairly small and their performance degrade when we use big buffers (up to ~20% for queries like Q23 from ClickBench). + if (use_external_buffer && read_type == ReadType::CACHED && settings.local_fs_buffer_size < internal_buffer.size()) + internal_buffer.resize(settings.local_fs_buffer_size); + // Pass a valid external buffer for implementation_buffer to read into. // We then take it back with another swap() after reading is done. // (If we get an exception in between, we'll be left with an invalid internal_buffer. That's ok, as long as diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp index 4d0f39357ab..b8ec98f6044 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp @@ -86,7 +86,7 @@ std::future ThreadPoolRemoteFSReader::submit(Reques auto * remote_fs_fd = assert_cast(request.descriptor.get()); - auto watch = std::make_unique(CLOCK_MONOTONIC); + auto watch = std::make_unique(CLOCK_REALTIME); Result result = remote_fs_fd->readInto(request.buf, request.size, request.offset, request.ignore); watch->stop(); diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 39399c9ce09..cc39255eb97 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -14,6 +14,20 @@ #include +namespace +{ + +size_t roundDownToMultiple(size_t num, size_t multiple) +{ + return (num / multiple) * multiple; +} + +size_t roundUpToMultiple(size_t num, size_t multiple) +{ + return roundDownToMultiple(num + multiple - 1, multiple); +} +} + namespace fs = std::filesystem; namespace DB @@ -37,6 +51,7 @@ FileCache::FileCache(const FileCacheSettings & settings) , main_priority(std::make_unique()) , stash_priority(std::make_unique()) , max_stash_element_size(settings.max_elements) + , boundary_alignment(settings.boundary_alignment) { } @@ -194,8 +209,7 @@ FileCache::FileSegmentCell * FileCache::getCell( return &cell_it->second; } -FileSegments FileCache::getImpl( - const Key & key, const FileSegment::Range & range, std::lock_guard & cache_lock) +FileSegments FileCache::getImpl(const Key & key, const FileSegment::Range & range, std::lock_guard & cache_lock) { /// Given range = [left, right] and non-overlapping ordered set of file segments, /// find list [segment1, ..., segmentN] of segments which intersect with given range. @@ -409,8 +423,13 @@ void FileCache::fillHolesWithEmptyFileSegments( } } -FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings) +FileSegmentsHolder +FileCache::getOrSet(const Key & key, size_t offset, size_t size, size_t file_size, const CreateFileSegmentSettings & settings) { + const auto aligned_offset = roundDownToMultiple(offset, boundary_alignment); + const auto aligned_end = std::min(roundUpToMultiple(offset + size, boundary_alignment), file_size); + const auto aligned_size = aligned_end - aligned_offset; + std::lock_guard cache_lock(mutex); assertInitialized(cache_lock); @@ -419,18 +438,25 @@ FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t si assertCacheCorrectness(key, cache_lock); #endif - FileSegment::Range range(offset, offset + size - 1); + FileSegment::Range range(aligned_offset, aligned_offset + aligned_size - 1); /// Get all segments which intersect with the given range. auto file_segments = getImpl(key, range, cache_lock); if (file_segments.empty()) { - file_segments = splitRangeIntoCells(key, offset, size, FileSegment::State::EMPTY, settings, cache_lock); + file_segments = splitRangeIntoCells(key, range.left, range.size(), FileSegment::State::EMPTY, settings, cache_lock); } else { - fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */false, settings, cache_lock); + fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */ false, settings, cache_lock); } + + while (!file_segments.empty() && file_segments.front()->range().right < offset) + file_segments.pop_front(); + + while (!file_segments.empty() && file_segments.back()->range().left >= offset + size) + file_segments.pop_back(); + assert(!file_segments.empty()); return FileSegmentsHolder(std::move(file_segments)); } diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 83435b67562..fa365f70200 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -58,7 +58,8 @@ public: * As long as pointers to returned file segments are held * it is guaranteed that these file segments are not removed from cache. */ - FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings); + FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size, size_t file_size, const CreateFileSegmentSettings & settings); + FileSegmentsHolder set(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings); /** @@ -214,6 +215,7 @@ private: FileCacheRecords stash_records; std::unique_ptr stash_priority; size_t max_stash_element_size; + size_t boundary_alignment; void loadCacheInfoIntoMemory(std::lock_guard & cache_lock); diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 9d5282047aa..afee69a1f94 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -48,6 +48,8 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & bypass_cache_threashold = REMOTE_FS_OBJECTS_CACHE_BYPASS_THRESHOLD; do_not_evict_index_and_mark_files = config.getUInt64(config_prefix + ".do_not_evict_index_and_mark_files", false); + + boundary_alignment = config.getUInt64(config_prefix + ".boundary_alignment", DBMS_DEFAULT_BUFFER_SIZE); } } diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index 689c3ef70fb..8c765631066 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -26,6 +27,8 @@ struct FileCacheSettings bool enable_bypass_cache_with_threashold = false; size_t bypass_cache_threashold = REMOTE_FS_OBJECTS_CACHE_BYPASS_THRESHOLD; + size_t boundary_alignment = DBMS_DEFAULT_BUFFER_SIZE; + void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); }; diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 72dc1144fb9..e9da0d21f7e 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -4,7 +4,7 @@ namespace DB { -static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 100 * 1024 * 1024; +static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 8 * 1024 * 1024; static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS = 1024 * 1024; static constexpr int REMOTE_FS_OBJECTS_CACHE_ENABLE_HITS_THRESHOLD = 0; static constexpr size_t REMOTE_FS_OBJECTS_CACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024;; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 6ae25c681d4..6a99ea322d3 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -426,6 +426,8 @@ void FileSegment::write(const char * from, size_t size, size_t offset) FileSegment::State FileSegment::wait() { + OpenTelemetry::SpanHolder span{fmt::format("FileSegment::wait({})", key().toString())}; + std::unique_lock segment_lock(mutex); if (is_detached) diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.cpp b/src/Interpreters/FilesystemReadPrefetchesLog.cpp index 9dd0fce84b0..664299017bc 100644 --- a/src/Interpreters/FilesystemReadPrefetchesLog.cpp +++ b/src/Interpreters/FilesystemReadPrefetchesLog.cpp @@ -1,9 +1,9 @@ -#include #include #include #include #include #include +#include namespace DB @@ -39,12 +39,12 @@ void FilesystemReadPrefetchesLogElement::appendToBlock(MutableColumns & columns) columns[i++]->insert(path); columns[i++]->insert(offset); columns[i++]->insert(size); - columns[i++]->insert(prefetch_submit_time); + columns[i++]->insert(std::chrono::duration_cast(prefetch_submit_time.time_since_epoch()).count()); columns[i++]->insert(priority); if (execution_watch) { - columns[i++]->insert(execution_watch->getStart()); - columns[i++]->insert(execution_watch->getEnd()); + columns[i++]->insert(execution_watch->getStart() / 1000); + columns[i++]->insert(execution_watch->getEnd() / 1000); columns[i++]->insert(execution_watch->elapsedMicroseconds()); } else diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.h b/src/Interpreters/FilesystemReadPrefetchesLog.h index a7672c49d91..685909d41b7 100644 --- a/src/Interpreters/FilesystemReadPrefetchesLog.h +++ b/src/Interpreters/FilesystemReadPrefetchesLog.h @@ -23,7 +23,7 @@ struct FilesystemReadPrefetchesLogElement String path; UInt64 offset; Int64 size; /// -1 means unknown - Decimal64 prefetch_submit_time{}; + std::chrono::system_clock::time_point prefetch_submit_time; std::optional execution_watch; size_t priority; FilesystemPrefetchState state; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 00a5d0ed1d8..96bcc89c1d2 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -321,6 +321,8 @@ static std::tuple executeQueryImpl( /// This does not have impact on the final span logs, because these internal queries are issued by external queries, /// we still have enough span logs for the execution of external queries. std::shared_ptr query_span = internal ? nullptr : std::make_shared("query"); + if (query_span) + LOG_DEBUG(&Poco::Logger::get("executeQuery"), "Query span trace_id for opentelemetry log: {}", query_span->trace_id); auto query_start_time = std::chrono::system_clock::now(); diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 0754c394f66..3ba13b218ec 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -138,6 +138,7 @@ TEST_F(FileCacheTest, get) settings.base_path = cache_base_path; settings.max_size = 30; settings.max_elements = 5; + settings.boundary_alignment = 1; { auto cache = DB::FileCache(settings); @@ -145,7 +146,7 @@ TEST_F(FileCacheTest, get) auto key = cache.hash("key1"); { - auto holder = cache.getOrSet(key, 0, 10, {}); /// Add range [0, 9] + auto holder = cache.getOrSet(key, 0, 10, 10, {}); /// Add range [0, 9] auto segments = fromHolder(holder); /// Range was not present in cache. It should be added in cache as one while file segment. ASSERT_EQ(segments.size(), 1); @@ -174,7 +175,7 @@ TEST_F(FileCacheTest, get) { /// Want range [5, 14], but [0, 9] already in cache, so only [10, 14] will be put in cache. - auto holder = cache.getOrSet(key, 5, 10, {}); + auto holder = cache.getOrSet(key, 5, 10, 15, {}); auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 2); @@ -194,14 +195,14 @@ TEST_F(FileCacheTest, get) ASSERT_EQ(cache.getUsedCacheSize(), 15); { - auto holder = cache.getOrSet(key, 9, 1, {}); /// Get [9, 9] + auto holder = cache.getOrSet(key, 9, 1, 10, {}); /// Get [9, 9] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 1); assertRange(7, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); } { - auto holder = cache.getOrSet(key, 9, 2, {}); /// Get [9, 10] + auto holder = cache.getOrSet(key, 9, 2, 11, {}); /// Get [9, 10] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 2); assertRange(8, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); @@ -209,15 +210,15 @@ TEST_F(FileCacheTest, get) } { - auto holder = cache.getOrSet(key, 10, 1, {}); /// Get [10, 10] + auto holder = cache.getOrSet(key, 10, 1, 11, {}); /// Get [10, 10] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 1); assertRange(10, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); } - complete(cache_base_path, cache.getOrSet(key, 17, 4, {})); /// Get [17, 20] - complete(cache_base_path, cache.getOrSet(key, 24, 3, {})); /// Get [24, 26] - /// completeWithState(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27] + complete(cache_base_path, cache.getOrSet(key, 17, 4, 21, {})); /// Get [17, 20] + complete(cache_base_path, cache.getOrSet(key, 24, 3, 27, {})); /// Get [24, 26] + /// completeWithState(cache.getOrSet(key, 27, 1, 28, false)); /// Get [27, 27] /// Current cache: [__________][_____] [____] [___][] /// ^ ^^ ^ ^ ^ ^ ^^^ @@ -227,7 +228,7 @@ TEST_F(FileCacheTest, get) ASSERT_EQ(cache.getUsedCacheSize(), 22); { - auto holder = cache.getOrSet(key, 0, 26, {}); /// Get [0, 25] + auto holder = cache.getOrSet(key, 0, 26, 26, {}); /// Get [0, 25] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 6); @@ -261,14 +262,14 @@ TEST_F(FileCacheTest, get) /// as max elements size is reached, next attempt to put something in cache should fail. /// This will also check that [27, 27] was indeed evicted. - auto holder1 = cache.getOrSet(key, 27, 1, {}); + auto holder1 = cache.getOrSet(key, 27, 1, 28, {}); auto segments_1 = fromHolder(holder1); /// Get [27, 27] ASSERT_EQ(segments_1.size(), 1); assertRange(17, segments_1[0], DB::FileSegment::Range(27, 27), DB::FileSegment::State::EMPTY); } { - auto holder = cache.getOrSet(key, 12, 10, {}); /// Get [12, 21] + auto holder = cache.getOrSet(key, 12, 10, 22, {}); /// Get [12, 21] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 4); @@ -292,7 +293,7 @@ TEST_F(FileCacheTest, get) ASSERT_EQ(cache.getFileSegmentsNum(), 5); { - auto holder = cache.getOrSet(key, 23, 5, {}); /// Get [23, 28] + auto holder = cache.getOrSet(key, 23, 5, 28, {}); /// Get [23, 28] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 3); @@ -313,12 +314,12 @@ TEST_F(FileCacheTest, get) /// 17 21 2324 26 28 { - auto holder5 = cache.getOrSet(key, 2, 3, {}); /// Get [2, 4] + auto holder5 = cache.getOrSet(key, 2, 3, 5, {}); /// Get [2, 4] auto s5 = fromHolder(holder5); ASSERT_EQ(s5.size(), 1); assertRange(25, s5[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::EMPTY); - auto holder1 = cache.getOrSet(key, 30, 2, {}); /// Get [30, 31] + auto holder1 = cache.getOrSet(key, 30, 2, 32, {}); /// Get [30, 31] auto s1 = fromHolder(holder1); ASSERT_EQ(s1.size(), 1); assertRange(26, s1[0], DB::FileSegment::Range(30, 31), DB::FileSegment::State::EMPTY); @@ -334,20 +335,20 @@ TEST_F(FileCacheTest, get) /// ^ ^ ^ ^ ^ ^ ^ ^ /// 2 4 23 24 26 27 30 31 - auto holder2 = cache.getOrSet(key, 23, 1, {}); /// Get [23, 23] + auto holder2 = cache.getOrSet(key, 23, 1, 24, {}); /// Get [23, 23] auto s2 = fromHolder(holder2); ASSERT_EQ(s2.size(), 1); - auto holder3 = cache.getOrSet(key, 24, 3, {}); /// Get [24, 26] + auto holder3 = cache.getOrSet(key, 24, 3, 27, {}); /// Get [24, 26] auto s3 = fromHolder(holder3); ASSERT_EQ(s3.size(), 1); - auto holder4 = cache.getOrSet(key, 27, 1, {}); /// Get [27, 27] + auto holder4 = cache.getOrSet(key, 27, 1, 28, {}); /// Get [27, 27] auto s4 = fromHolder(holder4); ASSERT_EQ(s4.size(), 1); /// All cache is now unreleasable because pointers are still hold - auto holder6 = cache.getOrSet(key, 0, 40, {}); + auto holder6 = cache.getOrSet(key, 0, 40, 40, {}); auto f = fromHolder(holder6); ASSERT_EQ(f.size(), 9); @@ -368,7 +369,7 @@ TEST_F(FileCacheTest, get) } { - auto holder = cache.getOrSet(key, 2, 3, {}); /// Get [2, 4] + auto holder = cache.getOrSet(key, 2, 3, 5, {}); /// Get [2, 4] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 1); assertRange(31, segments[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED); @@ -379,7 +380,7 @@ TEST_F(FileCacheTest, get) /// 2 4 23 24 26 27 30 31 { - auto holder = cache.getOrSet(key, 25, 5, {}); /// Get [25, 29] + auto holder = cache.getOrSet(key, 25, 5, 30, {}); /// Get [25, 29] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 3); @@ -403,7 +404,7 @@ TEST_F(FileCacheTest, get) chassert(&DB::CurrentThread::get() == &thread_status_1); DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); - auto holder_2 = cache.getOrSet(key, 25, 5, {}); /// Get [25, 29] once again. + auto holder_2 = cache.getOrSet(key, 25, 5, 30, {}); /// Get [25, 29] once again. auto segments_2 = fromHolder(holder_2); ASSERT_EQ(segments.size(), 3); @@ -446,7 +447,7 @@ TEST_F(FileCacheTest, get) /// and notify_all() is also called from destructor of holder. std::optional holder; - holder.emplace(cache.getOrSet(key, 3, 23, {})); /// Get [3, 25] + holder.emplace(cache.getOrSet(key, 3, 23, 26, {})); /// Get [3, 25] auto segments = fromHolder(*holder); ASSERT_EQ(segments.size(), 3); @@ -472,7 +473,7 @@ TEST_F(FileCacheTest, get) chassert(&DB::CurrentThread::get() == &thread_status_1); DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); - auto holder_2 = cache.getOrSet(key, 3, 23, {}); /// Get [3, 25] once again + auto holder_2 = cache.getOrSet(key, 3, 23, 26, {}); /// Get [3, 25] once again auto segments_2 = fromHolder(*holder); ASSERT_EQ(segments_2.size(), 3); @@ -521,7 +522,7 @@ TEST_F(FileCacheTest, get) cache2.initialize(); auto key = cache2.hash("key1"); - auto holder1 = cache2.getOrSet(key, 2, 28, {}); /// Get [2, 29] + auto holder1 = cache2.getOrSet(key, 2, 28, 30, {}); /// Get [2, 29] auto segments1 = fromHolder(holder1); ASSERT_EQ(segments1.size(), 5); @@ -543,7 +544,7 @@ TEST_F(FileCacheTest, get) cache2.initialize(); auto key = cache2.hash("key1"); - auto holder1 = cache2.getOrSet(key, 0, 25, {}); /// Get [0, 24] + auto holder1 = cache2.getOrSet(key, 0, 25, 25, {}); /// Get [0, 24] auto segments1 = fromHolder(holder1); ASSERT_EQ(segments1.size(), 3); @@ -668,7 +669,7 @@ TEST_F(FileCacheTest, temporaryData) auto tmp_data_scope = std::make_shared(nullptr, &file_cache, 0); - auto some_data_holder = file_cache.getOrSet(file_cache.hash("some_data"), 0, 5_KiB, CreateFileSegmentSettings{}); + auto some_data_holder = file_cache.getOrSet(file_cache.hash("some_data"), 0, 5_KiB, 5_KiB, CreateFileSegmentSettings{}); { auto segments = fromHolder(some_data_holder); diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 7428fd5c056..e51120f9305 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -366,6 +366,9 @@ MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInf part_info->column_name_set = {required_column_names.begin(), required_column_names.end()}; part_info->task_columns = task_columns; + if (settings.prefetch_buffer_size < DBMS_DEFAULT_BUFFER_SIZE) + throw Exception(ErrorCodes::LOGICAL_ERROR, "remove me"); + /// adjustBufferSize(), which is done in MergeTreeReaderStream and MergeTreeReaderCompact, /// lowers buffer size if file size (or required read range) is less. So we know that the /// settings.prefetch_buffer_size will be lowered there, therefore we account it here as well. diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 29be06b4e6a..931a1be6b30 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -21,6 +21,14 @@ namespace ErrorCodes namespace DB { +size_t getApproxSizeOfPart(const IMergeTreeDataPart & part, const Names & columns_to_read) +{ + ColumnSize columns_size{}; + for (const auto & col_name : columns_to_read) + columns_size.add(part.getColumnSize(col_name)); + return columns_size.data_compressed; +} + MergeTreeReadPool::MergeTreeReadPool( size_t threads_, size_t sum_marks_, @@ -44,13 +52,36 @@ MergeTreeReadPool::MergeTreeReadPool( , parts_ranges(std::move(parts_)) , predict_block_size_bytes(context_->getSettingsRef().preferred_block_size_bytes > 0) , do_not_steal_tasks(do_not_steal_tasks_) + , merge_tree_use_const_size_tasks_for_remote_reading(context_->getSettingsRef().merge_tree_use_const_size_tasks_for_remote_reading) , backoff_settings{context_->getSettingsRef()} , backoff_state{threads_} { + const auto & settings = context_->getSettingsRef(); + + size_t total_compressed_bytes = 0; + size_t total_marks = 0; + for (const auto & part : parts_ranges) + { + total_compressed_bytes += getApproxSizeOfPart( + *part.data_part, prewhere_info ? prewhere_info->prewhere_actions->getRequiredColumnsNames() : column_names_); + total_marks += part.getMarksCount(); + } + + if (total_marks) + { + const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; + const auto avg_mark_bytes = std::max(total_compressed_bytes / total_marks, 1); + const auto heuristic_min_marks = std::min(total_marks / threads_, min_bytes_per_task / avg_mark_bytes); + if (heuristic_min_marks > min_marks_for_concurrent_read) + { + min_marks_for_concurrent_read = heuristic_min_marks; + } + } + /// parts don't contain duplicate MergeTreeDataPart's. const auto per_part_sum_marks = fillPerPartInfo( parts_ranges, storage_snapshot, is_part_on_remote_disk, - do_not_steal_tasks, predict_block_size_bytes, + predict_block_size_bytes, column_names, virtual_column_names, prewhere_info, actions_settings, reader_settings, per_part_params); @@ -61,7 +92,6 @@ std::vector MergeTreeReadPool::fillPerPartInfo( const RangesInDataParts & parts, const StorageSnapshotPtr & storage_snapshot, std::vector & is_part_on_remote_disk, - bool & do_not_steal_tasks, bool & predict_block_size_bytes, const Names & column_names, const Names & virtual_column_names, @@ -83,7 +113,6 @@ std::vector MergeTreeReadPool::fillPerPartInfo( bool part_on_remote_disk = part.data_part->isStoredOnRemoteDisk(); is_part_on_remote_disk[i] = part_on_remote_disk; - do_not_steal_tasks |= part_on_remote_disk; /// Read marks for every data part. size_t sum_marks = 0; @@ -157,14 +186,13 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t thread) auto & marks_in_part = thread_tasks.sum_marks_in_parts.back(); size_t need_marks; - if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks + if (is_part_on_remote_disk[part_idx] && !merge_tree_use_const_size_tasks_for_remote_reading) need_marks = marks_in_part; else /// Get whole part to read if it is small enough. need_marks = std::min(marks_in_part, min_marks_for_concurrent_read); /// Do not leave too little rows in part for next time. - if (marks_in_part > need_marks && - marks_in_part - need_marks < min_marks_for_concurrent_read) + if (marks_in_part > need_marks && marks_in_part - need_marks < min_marks_for_concurrent_read / 2) need_marks = marks_in_part; MarkRanges ranges_to_get_from_part; @@ -294,6 +322,8 @@ void MergeTreeReadPool::fillPerThreadInfo( parts_queue.push(std::move(info.second)); } + LOG_DEBUG(log, "min_marks_for_concurrent_read={}", min_marks_for_concurrent_read); + const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1; for (size_t i = 0; i < threads && !parts_queue.empty(); ++i) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index b3356ec3351..514150566a6 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -92,7 +92,6 @@ public: const RangesInDataParts & parts, const StorageSnapshotPtr & storage_snapshot, std::vector & is_part_on_remote_disk, - bool & do_not_steal_tasks, bool & predict_block_size_bytes, const Names & column_names, const Names & virtual_column_names, @@ -117,6 +116,7 @@ private: RangesInDataParts parts_ranges; bool predict_block_size_bytes; bool do_not_steal_tasks; + bool merge_tree_use_const_size_tasks_for_remote_reading = false; std::vector per_part_params; std::vector is_part_on_remote_disk; @@ -188,7 +188,7 @@ public: , parts_ranges(std::move(parts_)) { MergeTreeReadPool::fillPerPartInfo( - parts_ranges, storage_snapshot, is_part_on_remote_disk, do_not_steal_tasks, + parts_ranges, storage_snapshot, is_part_on_remote_disk, predict_block_size_bytes, column_names, virtual_column_names, prewhere_info, actions_settings, reader_settings, per_part_params); @@ -225,7 +225,6 @@ private: const Names virtual_column_names; RangesInDataParts parts_ranges; - bool do_not_steal_tasks = false; bool predict_block_size_bytes = false; std::vector is_part_on_remote_disk; std::vector per_part_params; diff --git a/src/Storages/MergeTree/MergeTreeSource.cpp b/src/Storages/MergeTree/MergeTreeSource.cpp index 328336ff71a..6cf6cd48534 100644 --- a/src/Storages/MergeTree/MergeTreeSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSource.cpp @@ -207,6 +207,7 @@ std::optional MergeTreeSource::tryGenerate() try { + OpenTelemetry::SpanHolder span{"MergeTreeSource::tryGenerate()"}; holder->setResult(algorithm->read()); } catch (...) @@ -221,6 +222,7 @@ std::optional MergeTreeSource::tryGenerate() } #endif + OpenTelemetry::SpanHolder span{"MergeTreeSource::tryGenerate()"}; return processReadResult(algorithm->read()); } From 7dde282027891aa0afbd0889b4be896981354ca1 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 27 Apr 2023 18:50:35 -0400 Subject: [PATCH 045/308] Fix no case completion --- src/Client/LineReader.cpp | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Client/LineReader.cpp b/src/Client/LineReader.cpp index f6cd7bffef7..82dbe03e5d3 100644 --- a/src/Client/LineReader.cpp +++ b/src/Client/LineReader.cpp @@ -82,9 +82,13 @@ replxx::Replxx::completions_t LineReader::Suggest::getCompletions(const String & std::lock_guard lock(mutex); Words to_search; + bool no_case = false; /// Only perform case sensitive completion when the prefix string contains any uppercase characters if (std::none_of(prefix.begin(), prefix.end(), [](char32_t x) { return iswupper(static_cast(x)); })) + { to_search = words_no_case; + no_case = true; + } else to_search = words; @@ -95,11 +99,18 @@ replxx::Replxx::completions_t LineReader::Suggest::getCompletions(const String & addNewWords(to_search, new_words, std::less{}); } - range = std::equal_range( - to_search.begin(), to_search.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched) - { - return strncasecmp(s.data(), prefix_searched.data(), prefix_length) < 0; - }); + if (no_case) + range = std::equal_range( + to_search.begin(), to_search.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched) + { + return strncasecmp(s.data(), prefix_searched.data(), prefix_length) < 0; + }); + else + range = std::equal_range( + to_search.begin(), to_search.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched) + { + return strncmp(s.data(), prefix_searched.data(), prefix_length) < 0; + }); return replxx::Replxx::completions_t(range.first, range.second); } From 39ae0c1c4e4701f07846425a22d6e6c3f587b194 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 27 Apr 2023 22:51:52 -0400 Subject: [PATCH 046/308] Fix tests --- tests/integration/test_keeper_client/test.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 64ef62b6243..4fa8ce82aa6 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -30,7 +30,10 @@ def test_base_commands(started_cluster): [ started_cluster.server_bin_path, "keeper-client", - f"{cluster.get_instance_ip('zoo1')}:{cluster.zookeeper_port}", + "--host", + cluster.get_instance_ip('zoo1'), + "--port", + cluster.zookeeper_port, "-q", "create test_create_zk_node1 testvalue1;create test_create_zk_node_2 testvalue2;get test_create_zk_node1;", ], @@ -47,7 +50,10 @@ def test_four_letter_word_commands(started_cluster): [ started_cluster.server_bin_path, "keeper-client", - f"{cluster.get_instance_ip('zoo1')}:{cluster.zookeeper_port}", + "--host", + cluster.get_instance_ip('zoo1'), + "--port", + cluster.zookeeper_port, "-q", "ruok", ], From 711d8d4442cbe8f074d39c4cb2477a761037140e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 28 Apr 2023 03:19:21 +0000 Subject: [PATCH 047/308] Automatic style fix --- tests/integration/test_keeper_client/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 4fa8ce82aa6..f7aec7852b1 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -31,7 +31,7 @@ def test_base_commands(started_cluster): started_cluster.server_bin_path, "keeper-client", "--host", - cluster.get_instance_ip('zoo1'), + cluster.get_instance_ip("zoo1"), "--port", cluster.zookeeper_port, "-q", @@ -51,7 +51,7 @@ def test_four_letter_word_commands(started_cluster): started_cluster.server_bin_path, "keeper-client", "--host", - cluster.get_instance_ip('zoo1'), + cluster.get_instance_ip("zoo1"), "--port", cluster.zookeeper_port, "-q", From c862eca0f735433e7eb17199d21a8c8dd2d1dc07 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 28 Apr 2023 14:08:46 +0000 Subject: [PATCH 048/308] fix build --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- src/Interpreters/Cache/FileCache.cpp | 7 ++----- src/Interpreters/Cache/FileSegment.cpp | 15 ++++++++------- 3 files changed, 11 insertions(+), 13 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index ddc5b385b07..b186b9b1d28 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -536,7 +536,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) }); OpenTelemetry::SpanHolder span{ - fmt::format("CachedOnDiskReadBufferFromFile::predownload(key={}, size={})", file_segment->key().toString(), bytes_to_predownload)}; + fmt::format("CachedOnDiskReadBufferFromFile::predownload(key={}, size={})", file_segment.key().toString(), bytes_to_predownload)}; if (bytes_to_predownload) { diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 03cea3ee2ab..626ac7f5d49 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -405,11 +405,8 @@ FileSegmentsHolderPtr FileCache::set( return std::make_unique(std::move(file_segments)); } -FileSegmentsHolderPtr FileCache::getOrSet( - const Key & key, - size_t offset, - size_t size, - const CreateFileSegmentSettings & settings) +FileSegmentsHolderPtr +FileCache::getOrSet(const Key & key, size_t offset, size_t size, size_t file_size, const CreateFileSegmentSettings & settings) { assertInitialized(); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 91356f699e3..2b1801a46f0 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -1,13 +1,14 @@ #include "FileSegment.h" -#include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include +#include +#include #include From 30bc74c859ae4a33abe0d5bb7e42369865ea3b4c Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Apr 2023 14:09:24 -0400 Subject: [PATCH 049/308] Fix tests --- tests/integration/test_keeper_client/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 4fa8ce82aa6..eee30741bdd 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -31,9 +31,9 @@ def test_base_commands(started_cluster): started_cluster.server_bin_path, "keeper-client", "--host", - cluster.get_instance_ip('zoo1'), + str(cluster.get_instance_ip('zoo1')), "--port", - cluster.zookeeper_port, + str(cluster.zookeeper_port), "-q", "create test_create_zk_node1 testvalue1;create test_create_zk_node_2 testvalue2;get test_create_zk_node1;", ], @@ -51,9 +51,9 @@ def test_four_letter_word_commands(started_cluster): started_cluster.server_bin_path, "keeper-client", "--host", - cluster.get_instance_ip('zoo1'), + str(cluster.get_instance_ip('zoo1')), "--port", - cluster.zookeeper_port, + str(cluster.zookeeper_port), "-q", "ruok", ], From e0abb251e5344b431de768e91dbf4f791283467d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 28 Apr 2023 21:52:38 +0000 Subject: [PATCH 050/308] fix test --- tests/queries/0_stateless/02344_describe_cache.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 7561b32bae1..a803ca1fab1 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1,2 @@ -134217728 1048576 104857600 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 +134217728 1048576 8388608 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 134217728 1048576 104857600 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 From 979b68a488c88bad53a44e4c9a8c525572091de5 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 1 May 2023 18:24:30 +0000 Subject: [PATCH 051/308] fix --- src/Common/OpenTelemetryTraceContext.cpp | 2 +- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 7 +------ src/Storages/MergeTree/MergeTreeReadPool.cpp | 3 ++- 3 files changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 0d89c581318..b68795814fb 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -120,7 +120,7 @@ SpanHolder::SpanHolder(std::string_view _operation_name, SpanKind _kind) this->start_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - /// Add new initialization here + this->addAttribute("clickhouse.thread_id", getThreadId()); } catch (...) { diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index b186b9b1d28..de3edf29086 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -152,11 +152,6 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm if (use_external_buffer) local_read_settings.local_fs_buffer_size = 0; - // The buffer will unnecessarily allocate a Memory of size local_fs_buffer_size, which will then - // most likely be unused because we're swap()ping our own internal_buffer into - // implementation_buffer before each read. But we can't just set local_fs_buffer_size = 0 here - // because some buffer implementations actually use that memory (e.g. for prefetching). - auto buf = createReadBufferFromFileBase(path, local_read_settings); if (getFileSizeFromReadBuffer(*buf) == 0) @@ -827,7 +822,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } } - if (use_external_buffer && initialized) + if (use_external_buffer && !internal_buffer.empty()) internal_buffer.resize(original_buffer_size); chassert(!file_segment.isDownloader()); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 931a1be6b30..b3e3cf1361e 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -71,7 +71,8 @@ MergeTreeReadPool::MergeTreeReadPool( { const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; const auto avg_mark_bytes = std::max(total_compressed_bytes / total_marks, 1); - const auto heuristic_min_marks = std::min(total_marks / threads_, min_bytes_per_task / avg_mark_bytes); + /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. + const auto heuristic_min_marks = std::min(total_marks / threads_ / 8, min_bytes_per_task / avg_mark_bytes); if (heuristic_min_marks > min_marks_for_concurrent_read) { min_marks_for_concurrent_read = heuristic_min_marks; From 30d216f863583e7216b5cc5f9144f33068092d44 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 3 May 2023 13:49:46 +0000 Subject: [PATCH 052/308] fix --- src/Storages/MergeTree/MergeTreeReadPool.cpp | 43 +++++++++++--------- 1 file changed, 23 insertions(+), 20 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index b3e3cf1361e..34e270fdd24 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -1,10 +1,10 @@ -#include -#include #include +#include +#include +#include #include #include #include -#include namespace ProfileEvents @@ -56,26 +56,29 @@ MergeTreeReadPool::MergeTreeReadPool( , backoff_settings{context_->getSettingsRef()} , backoff_state{threads_} { - const auto & settings = context_->getSettingsRef(); - - size_t total_compressed_bytes = 0; - size_t total_marks = 0; - for (const auto & part : parts_ranges) + if (std::ranges::count(is_part_on_remote_disk, true)) { - total_compressed_bytes += getApproxSizeOfPart( - *part.data_part, prewhere_info ? prewhere_info->prewhere_actions->getRequiredColumnsNames() : column_names_); - total_marks += part.getMarksCount(); - } + const auto & settings = context_->getSettingsRef(); - if (total_marks) - { - const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; - const auto avg_mark_bytes = std::max(total_compressed_bytes / total_marks, 1); - /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. - const auto heuristic_min_marks = std::min(total_marks / threads_ / 8, min_bytes_per_task / avg_mark_bytes); - if (heuristic_min_marks > min_marks_for_concurrent_read) + size_t total_compressed_bytes = 0; + size_t total_marks = 0; + for (const auto & part : parts_ranges) { - min_marks_for_concurrent_read = heuristic_min_marks; + total_compressed_bytes += getApproxSizeOfPart( + *part.data_part, prewhere_info ? prewhere_info->prewhere_actions->getRequiredColumnsNames() : column_names_); + total_marks += part.getMarksCount(); + } + + if (total_marks) + { + const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; + const auto avg_mark_bytes = std::max(total_compressed_bytes / total_marks, 1); + /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. + const auto heuristic_min_marks = std::min(total_marks / threads_, min_bytes_per_task / avg_mark_bytes); + if (heuristic_min_marks > min_marks_for_concurrent_read) + { + min_marks_for_concurrent_read = heuristic_min_marks; + } } } From 1cc2499c70db4908331a02d1bbf71d89a27f8875 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 10 May 2023 16:10:43 +0000 Subject: [PATCH 053/308] fix build --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index fb25770ed8d..d956dac9fe0 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -49,7 +49,7 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( bool allow_seeks_after_first_read_, bool use_external_buffer_, std::optional read_until_position_, - std::shared_ptr cache_log_)) + std::shared_ptr cache_log_) : ReadBufferFromFileBase(use_external_buffer_ ? 0 : settings_.remote_fs_buffer_size, nullptr, 0, file_size_) #ifndef NDEBUG , log(&Poco::Logger::get("CachedOnDiskReadBufferFromFile(" + source_file_path_ + ")")) From 604bd24995b411cbf405e79a48b555beda604b6e Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 11 May 2023 11:58:08 +0000 Subject: [PATCH 054/308] Refactor, remove no more needed arguments --- .../functions/other-functions.md | 40 +- src/Functions/generateRandomStructure.cpp | 480 +++++++----------- .../02586_generate_random_structure.reference | 11 +- .../02586_generate_random_structure.sql | 26 +- 4 files changed, 189 insertions(+), 368 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 43330b75b8f..e235a3db393 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2617,18 +2617,13 @@ Generates random table structure in a format `column1_name column1_type, column2 **Syntax** ``` sql -generateRandomStructure([number_of_columns, seed, allow_big_numbers, allow_enums, allow_decimals, allow_ip, allow_only_string_map_keys]) +generateRandomStructure([number_of_columns, seed]) ``` **Arguments** - `number_of_columns` — The desired number of columns in the result table structure. If set to 0 or `Null`, the number of columns will be random from 1 to 128. Default value: `Null`. - `seed` - Random seed to produce stable results. If seed is not specified or set to `Null`, it is randomly generated. -- `allow_big_numbers` - Indicates if big number types (`Int128/UInt128/Int256/UInt256/Decimal128/Decinal256`) can be generated. Default value: true. -- `allow_enums` - Indicates if enum types (`Enum8/Enum16`) can be generated. Default - true. -- `allow_decimals` - Indicates if decimal types (`Decimal(P, S)`) can be generated. Default - true. -- `allow_ip` - Indicates if ip types (`IPv4/IPv6`) can be generated. Default - true. -- `allow_only_string_map_keys` - Indicates if Map key type can be only `String/FixedString`. Default - false. All arguments must be constant. @@ -2671,41 +2666,16 @@ Result: Query: ``` sql -SELECT generateRandomStructure(Null, 11) +SELECT generateRandomStructure(NULL, 33) ``` Result: ``` text -┌─generateRandomStructure(0, 11)──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ c1 Date32, c2 String, c3 IPv6, c4 DateTime, c5 UInt16, c6 Tuple(e1 UInt32, e2 Date, e3 Date, e4 IPv6, e5 Nested(e1 DateTime, e2 FixedString(110), e3 Int256, e4 Array(Decimal64(4)), e5 Decimal128(18), e6 Enum16('v0' = 0, 'v1' = 1, 'v2' = 2, 'v3' = 3, 'v4' = 4)), e6 DateTime64(4)), c7 DateTime, c8 DateTime64(6), c9 Bool │ -└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +┌─generateRandomStructure(NULL, 33)─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ c1 DateTime, c2 Enum8('c2V0' = 0, 'c2V1' = 1, 'c2V2' = 2, 'c2V3' = 3), c3 LowCardinality(Nullable(FixedString(30))), c4 Int16, c5 Enum8('c5V0' = 0, 'c5V1' = 1, 'c5V2' = 2, 'c5V3' = 3), c6 Nullable(UInt8), c7 String, c8 Nested(e1 IPv4, e2 UInt8, e3 UInt16, e4 UInt16, e5 Int32, e6 Map(Date, Decimal256(70))) │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` -``` sql -SELECT generateRandomStructure(6, Null, false, false) -``` - -Result: - -``` text -┌─generateRandomStructure(6, NULL, false, false)───────────────────────────────────────────────────────┐ -│ c1 Float32, c2 Tuple(DateTime), c3 UInt8, c4 UInt16, c5 Int64, c6 Array(Map(FixedString(108), Date)) │ -└──────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -``` sql -SELECT generateRandomStructure(6, Null, false, false, false, false, true) -``` - -Result: - -``` text -┌─generateRandomStructure(6, NULL, false, false, false, false, true)─────────────────────────────────────────────────┐ -│ c1 String, c2 UInt32, c3 Int32, c4 Int32, c5 Tuple(LowCardinality(Nullable(FixedString(101))), UInt8), c6 DateTime │ -└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - - This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables. diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index e6766e731b2..9fe321365e5 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -1,5 +1,3 @@ -#include "config.h" - #include #include #include @@ -24,130 +22,69 @@ namespace ErrorCodes class FunctionGenerateRandomStructure : public IFunction { private: - enum class Type + static constexpr std::array simple_types { - Int8, - UInt8, - Bool, - Int16, - UInt16, - Int32, - UInt32, - Int64, - UInt64, - Float32, - Float64, - DateTime64, - Decimal32, - Decimal64, - Date, - Date32, - DateTime, - String, - FixedString, - IPv4, - IPv6, - Int128, - UInt128, - Int256, - UInt256, - Decimal128, - Decimal256, - Enum8, - Enum16, - Nullable, - LowCardinality, - Array, - Tuple, - Map, - Nested, + TypeIndex::Int8, + TypeIndex::UInt8, + TypeIndex::Int16, + TypeIndex::UInt16, + TypeIndex::Int32, + TypeIndex::UInt32, + TypeIndex::Int64, + TypeIndex::UInt64, + TypeIndex::Int128, + TypeIndex::UInt128, + TypeIndex::Int256, + TypeIndex::UInt256, + TypeIndex::Float32, + TypeIndex::Float64, + TypeIndex::Decimal32, + TypeIndex::Decimal64, + TypeIndex::Decimal128, + TypeIndex::Decimal256, + TypeIndex::Date, + TypeIndex::Date32, + TypeIndex::DateTime, + TypeIndex::DateTime64, + TypeIndex::String, + TypeIndex::FixedString, + TypeIndex::Enum8, + TypeIndex::Enum16, + TypeIndex::IPv4, + TypeIndex::IPv6, + TypeIndex::UUID, }; - static constexpr std::array simple_types + static constexpr std::array complex_types { - Type::Int8, - Type::UInt8, - Type::Bool, - Type::Int16, - Type::UInt16, - Type::Int32, - Type::UInt32, - Type::Int64, - Type::UInt64, - Type::Float32, - Type::Float64, - Type::Date, - Type::Date32, - Type::DateTime, - Type::String, - Type::FixedString, + TypeIndex::Nullable, + TypeIndex::LowCardinality, + TypeIndex::Array, + TypeIndex::Tuple, + TypeIndex::Map, }; - static constexpr std::array big_integer_types + static constexpr std::array map_key_types { - Type::Int128, - Type::UInt128, - Type::Int256, - Type::UInt256, - }; - - static constexpr std::array decimal_types - { - Type::DateTime64, - Type::Decimal32, - Type::Decimal64, - }; - - static constexpr std::array big_decimal_types - { - Type::Decimal128, - Type::Decimal256, - }; - - static constexpr std::array enum_types - { - Type::Enum8, - Type::Enum16, - }; - - static constexpr std::array ip_types - { - Type::IPv4, - Type::IPv6, - }; - - static constexpr std::array complex_types - { - Type::Nullable, - Type::LowCardinality, - Type::Array, - Type::Tuple, - Type::Map, - Type::Nested, - }; - - static constexpr std::array map_key_types - { - Type::Int8, - Type::UInt8, - Type::Bool, - Type::Int16, - Type::UInt16, - Type::Int32, - Type::UInt32, - Type::Int64, - Type::UInt64, - Type::Date, - Type::Date32, - Type::DateTime, - Type::String, - Type::FixedString, - }; - - static constexpr std::array map_key_string_types - { - Type::String, - Type::FixedString + TypeIndex::Int8, + TypeIndex::UInt8, + TypeIndex::Int16, + TypeIndex::UInt16, + TypeIndex::Int32, + TypeIndex::UInt32, + TypeIndex::Int64, + TypeIndex::UInt64, + TypeIndex::Int128, + TypeIndex::UInt128, + TypeIndex::Int256, + TypeIndex::UInt256, + TypeIndex::Date, + TypeIndex::Date32, + TypeIndex::DateTime, + TypeIndex::String, + TypeIndex::FixedString, + TypeIndex::IPv4, + TypeIndex::UUID, }; static constexpr size_t MAX_NUMBER_OF_COLUMNS = 128; @@ -175,18 +112,19 @@ public: bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1, 2, 3, 4, 5, 6}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } bool useDefaultImplementationForConstants() const override { return false; } bool useDefaultImplementationForNulls() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments.size() > 7) + if (arguments.size() > 2) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 7", + "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 2", getName(), arguments.size()); + for (size_t i = 0; i != 2; ++i) { if (arguments.size() == i) @@ -203,22 +141,6 @@ public: } } - for (size_t i = 2; i != 7; ++i) - { - if (arguments.size() <= i) - break; - - if (!isUInt8(arguments[i])) - { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the {} argument of function {}, expected UInt8", - i + 1, - arguments[i]->getName(), - getName()); - } - } - return std::make_shared(); } @@ -241,40 +163,27 @@ public: if (arguments.size() > 1 && !arguments[1].column->onlyNull()) seed = arguments[1].column->getUInt(0); - bool allow_big_numbers = true; - if (arguments.size() > 2) - allow_big_numbers = arguments[2].column->getBool(0); - - bool allow_enums = true; - if (arguments.size() > 3) - allow_enums = arguments[3].column->getBool(0); - - bool allow_decimals = true; - if (arguments.size() > 4) - allow_decimals = arguments[4].column->getBool(0); - - bool allow_ip = true; - if (arguments.size() > 5) - allow_ip = arguments[5].column->getBool(0); - - bool only_string_map_key = false; - if (arguments.size() > 6) - only_string_map_key = arguments[6].column->getBool(0); - pcg64 rng(seed); if (number_of_columns == 0) number_of_columns = generateNumberOfColumns(rng); auto col_res = ColumnString::create(); - String generated_structure; + auto & string_column = assert_cast(*col_res); + auto & chars = string_column.getChars(); + WriteBufferFromVector buf(chars); for (size_t i = 0; i != number_of_columns; ++i) { if (i != 0) - generated_structure += ", "; - auto type = generateRandomType(rng, allow_big_numbers, allow_enums, allow_decimals, allow_ip, only_string_map_key); - generated_structure += "c" + std::to_string(i + 1) + " " + type; + writeCString(", ", buf); + String column_name = "c" + std::to_string(i + 1); + writeString(column_name, buf); + writeChar(' ', buf); + writeRandomType(column_name, rng, buf); } - col_res->insert(generated_structure); + + buf.finalize(); + chars.push_back(0); + string_column.getOffsets().push_back(chars.size()); return ColumnConst::create(std::move(col_res), input_rows_count); } @@ -285,181 +194,159 @@ private: return rng() % MAX_NUMBER_OF_COLUMNS + 1; } - /// Helper struct to call generateRandomTypeImpl with lots of bool template arguments without writing big if/else over all bool variables. - template - struct Dispatcher + template + void writeRandomType(const String & column_name, pcg64 & rng, WriteBuffer & buf, size_t depth = 0) const { - static auto call(const FunctionGenerateRandomStructure * f, pcg64 & rng) - { - return f->generateRandomTypeImpl(rng); - } - - template - static auto call(const FunctionGenerateRandomStructure * f, pcg64 & rng, bool b, Args1... ar1) - { - if (b) - return Dispatcher::call(f, rng, ar1...); - else - return Dispatcher::call(f, rng, ar1...); - } - - friend FunctionGenerateRandomStructure; - }; - - String generateRandomType(pcg64 & rng, bool allow_big_numbers, bool allow_enums, bool allow_decimals, bool allow_ip, bool allow_only_string_map_keys) const - { - return Dispatcher<>::call(this, rng, allow_big_numbers, allow_enums, allow_decimals, allow_ip, allow_only_string_map_keys, true); - } - - template - String generateRandomTypeImpl(pcg64 & rng, size_t depth = 0) const - { - constexpr auto all_types = getAllTypes(); + constexpr auto all_types = getAllTypes(); auto type = all_types[rng() % all_types.size()]; switch (type) { - case Type::FixedString: - return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - case Type::DateTime64: - return "DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")"; - case Type::Decimal32: - return "Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")"; - case Type::Decimal64: - return "Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")"; - case Type::Decimal128: - return "Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")"; - case Type::Decimal256: - return "Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")"; - case Type::Enum8: - return "Enum8(" + generateEnumValues(rng) + ")"; - case Type::Enum16: - return "Enum16(" + generateEnumValues(rng) + ")"; - case Type::LowCardinality: - return "LowCardinality(" + generateLowCardinalityNestedType(rng) + ")"; - case Type::Nullable: + case TypeIndex::UInt8: + if (rng() % 2) + writeCString("UInt8", buf); + else + writeCString("Bool", buf); + return; + case TypeIndex::FixedString: + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); + return; + case TypeIndex::DateTime64: + writeString("DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")", buf); + return; + case TypeIndex::Decimal32: + writeString("Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")", buf); + return; + case TypeIndex::Decimal64: + writeString("Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")", buf); + return; + case TypeIndex::Decimal128: + writeString("Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")", buf); + return; + case TypeIndex::Decimal256: + writeString("Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")", buf); + return; + case TypeIndex::Enum8: + writeCString("Enum8(", buf); + writeEnumValues(column_name, rng, buf); + writeChar(')', buf); + return; + case TypeIndex::Enum16: + writeCString("Enum16(", buf); + writeEnumValues(column_name, rng, buf); + writeChar(')', buf); + return; + case TypeIndex::LowCardinality: + writeCString("LowCardinality(", buf); + writeLowCardinalityNestedType(rng, buf); + writeChar(')', buf); + return; + case TypeIndex::Nullable: { - auto nested_type = generateRandomTypeImpl(rng, depth + 1); - return "Nullable(" + nested_type + ")"; + writeCString("Nullable(", buf); + writeRandomType(column_name, rng, buf, depth + 1); + writeChar(')', buf); + return; } - case Type::Array: + case TypeIndex::Array: { - auto nested_type = generateRandomTypeImpl(rng, depth + 1); - return "Array(" + nested_type + ")"; + writeCString("Array(", buf); + writeRandomType(column_name, rng, buf, depth + 1); + writeChar(')', buf); + return; } - case Type::Map: + case TypeIndex::Map: { - auto key_type = generateMapKeyType(rng); - auto value_type = generateRandomTypeImpl(rng, depth + 1); - return "Map(" + key_type + ", " + value_type + ")"; + writeCString("Map(", buf); + writeMapKeyType(rng, buf); + writeCString(", ", buf); + writeRandomType(column_name, rng, buf, depth + 1); + writeChar(')', buf); + return; } - case Type::Tuple: + case TypeIndex::Tuple: { size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; - bool named_tuple = rng() % 2; - String tuple_type = "Tuple("; + bool generate_nested = rng() % 2; + bool generate_named_tuple = rng() % 2; + if (generate_nested) + writeCString("Nested(", buf); + else + writeCString("Tuple(", buf); + for (size_t i = 0; i != elements; ++i) { if (i != 0) - tuple_type += ", "; - if (named_tuple) - tuple_type += "e" + std::to_string(i + 1) + " "; - tuple_type += generateRandomTypeImpl(rng, depth + 1); + writeCString(", ", buf); + + String element_name = "e" + std::to_string(i + 1); + if (generate_named_tuple || generate_nested) + { + writeString(element_name, buf); + writeChar(' ', buf); + } + writeRandomType(element_name, rng, buf, depth + 1); } - return tuple_type + ")"; - } - case Type::Nested: - { - size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; - String nested_type = "Nested("; - for (size_t i = 0; i != elements; ++i) - { - if (i != 0) - nested_type += ", "; - auto element_type = generateRandomTypeImpl(rng, depth + 1); - nested_type += "e" + std::to_string(i + 1) + " " + element_type; - } - return nested_type + ")"; + writeChar(')', buf); + return; } default: - return String(magic_enum::enum_name(type)); + writeString(magic_enum::enum_name(type), buf); + return; } } - template - String generateMapKeyType(pcg64 & rng) const + void writeMapKeyType(pcg64 & rng, WriteBuffer & buf) const { - Type type; - if constexpr (allow_only_string_map_keys) - type = map_key_string_types[rng() % map_key_string_types.size()]; + TypeIndex type = map_key_types[rng() % map_key_types.size()]; + if (type == TypeIndex::FixedString) + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); else - type = map_key_types[rng() % map_key_types.size()]; - - if (type == Type::FixedString) - return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - return String(magic_enum::enum_name(type)); + writeString(magic_enum::enum_name(type), buf); } - String generateLowCardinalityNestedType(pcg64 & rng) const + void writeLowCardinalityNestedType(pcg64 & rng, WriteBuffer & buf) const { - /// Support only String and FixedString. + /// Support only String and FixedString (maybe Nullable). String nested_type; + bool make_nullable = rng() % 2; + if (make_nullable) + writeCString("Nullable(", buf); + if (rng() % 2) - nested_type = "String"; + writeCString("String", buf); else - nested_type = "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - return rng() % 2 ? nested_type : "Nullable(" + nested_type + ")"; + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); + + if (make_nullable) + writeChar(')', buf); } - String generateEnumValues(pcg64 & rng) const + void writeEnumValues(const String & column_name, pcg64 & rng, WriteBuffer & buf) const { - /// Don't generate big enums, because it will lead to really big strings + /// Don't generate big enums, because it will lead to really big result /// and slowness of this function, and it can lead to `Max query size exceeded` /// while using this function with generateRandom. ssize_t num_values = rng() % 16 + 1; - String result; for (ssize_t i = 0; i != num_values; ++i) { if (i != 0) - result += ", "; - result += "'v" + std::to_string(i) + "' = " + std::to_string(i); + writeCString(", ", buf); + writeString("'" + column_name + "V" + std::to_string(i) + "' = " + std::to_string(i), buf); } - return result; } - template + template static constexpr auto getAllTypes() { - constexpr size_t big_integer_types_size = big_integer_types.size() * allow_big_numbers; - constexpr size_t enum_types_size = enum_types.size() * allow_enums; - constexpr size_t decimal_types_size = decimal_types.size() * allow_decimals; - constexpr size_t big_decimal_types_size = big_decimal_types.size() * allow_big_numbers * allow_decimals; - constexpr size_t ip_types_size = ip_types.size() * allow_ip; constexpr size_t complex_types_size = complex_types.size() * allow_complex_types; - - constexpr size_t result_size = simple_types.size() + big_integer_types_size + enum_types_size + decimal_types_size - + big_decimal_types_size + ip_types_size + complex_types_size; - std::array result; + constexpr size_t result_size = simple_types.size() + complex_types_size; + std::array result; size_t index = 0; for (size_t i = 0; i != simple_types.size(); ++i, ++index) result[index] = simple_types[i]; - for (size_t i = 0; i != big_integer_types_size; ++i, ++index) - result[index] = big_integer_types[i]; - - for (size_t i = 0; i != enum_types_size; ++i, ++index) - result[index] = enum_types[i]; - - for (size_t i = 0; i != decimal_types_size; ++i, ++index) - result[index] = decimal_types[i]; - - for (size_t i = 0; i != big_decimal_types_size; ++i, ++index) - result[index] = big_decimal_types[i]; - - for (size_t i = 0; i != ip_types_size; ++i, ++index) - result[index] = ip_types[i]; - for (size_t i = 0; i != complex_types_size; ++i, ++index) result[index] = complex_types[i]; @@ -474,26 +361,15 @@ REGISTER_FUNCTION(GenerateRandomStructure) { R"( Generates a random table structure. -This function takes 4 optional constant arguments: -1) the number of column in the result structure (random by default) -2) random seed (random by default) -3) flag that indicates if big number types can be used (true by default) -4) flag that indicates if enum types can be used (true by default) -5) flag that indicates if decimal types can be used (true by default) -6) flag that indicates if ip types (IPv4, IPv6) can be used (true by default) -7) flag that indicates if map keys should be only String or FixedString (false by default) +This function takes 2 optional constant arguments: +the number of columns in the result structure (random by default) and random seed (random by default) The maximum number of columns is 128. The function returns a value of type String. )", Documentation::Examples{ {"random", "SELECT generateRandomStructure()"}, - {"with specified number of arguments", "SELECT generateRandomStructure(10)"}, + {"with specified number of columns", "SELECT generateRandomStructure(10)"}, {"with specified seed", "SELECT generateRandomStructure(10, 42)"}, - {"without big number types", "SELECT generateRandomStructure(10, NULL, false)"}, - {"without enum types", "SELECT generateRandomStructure(10, NULL, true, false)"}, - {"without decimal types", "SELECT generateRandomStructure(10, NULL, true, true, false)"}, - {"without ip types", "SELECT generateRandomStructure(10, NULL, true, true, true, false)"}, - {"with only string mak key types", "SELECT generateRandomStructure(10, NULL, true, true, true, true, true)"}, }, Documentation::Categories{"Random"} }, diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index 76d89828071..65bdc530f10 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -1,11 +1,4 @@ -c1 Date, c2 Bool, c3 Int16, c4 Map(Int64, Array(Bool)), c5 Decimal256(30) -c1 String, c2 Float64, c3 Enum8(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4), c4 UInt64, c5 Date -c1 Nested(e1 Int64, e2 Int16, e3 Map(Int16, LowCardinality(Nullable(String))), e4 UInt8, e5 Nested(e1 Array(Nullable(DateTime)), e2 Nullable(Bool), e3 UInt8, e4 UInt64, e5 Decimal64(6), e6 DateTime), e6 LowCardinality(Nullable(String))), c2 Date, c3 Int32, c4 IPv4, c5 Decimal32(8) -c1 Date, c2 UInt16, c3 UInt256, c4 Nullable(IPv4), c5 Nullable(Decimal64(17)) -c1 Array(Int64), c2 Map(String, LowCardinality(String)), c3 Date, c4 Map(Int64, UInt128), c5 UInt8 -c1 Date, c2 UInt16, c3 UInt256, c4 Nullable(Decimal128(37)), c5 DateTime64(8) -c1 Date, c2 Bool, c3 Int16, c4 Map(FixedString(120), Bool), c5 Decimal256(30) -c1 String, c2 Float64, c3 Enum8(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4), c4 UInt64, c5 Date +c1 String, c2 UInt256, c3 String, c4 Decimal128(7), c5 UInt128 String Const(String) -1977-07-28 true 5389 +` 90465455320735604871982424534384518837533904778028808627865442405232847164685 5& -3034771008825448884614719061068.2821046 75820566154622566322847299106656624693 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql index 061fbc24219..a28c159cff5 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.sql +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -1,29 +1,11 @@ select generateRandomStructure(5, 42); -select generateRandomStructure(5, 42, false); -select generateRandomStructure(5, 42, false, false); -select generateRandomStructure(5, 42, true, false); -select generateRandomStructure(5, 42, true, true, false); -select generateRandomStructure(5, 42, true, true, true, false); -select generateRandomStructure(5, 42, true, true, true, true, true); -select generateRandomStructure(5, 42, false, true, true); select toTypeName(generateRandomStructure(5, 42)); select toColumnTypeName(generateRandomStructure(5, 42)); -SELECT * FROM generateRandom(generateRandomStructure(3, 42), 42) LIMIT 1; +SELECT * FROM generateRandom(generateRandomStructure(5, 42), 42) LIMIT 1; -select generateRandomStructure(5, 42, false, false, false, false, true, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select generateRandomStructure(5, 42, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} select generateRandomStructure('5'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(5, 42, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(5, 42, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(5, 42, false, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(5, 42, false, false, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(5, 42, false, false, false, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(materialize(5), 42, false, false); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, materialize(42), false, false); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, 42, materialize(false), false); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, 42, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, 42, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, 42, false, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, 42, false, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, 42, false, false, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(materialize(5), 42); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, materialize(42)); -- {serverError ILLEGAL_COLUMN} From 9d1ee044b080237df41ff40f3376d433e6b274a4 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 11 May 2023 12:02:00 +0000 Subject: [PATCH 055/308] Update example --- docs/en/sql-reference/table-functions/generate.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/table-functions/generate.md b/docs/en/sql-reference/table-functions/generate.md index 7a17c61eeca..b2777418e4f 100644 --- a/docs/en/sql-reference/table-functions/generate.md +++ b/docs/en/sql-reference/table-functions/generate.md @@ -56,15 +56,15 @@ SELECT * FROM random; In combination with [generateRandomStructure](../../sql-reference/functions/other-functions.md#generateRandomStructure): ```sql -SELECT * FROM generateRandom(generateRandomStructure(3, 24), 24) LIMIT 3; +SELECT * FROM generateRandom(generateRandomStructure(4, 101), 101) LIMIT 3; ``` ```text -┌─────────────────────────c1─┬─────c2─┬───────────────────c3─┬───────────────────────────────────────c4─┐ -│ 2085-07-05 23:48:43.345759 │ -20656 │ 1632406185424686785 │ -210464718903845545171230673454802.15238 │ -│ 1971-07-17 16:32:36.390777 │ -27071 │ -1553021742787219162 │ 1095158319964381336405161704296125.08074 │ -│ 2024-02-19 13:14:32.902513 │ 24913 │ 7727442383333447640 │ 1090748832613398997057187200834127.07109 │ -└────────────────────────────┴────────┴──────────────────────┴──────────────────────────────────────────┘ +┌──────────────────c1─┬──────────────────c2─┬─c3─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─c4──────────────────────────────────────┐ +│ 1996-04-15 06:40:05 │ 33954608387.2844801 │ ['232.78.216.176','9.244.59.211','211.21.80.152','44.49.94.109','165.77.195.182','68.167.134.239','212.13.24.185','1.197.255.35','192.55.131.232'] │ 45d9:2b52:ab6:1c59:185b:515:c5b6:b781 │ +│ 2063-01-13 01:22:27 │ 36155064970.9514454 │ ['176.140.188.101'] │ c65a:2626:41df:8dee:ec99:f68d:c6dd:6b30 │ +│ 2090-02-28 14:50:56 │ 3864327452.3901373 │ ['155.114.30.32'] │ 57e9:5229:93ab:fbf3:aae7:e0e4:d1eb:86b │ +└─────────────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────┘ ``` ## Related content From 5ed1b12e19939e399dccc1776901c4e4ddede29a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 11 May 2023 12:12:43 +0000 Subject: [PATCH 056/308] Fix build --- src/Functions/generateRandomStructure.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index 9fe321365e5..eccccdf563f 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes class FunctionGenerateRandomStructure : public IFunction { private: - static constexpr std::array simple_types + static constexpr std::array simple_types { TypeIndex::Int8, TypeIndex::UInt8, @@ -64,7 +64,7 @@ private: TypeIndex::Map, }; - static constexpr std::array map_key_types + static constexpr std::array map_key_types { TypeIndex::Int8, TypeIndex::UInt8, From 20007504a7d475274e07e46a0f4c27e577cad43f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 11 May 2023 14:16:48 +0200 Subject: [PATCH 057/308] Handle exception in finalize inside WriteBufferFromPocoSocket destructor --- src/IO/WriteBufferFromPocoSocket.cpp | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 039110dfb62..79534b0f030 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -106,7 +106,22 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() { - finalize(); +#ifndef NDEBUG + if (!finalized) + { + LOG_ERROR(log, "WriteBufferFromPocoSocket is not finalized in destructor. It's a bug"); + std::terminate(); + } +#else + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +#endif } } From 602b9a740ec2bf3064d2970c54fbc92da9304991 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 May 2023 19:39:33 +0000 Subject: [PATCH 058/308] Make better, allow generateRandom without structure argument --- .../sql-reference/table-functions/generate.md | 30 +- ...pp => FunctionGenerateRandomStructure.cpp} | 402 ++++++++++-------- .../FunctionGenerateRandomStructure.h | 45 ++ .../TableFunctionGenerateRandom.cpp | 67 ++- .../02586_generate_random_structure.reference | 16 +- .../02586_generate_random_structure.sql | 6 + 6 files changed, 369 insertions(+), 197 deletions(-) rename src/Functions/{generateRandomStructure.cpp => FunctionGenerateRandomStructure.cpp} (51%) create mode 100644 src/Functions/FunctionGenerateRandomStructure.h diff --git a/docs/en/sql-reference/table-functions/generate.md b/docs/en/sql-reference/table-functions/generate.md index b2777418e4f..6ceeb63cbb3 100644 --- a/docs/en/sql-reference/table-functions/generate.md +++ b/docs/en/sql-reference/table-functions/generate.md @@ -11,7 +11,7 @@ Allows to populate test tables with data. Not all types are supported. ``` sql -generateRandom('name TypeName[, name TypeName]...', [, 'random_seed'[, 'max_string_length'[, 'max_array_length']]]) +generateRandom(['name TypeName[, name TypeName]...', [, 'random_seed'[, 'max_string_length'[, 'max_array_length']]]]) ``` **Arguments** @@ -67,5 +67,33 @@ SELECT * FROM generateRandom(generateRandomStructure(4, 101), 101) LIMIT 3; └─────────────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────┘ ``` +With missing `structure` argument (in this case the structure is random): + +```sql +SELECT * FROM generateRandom() LIMIT 3; +``` + +```text +┌───c1─┬─────────c2─┬─────────────────────c3─┬──────────────────────c4─┬─c5───────┐ +│ -128 │ 317300854 │ 2030-08-16 08:22:20.65 │ 1994-08-16 12:08:56.745 │ R0qgiC46 │ +│ 40 │ -744906827 │ 2059-04-16 06:31:36.98 │ 1975-07-16 16:28:43.893 │ PuH4M*MZ │ +│ -55 │ 698652232 │ 2052-08-04 20:13:39.68 │ 1998-09-20 03:48:29.279 │ │ +└──────┴────────────┴────────────────────────┴─────────────────────────┴──────────┘ +``` + +With random seed both for random structure and random data: + +```sql +SELECT * FROM generateRandom(11) LIMIT 3; +``` + +```text +┌───────────────────────────────────────c1─┬─────────────────────────────────────────────────────────────────────────────c2─┬─────────────────────────────────────────────────────────────────────────────c3─┬─────────c4─┬─────────────────────────────────────────────────────────────────────────────c5─┬──────────────────────c6─┬─c7──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─c8──────────────────────────────────────┬─────────c9─┐ +│ -77422512305044606600216318673365695785 │ 636812099959807642229.503817849012019401335326013846687285151335352272727523 │ -34944452809785978175157829109276115789694605299387223845886143311647505037529 │ 544473976 │ 111220388331710079615337037674887514156741572807049614590010583571763691328563 │ 22016.22623506465 │ {'2052-01-31 20:25:33':4306400876908509081044405485378623663,'1993-04-16 15:58:49':164367354809499452887861212674772770279,'2101-08-19 03:07:18':-60676948945963385477105077735447194811,'2039-12-22 22:31:39':-59227773536703059515222628111999932330} │ a7b2:8f58:4d07:6707:4189:80cf:92f5:902d │ 1950-07-14 │ +│ -159940486888657488786004075627859832441 │ 629206527868163085099.8195700356331771569105231840157308480121506729741348442 │ -53203761250367440823323469081755775164053964440214841464405368882783634063735 │ 2187136525 │ 94881662451116595672491944222189810087991610568040618106057495823910493624275 │ 1.3095786748458954e-104 │ {} │ a051:e3da:2e0a:c69:7835:aed6:e8b:3817 │ 1943-03-25 │ +│ -5239084224358020595591895205940528518 │ -529937657954363597180.1709207212648004850138812370209091520162977548101577846 │ 47490343304582536176125359129223180987770215457970451211489086575421345731671 │ 1637451978 │ 101899445785010192893461828129714741298630410942962837910400961787305271699002 │ 2.4344456058391296e223 │ {'2013-12-22 17:42:43':80271108282641375975566414544777036006,'2041-03-08 10:28:17':169706054082247533128707458270535852845,'1986-08-31 23:07:38':-54371542820364299444195390357730624136,'2094-04-23 21:26:50':7944954483303909347454597499139023465} │ 1293:a726:e899:9bfc:8c6f:2aa1:22c9:b635 │ 1924-11-20 │ +└──────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────┴────────────┴────────────────────────────────────────────────────────────────────────────────┴─────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────┴────────────┘ +``` + ## Related content - Blog: [Generating random data in ClickHouse](https://clickhouse.com/blog/generating-random-test-distribution-data-for-clickhouse) diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp similarity index 51% rename from src/Functions/generateRandomStructure.cpp rename to src/Functions/FunctionGenerateRandomStructure.cpp index eccccdf563f..4cf783212cb 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -1,9 +1,10 @@ -#include -#include -#include +#include #include #include #include +#include +#include +#include #include #include @@ -19,10 +20,18 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -class FunctionGenerateRandomStructure : public IFunction +namespace { -private: - static constexpr std::array simple_types + const size_t MAX_NUMBER_OF_COLUMNS = 128; + const size_t MAX_TUPLE_ELEMENTS = 16; + const size_t MAX_DATETIME64_PRECISION = 9; + const size_t MAX_DECIMAL32_PRECISION = 9; + const size_t MAX_DECIMAL64_PRECISION = 18; + const size_t MAX_DECIMAL128_PRECISION = 38; + const size_t MAX_DECIMAL256_PRECISION = 76; + const size_t MAX_DEPTH = 32; + + constexpr std::array simple_types { TypeIndex::Int8, TypeIndex::UInt8, @@ -55,7 +64,7 @@ private: TypeIndex::UUID, }; - static constexpr std::array complex_types + constexpr std::array complex_types { TypeIndex::Nullable, TypeIndex::LowCardinality, @@ -64,7 +73,7 @@ private: TypeIndex::Map, }; - static constexpr std::array map_key_types + constexpr std::array map_key_types { TypeIndex::Int8, TypeIndex::UInt8, @@ -84,119 +93,147 @@ private: TypeIndex::String, TypeIndex::FixedString, TypeIndex::IPv4, + TypeIndex::Enum8, + TypeIndex::Enum16, + TypeIndex::UUID, + TypeIndex::LowCardinality, + }; + + constexpr std::array suspicious_lc_types + { + TypeIndex::Int8, + TypeIndex::UInt8, + TypeIndex::Int16, + TypeIndex::UInt16, + TypeIndex::Int32, + TypeIndex::UInt32, + TypeIndex::Int64, + TypeIndex::UInt64, + TypeIndex::Int128, + TypeIndex::UInt128, + TypeIndex::Int256, + TypeIndex::UInt256, + TypeIndex::Float32, + TypeIndex::Float64, + TypeIndex::Date, + TypeIndex::Date32, + TypeIndex::DateTime, + TypeIndex::String, + TypeIndex::FixedString, + TypeIndex::IPv4, + TypeIndex::IPv6, TypeIndex::UUID, }; - static constexpr size_t MAX_NUMBER_OF_COLUMNS = 128; - static constexpr size_t MAX_TUPLE_ELEMENTS = 16; - static constexpr size_t MAX_DATETIME64_PRECISION = 9; - static constexpr size_t MAX_DECIMAL32_PRECISION = 9; - static constexpr size_t MAX_DECIMAL64_PRECISION = 18; - static constexpr size_t MAX_DECIMAL128_PRECISION = 38; - static constexpr size_t MAX_DECIMAL256_PRECISION = 76; - static constexpr size_t MAX_DEPTH = 32; - -public: - static constexpr auto name = "generateRandomStructure"; - - static FunctionPtr create(ContextPtr /*context*/) + template + constexpr auto getAllTypes() { - return std::make_shared(); + constexpr size_t complex_types_size = complex_types.size() * allow_complex_types; + constexpr size_t result_size = simple_types.size() + complex_types_size; + std::array result; + size_t index = 0; + + for (size_t i = 0; i != simple_types.size(); ++i, ++index) + result[index] = simple_types[i]; + + for (size_t i = 0; i != complex_types_size; ++i, ++index) + result[index] = complex_types[i]; + + return result; } - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 0; } - - bool isVariadic() const override { return true; } - bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } - bool useDefaultImplementationForConstants() const override { return false; } - bool useDefaultImplementationForNulls() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments.size() > 2) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 2", - getName(), arguments.size()); - - - for (size_t i = 0; i != 2; ++i) - { - if (arguments.size() == i) - break; - - if (!isUnsignedInteger(arguments[i]) && !arguments[i]->onlyNull()) - { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the {} argument of function {}, expected unsigned integer or Null", - i + 1, - arguments[i]->getName(), - getName()); - } - } - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override - { - size_t seed = randomSeed(); - size_t number_of_columns = 0; - - if (!arguments.empty() && !arguments[0].column->onlyNull()) - { - number_of_columns = arguments[0].column->getUInt(0); - if (number_of_columns > MAX_NUMBER_OF_COLUMNS) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Maximum allowed number of columns is {}, got {}", - MAX_NUMBER_OF_COLUMNS, - number_of_columns); - } - - if (arguments.size() > 1 && !arguments[1].column->onlyNull()) - seed = arguments[1].column->getUInt(0); - - pcg64 rng(seed); - if (number_of_columns == 0) - number_of_columns = generateNumberOfColumns(rng); - - auto col_res = ColumnString::create(); - auto & string_column = assert_cast(*col_res); - auto & chars = string_column.getChars(); - WriteBufferFromVector buf(chars); - for (size_t i = 0; i != number_of_columns; ++i) - { - if (i != 0) - writeCString(", ", buf); - String column_name = "c" + std::to_string(i + 1); - writeString(column_name, buf); - writeChar(' ', buf); - writeRandomType(column_name, rng, buf); - } - - buf.finalize(); - chars.push_back(0); - string_column.getOffsets().push_back(chars.size()); - return ColumnConst::create(std::move(col_res), input_rows_count); - } - -private: - - size_t generateNumberOfColumns(pcg64 & rng) const + size_t generateNumberOfColumns(pcg64 & rng) { return rng() % MAX_NUMBER_OF_COLUMNS + 1; } - template - void writeRandomType(const String & column_name, pcg64 & rng, WriteBuffer & buf, size_t depth = 0) const + void writeLowCardinalityNestedType(pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types, bool allow_nullable = true) { + bool make_nullable = allow_nullable & rng() % 2; + if (make_nullable) + writeCString("Nullable(", buf); + + if (allow_suspicious_lc_types) + { + TypeIndex type = suspicious_lc_types[rng() % map_key_types.size()]; + if (type == TypeIndex::FixedString) + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); + else + writeString(magic_enum::enum_name(type), buf); + } + else + { + /// Support only String and FixedString. + if (rng() % 2) + writeCString("String", buf); + else + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); + } + + if (make_nullable) + writeChar(')', buf); + } + + + void writeEnumValues(const String & column_name, pcg64 & rng, WriteBuffer & buf, ssize_t max_value) + { + /// Don't generate big enums, because it will lead to really big result + /// and slowness of this function, and it can lead to `Max query size exceeded` + /// while using this function with generateRandom. + size_t num_values = rng() % 16 + 1; + std::vector values(num_values); + + /// Generate random numbers from range [-(max_value + 1), max_value - num_values + 1] + for (Int16 & x : values) + x = rng() % (2 * (max_value + 1) - num_values) - max_value - 1; + /// Make all numbers unique. + std::sort(values.begin(), values.end()); + for (size_t i = 0; i < num_values; ++i) + values[i] += i; + std::shuffle(values.begin(), values.end(), rng); + for (size_t i = 0; i != num_values; ++i) + { + if (i != 0) + writeCString(", ", buf); + writeString("'" + column_name + "V" + std::to_string(i) + "' = " + std::to_string(values[i]), buf); + } + } + + void writeMapKeyType(const String & column_name, pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types) + { + TypeIndex type = map_key_types[rng() % map_key_types.size()]; + switch (type) + { + case TypeIndex::FixedString: + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); + break; + case TypeIndex::LowCardinality: + writeCString("LowCardinality(", buf); + writeLowCardinalityNestedType(rng, buf, allow_suspicious_lc_types, false); + writeChar(')', buf); + break; + case TypeIndex::Enum8: + writeCString("Enum8(", buf); + writeEnumValues(column_name, rng, buf, INT8_MAX); + writeChar(')', buf); + break; + case TypeIndex::Enum16: + writeCString("Enum16(", buf); + writeEnumValues(column_name, rng, buf, INT16_MAX); + writeChar(')', buf); + break; + default: + writeString(magic_enum::enum_name(type), buf); + break; + } + } + + template + void writeRandomType(const String & column_name, pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types, size_t depth = 0) + { + if (allow_complex_types && depth > MAX_DEPTH) + writeRandomType(column_name, rng, buf, depth); + constexpr auto all_types = getAllTypes(); auto type = all_types[rng() % all_types.size()]; @@ -212,55 +249,55 @@ private: writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); return; case TypeIndex::DateTime64: - writeString("DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")", buf); + writeString("DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION + 1) + ")", buf); return; case TypeIndex::Decimal32: - writeString("Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")", buf); + writeString("Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION + 1) + ")", buf); return; case TypeIndex::Decimal64: - writeString("Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")", buf); + writeString("Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION + 1) + ")", buf); return; case TypeIndex::Decimal128: - writeString("Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")", buf); + writeString("Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION + 1) + ")", buf); return; case TypeIndex::Decimal256: - writeString("Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")", buf); + writeString("Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION + 1) + ")", buf); return; case TypeIndex::Enum8: writeCString("Enum8(", buf); - writeEnumValues(column_name, rng, buf); + writeEnumValues(column_name, rng, buf, INT8_MAX); writeChar(')', buf); return; case TypeIndex::Enum16: writeCString("Enum16(", buf); - writeEnumValues(column_name, rng, buf); + writeEnumValues(column_name, rng, buf, INT16_MAX); writeChar(')', buf); return; case TypeIndex::LowCardinality: writeCString("LowCardinality(", buf); - writeLowCardinalityNestedType(rng, buf); + writeLowCardinalityNestedType(rng, buf, allow_suspicious_lc_types); writeChar(')', buf); return; case TypeIndex::Nullable: { writeCString("Nullable(", buf); - writeRandomType(column_name, rng, buf, depth + 1); + writeRandomType(column_name, rng, buf, allow_suspicious_lc_types, depth + 1); writeChar(')', buf); return; } case TypeIndex::Array: { writeCString("Array(", buf); - writeRandomType(column_name, rng, buf, depth + 1); + writeRandomType(column_name, rng, buf, allow_suspicious_lc_types, depth + 1); writeChar(')', buf); return; } case TypeIndex::Map: { writeCString("Map(", buf); - writeMapKeyType(rng, buf); + writeMapKeyType(column_name, rng, buf, allow_suspicious_lc_types); writeCString(", ", buf); - writeRandomType(column_name, rng, buf, depth + 1); + writeRandomType(column_name, rng, buf, allow_suspicious_lc_types, depth + 1); writeChar(')', buf); return; } @@ -285,7 +322,7 @@ private: writeString(element_name, buf); writeChar(' ', buf); } - writeRandomType(element_name, rng, buf, depth + 1); + writeRandomType(element_name, rng, buf, allow_suspicious_lc_types, depth + 1); } writeChar(')', buf); return; @@ -296,64 +333,87 @@ private: } } - void writeMapKeyType(pcg64 & rng, WriteBuffer & buf) const + void writeRandomStructure(pcg64 & rng, size_t number_of_columns, WriteBuffer & buf, bool allow_suspicious_lc_types) { - TypeIndex type = map_key_types[rng() % map_key_types.size()]; - if (type == TypeIndex::FixedString) - writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); - else - writeString(magic_enum::enum_name(type), buf); - } - - void writeLowCardinalityNestedType(pcg64 & rng, WriteBuffer & buf) const - { - /// Support only String and FixedString (maybe Nullable). - String nested_type; - bool make_nullable = rng() % 2; - if (make_nullable) - writeCString("Nullable(", buf); - - if (rng() % 2) - writeCString("String", buf); - else - writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); - - if (make_nullable) - writeChar(')', buf); - } - - void writeEnumValues(const String & column_name, pcg64 & rng, WriteBuffer & buf) const - { - /// Don't generate big enums, because it will lead to really big result - /// and slowness of this function, and it can lead to `Max query size exceeded` - /// while using this function with generateRandom. - ssize_t num_values = rng() % 16 + 1; - for (ssize_t i = 0; i != num_values; ++i) + for (size_t i = 0; i != number_of_columns; ++i) { if (i != 0) writeCString(", ", buf); - writeString("'" + column_name + "V" + std::to_string(i) + "' = " + std::to_string(i), buf); + String column_name = "c" + std::to_string(i + 1); + writeString(column_name, buf); + writeChar(' ', buf); + writeRandomType(column_name, rng, buf, allow_suspicious_lc_types); + } + } +} + +DataTypePtr FunctionGenerateRandomStructure::getReturnTypeImpl(const DataTypes & arguments) const +{ + if (arguments.size() > 2) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 2", + getName(), arguments.size()); + + + for (size_t i = 0; i != arguments.size(); ++i) + { + if (!isUnsignedInteger(arguments[i]) && !arguments[i]->onlyNull()) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the {} argument of function {}, expected unsigned integer or Null", + arguments[i]->getName(), + i + 1, + getName()); } } - template - static constexpr auto getAllTypes() + return std::make_shared(); +} + +ColumnPtr FunctionGenerateRandomStructure::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const +{ + size_t seed = randomSeed(); + size_t number_of_columns = 0; + + if (!arguments.empty() && !arguments[0].column->onlyNull()) { - constexpr size_t complex_types_size = complex_types.size() * allow_complex_types; - constexpr size_t result_size = simple_types.size() + complex_types_size; - std::array result; - size_t index = 0; - - for (size_t i = 0; i != simple_types.size(); ++i, ++index) - result[index] = simple_types[i]; - - for (size_t i = 0; i != complex_types_size; ++i, ++index) - result[index] = complex_types[i]; - - return result; + number_of_columns = arguments[0].column->getUInt(0); + if (number_of_columns > MAX_NUMBER_OF_COLUMNS) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Maximum allowed number of columns is {}, got {}", + MAX_NUMBER_OF_COLUMNS, + number_of_columns); } -}; + if (arguments.size() > 1 && !arguments[1].column->onlyNull()) + seed = arguments[1].column->getUInt(0); + + pcg64 rng(seed); + if (number_of_columns == 0) + number_of_columns = generateNumberOfColumns(rng); + + auto col_res = ColumnString::create(); + auto & string_column = assert_cast(*col_res); + auto & chars = string_column.getChars(); + WriteBufferFromVector buf(chars); + writeRandomStructure(rng, number_of_columns, buf, allow_suspicious_lc_types); + buf.finalize(); + chars.push_back(0); + string_column.getOffsets().push_back(chars.size()); + return ColumnConst::create(std::move(col_res), input_rows_count); +} + +String FunctionGenerateRandomStructure::generateRandomStructure(size_t seed, const ContextPtr & context) +{ + pcg64 rng(seed); + size_t number_of_columns = generateNumberOfColumns(rng); + WriteBufferFromOwnString buf; + writeRandomStructure(rng, number_of_columns, buf, context->getSettingsRef().allow_suspicious_low_cardinality_types); + return buf.str(); +} REGISTER_FUNCTION(GenerateRandomStructure) { diff --git a/src/Functions/FunctionGenerateRandomStructure.h b/src/Functions/FunctionGenerateRandomStructure.h new file mode 100644 index 00000000000..1d1bcb1a0a8 --- /dev/null +++ b/src/Functions/FunctionGenerateRandomStructure.h @@ -0,0 +1,45 @@ +#include +#include + +#include + +namespace DB +{ + +class FunctionGenerateRandomStructure : public IFunction +{ +public: + static constexpr auto name = "generateRandomStructure"; + + explicit FunctionGenerateRandomStructure(bool allow_suspicious_lc_types_) : allow_suspicious_lc_types(allow_suspicious_lc_types_) + { + } + + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context->getSettingsRef().allow_suspicious_low_cardinality_types.value); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isVariadic() const override { return true; } + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } + bool useDefaultImplementationForConstants() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override; + + static String generateRandomStructure(size_t seed, const ContextPtr & context); + +private: + bool allow_suspicious_lc_types; +}; + +} diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 12cbda334a3..a9e81c78c07 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -1,20 +1,20 @@ -#include #include -#include #include #include #include #include -#include #include #include #include +#include #include #include +#include + #include "registerTableFunctions.h" @@ -41,13 +41,21 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co if (args.empty()) return; - if (args.size() > 4) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Table function '{}' requires at most four arguments: " - " structure, [random_seed, max_string_length, max_array_length].", getName()); + /// First, check if first argument is structure or seed. + const auto * first_arg_literal = args[0]->as(); + bool first_argument_is_structure = !first_arg_literal || first_arg_literal->value.getType() == Field::Types::String; + size_t max_args = first_argument_is_structure ? 4 : 3; - /// Allow constant expression for structure argument, it can be generated using generateRandomStructure function. - args[0] = evaluateConstantExpressionAsLiteral(args[0], context); + if (args.size() > max_args) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Table function '{}' requires at most four (or three if structure is missing) arguments: " + " [structure, random_seed, max_string_length, max_array_length].", getName()); + + if (first_argument_is_structure) + { + /// Allow constant expression for structure argument, it can be generated using generateRandomStructure function. + args[0] = evaluateConstantExpressionAsLiteral(args[0], context); + } // All the arguments must be literals. for (const auto & arg : args) @@ -55,26 +63,39 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co if (!arg->as()) { throw Exception(ErrorCodes::BAD_ARGUMENTS, - "All arguments of table function '{}' must be literals. " + "All arguments of table function '{}' except structure argument must be literals. " "Got '{}' instead", getName(), arg->formatForErrorMessage()); } } - /// Parsing first argument as table structure and creating a sample block - structure = checkAndGetLiteralArgument(args[0], "structure"); + size_t arg_index = 0; - if (args.size() >= 2) + if (first_argument_is_structure) { - const auto & literal = args[1]->as(); + /// Parsing first argument as table structure and creating a sample block + structure = checkAndGetLiteralArgument(args[arg_index], "structure"); + ++arg_index; + } + + if (args.size() >= arg_index + 1) + { + const auto & literal = args[arg_index]->as(); + ++arg_index; if (!literal.value.isNull()) random_seed = checkAndGetLiteralArgument(literal, "random_seed"); } - if (args.size() >= 3) - max_string_length = checkAndGetLiteralArgument(args[2], "max_string_length"); + if (args.size() >= arg_index + 1) + { + max_string_length = checkAndGetLiteralArgument(args[arg_index], "max_string_length"); + ++arg_index; + } - if (args.size() == 4) - max_array_length = checkAndGetLiteralArgument(args[3], "max_string_length"); + if (args.size() == arg_index + 1) + { + max_array_length = checkAndGetLiteralArgument(args[arg_index], "max_string_length"); + ++arg_index; + } } ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(ContextPtr context) const @@ -82,11 +103,11 @@ ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(ContextP if (structure == "auto") { if (structure_hint.empty()) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Table function '{}' was used without structure argument but structure could not be determined automatically. Please, " - "provide structure manually", - getName()); + { + auto random_structure = FunctionGenerateRandomStructure::generateRandomStructure(random_seed.value_or(randomSeed()), context); + return parseColumnsListFromString(random_structure, context); + } + return structure_hint; } diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index 65bdc530f10..bd2009830f9 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -1,4 +1,16 @@ -c1 String, c2 UInt256, c3 String, c4 Decimal128(7), c5 UInt128 +c1 String, c2 UInt256, c3 String, c4 Decimal128(8), c5 UInt128 String Const(String) -` 90465455320735604871982424534384518837533904778028808627865442405232847164685 5& -3034771008825448884614719061068.2821046 75820566154622566322847299106656624693 +` 90465455320735604871982424534384518837533904778028808627865442405232847164685 5& -303477100882544888461471906106.82821046 75820566154622566322847299106656624693 +c1 Int128 +c2 Decimal(76, 55) +c3 Int256 +c4 UInt32 +c5 UInt256 +c6 Float64 +c7 Map(DateTime, Int128) +c8 IPv6 +c9 Date32 +-77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':127575633389498667752072479581409788016,'2052-11-08 23:07:13':-53938349319031918752329205601430421582,'2104-07-17 19:59:35':84394139582225600743319968813775553298,'2034-11-07 21:10:22':151550220355687100498925996413330909596} 328a:eccb:530f:23c3:275d:7eec:2b1b:9c29 2112-05-13 +-77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':127575633389498667752072479581409788016,'2052-11-08 23:07:13':-53938349319031918752329205601430421582,'2104-07-17 19:59:35':84394139582225600743319968813775553298,'2034-11-07 21:10:22':151550220355687100498925996413330909596} 328a:eccb:530f:23c3:275d:7eec:2b1b:9c29 2112-05-13 +-77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':166979754159728572703419507823025932071} 8eff:8d3c:5a2c:fa5f:b2bf:2b0e:ff23:beb2 2143-03-03 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql index a28c159cff5..174e2be261e 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.sql +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -9,3 +9,9 @@ select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMEN select generateRandomStructure(materialize(5), 42); -- {serverError ILLEGAL_COLUMN} select generateRandomStructure(5, materialize(42)); -- {serverError ILLEGAL_COLUMN} +desc generateRandom(11); +select * from generateRandom(11) limit 1; +select * from generateRandom(11, 2) limit 1; +select * from generateRandom(11, 2, 2) limit 1; +select * from generateRandom(11, 2, 2, 2) limit 1; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} + From aa7ab1f23badfc798116e7eeece8e62a0dfcfa9b Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 15 May 2023 11:20:03 +0000 Subject: [PATCH 059/308] Fix comments --- .../functions/other-functions.md | 2 ++ .../FunctionGenerateRandomStructure.cpp | 25 +++++++++++-------- .../02586_generate_random_structure.reference | 1 + .../02586_generate_random_structure.sql | 3 +++ 4 files changed, 21 insertions(+), 10 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index e235a3db393..20e1168ed5a 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2677,5 +2677,7 @@ Result: └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` +**Note**: the maximum nesting depth of complex types (Array, Tuple, Map, Nested) is limited to 16. + This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables. diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index 4cf783212cb..75455cdda85 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -29,7 +30,7 @@ namespace const size_t MAX_DECIMAL64_PRECISION = 18; const size_t MAX_DECIMAL128_PRECISION = 38; const size_t MAX_DECIMAL256_PRECISION = 76; - const size_t MAX_DEPTH = 32; + const size_t MAX_DEPTH = 16; constexpr std::array simple_types { @@ -147,15 +148,16 @@ namespace return rng() % MAX_NUMBER_OF_COLUMNS + 1; } - void writeLowCardinalityNestedType(pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types, bool allow_nullable = true) + void writeLowCardinalityNestedType(pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types) { - bool make_nullable = allow_nullable & rng() % 2; + bool make_nullable = rng() % 2; if (make_nullable) writeCString("Nullable(", buf); if (allow_suspicious_lc_types) { - TypeIndex type = suspicious_lc_types[rng() % map_key_types.size()]; + TypeIndex type = suspicious_lc_types[rng() % suspicious_lc_types.size()]; + if (type == TypeIndex::FixedString) writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); else @@ -174,7 +176,6 @@ namespace writeChar(')', buf); } - void writeEnumValues(const String & column_name, pcg64 & rng, WriteBuffer & buf, ssize_t max_value) { /// Don't generate big enums, because it will lead to really big result @@ -183,9 +184,9 @@ namespace size_t num_values = rng() % 16 + 1; std::vector values(num_values); - /// Generate random numbers from range [-(max_value + 1), max_value - num_values + 1] + /// Generate random numbers from range [-(max_value + 1), max_value - num_values + 1]. for (Int16 & x : values) - x = rng() % (2 * (max_value + 1) - num_values) - max_value - 1; + x = rng() % (2 * max_value + 3 - num_values) - max_value - 1; /// Make all numbers unique. std::sort(values.begin(), values.end()); for (size_t i = 0; i < num_values; ++i) @@ -199,7 +200,7 @@ namespace } } - void writeMapKeyType(const String & column_name, pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types) + void writeMapKeyType(const String & column_name, pcg64 & rng, WriteBuffer & buf) { TypeIndex type = map_key_types[rng() % map_key_types.size()]; switch (type) @@ -209,7 +210,11 @@ namespace break; case TypeIndex::LowCardinality: writeCString("LowCardinality(", buf); - writeLowCardinalityNestedType(rng, buf, allow_suspicious_lc_types, false); + /// Map key supports only String and FixedString inside LowCardinality. + if (rng() % 2) + writeCString("String", buf); + else + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); writeChar(')', buf); break; case TypeIndex::Enum8: @@ -295,7 +300,7 @@ namespace case TypeIndex::Map: { writeCString("Map(", buf); - writeMapKeyType(column_name, rng, buf, allow_suspicious_lc_types); + writeMapKeyType(column_name, rng, buf); writeCString(", ", buf); writeRandomType(column_name, rng, buf, allow_suspicious_lc_types, depth + 1); writeChar(')', buf); diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index bd2009830f9..e6e2c73ad87 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -14,3 +14,4 @@ c9 Date32 -77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':127575633389498667752072479581409788016,'2052-11-08 23:07:13':-53938349319031918752329205601430421582,'2104-07-17 19:59:35':84394139582225600743319968813775553298,'2034-11-07 21:10:22':151550220355687100498925996413330909596} 328a:eccb:530f:23c3:275d:7eec:2b1b:9c29 2112-05-13 -77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':127575633389498667752072479581409788016,'2052-11-08 23:07:13':-53938349319031918752329205601430421582,'2104-07-17 19:59:35':84394139582225600743319968813775553298,'2034-11-07 21:10:22':151550220355687100498925996413330909596} 328a:eccb:530f:23c3:275d:7eec:2b1b:9c29 2112-05-13 -77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':166979754159728572703419507823025932071} 8eff:8d3c:5a2c:fa5f:b2bf:2b0e:ff23:beb2 2143-03-03 +c1 LowCardinality(Nullable(UInt64)), c2 Date32, c3 LowCardinality(Nullable(Float64)), c4 Int256, c5 Date32 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql index 174e2be261e..b9cec1a436a 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.sql +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -15,3 +15,6 @@ select * from generateRandom(11, 2) limit 1; select * from generateRandom(11, 2, 2) limit 1; select * from generateRandom(11, 2, 2, 2) limit 1; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +set allow_suspicious_low_cardinality_types=1; +select generateRandomStructure(5, 4); + From eef0a433e57598405009cf3f859da767f7415972 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 15 May 2023 11:24:18 +0000 Subject: [PATCH 060/308] Add note about possible huge output --- docs/en/sql-reference/table-functions/generate.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/table-functions/generate.md b/docs/en/sql-reference/table-functions/generate.md index 6ceeb63cbb3..724f6d4a1f2 100644 --- a/docs/en/sql-reference/table-functions/generate.md +++ b/docs/en/sql-reference/table-functions/generate.md @@ -95,5 +95,7 @@ SELECT * FROM generateRandom(11) LIMIT 3; └──────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────┴────────────┴────────────────────────────────────────────────────────────────────────────────┴─────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────┴────────────┘ ``` +**Note:** `generateRandom(generateRandomStructure(), [random seed], max_string_length, max_array_length)` with large enough `max_array_length` can generate really huge output due to possible big nesting depth (up to 16) of complex types (`Array`, `Tuple`, `Map`, `Nested`). + ## Related content - Blog: [Generating random data in ClickHouse](https://clickhouse.com/blog/generating-random-test-distribution-data-for-clickhouse) From c901d2a7be1f1791b75567f381bfddb7416c4beb Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 15 May 2023 13:46:18 +0200 Subject: [PATCH 061/308] Fix style --- src/Functions/FunctionGenerateRandomStructure.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/FunctionGenerateRandomStructure.h b/src/Functions/FunctionGenerateRandomStructure.h index 1d1bcb1a0a8..894096a6e07 100644 --- a/src/Functions/FunctionGenerateRandomStructure.h +++ b/src/Functions/FunctionGenerateRandomStructure.h @@ -1,3 +1,5 @@ +#pragma once + #include #include From a07db551a7f0233b74d47555b98ca46cffb8b2a3 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 15 May 2023 13:46:36 +0200 Subject: [PATCH 062/308] Fix style --- src/TableFunctions/TableFunctionGenerateRandom.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index a9e81c78c07..08059796660 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -26,7 +26,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int LOGICAL_ERROR; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, ContextPtr context) From 362fa4849f0beccc988231ba3b107ca0868ccb16 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 15 May 2023 17:56:53 +0200 Subject: [PATCH 063/308] Try to fix build --- src/Functions/FunctionGenerateRandomStructure.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index 75455cdda85..9d818350fac 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include @@ -431,12 +431,12 @@ the number of columns in the result structure (random by default) and random see The maximum number of columns is 128. The function returns a value of type String. )", - Documentation::Examples{ + FunctionDocumentation::Examples{ {"random", "SELECT generateRandomStructure()"}, {"with specified number of columns", "SELECT generateRandomStructure(10)"}, {"with specified seed", "SELECT generateRandomStructure(10, 42)"}, }, - Documentation::Categories{"Random"} + FunctionDocumentation::Categories{"Random"} }, FunctionFactory::CaseSensitive); } From d9be88a36a0a9221345e8300e954440d15605b8b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 15 May 2023 21:40:10 +0000 Subject: [PATCH 064/308] Add UUID data type to PostgreSQL --- src/Storages/StoragePostgreSQL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index e013199c584..09198e5bdad 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -267,6 +267,7 @@ public: else if (which.isFloat64()) nested_column = ColumnFloat64::create(); else if (which.isDate()) nested_column = ColumnUInt16::create(); else if (which.isDateTime()) nested_column = ColumnUInt32::create(); + else if (which.isUUID()) nested_column = ColumnUUID::create(); else if (which.isDateTime64()) { nested_column = ColumnDecimal::create(0, 6); From b6d2a84e830dbcb74343c70e32d5de06ddea3c70 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 16 May 2023 12:01:55 +0200 Subject: [PATCH 065/308] Try to fix build --- src/Functions/FunctionGenerateRandomStructure.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index 9d818350fac..4cbbdd84c4a 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -8,6 +8,8 @@ #include #include #include +#include +#include #include From 8436a093e78880983d4340e0b52822c1782dc84e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 16 May 2023 13:36:12 +0200 Subject: [PATCH 066/308] Fix build --- src/IO/WriteBufferFromPocoSocket.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 79534b0f030..df34e8003cb 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace ProfileEvents From d50e6fe8682a7a77572dfb3ea11541fecad25702 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 16 May 2023 15:35:16 +0200 Subject: [PATCH 067/308] Fix build after bad conflicts resolution --- src/Functions/FunctionGenerateRandomStructure.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index 4cbbdd84c4a..16dac4f5112 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -426,19 +426,19 @@ REGISTER_FUNCTION(GenerateRandomStructure) { factory.registerFunction( { - R"( + .description=R"( Generates a random table structure. This function takes 2 optional constant arguments: the number of columns in the result structure (random by default) and random seed (random by default) The maximum number of columns is 128. The function returns a value of type String. )", - FunctionDocumentation::Examples{ - {"random", "SELECT generateRandomStructure()"}, - {"with specified number of columns", "SELECT generateRandomStructure(10)"}, - {"with specified seed", "SELECT generateRandomStructure(10, 42)"}, + .examples{ + {"random", "SELECT generateRandomStructure()", "c1 UInt32, c2 FixedString(25)"}, + {"with specified number of columns", "SELECT generateRandomStructure(3)", "c1 String, c2 Array(Int32), c3 LowCardinality(String)"}, + {"with specified seed", "SELECT generateRandomStructure(1, 42)", "c1 UInt128"}, }, - FunctionDocumentation::Categories{"Random"} + .categories{"Random"} }, FunctionFactory::CaseSensitive); } From 846804fed085680d4d4ae1ac5f34329e39006486 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 17 May 2023 11:39:04 +0000 Subject: [PATCH 068/308] Add separate handshake_timeout for receiving Hello packet from replica --- docs/en/operations/settings/settings.md | 6 +++ src/Client/Connection.cpp | 6 ++- src/Client/Connection.h | 2 +- src/Client/ConnectionParameters.cpp | 3 +- src/Core/Settings.h | 1 + src/IO/ConnectionTimeouts.cpp | 58 +++++++++++++++---------- src/IO/ConnectionTimeouts.h | 14 ++++-- src/Server/TCPHandler.cpp | 7 +++ 8 files changed, 65 insertions(+), 32 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cddde2090f8..20779eba360 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1048,6 +1048,12 @@ Timeouts in seconds on the socket used for communicating with the client. Default value: 10, 300, 300. +## handshake_timeout_ms {#handshake-timeout-ms} + +Timeout in milliseconds for receiving Hello packet from replicas during handshake. + +Default value: 300000. + ## cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response. diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index d39148d3016..09145bcdf1b 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -190,7 +190,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) connected = true; sendHello(); - receiveHello(); + receiveHello(timeouts.handshake_timeout); if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) sendAddendum(); @@ -305,8 +305,10 @@ void Connection::sendAddendum() } -void Connection::receiveHello() +void Connection::receiveHello(const Poco::Timespan & handshake_timeout) { + TimeoutSetter timeout_setter(*socket, socket->getSendTimeout(), handshake_timeout); + /// Receive hello packet. UInt64 packet_type = 0; diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 77dbe5e3398..cb3f2507cb9 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -256,7 +256,7 @@ private: void connect(const ConnectionTimeouts & timeouts); void sendHello(); void sendAddendum(); - void receiveHello(); + void receiveHello(const Poco::Timespan & handshake_timeout); #if USE_SSL void sendClusterNameAndSalt(); diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 2031036eb58..c47d217d432 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -67,7 +67,8 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati Poco::Timespan(config.getInt("connect_timeout", DBMS_DEFAULT_CONNECT_TIMEOUT_SEC), 0), Poco::Timespan(config.getInt("send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0), Poco::Timespan(config.getInt("receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0), - Poco::Timespan(config.getInt("tcp_keep_alive_timeout", 0), 0)); + Poco::Timespan(config.getInt("tcp_keep_alive_timeout", 0), 0), + Poco::Timespan(config.getInt("handshake_timeout_ms", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC * 1000), 0)); timeouts.sync_request_timeout = Poco::Timespan(config.getInt("sync_request_timeout", DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC), 0); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 610c7135a75..d5fb864ca6b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -55,6 +55,7 @@ class IColumn; M(UInt64, max_query_size, DBMS_DEFAULT_MAX_QUERY_SIZE, "The maximum number of bytes of a query string parsed by the SQL parser. Data in the VALUES clause of INSERT queries is processed by a separate stream parser (that consumes O(1) RAM) and not affected by this restriction.", 0) \ M(UInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \ M(Seconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \ + M(Milliseconds, handshake_timeout_ms, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC * 1000, "Timeout for receiving HELLO packet from replicas.", 0) \ M(Milliseconds, connect_timeout_with_failover_ms, 1000, "Connection timeout for selecting first healthy replica.", 0) \ M(Milliseconds, connect_timeout_with_failover_secure_ms, 1000, "Connection timeout for selecting first healthy replica (for secure connections).", 0) \ M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "Timeout for receiving data from network, in seconds. If no bytes were received in this interval, exception is thrown. If you set this setting on client, the 'send_timeout' for the socket will be also set on the corresponding connection end on the server.", 0) \ diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index 401afb7baac..01fbaa4f817 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -17,22 +17,7 @@ ConnectionTimeouts::ConnectionTimeouts( , secure_connection_timeout(connection_timeout) , hedged_connection_timeout(receive_timeout_) , receive_data_timeout(receive_timeout_) -{ -} - -ConnectionTimeouts::ConnectionTimeouts( - Poco::Timespan connection_timeout_, - Poco::Timespan send_timeout_, - Poco::Timespan receive_timeout_, - Poco::Timespan tcp_keep_alive_timeout_) - : connection_timeout(connection_timeout_) - , send_timeout(send_timeout_) - , receive_timeout(receive_timeout_) - , tcp_keep_alive_timeout(tcp_keep_alive_timeout_) - , http_keep_alive_timeout(0) - , secure_connection_timeout(connection_timeout) - , hedged_connection_timeout(receive_timeout_) - , receive_data_timeout(receive_timeout_) + , handshake_timeout(receive_timeout_) { } @@ -41,7 +26,26 @@ ConnectionTimeouts::ConnectionTimeouts( Poco::Timespan send_timeout_, Poco::Timespan receive_timeout_, Poco::Timespan tcp_keep_alive_timeout_, - Poco::Timespan http_keep_alive_timeout_) + Poco::Timespan handshake_timeout_) + : connection_timeout(connection_timeout_) + , send_timeout(send_timeout_) + , receive_timeout(receive_timeout_) + , tcp_keep_alive_timeout(tcp_keep_alive_timeout_) + , http_keep_alive_timeout(0) + , secure_connection_timeout(connection_timeout) + , hedged_connection_timeout(receive_timeout_) + , receive_data_timeout(receive_timeout_) + , handshake_timeout(handshake_timeout_) +{ +} + +ConnectionTimeouts::ConnectionTimeouts( + Poco::Timespan connection_timeout_, + Poco::Timespan send_timeout_, + Poco::Timespan receive_timeout_, + Poco::Timespan tcp_keep_alive_timeout_, + Poco::Timespan http_keep_alive_timeout_, + Poco::Timespan handshake_timeout_) : connection_timeout(connection_timeout_) , send_timeout(send_timeout_) , receive_timeout(receive_timeout_) @@ -50,6 +54,7 @@ ConnectionTimeouts::ConnectionTimeouts( , secure_connection_timeout(connection_timeout) , hedged_connection_timeout(receive_timeout_) , receive_data_timeout(receive_timeout_) + , handshake_timeout(handshake_timeout_) { } @@ -60,16 +65,18 @@ ConnectionTimeouts::ConnectionTimeouts( Poco::Timespan tcp_keep_alive_timeout_, Poco::Timespan http_keep_alive_timeout_, Poco::Timespan secure_connection_timeout_, - Poco::Timespan receive_hello_timeout_, - Poco::Timespan receive_data_timeout_) + Poco::Timespan hedged_connection_timeout_, + Poco::Timespan receive_data_timeout_, + Poco::Timespan handshake_timeout_) : connection_timeout(connection_timeout_) , send_timeout(send_timeout_) , receive_timeout(receive_timeout_) , tcp_keep_alive_timeout(tcp_keep_alive_timeout_) , http_keep_alive_timeout(http_keep_alive_timeout_) , secure_connection_timeout(secure_connection_timeout_) - , hedged_connection_timeout(receive_hello_timeout_) + , hedged_connection_timeout(hedged_connection_timeout_) , receive_data_timeout(receive_data_timeout_) + , handshake_timeout(handshake_timeout_) { } @@ -90,13 +97,14 @@ ConnectionTimeouts ConnectionTimeouts::getSaturated(Poco::Timespan limit) const saturate(http_keep_alive_timeout, limit), saturate(secure_connection_timeout, limit), saturate(hedged_connection_timeout, limit), - saturate(receive_data_timeout, limit)); + saturate(receive_data_timeout, limit), + saturate(handshake_timeout, limit)); } /// Timeouts for the case when we have just single attempt to connect. ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithoutFailover(const Settings & settings) { - return ConnectionTimeouts(settings.connect_timeout, settings.send_timeout, settings.receive_timeout, settings.tcp_keep_alive_timeout); + return ConnectionTimeouts(settings.connect_timeout, settings.send_timeout, settings.receive_timeout, settings.tcp_keep_alive_timeout, settings.handshake_timeout_ms); } /// Timeouts for the case when we will try many addresses in a loop. @@ -110,7 +118,8 @@ ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithFailover(const Settings 0, settings.connect_timeout_with_failover_secure_ms, settings.hedged_connection_timeout_ms, - settings.receive_data_timeout_ms); + settings.receive_data_timeout_ms, + settings.handshake_timeout_ms); } ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings, Poco::Timespan http_keep_alive_timeout) @@ -120,7 +129,8 @@ ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings settings.http_send_timeout, settings.http_receive_timeout, settings.tcp_keep_alive_timeout, - http_keep_alive_timeout); + http_keep_alive_timeout, + settings.http_receive_timeout); } } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 368288ee022..684af42827f 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -23,6 +23,9 @@ struct ConnectionTimeouts Poco::Timespan hedged_connection_timeout; Poco::Timespan receive_data_timeout; + /// Timeout for receiving HELLO packet + Poco::Timespan handshake_timeout; + /// Timeout for synchronous request-result protocol call (like Ping or TablesStatus) Poco::Timespan sync_request_timeout = Poco::Timespan(DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC, 0); @@ -35,13 +38,15 @@ struct ConnectionTimeouts ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, Poco::Timespan receive_timeout_, - Poco::Timespan tcp_keep_alive_timeout_); + Poco::Timespan tcp_keep_alive_timeout_, + Poco::Timespan handshake_timeout_); ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, Poco::Timespan receive_timeout_, Poco::Timespan tcp_keep_alive_timeout_, - Poco::Timespan http_keep_alive_timeout_); + Poco::Timespan http_keep_alive_timeout_, + Poco::Timespan handshake_timeout_); ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, @@ -49,8 +54,9 @@ struct ConnectionTimeouts Poco::Timespan tcp_keep_alive_timeout_, Poco::Timespan http_keep_alive_timeout_, Poco::Timespan secure_connection_timeout_, - Poco::Timespan receive_hello_timeout_, - Poco::Timespan receive_data_timeout_); + Poco::Timespan hedged_connection_timeout_, + Poco::Timespan receive_data_timeout_, + Poco::Timespan handshake_timeout_); static Poco::Timespan saturate(Poco::Timespan timespan, Poco::Timespan limit); ConnectionTimeouts getSaturated(Poco::Timespan limit) const; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 0522b6d8a48..a076e248a9f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1296,6 +1296,13 @@ void TCPHandler::receiveUnexpectedHello() void TCPHandler::sendHello() { + if (unlikely(sleep_in_send_tables_status.totalMilliseconds())) + { + out->next(); + std::chrono::milliseconds ms(sleep_in_send_tables_status.totalMilliseconds()); + std::this_thread::sleep_for(ms); + } + writeVarUInt(Protocol::Server::Hello, *out); writeStringBinary(DBMS_NAME, *out); writeVarUInt(DBMS_VERSION_MAJOR, *out); From 2ff3c8badd5a5c18f14ca76438978a415fe73d74 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 17 May 2023 11:41:00 +0000 Subject: [PATCH 069/308] Remove testing code --- src/Server/TCPHandler.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a076e248a9f..0522b6d8a48 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1296,13 +1296,6 @@ void TCPHandler::receiveUnexpectedHello() void TCPHandler::sendHello() { - if (unlikely(sleep_in_send_tables_status.totalMilliseconds())) - { - out->next(); - std::chrono::milliseconds ms(sleep_in_send_tables_status.totalMilliseconds()); - std::this_thread::sleep_for(ms); - } - writeVarUInt(Protocol::Server::Hello, *out); writeStringBinary(DBMS_NAME, *out); writeVarUInt(DBMS_VERSION_MAJOR, *out); From 194ce2d881aa6c3598f24e93cce29671ec9f67c3 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 17 May 2023 13:13:57 +0000 Subject: [PATCH 070/308] Better --- src/Core/PostgreSQL/insertPostgreSQLValue.cpp | 2 + src/Storages/StoragePostgreSQL.cpp | 40 ++++++++++++++----- 2 files changed, 31 insertions(+), 11 deletions(-) diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp index 08b67a470f1..d2e8071c5de 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp @@ -202,6 +202,8 @@ void preparePostgreSQLArrayInfo( parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isFloat64()) parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isUUID()) + parser = [](std::string & field) -> Field { return parse(field); }; else if (which.isString() || which.isFixedString()) parser = [](std::string & field) -> Field { return field; }; else if (which.isDate()) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 09198e5bdad..007f83165fd 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -197,7 +197,7 @@ public: /// Cannot just use serializeAsText for array data type even though it converts perfectly /// any dimension number array into text format, because it encloses in '[]' and for postgres it must be '{}'. /// Check if array[...] syntax from PostgreSQL will be applicable. - void parseArray(const Field & array_field, const DataTypePtr & data_type, WriteBuffer & ostr) + static void parseArray(const Field & array_field, const DataTypePtr & data_type, WriteBuffer & ostr) { const auto * array_type = typeid_cast(data_type.get()); const auto & nested = array_type->getNestedType(); @@ -205,7 +205,7 @@ public: if (!isArray(nested)) { - writeText(clickhouseToPostgresArray(array, data_type), ostr); + parseArrayContent(array, data_type, ostr); return; } @@ -219,7 +219,7 @@ public: if (!isArray(nested_array_type->getNestedType())) { - writeText(clickhouseToPostgresArray(iter->get(), nested), ostr); + parseArrayContent(iter->get(), nested, ostr); } else { @@ -232,17 +232,35 @@ public: /// Conversion is done via column casting because with writeText(Array..) got incorrect conversion /// of Date and DateTime data types and it added extra quotes for values inside array. - static std::string clickhouseToPostgresArray(const Array & array_field, const DataTypePtr & data_type) + static void parseArrayContent(const Array & array_field, const DataTypePtr & data_type, WriteBuffer & ostr) { - auto nested = typeid_cast(data_type.get())->getNestedType(); - auto array_column = ColumnArray::create(createNested(nested)); + auto nested_type = typeid_cast(data_type.get())->getNestedType(); + auto array_column = ColumnArray::create(createNested(nested_type)); array_column->insert(array_field); - WriteBufferFromOwnString ostr; - data_type->getDefaultSerialization()->serializeText(*array_column, 0, ostr, FormatSettings{}); - /// ostr is guaranteed to be at least '[]', i.e. size is at least 2 and 2 only if ostr.str() == '[]' - assert(ostr.str().size() >= 2); - return '{' + std::string(ostr.str().begin() + 1, ostr.str().end() - 1) + '}'; + const IColumn & nested_column = array_column->getData(); + const auto serialization = nested_type->getDefaultSerialization(); + + FormatSettings settings; + settings.pretty.charset = FormatSettings::Pretty::Charset::ASCII; + + if (nested_type->isNullable()) + nested_type = static_cast(nested_type.get())->getNestedType(); + + const bool quoted = !isUUID(nested_type); + + writeChar('{', ostr); + for (size_t i = 0, size = array_field.size(); i < size; ++i) + { + if (i != 0) + writeChar(',', ostr); + + if (quoted) + serialization->serializeTextQuoted(nested_column, i, ostr, settings); + else + serialization->serializeText(nested_column, i, ostr, settings); + } + writeChar('}', ostr); } static MutableColumnPtr createNested(DataTypePtr nested) From ad637c682418caa8fdffd28795a2edb415f2bfce Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 17 May 2023 13:14:10 +0000 Subject: [PATCH 071/308] Add test --- .../test_storage_postgresql/test.py | 41 +++++++++++-------- 1 file changed, 25 insertions(+), 16 deletions(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 3b7aae1ccdc..6ceddfe831b 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -113,7 +113,9 @@ def test_postgres_conversions(started_cluster): g Text[][][][][] NOT NULL, -- String h Integer[][][], -- Nullable(Int32) i Char(2)[][][][], -- Nullable(String) - k Char(2)[] -- Nullable(String) + j Char(2)[], -- Nullable(String) + k UUID[], -- Nullable(UUID) + l UUID[][] -- Nullable(UUID) )""" ) @@ -123,15 +125,18 @@ def test_postgres_conversions(started_cluster): ) expected = ( "a\tArray(Date)\t\t\t\t\t\n" - + "b\tArray(DateTime64(6))\t\t\t\t\t\n" - + "c\tArray(Array(Float32))\t\t\t\t\t\n" - + "d\tArray(Array(Float64))\t\t\t\t\t\n" - + "e\tArray(Array(Array(Decimal(5, 5))))\t\t\t\t\t\n" - + "f\tArray(Array(Array(Int32)))\t\t\t\t\t\n" - + "g\tArray(Array(Array(Array(Array(String)))))\t\t\t\t\t\n" - + "h\tArray(Array(Array(Nullable(Int32))))\t\t\t\t\t\n" - + "i\tArray(Array(Array(Array(Nullable(String)))))\t\t\t\t\t\n" - + "k\tArray(Nullable(String))" + "b\tArray(DateTime64(6))\t\t\t\t\t\n" + "c\tArray(Array(Float32))\t\t\t\t\t\n" + "d\tArray(Array(Float64))\t\t\t\t\t\n" + "e\tArray(Array(Array(Decimal(5, 5))))\t\t\t\t\t\n" + "f\tArray(Array(Array(Int32)))\t\t\t\t\t\n" + "g\tArray(Array(Array(Array(Array(String)))))\t\t\t\t\t\n" + "h\tArray(Array(Array(Nullable(Int32))))\t\t\t\t\t\n" + "i\tArray(Array(Array(Array(Nullable(String)))))\t\t\t\t\t\n" + "j\tArray(Nullable(String))\t\t\t\t\t\n" + "k\tArray(Nullable(UUID))\t\t\t\t\t\n" + "l\tArray(Array(Nullable(UUID)))" + "" ) assert result.rstrip() == expected @@ -147,7 +152,9 @@ def test_postgres_conversions(started_cluster): "[[[[['winx', 'winx', 'winx']]]]], " "[[[1, NULL], [NULL, 1]], [[NULL, NULL], [NULL, NULL]], [[4, 4], [5, 5]]], " "[[[[NULL]]]], " - "[]" + "[], " + "['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a', '42209d53-d641-4d73-a8b6-c038db1e75d6', NULL], " + "[[NULL, '42209d53-d641-4d73-a8b6-c038db1e75d6'], ['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a', NULL], [NULL, NULL]]" ")" ) @@ -157,15 +164,17 @@ def test_postgres_conversions(started_cluster): ) expected = ( "['2000-05-12','2000-05-12']\t" - + "['2000-05-12 12:12:12.012345','2000-05-12 12:12:12.012345']\t" - + "[[1.12345],[1.12345],[1.12345]]\t" - + "[[1.1234567891],[1.1234567891],[1.1234567891]]\t" - + "[[[0.11111,0.11111]],[[0.22222,0.22222]],[[0.33333,0.33333]]]\t" + "['2000-05-12 12:12:12.012345','2000-05-12 12:12:12.012345']\t" + "[[1.12345],[1.12345],[1.12345]]\t" + "[[1.1234567891],[1.1234567891],[1.1234567891]]\t" + "[[[0.11111,0.11111]],[[0.22222,0.22222]],[[0.33333,0.33333]]]\t" "[[[1,1],[1,1]],[[3,3],[3,3]],[[4,4],[5,5]]]\t" "[[[[['winx','winx','winx']]]]]\t" "[[[1,NULL],[NULL,1]],[[NULL,NULL],[NULL,NULL]],[[4,4],[5,5]]]\t" "[[[[NULL]]]]\t" - "[]\n" + "[]\t" + "['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a','42209d53-d641-4d73-a8b6-c038db1e75d6',NULL]\t" + "[[NULL,'42209d53-d641-4d73-a8b6-c038db1e75d6'],['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a',NULL],[NULL,NULL]]\n" ) assert result == expected From 971cc092d4da472fa6a3a0726616218d6a783b58 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 18 May 2023 15:16:47 +0200 Subject: [PATCH 072/308] Update src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp --- src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 2510d6f2d19..30f31910bee 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -366,9 +366,6 @@ MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInf part_info->column_name_set = {required_column_names.begin(), required_column_names.end()}; part_info->task_columns = task_columns; - if (settings.prefetch_buffer_size < DBMS_DEFAULT_BUFFER_SIZE) - throw Exception(ErrorCodes::LOGICAL_ERROR, "remove me"); - /// adjustBufferSize(), which is done in MergeTreeReaderStream and MergeTreeReaderCompact, /// lowers buffer size if file size (or required read range) is less. So we know that the /// settings.prefetch_buffer_size will be lowered there, therefore we account it here as well. From 3a3e41355239a27d1748f0d6c220d0f6ccabae15 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 18 May 2023 21:47:52 +0000 Subject: [PATCH 073/308] Implement toLastDayWeek function --- src/Common/DateLUTImpl.h | 51 +++++++++++++++++++++++++--- src/Functions/CustomWeekTransforms.h | 32 +++++++++++++++++ src/Functions/toCustomWeek.cpp | 2 ++ 3 files changed, 80 insertions(+), 5 deletions(-) diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index b40b4d7c65b..93af04456b2 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -311,11 +311,6 @@ public: /// All functions below are thread-safe; arguments are not checked. - static ExtendedDayNum toDayNum(ExtendedDayNum d) - { - return d; - } - static UInt32 saturateMinus(UInt32 x, UInt32 y) { UInt32 res = x - y; @@ -323,6 +318,11 @@ public: return res; } + static ExtendedDayNum toDayNum(ExtendedDayNum d) + { + return d; + } + static ExtendedDayNum toDayNum(LUTIndex d) { return ExtendedDayNum{static_cast(d.toUnderType() - daynum_offset_epoch)}; @@ -363,6 +363,27 @@ public: return toDayNum(LUTIndex(i - (lut[i].day_of_week - 1))); } + /// Round up to the last day of week. + template + inline Time toLastDayOfWeek(DateOrTime v) const + { + const LUTIndex i = toLUTIndex(v); + if constexpr (std::is_unsigned_v || std::is_same_v) + return lut_saturated[i + (7 - lut[i].day_of_week)].date; + else + return lut[i + (7 - lut[i].day_of_week)].date; + } + + template + inline auto toLastDayNumOfWeek(DateOrTime v) const + { + const LUTIndex i = toLUTIndex(v); + if constexpr (std::is_unsigned_v || std::is_same_v) + return toDayNum(LUTIndexWithSaturation(i + (7 - lut[i].day_of_week))); + else + return toDayNum(LUTIndex(i + (7 - lut[i].day_of_week))); + } + /// Round down to start of month. template inline Time toFirstDayOfMonth(DateOrTime v) const @@ -863,6 +884,26 @@ public: } } + /// Get last day of week with week_mode, return Saturday or Sunday + template + inline auto toLastDayNumOfWeek(DateOrTime v, UInt8 week_mode) const + { + bool monday_first_mode = week_mode & static_cast(WeekModeFlag::MONDAY_FIRST); + if (monday_first_mode) + { + return toLastDayNumOfWeek(v); + } + else + { + const auto day_of_week = toDayOfWeek(v); + v += 6; + if constexpr (std::is_unsigned_v || std::is_same_v) + return (day_of_week != 7) ? DayNum(saturateMinus(v, day_of_week)) : toDayNum(v); + else + return (day_of_week != 7) ? ExtendedDayNum(v - day_of_week) : toDayNum(v); + } + } + /// Check and change mode to effective. inline UInt8 check_week_mode(UInt8 mode) const /// NOLINT { diff --git a/src/Functions/CustomWeekTransforms.h b/src/Functions/CustomWeekTransforms.h index 413c81d1400..2a82912d5d6 100644 --- a/src/Functions/CustomWeekTransforms.h +++ b/src/Functions/CustomWeekTransforms.h @@ -88,6 +88,38 @@ struct ToStartOfWeekImpl using FactorTransform = ZeroTransform; }; +struct ToLastDayOfWeekImpl +{ + static constexpr auto name = "toLastDayOfWeek"; + + static inline UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode); + } + static inline UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode); + } + static inline UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(ExtendedDayNum(d), week_mode); + } + static inline UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(DayNum(d), week_mode); + } + static inline Int64 executeExtendedResult(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode); + } + static inline Int32 executeExtendedResult(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(ExtendedDayNum(d), week_mode); + } + + using FactorTransform = ZeroTransform; +}; + struct ToWeekImpl { static constexpr auto name = "toWeek"; diff --git a/src/Functions/toCustomWeek.cpp b/src/Functions/toCustomWeek.cpp index b773cc7df96..98e7aaf1d6b 100644 --- a/src/Functions/toCustomWeek.cpp +++ b/src/Functions/toCustomWeek.cpp @@ -11,12 +11,14 @@ namespace DB using FunctionToWeek = FunctionCustomWeekToSomething; using FunctionToYearWeek = FunctionCustomWeekToSomething; using FunctionToStartOfWeek = FunctionCustomWeekToDateOrDate32; +using FunctionToLastDayOfWeek = FunctionCustomWeekToDateOrDate32; REGISTER_FUNCTION(ToCustomWeek) { factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); /// Compatibility aliases for mysql. factory.registerAlias("week", "toWeek", FunctionFactory::CaseInsensitive); From 3121a57912752d70ac46402f46c695181571dea3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 May 2023 14:21:07 +0200 Subject: [PATCH 074/308] Add some assertions --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 47 +++++++++++-------- 1 file changed, 28 insertions(+), 19 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 00d23183f6a..b4ea30e54c8 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -708,14 +708,18 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() } else if (current_write_offset < file_offset_of_buffer_end) { + const auto path = file_segment.getPathInLocalCache(); + size_t file_size = 0; + if (fs::exists(path)) + file_size = fs::file_size(path); + throw Exception( ErrorCodes::LOGICAL_ERROR, - "Expected {} >= {} ({})", - current_write_offset, file_offset_of_buffer_end, getInfoForLog()); + "Invariant failed. Expected {} >= {} (size on fs: {}, {})", + current_write_offset, file_offset_of_buffer_end, file_size, getInfoForLog()); } } - - if (read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE) + else if (read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE) { /** * ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE means that on previous getImplementationBuffer() call @@ -884,25 +888,28 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (!result) { -#ifndef NDEBUG - if (read_type == ReadType::CACHED) + auto debug_check = [&]() { - size_t cache_file_size = getFileSizeFromReadBuffer(*implementation_buffer); - if (cache_file_size == 0) + if (read_type == ReadType::CACHED) { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Attempt to read from an empty cache file: {} (just before actual read)", - cache_file_size); + size_t cache_file_size = getFileSizeFromReadBuffer(*implementation_buffer); + if (cache_file_size == 0) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Attempt to read from an empty cache file: {} (just before actual read)", + cache_file_size); + } } - } - else - { - assert(file_offset_of_buffer_end == static_cast(implementation_buffer->getFileOffsetOfBufferEnd())); - } + else + { + chassert(file_offset_of_buffer_end == static_cast(implementation_buffer->getFileOffsetOfBufferEnd())); + } + chassert(!implementation_buffer->hasPendingData()); + return true; + }; - assert(!implementation_buffer->hasPendingData()); -#endif + chassert(debug_check()); Stopwatch watch(CLOCK_MONOTONIC); @@ -927,6 +934,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() { ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed); + + chassert(file_offset_of_buffer_end + size <= file_segment.range().size()); } else { From c93836b9620f2bd424d5f6132404a455c94a39dd Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 19 May 2023 22:26:53 +0000 Subject: [PATCH 075/308] fix --- src/Storages/MergeTree/MergeTreeReadPool.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 1de5361a7db..4e95a210bae 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -56,6 +56,13 @@ MergeTreeReadPool::MergeTreeReadPool( , backoff_settings{context_->getSettingsRef()} , backoff_state{threads_} { + /// parts don't contain duplicate MergeTreeDataPart's. + const auto per_part_sum_marks = fillPerPartInfo( + parts_ranges, storage_snapshot, is_part_on_remote_disk, + predict_block_size_bytes, + column_names, virtual_column_names, prewhere_info, + actions_settings, reader_settings, per_part_params); + if (std::ranges::count(is_part_on_remote_disk, true)) { const auto & settings = context_->getSettingsRef(); @@ -82,13 +89,6 @@ MergeTreeReadPool::MergeTreeReadPool( } } - /// parts don't contain duplicate MergeTreeDataPart's. - const auto per_part_sum_marks = fillPerPartInfo( - parts_ranges, storage_snapshot, is_part_on_remote_disk, - predict_block_size_bytes, - column_names, virtual_column_names, prewhere_info, - actions_settings, reader_settings, per_part_params); - fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_ranges); } From 7189481fad990824fddeee045c97e66d2cd4985c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 May 2023 09:00:28 +0200 Subject: [PATCH 076/308] Preserve backward incompatibility for renamed settings by using aliases - optimize_use_projections/allow_experimental_projection_optimization - enable_lightweight_delete/allow_experimental_lightweight_delete Signed-off-by: Azat Khuzhin --- src/Core/Settings.h | 6 ++---- .../0_stateless/02319_lightweight_delete_on_merge_tree.sql | 2 +- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 21dc267749b..b9e728a3ca4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -507,7 +507,7 @@ class IColumn; M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ - M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) \ + M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \ M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ @@ -557,7 +557,7 @@ class IColumn; M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \ M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ - M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) \ + M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) ALIAS(allow_experimental_projection_optimization) \ M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \ @@ -764,7 +764,6 @@ class IColumn; MAKE_OBSOLETE(M, Bool, allow_experimental_database_atomic, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_bigint_types, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_window_functions, true) \ - MAKE_OBSOLETE(M, Bool, allow_experimental_lightweight_delete, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_geo_types, true) \ \ MAKE_OBSOLETE(M, Milliseconds, async_insert_stale_timeout_ms, 0) \ @@ -777,7 +776,6 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \ MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \ - MAKE_OBSOLETE(M, Bool, allow_experimental_projection_optimization, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_query_cache, true) \ /* moved to config.xml: see also src/Core/ServerSettings.h */ \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_buffer_flush_schedule_pool_size, 16) \ diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql index 9413c664293..050b8e37722 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -110,7 +110,7 @@ DROP TABLE t_proj; CREATE TABLE merge_table_standard_delete(id Int32, name String) ENGINE = MergeTree order by id settings min_bytes_for_wide_part=0; SET allow_experimental_lightweight_delete = false; -DELETE FROM merge_table_standard_delete WHERE id = 10; -- allow_experimental_lightweight_delete=false is now ignored +DELETE FROM merge_table_standard_delete WHERE id = 10; -- { serverError SUPPORT_IS_DISABLED } SET enable_lightweight_delete = false; DELETE FROM merge_table_standard_delete WHERE id = 10; -- { serverError SUPPORT_IS_DISABLED } DROP TABLE merge_table_standard_delete; From cee6c3914fef9913b0b249fcec359a692f413a32 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 22 May 2023 21:36:55 +0200 Subject: [PATCH 077/308] Fix build --- src/IO/WriteBufferFromPocoSocket.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index df34e8003cb..c316ff17e2a 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -110,7 +110,7 @@ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() #ifndef NDEBUG if (!finalized) { - LOG_ERROR(log, "WriteBufferFromPocoSocket is not finalized in destructor. It's a bug"); + LOG_ERROR(&Poco::Logger::get("WriteBufferFromPocoSocket"), "WriteBufferFromPocoSocket is not finalized in destructor. It's a bug"); std::terminate(); } #else From 646eeb63a4cc720b05ff9de48364be32a6936d94 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 May 2023 19:46:05 +0000 Subject: [PATCH 078/308] Fix build --- src/Functions/FunctionGenerateRandomStructure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index 16dac4f5112..f85b2596530 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -424,7 +424,7 @@ String FunctionGenerateRandomStructure::generateRandomStructure(size_t seed, con REGISTER_FUNCTION(GenerateRandomStructure) { - factory.registerFunction( + factory.registerFunction(FunctionDocumentation { .description=R"( Generates a random table structure. From 07d9f33b2e1c503438a0b257b8ea321abd70f7e4 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 23 May 2023 04:01:44 +0000 Subject: [PATCH 079/308] Improve toFirstDayNumOfWeek infinitesimally --- src/Common/DateLUTImpl.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index 93af04456b2..c141ab07d78 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -877,10 +877,11 @@ public: } else { + const auto day_of_week = toDayOfWeek(v); if constexpr (std::is_unsigned_v || std::is_same_v) - return (toDayOfWeek(v) != 7) ? DayNum(saturateMinus(v, toDayOfWeek(v))) : toDayNum(v); + return (day_of_week != 7) ? DayNum(saturateMinus(v, day_of_week)) : toDayNum(v); else - return (toDayOfWeek(v) != 7) ? ExtendedDayNum(v - toDayOfWeek(v)) : toDayNum(v); + return (day_of_week != 7) ? ExtendedDayNum(v - day_of_week) : toDayNum(v); } } From c0bc75eacd0624e38d2a1581e19906778ea8b676 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 May 2023 09:50:34 +0000 Subject: [PATCH 080/308] Try to fix test --- .../02586_generate_random_structure.reference | 26 ++++++++++--------- .../02586_generate_random_structure.sql | 10 +++---- 2 files changed, 19 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index e6e2c73ad87..d2929fb4564 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -2,16 +2,18 @@ c1 String, c2 UInt256, c3 String, c4 Decimal128(8), c5 UInt128 String Const(String) ` 90465455320735604871982424534384518837533904778028808627865442405232847164685 5& -303477100882544888461471906106.82821046 75820566154622566322847299106656624693 -c1 Int128 -c2 Decimal(76, 55) -c3 Int256 -c4 UInt32 -c5 UInt256 -c6 Float64 -c7 Map(DateTime, Int128) -c8 IPv6 -c9 Date32 --77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':127575633389498667752072479581409788016,'2052-11-08 23:07:13':-53938349319031918752329205601430421582,'2104-07-17 19:59:35':84394139582225600743319968813775553298,'2034-11-07 21:10:22':151550220355687100498925996413330909596} 328a:eccb:530f:23c3:275d:7eec:2b1b:9c29 2112-05-13 --77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':127575633389498667752072479581409788016,'2052-11-08 23:07:13':-53938349319031918752329205601430421582,'2104-07-17 19:59:35':84394139582225600743319968813775553298,'2034-11-07 21:10:22':151550220355687100498925996413330909596} 328a:eccb:530f:23c3:275d:7eec:2b1b:9c29 2112-05-13 --77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':166979754159728572703419507823025932071} 8eff:8d3c:5a2c:fa5f:b2bf:2b0e:ff23:beb2 2143-03-03 +c1 FixedString(125) +c2 IPv4 +c3.e1 Array(Enum16(\'e1V3\' = -24827, \'e1V14\' = -24479, \'e1V8\' = -22478, \'e1V10\' = -13735, \'e1V15\' = -12641, \'e1V11\' = -10191, \'e1V0\' = -8579, \'e1V7\' = -8104, \'e1V6\' = 712, \'e1V12\' = 5683, \'e1V13\' = 13678, \'e1V9\' = 19740, \'e1V5\' = 23066, \'e1V2\' = 23292, \'e1V4\' = 23736, \'e1V1\' = 31672)) +c3.e2 Array(Map(Int8, Int32)) +c3.e3 Array(Decimal(76, 64)) +c3.e4 Array(Int32) +c3.e5 Array(Nullable(Int64)) +c3.e6 Array(Int256) +c4 FixedString(183) +c5 IPv4 +c6 UInt256 +Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10','e1V0','e1V10','e1V14','e1V10','e1V14'] [{-13:777622572,102:-1122882357,62:1647813163,-94:2094022166},{-32:1448633509},{},{},{34:1536340393,19:-2049677851,74:65643868,-46:-1990799930,97:-531041081,46:-2634833,14:1581632600,89:-771229823,-105:1238603584},{47:1458809010,109:1640682510,86:1945730198,85:1505847247,35:-35189402}] [153363749503.3642648494826450951141750747382772821825909005880434540971999557,79828591186.7378041015337066268618633118713347614941338787453473118807106292,81672688565.9633830721322966111551266731935181670389237071708068971548883315,573768486971.1812413548839655834002608768736215115033958693122764224003897029,-393925092368.4893467278351090742501814120269109477445490969167853713051140487,46027399426.0865278566391382610843315130162915324295037009704113636499519839] [755855942,1804001770,-78103159,-866181765,731736602,-79599206] [5253556148991564114,4681434929596395351,-7302160004580855709,-3686747220178471318,6288582051009949273,646864891160092871] [17035203905051045016266537043565487029724162173062647021612805252288722534904,-42105881403933504641593145676742477006499618886131028341247993701618141933523,45346626822580305846120377917274679004279343244238782744860626882886217433843,-3660165069803677989574889324494857545543653453780976182221584349306428201647,-23316760935816288837287058499520670431785615691220162210524162590241529297823,6184785563808848524970564618169964412151721224362412457508264894603779018817] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 +Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10','e1V0','e1V10','e1V14','e1V10','e1V14'] [{-13:777622572,102:-1122882357,62:1647813163,-94:2094022166},{-32:1448633509},{},{},{34:1536340393,19:-2049677851,74:65643868,-46:-1990799930,97:-531041081,46:-2634833,14:1581632600,89:-771229823,-105:1238603584},{47:1458809010,109:1640682510,86:1945730198,85:1505847247,35:-35189402}] [153363749503.3642648494826450951141750747382772821825909005880434540971999557,79828591186.7378041015337066268618633118713347614941338787453473118807106292,81672688565.9633830721322966111551266731935181670389237071708068971548883315,573768486971.1812413548839655834002608768736215115033958693122764224003897029,-393925092368.4893467278351090742501814120269109477445490969167853713051140487,46027399426.0865278566391382610843315130162915324295037009704113636499519839] [755855942,1804001770,-78103159,-866181765,731736602,-79599206] [5253556148991564114,4681434929596395351,-7302160004580855709,-3686747220178471318,6288582051009949273,646864891160092871] [17035203905051045016266537043565487029724162173062647021612805252288722534904,-42105881403933504641593145676742477006499618886131028341247993701618141933523,45346626822580305846120377917274679004279343244238782744860626882886217433843,-3660165069803677989574889324494857545543653453780976182221584349306428201647,-23316760935816288837287058499520670431785615691220162210524162590241529297823,6184785563808848524970564618169964412151721224362412457508264894603779018817] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 +Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10'] [{}] [825002272867.1157788721157301271303736024856710948164507982705676578804195475] [1865150610] [7514464811443271056] [33504961604882608369857530219353040639899064613284394558131808339620328539033] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 c1 LowCardinality(Nullable(UInt64)), c2 Date32, c3 LowCardinality(Nullable(Float64)), c4 Int256, c5 Date32 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql index b9cec1a436a..e2e8409b35c 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.sql +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -9,11 +9,11 @@ select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMEN select generateRandomStructure(materialize(5), 42); -- {serverError ILLEGAL_COLUMN} select generateRandomStructure(5, materialize(42)); -- {serverError ILLEGAL_COLUMN} -desc generateRandom(11); -select * from generateRandom(11) limit 1; -select * from generateRandom(11, 2) limit 1; -select * from generateRandom(11, 2, 2) limit 1; -select * from generateRandom(11, 2, 2, 2) limit 1; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +desc generateRandom(10000000); +select * from generateRandom(10000000) limit 1; +select * from generateRandom(10000000, 2) limit 1; +select * from generateRandom(10000000, 2, 2) limit 1; +select * from generateRandom(10000000, 2, 2, 2) limit 1; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} set allow_suspicious_low_cardinality_types=1; select generateRandomStructure(5, 4); From 66e111a6aa49d0cd9a16f7e6050b23f8ad4a6e68 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 23 May 2023 11:52:44 +0200 Subject: [PATCH 081/308] Fix tests --- src/IO/WriteBufferFromPocoSocket.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index c316ff17e2a..cf3944e019d 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -10,7 +10,6 @@ #include #include #include -#include namespace ProfileEvents @@ -107,13 +106,6 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() { -#ifndef NDEBUG - if (!finalized) - { - LOG_ERROR(&Poco::Logger::get("WriteBufferFromPocoSocket"), "WriteBufferFromPocoSocket is not finalized in destructor. It's a bug"); - std::terminate(); - } -#else try { finalize(); @@ -122,7 +114,6 @@ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() { tryLogCurrentException(__PRETTY_FUNCTION__); } -#endif } } From 3c1aeaaa793f080e20005bfa42a18746f969ad39 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 May 2023 11:39:40 +0000 Subject: [PATCH 082/308] Change default value of handshake_timeout to 10 sec, fix possible use-after-free --- docs/en/operations/settings/settings.md | 2 +- src/Client/Connection.cpp | 4 ++++ src/Core/Settings.h | 2 +- src/IO/TimeoutSetter.cpp | 21 +++++++++++++------ src/IO/TimeoutSetter.h | 6 +++++- .../test_reload_clusters_config/test.py | 2 +- .../test.py | 4 ++-- 7 files changed, 29 insertions(+), 12 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b6ade78b1e1..0810b642039 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1054,7 +1054,7 @@ Default value: 10, 300, 300. Timeout in milliseconds for receiving Hello packet from replicas during handshake. -Default value: 300000. +Default value: 10000. ## cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 09145bcdf1b..35423012424 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -361,6 +361,10 @@ void Connection::receiveHello(const Poco::Timespan & handshake_timeout) receiveException()->rethrow(); else { + /// Reset timeout_setter before disconnect, + /// because after disconnect socket will be invalid. + timeout_setter.reset(); + /// Close connection, to not stay in unsynchronised state. disconnect(); throwUnexpectedPacket(packet_type, "Hello or Exception"); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ba3a28af614..03cf32e5aaa 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -55,7 +55,7 @@ class IColumn; M(UInt64, max_query_size, DBMS_DEFAULT_MAX_QUERY_SIZE, "The maximum number of bytes of a query string parsed by the SQL parser. Data in the VALUES clause of INSERT queries is processed by a separate stream parser (that consumes O(1) RAM) and not affected by this restriction.", 0) \ M(UInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \ M(Seconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \ - M(Milliseconds, handshake_timeout_ms, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC * 1000, "Timeout for receiving HELLO packet from replicas.", 0) \ + M(Milliseconds, handshake_timeout_ms, 10000, "Timeout for receiving HELLO packet from replicas.", 0) \ M(Milliseconds, connect_timeout_with_failover_ms, 1000, "Connection timeout for selecting first healthy replica.", 0) \ M(Milliseconds, connect_timeout_with_failover_secure_ms, 1000, "Connection timeout for selecting first healthy replica (for secure connections).", 0) \ M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "Timeout for receiving data from network, in seconds. If no bytes were received in this interval, exception is thrown. If you set this setting on client, the 'send_timeout' for the socket will be also set on the corresponding connection end on the server.", 0) \ diff --git a/src/IO/TimeoutSetter.cpp b/src/IO/TimeoutSetter.cpp index ed21383ccd4..b8b7a814703 100644 --- a/src/IO/TimeoutSetter.cpp +++ b/src/IO/TimeoutSetter.cpp @@ -29,14 +29,12 @@ TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_, Poco::Timespan t TimeoutSetter::~TimeoutSetter() { + if (was_reset) + return; + try { - bool connected = socket.impl()->initialized(); - if (!connected) - return; - - socket.setSendTimeout(old_send_timeout); - socket.setReceiveTimeout(old_receive_timeout); + reset(); } catch (...) { @@ -44,4 +42,15 @@ TimeoutSetter::~TimeoutSetter() } } +void TimeoutSetter::reset() +{ + bool connected = socket.impl()->initialized(); + if (!connected) + return; + + socket.setSendTimeout(old_send_timeout); + socket.setReceiveTimeout(old_receive_timeout); + was_reset = true; +} + } diff --git a/src/IO/TimeoutSetter.h b/src/IO/TimeoutSetter.h index 31c37ea07af..3479986d7fe 100644 --- a/src/IO/TimeoutSetter.h +++ b/src/IO/TimeoutSetter.h @@ -6,7 +6,7 @@ namespace DB { -/// Temporarily overrides socket send/receive timeouts and reset them back into destructor +/// Temporarily overrides socket send/receive timeouts and reset them back into destructor (or manually by calling reset method) /// If "limit_max_timeout" is true, timeouts could be only decreased (maxed by previous value). struct TimeoutSetter { @@ -19,6 +19,9 @@ struct TimeoutSetter ~TimeoutSetter(); + /// Reset timeouts back. + void reset(); + Poco::Net::StreamSocket & socket; Poco::Timespan send_timeout; @@ -26,5 +29,6 @@ struct TimeoutSetter Poco::Timespan old_send_timeout; Poco::Timespan old_receive_timeout; + bool was_reset = false; }; } diff --git a/tests/integration/test_reload_clusters_config/test.py b/tests/integration/test_reload_clusters_config/test.py index a52871890e9..73ca4a01f34 100644 --- a/tests/integration/test_reload_clusters_config/test.py +++ b/tests/integration/test_reload_clusters_config/test.py @@ -169,7 +169,7 @@ test_config3 = """ def send_repeated_query(table, count=5): for i in range(count): node.query_and_get_error( - "SELECT count() FROM {} SETTINGS receive_timeout=1".format(table) + "SELECT count() FROM {} SETTINGS receive_timeout=1, handshake_timeout_ms=1".format(table) ) diff --git a/tests/integration/test_system_clusters_actual_information/test.py b/tests/integration/test_system_clusters_actual_information/test.py index 0658d0c7576..e90a6cdeb3f 100644 --- a/tests/integration/test_system_clusters_actual_information/test.py +++ b/tests/integration/test_system_clusters_actual_information/test.py @@ -40,8 +40,8 @@ def test(started_cluster): cluster.pause_container("node_1") node.query("SYSTEM RELOAD CONFIG") - node.query_and_get_error( - "SELECT count() FROM distributed SETTINGS receive_timeout=1" + error = node.query_and_get_error( + "SELECT count() FROM distributed SETTINGS receive_timeout=1, handshake_timeout_ms=1" ) result = node.query( From b82ff979d014ef63f5661f83c1cf29309fe340be Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 23 May 2023 23:10:34 +0800 Subject: [PATCH 083/308] Fix invalid index analysis for date related keys --- src/Storages/MergeTree/KeyCondition.cpp | 35 +++++++++++++++++-- .../02764_index_analysis_fix.reference | 1 + .../0_stateless/02764_index_analysis_fix.sql | 9 +++++ 3 files changed, 42 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02764_index_analysis_fix.reference create mode 100644 tests/queries/0_stateless/02764_index_analysis_fix.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index b8ef2152a99..dea2091f115 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -943,6 +943,19 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & return {field.columns, field.row_idx, result_idx}; } +static std::set date_time_parsing_functions = { + "toDate", + "toDate32", + "toDateTime", + "toDateTime64", + "ParseDateTimeBestEffort", + "ParseDateTimeBestEffortUS", + "ParseDateTime32BestEffort", + "ParseDateTime64BestEffort", + "parseDateTime", + "parseDateTimeInJodaSyntax", +}; + /** The key functional expression constraint may be inferred from a plain column in the expression. * For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`, * it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())` @@ -1026,10 +1039,23 @@ bool KeyCondition::transformConstantWithValidFunctions( if (func->type != ActionsDAG::ActionType::FUNCTION) continue; + const auto & func_name = func->function_base->getName(); + auto func_base = func->function_base; + if (date_time_parsing_functions.contains(func_name)) + { + auto func_or_null = FunctionFactory::instance().get(func_name + "OrNull", context); + ColumnsWithTypeAndName arguments; + int i = 0; + for (const auto & type : func->function_base->getArgumentTypes()) + arguments.push_back({nullptr, type, fmt::format("_{}", i++)}); + + func_base = func_or_null->build(arguments); + } + if (func->children.size() == 1) { std::tie(const_value, const_type) - = applyFunctionForFieldOfUnknownType(func->function_base, const_type, const_value); + = applyFunctionForFieldOfUnknownType(func_base, const_type, const_value); } else if (func->children.size() == 2) { @@ -1040,7 +1066,7 @@ bool KeyCondition::transformConstantWithValidFunctions( auto left_arg_type = left->result_type; auto left_arg_value = (*left->column)[0]; std::tie(const_value, const_type) = applyBinaryFunctionForFieldOfUnknownType( - FunctionFactory::instance().get(func->function_base->getName(), context), + FunctionFactory::instance().get(func_base->getName(), context), left_arg_type, left_arg_value, const_type, const_value); } else @@ -1048,10 +1074,13 @@ bool KeyCondition::transformConstantWithValidFunctions( auto right_arg_type = right->result_type; auto right_arg_value = (*right->column)[0]; std::tie(const_value, const_type) = applyBinaryFunctionForFieldOfUnknownType( - FunctionFactory::instance().get(func->function_base->getName(), context), + FunctionFactory::instance().get(func_base->getName(), context), const_type, const_value, right_arg_type, right_arg_value); } } + + if (const_value.isNull()) + return false; } out_key_column_num = it->second; diff --git a/tests/queries/0_stateless/02764_index_analysis_fix.reference b/tests/queries/0_stateless/02764_index_analysis_fix.reference new file mode 100644 index 00000000000..8eeacf99fa8 --- /dev/null +++ b/tests/queries/0_stateless/02764_index_analysis_fix.reference @@ -0,0 +1 @@ +2022-10-01 10:10:10 diff --git a/tests/queries/0_stateless/02764_index_analysis_fix.sql b/tests/queries/0_stateless/02764_index_analysis_fix.sql new file mode 100644 index 00000000000..541a3444ef3 --- /dev/null +++ b/tests/queries/0_stateless/02764_index_analysis_fix.sql @@ -0,0 +1,9 @@ +drop table if exists x; + +create table x (dt String) engine MergeTree partition by toYYYYMM(toDate(dt)) order by tuple(); + +insert into x values ('2022-10-01 10:10:10'); + +select * from x where dt like '2022-10-01%'; + +drop table x; From b11aa42db9337ea652f8a0b36c9e3f21e9f675af Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 24 May 2023 14:27:49 +0800 Subject: [PATCH 084/308] Fix tests --- src/Storages/MergeTree/KeyCondition.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index dea2091f115..923e5237420 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1041,7 +1041,8 @@ bool KeyCondition::transformConstantWithValidFunctions( const auto & func_name = func->function_base->getName(); auto func_base = func->function_base; - if (date_time_parsing_functions.contains(func_name)) + const auto & arg_types = func_base->getArgumentTypes(); + if (date_time_parsing_functions.contains(func_name) && !arg_types.empty() && isStringOrFixedString(arg_types[0])) { auto func_or_null = FunctionFactory::instance().get(func_name + "OrNull", context); ColumnsWithTypeAndName arguments; From 8bbfdcc56c1ad77729529b1bcbb65d4a5b7c2b6d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 24 May 2023 15:47:38 +0800 Subject: [PATCH 085/308] Fix index analysis with binary operator null --- src/Storages/MergeTree/KeyCondition.cpp | 4 ++++ ...ndex_analysis_binary_operator_with_null.reference | 0 ...2746_index_analysis_binary_operator_with_null.sql | 12 ++++++++++++ 3 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.reference create mode 100644 tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index b8ef2152a99..239a534ca93 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1334,6 +1334,10 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( arguments.push_back(const_arg); kind = FunctionWithOptionalConstArg::Kind::RIGHT_CONST; } + + /// If constant arg of binary operator is NULL, there will be no monotonicity. + if (const_arg.column->isNullAt(0)) + return false; } else arguments.push_back({ nullptr, key_column_type, "" }); diff --git a/tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.reference b/tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.sql b/tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.sql new file mode 100644 index 00000000000..f9613735bbf --- /dev/null +++ b/tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.sql @@ -0,0 +1,12 @@ +drop table if exists tab; + +create table tab (x DateTime) engine MergeTree order by x; + +SELECT toDateTime(65537, toDateTime(NULL), NULL) +FROM tab +WHERE ((x + CAST('1', 'Nullable(UInt8)')) <= 2) AND ((x + CAST('', 'Nullable(UInt8)')) <= 256) +ORDER BY + toDateTime(toDateTime(-2, NULL, NULL) + 100.0001, NULL, -2, NULL) DESC NULLS LAST, + x ASC NULLS LAST; + +drop table tab; From 4935b181604b718420fd05e1a3a0a00e6781f963 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 24 May 2023 11:07:10 +0000 Subject: [PATCH 086/308] Add 02763_last_day_of_week test draft --- .../0_stateless/02763_last_day_of_week.reference | 10 ++++++++++ tests/queries/0_stateless/02763_last_day_of_week.sql | 9 +++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02763_last_day_of_week.reference create mode 100644 tests/queries/0_stateless/02763_last_day_of_week.sql diff --git a/tests/queries/0_stateless/02763_last_day_of_week.reference b/tests/queries/0_stateless/02763_last_day_of_week.reference new file mode 100644 index 00000000000..9c136d856f6 --- /dev/null +++ b/tests/queries/0_stateless/02763_last_day_of_week.reference @@ -0,0 +1,10 @@ +2018-12-25 2018-12-25 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 +2018-12-26 2018-12-26 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 +2018-12-27 2018-12-27 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 +2018-12-28 2018-12-28 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 +2018-12-29 2018-12-29 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 +2018-12-30 2018-12-30 00:00:00 2019-01-05 2019-01-05 2018-12-30 2018-12-30 +2018-12-31 2018-12-31 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 +2019-01-01 2019-01-01 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 +2019-01-02 2019-01-02 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 +2019-01-03 2019-01-03 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 diff --git a/tests/queries/0_stateless/02763_last_day_of_week.sql b/tests/queries/0_stateless/02763_last_day_of_week.sql new file mode 100644 index 00000000000..600cebd2636 --- /dev/null +++ b/tests/queries/0_stateless/02763_last_day_of_week.sql @@ -0,0 +1,9 @@ +SELECT + toDate('2018-12-25') + number AS x, + toDateTime(x) AS x_t, + toLastDayOfWeek(x) AS w0, + toLastDayOfWeek(x_t) AS wt0, + toLastDayOfWeek(x, 3) AS w3, + toLastDayOfWeek(x_t, 3) AS wt3 +FROM numbers(10); + From 3bce2bbc37a070dc967ef071a510564b3a5066b7 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 24 May 2023 11:40:41 +0000 Subject: [PATCH 087/308] Fix settings.md in regard to enable_extended_results_for_datetime_functions --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cddde2090f8..0147ff6c377 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3791,8 +3791,8 @@ Result: ## enable_extended_results_for_datetime_functions {#enable-extended-results-for-datetime-functions} Enables or disables returning results of type: -- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md/#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md/#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md/#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md/#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md/#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md/#tomonday) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md/#tolastdayofmonth). -- `DateTime64` with extended range (compared to type `DateTime`) for functions [toStartOfDay](../../sql-reference/functions/date-time-functions.md/#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md/#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md/#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md/#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md/#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md/#tostartoffifteenminutes) and [timeSlot](../../sql-reference/functions/date-time-functions.md/#timeslot). +- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). +- `DateTime64` with extended range (compared to type `DateTime`) for functions [toStartOfDay](../../sql-reference/functions/date-time-functions.md#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffifteenminutes) and [timeSlot](../../sql-reference/functions/date-time-functions.md#timeslot). Possible values: From 99a7967f11050b1cc4e3a61365ff829b6084012f Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 24 May 2023 11:58:39 +0000 Subject: [PATCH 088/308] Amend the dicumentation with regard to toLastDayOfWeek --- docs/en/operations/settings/settings.md | 2 +- .../functions/date-time-functions.md | 20 ++++++++++++++++--- docs/ru/operations/settings/settings.md | 2 +- .../functions/date-time-functions.md | 20 +++++++++++++------ 4 files changed, 33 insertions(+), 11 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 0147ff6c377..52ae82b8f36 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3791,7 +3791,7 @@ Result: ## enable_extended_results_for_datetime_functions {#enable-extended-results-for-datetime-functions} Enables or disables returning results of type: -- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). +- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday), [toLastDayOfWeek](../../sql-reference/functions/date-time-functions.md#tolastdayofweek) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). - `DateTime64` with extended range (compared to type `DateTime`) for functions [toStartOfDay](../../sql-reference/functions/date-time-functions.md#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffifteenminutes) and [timeSlot](../../sql-reference/functions/date-time-functions.md#timeslot). Possible values: diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 4710a5617c3..42b25de7f61 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -389,12 +389,14 @@ Result: ``` :::note -The return type of `toStartOf*`, `toLastDayOfMonth`, `toMonday`, `timeSlot` functions described below is determined by the configuration parameter [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) which is `0` by default. +The return type of `toStartOf*`, `toLastDayOf*`, `toMonday`, `timeSlot` functions described below is determined by the configuration parameter [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) which is `0` by default. Behavior for -* `enable_extended_results_for_datetime_functions = 0`: Functions `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfMonth`, `toMonday` return `Date` or `DateTime`. Functions `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` return `DateTime`. Though these functions can take values of the extended types `Date32` and `DateTime64` as an argument, passing them a time outside the normal range (year 1970 to 2149 for `Date` / 2106 for `DateTime`) will produce wrong results. +* `enable_extended_results_for_datetime_functions = 0`: + * Functions `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfWeek`, `toLastDayOfMonth`, `toMonday` return `Date` or `DateTime`. + * Functions `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` return `DateTime`. Though these functions can take values of the extended types `Date32` and `DateTime64` as an argument, passing them a time outside the normal range (year 1970 to 2149 for `Date` / 2106 for `DateTime`) will produce wrong results. * `enable_extended_results_for_datetime_functions = 1`: - * Functions `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfMonth`, `toMonday` return `Date` or `DateTime` if their argument is a `Date` or `DateTime`, and they return `Date32` or `DateTime64` if their argument is a `Date32` or `DateTime64`. + * Functions `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfWeek`, `toLastDayOfMonth`, `toMonday` return `Date` or `DateTime` if their argument is a `Date` or `DateTime`, and they return `Date32` or `DateTime64` if their argument is a `Date32` or `DateTime64`. * Functions `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` return `DateTime` if their argument is a `Date` or `DateTime`, and they return `DateTime64` if their argument is a `Date32` or `DateTime64`. ::: @@ -449,6 +451,18 @@ The mode argument works exactly like the mode argument in function `toWeek()`. I toStartOfWeek(t[, mode[, timezone]]) ``` +## toLastDayOfWeek + +Rounds a date or date with time up to the nearest Saturday or Sunday. +Returns the date. +The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, mode is assumed as 0. + +**Syntax** + +``` sql +toLastDayOfWeek(t[, mode[, timezone]]) +``` + ## toStartOfDay Rounds down a date with time to the start of the day. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 066e71c25a5..65b75ff3d74 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3800,7 +3800,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; ## enable_extended_results_for_datetime_functions {#enable-extended-results-for-datetime-functions} Включает или отключает возвращение результатов типа: -- `Date32` с расширенным диапазоном (по сравнению с типом `Date`) для функций [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday) и [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). +- `Date32` с расширенным диапазоном (по сравнению с типом `Date`) для функций [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday), [toLastDayOfWeek](../../sql-reference/functions/date-time-functions.md#tolastdayofweek) и [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). - `DateTime64` с расширенным диапазоном (по сравнению с типом `DateTime`) для функций [toStartOfDay](../../sql-reference/functions/date-time-functions.md#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffifteenminutes) и [timeSlot](../../sql-reference/functions/date-time-functions.md#timeslot). Возможные значения: diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index a7e8a478edb..ee86f0d3d13 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -268,13 +268,15 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp; ``` :::note -Тип возвращаемого значения описанными далее функциями `toStartOf*`, `toLastDayOfMonth`, `toMonday`, `timeSlot` определяется конфигурационным параметром [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) имеющим по умолчанию значение `0`. +Тип возвращаемого значения описанными далее функциями `toStartOf*`, `toLastDayOf*`, `toMonday`, `timeSlot` определяется конфигурационным параметром [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) имеющим по умолчанию значение `0`. Поведение для -* `enable_extended_results_for_datetime_functions = 0`: Функции `toStartOf*`, `toLastDayOfMonth`, `toMonday` возвращают `Date` или `DateTime`. Функции `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` возвращают `DateTime`. Хотя эти функции могут принимать значения типа `Date32` или `DateTime64` в качестве аргумента, при обработке аргумента вне нормального диапазона значений (`1970` - `2148` для `Date` и `1970-01-01 00:00:00`-`2106-02-07 08:28:15` для `DateTime`) будет получен некорректный результат. +* `enable_extended_results_for_datetime_functions = 0`: + * Функции `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfWeek`, `toLastDayOfMonth`, `toMonday` возвращают `Date` или `DateTime`. + * Функции `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` возвращают `DateTime`. Хотя эти функции могут принимать значения расширенных типов `Date32` и `DateTime64` в качестве аргумента, при обработке аргумента вне нормального диапазона значений (`1970` - `2148` для `Date` и `1970-01-01 00:00:00`-`2106-02-07 08:28:15` для `DateTime`) будет получен некорректный результат. * `enable_extended_results_for_datetime_functions = 1`: - * Функции `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfMonth`, `toMonday` возвращают `Date` или `DateTime` если их аргумент `Date` или `DateTime` и они возвращают `Date32` или `DateTime64` если их аргумент `Date32` или `DateTime64`. - * Функции `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` возвращают `DateTime` если их аргумент `Date` или `DateTime` и они возвращают `DateTime64` если их аргумент `Date32` или `DateTime64`. + * Функции `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfWeek`, `toLastDayOfMonth`, `toMonday` возвращают `Date` или `DateTime` если их аргумент `Date` или `DateTime` и они возвращают `Date32` или `DateTime64` если их аргумент `Date32` или `DateTime64`. + * Функции `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` возвращают `DateTime`, если их аргумент имеет тип `Date` или `DateTime`, и `DateTime64` если их аргумент имеет тип `Date32` или `DateTime64`. ::: ## toStartOfYear {#tostartofyear} @@ -324,9 +326,15 @@ SELECT toStartOfISOYear(toDate('2017-01-01')) AS ISOYear20170101; Округляет дату или дату-с-временем вниз до ближайшего понедельника. Возвращается дата. -## toStartOfWeek(t[,mode]) {#tostartofweek} +## toStartOfWeek(t[, mode[, timezone]]) -Округляет дату или дату со временем до ближайшего воскресенья или понедельника в соответствии с mode. +Округляет дату или дату-с-временем назад, до ближайшего воскресенья или понедельника, в соответствии с mode. +Возвращается дата. +Аргумент mode работает точно так же, как аргумент mode [toWeek()](#toweek). Если аргумент mode опущен, то используется режим 0. + +## toLastDayOfWeek(t[, mode[, timezone]]) + +Округляет дату или дату-с-временем вперёд, до ближайшей субботы или воскресенья, в соответствии с mode. Возвращается дата. Аргумент mode работает точно так же, как аргумент mode [toWeek()](#toweek). Если аргумент mode опущен, то используется режим 0. From 8bc25c4ea3a1359affc36599bcc982b741ea5360 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 24 May 2023 14:40:27 +0200 Subject: [PATCH 089/308] Fix style --- tests/integration/test_reload_clusters_config/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_reload_clusters_config/test.py b/tests/integration/test_reload_clusters_config/test.py index 73ca4a01f34..cb003bbe04e 100644 --- a/tests/integration/test_reload_clusters_config/test.py +++ b/tests/integration/test_reload_clusters_config/test.py @@ -169,7 +169,9 @@ test_config3 = """ def send_repeated_query(table, count=5): for i in range(count): node.query_and_get_error( - "SELECT count() FROM {} SETTINGS receive_timeout=1, handshake_timeout_ms=1".format(table) + "SELECT count() FROM {} SETTINGS receive_timeout=1, handshake_timeout_ms=1".format( + table + ) ) From bc527c75889b321a01c30f665eb0d4ef47e61d68 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 24 May 2023 17:07:31 +0000 Subject: [PATCH 090/308] Don't send head request for all keys in Iceberg schema inference --- src/Storages/StorageS3.cpp | 19 ++++++++++++++----- src/Storages/StorageS3.h | 2 ++ 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index afaafcc75a2..f3cad4de31a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -418,11 +418,13 @@ public: ASTPtr query_, const Block & virtual_header_, ContextPtr context_, + bool need_total_size, KeysWithInfo * read_keys_) : WithContext(context_) , bucket(bucket_) , query(query_) , virtual_header(virtual_header_) + { Strings all_keys = keys_; @@ -458,8 +460,13 @@ public: for (auto && key : all_keys) { - auto info = S3::getObjectInfo(client_, bucket, key, version_id_, request_settings_); - total_size += info.size; + std::optional info; + if (need_total_size) + { + info = S3::getObjectInfo(client_, bucket, key, version_id_, request_settings_); + total_size += info->size; + } + keys.emplace_back(std::move(key), std::move(info)); } @@ -501,10 +508,11 @@ StorageS3Source::KeysIterator::KeysIterator( ASTPtr query, const Block & virtual_header, ContextPtr context, + bool need_total_size, KeysWithInfo * read_keys) : pimpl(std::make_shared( client_, version_id_, keys_, bucket_, request_settings_, - query, virtual_header, context, read_keys)) + query, virtual_header, context, need_total_size, read_keys)) { } @@ -979,6 +987,7 @@ std::shared_ptr StorageS3::createFileIterator( ContextPtr local_context, ASTPtr query, const Block & virtual_block, + bool need_total_size, KeysWithInfo * read_keys) { if (distributed_processing) @@ -997,7 +1006,7 @@ std::shared_ptr StorageS3::createFileIterator( return std::make_shared( *configuration.client, configuration.url.version_id, configuration.keys, configuration.url.bucket, configuration.request_settings, query, - virtual_block, local_context, read_keys); + virtual_block, local_context, need_total_size, read_keys); } } @@ -1442,7 +1451,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( { KeysWithInfo read_keys; - auto file_iterator = createFileIterator(configuration, false, ctx, nullptr, {}, &read_keys); + auto file_iterator = createFileIterator(configuration, false, ctx, nullptr, {}, false, &read_keys); std::optional columns_from_cache; size_t prev_read_keys_size = read_keys.size(); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 12573ab513f..1ca8f80e7a0 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -95,6 +95,7 @@ public: ASTPtr query, const Block & virtual_header, ContextPtr context, + bool need_total_size = true, KeysWithInfo * read_keys = nullptr); KeyWithInfo next() override; @@ -354,6 +355,7 @@ private: ContextPtr local_context, ASTPtr query, const Block & virtual_block, + bool need_total_size = true, KeysWithInfo * read_keys = nullptr); static ColumnsDescription getTableStructureFromDataImpl( From 03ca3f96d2bbb455dbef82e7d1937771f49382e2 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 24 May 2023 17:40:21 +0000 Subject: [PATCH 091/308] Add built-in documentation to toStartOfWeek and toLastDayOfWeek functions --- src/Functions/toCustomWeek.cpp | 26 ++++++++++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/src/Functions/toCustomWeek.cpp b/src/Functions/toCustomWeek.cpp index 98e7aaf1d6b..0dbd60b3eff 100644 --- a/src/Functions/toCustomWeek.cpp +++ b/src/Functions/toCustomWeek.cpp @@ -17,8 +17,30 @@ REGISTER_FUNCTION(ToCustomWeek) { factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + + factory.registerFunction(FunctionDocumentation{.description=R"( +Rounds a date or date with time down to the nearest Sunday or Monday. Returns the date. +Syntax: toStartOfWeek(t[, mode[, timezone]]) +The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, mode is assumed as 0. + +Example: +[example:typical] +)", + .examples{ + {"typical", "SELECT toStartOfWeek(today(), 1);", ""}}, + .categories{"Dates and Times"}}, FunctionFactory::CaseSensitive); + + factory.registerFunction(FunctionDocumentation{.description=R"( +Rounds a date or date with time up to the nearest Saturday or Sunday. Returns the date. +Syntax: toLastDayOfWeek(t[, mode[, timezone]]) +The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, mode is assumed as 0. + +Example: +[example:typical] +)", + .examples{ + {"typical", "SELECT toLastDayOfWeek(today(), 1);", ""}}, + .categories{"Dates and Times"}}, FunctionFactory::CaseSensitive); /// Compatibility aliases for mysql. factory.registerAlias("week", "toWeek", FunctionFactory::CaseInsensitive); From d19d36aad1a2c887ec11ce478f3a8c00689a1cb5 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 24 May 2023 20:53:05 +0000 Subject: [PATCH 092/308] Extend 02763_last_day_of_week test: add Date32 and DateTime64 types --- .../02763_last_day_of_week.reference | 20 +++++++++---------- .../0_stateless/02763_last_day_of_week.sql | 20 +++++++++++++------ 2 files changed, 24 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02763_last_day_of_week.reference b/tests/queries/0_stateless/02763_last_day_of_week.reference index 9c136d856f6..140ee9bfe24 100644 --- a/tests/queries/0_stateless/02763_last_day_of_week.reference +++ b/tests/queries/0_stateless/02763_last_day_of_week.reference @@ -1,10 +1,10 @@ -2018-12-25 2018-12-25 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 -2018-12-26 2018-12-26 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 -2018-12-27 2018-12-27 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 -2018-12-28 2018-12-28 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 -2018-12-29 2018-12-29 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 -2018-12-30 2018-12-30 00:00:00 2019-01-05 2019-01-05 2018-12-30 2018-12-30 -2018-12-31 2018-12-31 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 -2019-01-01 2019-01-01 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 -2019-01-02 2019-01-02 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 -2019-01-03 2019-01-03 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 +2018-12-25 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-26 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-27 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-28 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-29 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-30 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-31 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 +2019-01-01 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 +2019-01-02 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 +2019-01-03 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 diff --git a/tests/queries/0_stateless/02763_last_day_of_week.sql b/tests/queries/0_stateless/02763_last_day_of_week.sql index 600cebd2636..533ecfaa5b2 100644 --- a/tests/queries/0_stateless/02763_last_day_of_week.sql +++ b/tests/queries/0_stateless/02763_last_day_of_week.sql @@ -1,9 +1,17 @@ +WITH + toDate('2018-12-25') + number AS d, + toDate32(d) AS d32, + toDateTime(d) AS dt, + toDateTime64(d, 0) AS dt64 SELECT - toDate('2018-12-25') + number AS x, - toDateTime(x) AS x_t, - toLastDayOfWeek(x) AS w0, - toLastDayOfWeek(x_t) AS wt0, - toLastDayOfWeek(x, 3) AS w3, - toLastDayOfWeek(x_t, 3) AS wt3 + dt64, + toLastDayOfWeek(d) AS wd_0, + toLastDayOfWeek(d32) AS wd32_0, + toLastDayOfWeek(dt) AS wdt_0, + toLastDayOfWeek(dt64) AS wdt64_0, + toLastDayOfWeek(d, 3) AS wd_3, + toLastDayOfWeek(d32, 3) AS wd32_3, + toLastDayOfWeek(dt, 3) AS wdt_3, + toLastDayOfWeek(dt64, 3) AS wdt64_3 FROM numbers(10); From d48a7f9b2cbaa6e905611d58321a7515309051ac Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 24 May 2023 21:08:25 +0000 Subject: [PATCH 093/308] Grant toStartOfWeek function a permission to get documentation --- .../02415_all_new_functions_must_be_documented.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index eb574c4e4f5..a22a1552156 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -813,7 +813,6 @@ toStartOfNanosecond toStartOfQuarter toStartOfSecond toStartOfTenMinutes -toStartOfWeek toStartOfYear toString toStringCutToZero From 75791d7a63b9a6a579e1f036cdffc321bcc9fa2d Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 25 May 2023 07:51:32 +0000 Subject: [PATCH 094/308] Added input_format_csv_trim_whitespaces parameter --- docs/en/interfaces/formats.md | 1 + .../operations/settings/settings-formats.md | 32 +++++ docs/ru/interfaces/formats.md | 17 +++ docs/ru/operations/settings/settings.md | 62 +++++++++ src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + src/IO/ReadHelpers.cpp | 19 +-- .../Formats/Impl/CSVRowInputFormat.cpp | 17 ++- .../02764_csv_trim_whitespaces.reference | 122 ++++++++++++++++++ .../0_stateless/02764_csv_trim_whitespaces.sh | 55 ++++++++ 11 files changed, 317 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02764_csv_trim_whitespaces.reference create mode 100755 tests/queries/0_stateless/02764_csv_trim_whitespaces.sh diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 2ab9e8caec4..f19fd94dcd8 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -467,6 +467,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [output_format_csv_crlf_end_of_line](/docs/en/operations/settings/settings-formats.md/#output_format_csv_crlf_end_of_line) - if it is set to true, end of line in CSV output format will be `\r\n` instead of `\n`. Default value - `false`. - [input_format_csv_skip_first_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_first_lines) - skip the specified number of lines at the beginning of data. Default value - `0`. - [input_format_csv_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_csv_detect_header) - automatically detect header with names and types in CSV format. Default value - `true`. +- [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 3b87b829c92..cb7d98a4876 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -882,6 +882,38 @@ My NULL My NULL ``` +### input_format_csv_trim_whitespaces {#input_format_csv_trim_whitespaces} + +Trims spaces and tabs in non-quoted CSV strings. + +Default value: `true`. + +**Examples** + +Query + +```bash +echo ' string ' |./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true +``` + +Result + +```text +"string" +``` + +Query + +```bash +echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=false +``` + +Result + +```text +" string " +``` + ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index bef5c223281..48a6132170a 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -387,6 +387,23 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR Формат CSV поддерживает вывод totals и extremes аналогично `TabSeparated`. + +### CSV опции форматирования {#csv-format-settings} + +- [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter) - символ, который будет считаться разделителем в CSV данных. Значение по умолчанию - `,`. +- [format_csv_allow_single_quotes](../operations/settings/settings.md#format_csv_allow_single_quotes) - разрешить строки в одинарных кавычках. Значение по умолчанию - `true`. +- [format_csv_allow_double_quotes](../operations/settings/settings.md#format_csv_allow_double_quotes) - разрешить строки в двойных кавычках. Значение по умолчанию - `true`. +- [format_csv_null_representation](../operations/settings/settings.md#format_tsv_null_representation) - пользовательское представление NULL в формате CSV. Значение по умолчанию - `\N`. +- [input_format_csv_empty_as_default](../operations/settings/settings.md#input_format_csv_empty_as_default) - рассматривать пустые поля в CSV в качестве значений по умолчанию. Значение по умолчанию - `true`. Для сложных выражений по умолчанию необходимо также включить [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#input_format_defaults_for_omitted_fields). +- [input_format_csv_enum_as_number](../operations/settings/settings.md#input_format_csv_enum_as_number) - рассматривать вставленные значения enum в форматах CSV как индексы enum. Значение по умолчанию - `false`. +- [input_format_csv_use_best_effort_in_schema_inference](../operations/settings/settings.md#input_format_csv_use_best_effort_in_schema_inference) - использовать некоторые твики и эвристики для вывода схемы в формате CSV. Если параметр отключен, все поля будут определяться как строки. Значение по умолчанию - `true`. +- [input_format_csv_arrays_as_nested_csv](../operations/settings/settings.md#input_format_csv_arrays_as_nested_csv) - при чтении массива из CSV ожидать, что его элементы были сериализованы во вложенный CSV и затем помещены в строку. Значение по умолчанию - `false`. +- [output_format_csv_crlf_end_of_line](../operations/settings/settings.md#output_format_csv_crlf_end_of_line) - если установлено значение true, конец строки в формате вывода CSV будет `\r\n` вместо `\n`. Значение по умолчанию - `false`. +- [input_format_csv_skip_first_lines](../operations/settings/settings.md#input_format_csv_skip_first_lines) - пропустить указанное количество строк в начале данных. Значение по умолчанию - `0`. +- [input_format_csv_detect_header](../operations/settings/settings.md#input_format_csv_detect_header) - обнаружить заголовок с именами и типами в формате CSV. Значение по умолчанию - `true`. +- [input_format_csv_trim_whitespaces](../operations/settings/settings.md#input_format_csv_trim_whitespaces) - удалить пробелы и символы табуляции из строк без кавычек. +Значение по умолчанию - `true`. + ## CSVWithNames {#csvwithnames} Выводит также заголовок, аналогично [TabSeparatedWithNames](#tabseparatedwithnames). diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index fa3ea582c55..e9b7091c8b8 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1589,6 +1589,24 @@ SELECT area/period FROM account_orders FORMAT JSON; Символ, интерпретируемый как разделитель в данных формата CSV. По умолчанию — `,`. +## format_csv_allow_double_quotes {#format_csv_allow_double_quotes} + +Если установлено значение true, разрешить строки в двойных кавычках. + +Включено по умолчанию. + +## input_format_csv_empty_as_default {#input_format_csv_empty_as_default} + +Если включено, заменяет пустые поля ввода в CSV значениями по умолчанию. Для сложных выражений по умолчанию `input_format_defaults_for_omitted_fields` также должен быть включен. + +Включено по умолчанию. + +## input_format_csv_arrays_as_nested_csv {#input_format_csv_arrays_as_nested_csv} + +При чтении массива из CSV ожидайте, что его элементы были сериализованы во вложенный CSV, а затем помещены в строку. Пример: "[""Hello"", ""world"", ""42"""" TV""]". Скобки вокруг массива могут быть опущены. + +По умолчанию отключены. + ## input_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null} Для формата CSV включает или выключает парсинг неэкранированной строки `NULL` как литерала (синоним для `\N`) @@ -1665,6 +1683,50 @@ SELECT * FROM table_with_enum_column_for_csv_insert; Использовать в качестве разделителя строк для CSV формата CRLF (DOS/Windows стиль) вместо LF (Unix стиль). +## input_format_csv_detect_header {#input_format_csv_detect_header} + +Обнаружить заголовок с именами и типами в формате CSV. + +Значение по умолчанию - `true`. + +## input_format_csv_skip_first_lines {#input_format_csv_skip_first_lines} + +Количество строк, пропускаемых в начале данных в формате ввода CSV. + +Значение по умолчанию: `0`. + +## input_format_csv_trim_whitespaces {#input_format_csv_trim_whitespaces} + +Удалить пробелы и символы табуляции из строк без кавычек. + +Значение по умолчанию: `true`. + +**Примеры** + +Запрос + +```bash +echo ' string ' |./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true +``` + +Результат + +```text +"string" +``` + +Запрос + +```bash +echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=false +``` + +Результат + +```text +" string " +``` + ## output_format_tsv_crlf_end_of_line {#settings-output-format-tsv-crlf-end-of-line} Использовать в качестве разделителя строк для TSV формата CRLF (DOC/Windows стиль) вместо LF (Unix стиль). diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1df0a8af24f..750b6e16c4b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -844,6 +844,7 @@ class IColumn; M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \ M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \ M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ + M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index dd6252b96f1..ada8751545c 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -69,6 +69,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.use_best_effort_in_schema_inference = settings.input_format_csv_use_best_effort_in_schema_inference; format_settings.csv.skip_first_lines = settings.input_format_csv_skip_first_lines; format_settings.csv.try_detect_header = settings.input_format_csv_detect_header; + format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index c88af650671..3ae579cd552 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -136,6 +136,7 @@ struct FormatSettings UInt64 skip_first_lines = 0; String custom_delimiter; bool try_detect_header = true; + bool trim_whitespaces = true; } csv; struct HiveText diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 99d25ee6613..8dc05e75855 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -849,15 +849,18 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & if constexpr (WithResize) { - /** CSV format can contain insignificant spaces and tabs. - * Usually the task of skipping them is for the calling code. - * But in this case, it will be difficult to do this, so remove the trailing whitespace by ourself. - */ - size_t size = s.size(); - while (size > 0 && (s[size - 1] == ' ' || s[size - 1] == '\t')) - --size; + if (settings.trim_whitespaces) [[likely]] + { + /** CSV format can contain insignificant spaces and tabs. + * Usually the task of skipping them is for the calling code. + * But in this case, it will be difficult to do this, so remove the trailing whitespace by ourself. + */ + size_t size = s.size(); + while (size > 0 && (s[size - 1] == ' ' || s[size - 1] == '\t')) + --size; - s.resize(size); + s.resize(size); + } } return; } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index de955d81651..9922bd41442 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -152,7 +153,9 @@ void CSVFormatReader::skipFieldDelimiter() template String CSVFormatReader::readCSVFieldIntoString() { - skipWhitespacesAndTabs(*buf); + if (format_settings.csv.trim_whitespaces) [[likely]] + skipWhitespacesAndTabs(*buf); + String field; if constexpr (read_string) readCSVString(field, *buf, format_settings.csv); @@ -200,7 +203,6 @@ void CSVFormatReader::skipHeaderRow() template std::vector CSVFormatReader::readRowImpl() { - std::vector fields; do { @@ -280,7 +282,16 @@ bool CSVFormatReader::readField( bool is_last_file_column, const String & /*column_name*/) { - skipWhitespacesAndTabs(*buf); + if (format_settings.csv.trim_whitespaces) [[likely]] + skipWhitespacesAndTabs(*buf); + else if (type->isNullable()) + { + auto nested_type = typeid_cast(type.get())->getNestedType(); + if (!isStringOrFixedString(nested_type)) + skipWhitespacesAndTabs(*buf); + } + else if (!isStringOrFixedString(type)) + skipWhitespacesAndTabs(*buf); const bool at_delimiter = !buf->eof() && *buf->position() == format_settings.csv.delimiter; const bool at_last_column_line_end = is_last_file_column && (buf->eof() || *buf->position() == '\n' || *buf->position() == '\r'); diff --git a/tests/queries/0_stateless/02764_csv_trim_whitespaces.reference b/tests/queries/0_stateless/02764_csv_trim_whitespaces.reference new file mode 100644 index 00000000000..a2ea31ddae7 --- /dev/null +++ b/tests/queries/0_stateless/02764_csv_trim_whitespaces.reference @@ -0,0 +1,122 @@ +" trim_false_tab_left" +"trim_false_tab_right " +"trim_false_ tab_middle" +" trim_false_ tab_everywhere " +" trim_false_fixed_string_ tab_everywhere " +" quoted_trim_false_ tab_everywhere " +" trim_false_csv_field1 ","123 ","5.0 "," 12.0123"," quoted_string1" +" trim_false_csv_field2 "," 321 "," 0.5","21.321 "," quoted_ string2 " +" trim_false_csv_field1_with_ structure ",123,5,12.0123," quoted_string " +" trim_false_csv_field2_with_structure ",321,0.5,21.321," quoted_ _string2 " +" trim_false_space_left" +"trim_false_space_right " +"trim_false_ space_middle" +" trim_false_ space_everywhere " +" trim_false_fixed_string_ space_everywhere " +" quoted_trim_false_ space_everywhere " +" trim_false_csv_field1 ","123 ","5.0 "," 12.0123"," quoted_string1" +" trim_false_csv_field2 "," 321 "," 0.5","21.321 "," quoted_ string2 " +" trim_false_csv_field1_with_ structure ",123,5,12.0123," quoted_string " +" trim_false_csv_field2_with_structure ",321,0.5,21.321," quoted_ _string2 " +" trim_false_tab_space_left" +"trim_false_tab_space_right " +"trim_false_ tab_space_middle" +" trim_false_ tab_space_everywhere " +" trim_false_fixed_string_ tab_space_everywhere " +" quoted_trim_false_ tab_space_everywhere " +" trim_false_csv_field1 ","123 ","5.0 "," 12.0123"," quoted_string1" +" trim_false_csv_field2 "," 321 "," 0.5","21.321 "," quoted_ string2 " +" trim_false_csv_field1_with_ structure ",123,5,12.0123," quoted_string " +" trim_false_csv_field2_with_structure ",321,0.5,21.321," quoted_ _string2 " +8 +8 +16 +16 +32.32 +32.32 +64.64 +64.64 +"2023-05-22" +"2023-05-22" +"2023-05-22" +"2023-05-22" +"2023-05-22 00:00:00" +"2023-05-22 00:00:00" +"2023-05-22 00:00:00.000" +"2023-05-22 00:00:00.000" +"trim_true_tab_left" +"trim_true_tab_right" +"trim_true_ tab_middle" +"trim_true_ tab_everywhere" +"trim_true_fixed_string_ tab_everywhere" +" quoted_trim_true_ tab_everywhere " +"trim_true_csv_field1",123,5,12.0123," quoted_string1" +"trim_true_csv_field2",321,0.5,21.321," quoted_ string2 " +"trim_true_csv_field1_with_ structure",123,5,12.0123," quoted_string " +"trim_true_csv_field2_with_structure",321,0.5,21.321," quoted_ _string2 " +"trim_true_space_left" +"trim_true_space_right" +"trim_true_ space_middle" +"trim_true_ space_everywhere" +"trim_true_fixed_string_ space_everywhere" +" quoted_trim_true_ space_everywhere " +"trim_true_csv_field1",123,5,12.0123," quoted_string1" +"trim_true_csv_field2",321,0.5,21.321," quoted_ string2 " +"trim_true_csv_field1_with_ structure",123,5,12.0123," quoted_string " +"trim_true_csv_field2_with_structure",321,0.5,21.321," quoted_ _string2 " +"trim_true_tab_space_left" +"trim_true_tab_space_right" +"trim_true_ tab_space_middle" +"trim_true_ tab_space_everywhere" +"trim_true_fixed_string_ tab_space_everywhere" +" quoted_trim_true_ tab_space_everywhere " +"trim_true_csv_field1",123,5,12.0123," quoted_string1" +"trim_true_csv_field2",321,0.5,21.321," quoted_ string2 " +"trim_true_csv_field1_with_ structure",123,5,12.0123," quoted_string " +"trim_true_csv_field2_with_structure",321,0.5,21.321," quoted_ _string2 " +8 +8 +16 +16 +32.32 +32.32 +64.64 +64.64 +"2023-05-22" +"2023-05-22" +"2023-05-22" +"2023-05-22" +"2023-05-22 00:00:00" +"2023-05-22 00:00:00" +"2023-05-22 00:00:00.000" +"2023-05-22 00:00:00.000" +" custom_csv_tab_left" +"custom_csv_tab_right " +"custom_csv_ tab_middle" +" custom_csv_ tab_everywhere " +" custom_csv_fixed_string_ tab_everywhere " +" quoted_custom_csv_ tab_everywhere " +" custom_csv_field_with_ structure ",123,5,12.0123," custom_csv_quoted_string " +" custom_csv_field2_with_structure ",321,0.5,21.321," custom_csv_quoted_ _string2 " +" custom_csv_field_with_ structure ",123,5,12.0123," custom_csv_quoted_string " +" custom_csv_field2_with_structure ",321,0.5,21.321," custom_csv_quoted_ _string2 " +" custom_csv_space_left" +"custom_csv_space_right " +"custom_csv_ space_middle" +" custom_csv_ space_everywhere " +" custom_csv_fixed_string_ space_everywhere " +" quoted_custom_csv_ space_everywhere " +" custom_csv_field_with_ structure ",123,5,12.0123," custom_csv_quoted_string " +" custom_csv_field2_with_structure ",321,0.5,21.321," custom_csv_quoted_ _string2 " +" custom_csv_field_with_ structure ",123,5,12.0123," custom_csv_quoted_string " +" custom_csv_field2_with_structure ",321,0.5,21.321," custom_csv_quoted_ _string2 " +" custom_csv_tab_space_left" +"custom_csv_tab_space_right " +"custom_csv_ tab_space_middle" +" custom_csv_ tab_space_everywhere " +" custom_csv_fixed_string_ tab_space_everywhere " +" quoted_custom_csv_ tab_space_everywhere " +" custom_csv_field_with_ structure ",123,5,12.0123," custom_csv_quoted_string " +" custom_csv_field2_with_structure ",321,0.5,21.321," custom_csv_quoted_ _string2 " +" custom_csv_field_with_ structure ",123,5,12.0123," custom_csv_quoted_string " +" custom_csv_field2_with_structure ",321,0.5,21.321," custom_csv_quoted_ _string2 " diff --git a/tests/queries/0_stateless/02764_csv_trim_whitespaces.sh b/tests/queries/0_stateless/02764_csv_trim_whitespaces.sh new file mode 100755 index 00000000000..9b3b7231f6e --- /dev/null +++ b/tests/queries/0_stateless/02764_csv_trim_whitespaces.sh @@ -0,0 +1,55 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +BOOLS=('false' 'true') + +WHITESPACES=( '\t' ' ' '\t ') +WHITESPACES_NAMES=('tab' 'space' 'tab_space') + +DATA_TYPES=( 'Int8' 'Int16' 'Float32' 'Float64' 'Date' 'Date32' 'DateTime' 'DateTime64') +DATA_VALUES=( '8' '16' '32.32' '64.64' '2023-05-22' '2023-05-22' '2023-05-22 00:00:00' '2023-05-22 00:00:00.000') + +for trim in "${BOOLS[@]}" +do + for wsIndex in "${!WHITESPACES[@]}"; + do + whitespace=${WHITESPACES[$wsIndex]} + whitespace_name=${WHITESPACES_NAMES[$wsIndex]} + echo -e "${whitespace}trim_${trim}_${whitespace_name}_left" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + echo -e "trim_${trim}_${whitespace_name}_right${whitespace}" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + echo -e "trim_${trim}_${whitespace}${whitespace_name}_middle" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + echo -e "${whitespace}trim_${trim}_${whitespace}${whitespace_name}_everywhere${whitespace}" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + echo -e "${whitespace}trim_${trim}_fixed_string_${whitespace}${whitespace_name}_everywhere${whitespace}" | $CLICKHOUSE_LOCAL -S "c1 FixedString(64)" --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select toString(c1) from table FORMAT CSV" + echo -e "\"${whitespace}quoted_trim_${trim}_${whitespace}${whitespace_name}_everywhere${whitespace}\"" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + echo -e "${whitespace}trim_${trim}_csv_field1${whitespace},123${whitespace},5.0${whitespace},${whitespace}12.0123,\"${whitespace}quoted_string1\"\n${whitespace}trim_${trim}_csv_field2${whitespace},${whitespace}321${whitespace},${whitespace}0.5,21.321${whitespace},\"${whitespace}quoted_${whitespace}string2${whitespace}\"${whitespace}" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + echo -e "${whitespace}trim_${trim}_csv_field1_with_${whitespace}structure${whitespace},${whitespace}123,${whitespace}5.0${whitespace},12.0123${whitespace},\"${whitespace}quoted_string${whitespace}\"\n${whitespace}trim_${trim}_csv_field2_with_structure${whitespace},${whitespace}321${whitespace},0.5,21.321,\"${whitespace}quoted_${whitespace}_string2${whitespace}\"${whitespace}" | $CLICKHOUSE_LOCAL -S "c1 String, c2 Int32, c3 Float, c4 Double, c5 String" --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + done + + for type_index in "${!DATA_TYPES[@]}"; + do + type=${DATA_TYPES[$type_index]} + value=${DATA_VALUES[$type_index]} + echo -e "\t ${value} \t" | $CLICKHOUSE_LOCAL -S "c1 ${type}" --input-format="CSV" --input_format_csv_trim_whitespaces=${trim} -q "select * from table FORMAT CSV" + echo -e "\t ${value} \t" | $CLICKHOUSE_LOCAL -S "c1 Nullable(${type})" --input-format="CSV" --input_format_csv_trim_whitespaces=${trim} -q "select * from table FORMAT CSV" + done +done + +## Custom CSV tested with input_format_csv_trim_whitespaces = false. +## Custom CSV with input_format_csv_trim_whitespaces=true doesn't trim whitespaces from the left side at the moment +for wsIndex in "${!WHITESPACES[@]}"; +do + whitespace=${WHITESPACES[$wsIndex]} + whitespace_name=${WHITESPACES_NAMES[$wsIndex]} + echo -e "${whitespace}custom_csv_${whitespace_name}_left" | $CLICKHOUSE_LOCAL --input-format="CustomSeparated" --input_format_csv_trim_whitespaces=false --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" + echo -e "custom_csv_${whitespace_name}_right${whitespace}" | $CLICKHOUSE_LOCAL --input-format="CustomSeparated" --input_format_csv_trim_whitespaces=false --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" + echo -e "custom_csv_${whitespace}${whitespace_name}_middle" | $CLICKHOUSE_LOCAL --input-format="CustomSeparated" --input_format_csv_trim_whitespaces=false --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" + echo -e "${whitespace}custom_csv_${whitespace}${whitespace_name}_everywhere${whitespace}" | $CLICKHOUSE_LOCAL --input-format="CustomSeparated" --input_format_csv_trim_whitespaces=false --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" + echo -e "${whitespace}custom_csv_fixed_string_${whitespace}${whitespace_name}_everywhere${whitespace}" | $CLICKHOUSE_LOCAL -S "c1 FixedString(64)" --input-format="CustomSeparated" --input_format_csv_trim_whitespaces=false --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select toString(c1) from table FORMAT CSV" + echo -e "\"${whitespace}quoted_custom_csv_${whitespace}${whitespace_name}_everywhere${whitespace}\"" | $CLICKHOUSE_LOCAL --input-format="CustomSeparated" --input_format_csv_trim_whitespaces=false --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" + + echo -e "${whitespace}custom_csv_field_with_${whitespace}structure${whitespace},123,5.0,12.0123,\"${whitespace}custom_csv_quoted_string${whitespace}\"\n${whitespace}custom_csv_field2_with_structure${whitespace},321,0.5,21.321,\"${whitespace}custom_csv_quoted_${whitespace}_string2${whitespace}\"" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=false --input-format="CustomSeparated" --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" + echo -e "${whitespace}custom_csv_field_with_${whitespace}structure${whitespace},123,5.0,12.0123,\"${whitespace}custom_csv_quoted_string${whitespace}\"\n${whitespace}custom_csv_field2_with_structure${whitespace},321,0.5,21.321,\"${whitespace}custom_csv_quoted_${whitespace}_string2${whitespace}\"" | $CLICKHOUSE_LOCAL -S "c1 String, c2 Int32, c3 Float, c4 Double, c5 String" --input_format_csv_trim_whitespaces=false --input-format="CustomSeparated" --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" +done From 4eb944fef11dacc95873f9e5de9949c381d4dc79 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 25 May 2023 08:05:11 +0000 Subject: [PATCH 095/308] minor changes in documentation --- docs/en/operations/settings/settings-formats.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index cb7d98a4876..1db1b5066c3 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -893,7 +893,7 @@ Default value: `true`. Query ```bash -echo ' string ' |./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true +echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true ``` Result diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e9b7091c8b8..4c2117b2b87 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1706,7 +1706,7 @@ SELECT * FROM table_with_enum_column_for_csv_insert; Запрос ```bash -echo ' string ' |./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true +echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true ``` Результат From fd49821e98c9324e0a6db56dbbec55b52551225b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 25 May 2023 12:46:53 +0200 Subject: [PATCH 096/308] Fix tests --- tests/integration/test_storage_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index d9ac70f51ad..f983bd618e3 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1636,7 +1636,7 @@ def test_ast_auth_headers(started_cluster): filename = "test.csv" result = instance.query_and_get_error( - f"select count() from s3('http://resolver:8080/{bucket}/{filename}', 'CSV')" + f"select count() from s3('http://resolver:8080/{bucket}/{filename}', 'CSV', 'dummy String')" ) assert "HTTP response code: 403" in result From 58610f11ab90a017a1275e1a9a0b843d17e948d8 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 May 2023 10:54:02 +0000 Subject: [PATCH 097/308] Fix tests --- tests/queries/0_stateless/01256_negative_generate_random.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/01256_negative_generate_random.sql b/tests/queries/0_stateless/01256_negative_generate_random.sql index 14f1d947108..7e05a394b8d 100644 --- a/tests/queries/0_stateless/01256_negative_generate_random.sql +++ b/tests/queries/0_stateless/01256_negative_generate_random.sql @@ -1,5 +1,4 @@ SELECT * FROM generateRandom('i8', 1, 10, 10); -- { serverError 62 } SELECT * FROM generateRandom; -- { serverError 60 } -SELECT * FROM generateRandom(); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } SELECT * FROM generateRandom('i8 UInt8', 1, 10, 10, 10, 10); -- { serverError 42 } SELECT * FROM generateRandom('', 1, 10, 10); -- { serverError 62 } From 20287055be4dada6d5eef2bc21004911f2649628 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 25 May 2023 13:43:05 +0000 Subject: [PATCH 098/308] Add toLastDayOfWeek function to 02403_enable_extended_results_for_datetime_functions group test --- ...able_extended_results_for_datetime_functions.reference | 8 ++++++++ ..._enable_extended_results_for_datetime_functions.sql.j2 | 2 +- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference b/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference index 025191c234a..c830d790000 100644 --- a/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference +++ b/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference @@ -22,6 +22,10 @@ toMonday;toDate32;true 1920-02-02 type;toMonday;toDate32;true Date32 toMonday;toDateTime64;true 1920-02-02 type;toMonday;toDateTime64;true Date32 +toLastDayOfWeek;toDate32;true 1920-02-07 +type;toLastDayOfWeek;toDate32;true Date32 +toLastDayOfWeek;toDateTime64;true 1920-02-07 +type;toLastDayOfWeek;toDateTime64;true Date32 toLastDayOfMonth;toDate32;true 1920-02-29 type;toLastDayOfMonth;toDate32;true Date32 toLastDayOfMonth;toDateTime64;true 1920-02-29 @@ -66,6 +70,10 @@ toMonday;toDate32;false 2099-07-08 type;toMonday;toDate32;false Date toMonday;toDateTime64;false 2099-07-08 type;toMonday;toDateTime64;false Date +toLastDayOfWeek;toDate32;false 2099-07-13 +type;toLastDayOfWeek;toDate32;false Date +toLastDayOfWeek;toDateTime64;false 2099-07-13 +type;toLastDayOfWeek;toDateTime64;false Date toLastDayOfMonth;toDate32;false 2099-08-04 type;toLastDayOfMonth;toDate32;false Date toLastDayOfMonth;toDateTime64;false 2099-08-04 diff --git a/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.sql.j2 b/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.sql.j2 index 00a1a0030aa..27e950e4fce 100644 --- a/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.sql.j2 +++ b/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.sql.j2 @@ -1,5 +1,5 @@ {% for option_value in ['true', 'false'] -%} -{% for date_fun in ['toStartOfYear', 'toStartOfISOYear', 'toStartOfQuarter', 'toStartOfMonth', 'toStartOfWeek', 'toMonday', 'toLastDayOfMonth'] -%} +{% for date_fun in ['toStartOfYear', 'toStartOfISOYear', 'toStartOfQuarter', 'toStartOfMonth', 'toStartOfWeek', 'toMonday', 'toLastDayOfWeek', 'toLastDayOfMonth'] -%} SELECT '{{ date_fun }};toDate32;{{ option_value }}', {{ date_fun }}(toDate32('1920-02-02')) SETTINGS enable_extended_results_for_datetime_functions = {{ option_value }}; SELECT 'type;{{ date_fun }};toDate32;{{ option_value }}', toTypeName({{ date_fun }}(toDate32('1920-02-02'))) SETTINGS enable_extended_results_for_datetime_functions = {{ option_value }}; SELECT '{{ date_fun }};toDateTime64;{{ option_value }}', {{ date_fun }}(toDateTime64('1920-02-02 10:20:30', 3)) SETTINGS enable_extended_results_for_datetime_functions = {{ option_value }}; From 0580859e6fa70102d3cde058040c4722d51170fc Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 May 2023 14:05:44 +0000 Subject: [PATCH 099/308] Better --- src/Parsers/TokenIterator.cpp | 4 ++-- src/Parsers/TokenIterator.h | 2 +- src/Parsers/parseQuery.cpp | 4 ++-- src/Parsers/parseQuery.h | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Parsers/TokenIterator.cpp b/src/Parsers/TokenIterator.cpp index 6b798f6f576..fa792e7c8b5 100644 --- a/src/Parsers/TokenIterator.cpp +++ b/src/Parsers/TokenIterator.cpp @@ -4,7 +4,7 @@ namespace DB { -Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool skipp_insignificant) +Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool skip_insignificant) { Lexer lexer(begin, end, max_query_size); @@ -13,7 +13,7 @@ Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool { Token token = lexer.nextToken(); stop = token.isEnd() || token.type == TokenType::ErrorMaxQuerySizeExceeded; - if (token.isSignificant() || (!skipp_insignificant && !data.empty() && data.back().isSignificant())) + if (token.isSignificant() || (!skip_insignificant && !data.empty() && data.back().isSignificant())) data.emplace_back(std::move(token)); } while (!stop); } diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index 31cb644d879..192f2f55e6a 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -24,7 +24,7 @@ private: std::size_t last_accessed_index = 0; public: - Tokens(const char * begin, const char * end, size_t max_query_size = 0, bool skipp_insignificant = true); + Tokens(const char * begin, const char * end, size_t max_query_size = 0, bool skip_insignificant = true); ALWAYS_INLINE inline const Token & operator[](size_t index) { diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 9f688f204a2..dd9a6023b0b 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -234,10 +234,10 @@ ASTPtr tryParseQuery( bool allow_multi_statements, size_t max_query_size, size_t max_parser_depth, - bool skipp_insignificant) + bool skip_insignificant) { const char * query_begin = _out_query_end; - Tokens tokens(query_begin, all_queries_end, max_query_size, skipp_insignificant); + Tokens tokens(query_begin, all_queries_end, max_query_size, skip_insignificant); /// NOTE: consider use UInt32 for max_parser_depth setting. IParser::Pos token_iterator(tokens, static_cast(max_parser_depth)); diff --git a/src/Parsers/parseQuery.h b/src/Parsers/parseQuery.h index 30f43261103..a087f145d2c 100644 --- a/src/Parsers/parseQuery.h +++ b/src/Parsers/parseQuery.h @@ -19,7 +19,7 @@ ASTPtr tryParseQuery( size_t max_query_size, /// If (end - pos) > max_query_size and query is longer than max_query_size then throws "Max query size exceeded". /// Disabled if zero. Is used in order to check query size if buffer can contains data for INSERT query. size_t max_parser_depth, - bool skipp_insignificant = true); /// If true, lexer will skip all insignificant tokens (e.g. whitespaces) + bool skip_insignificant = true); /// If true, lexer will skip all insignificant tokens (e.g. whitespaces) /// Parse query or throw an exception with error message. From ad4a21034f79ba296153ea153a34d8829e7d1f6b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 17:41:38 +0000 Subject: [PATCH 100/308] Fix msan issue in keyed siphash Issue: https://s3.amazonaws.com/clickhouse-test-reports/0/ffdd91669471f4934704f98f0191524496b4e85b/fuzzer_astfuzzermsan/report.html Repro: SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1) Minimal repro: SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2) --- docs/en/sql-reference/functions/hash-functions.md | 8 ++++---- src/Functions/FunctionsHashing.h | 10 ++++++++-- .../queries/0_stateless/02534_keyed_siphash.reference | 2 ++ tests/queries/0_stateless/02534_keyed_siphash.sql | 4 ++++ 4 files changed, 18 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 635c8f4e0ec..8dfa03ceaf2 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -51,7 +51,7 @@ Calculates the MD5 from a string and returns the resulting set of bytes as Fixed If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead. If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))). -## sipHash64 (#hash_functions-siphash64) +## sipHash64 {#hash_functions-siphash64} Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value. @@ -63,9 +63,9 @@ This is a cryptographic hash function. It works at least three times faster than The function [interprets](/docs/en/sql-reference/functions/type-conversion-functions.md/#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. It then combines the hashes by the following algorithm: -1. The first and the second hash value are concatenated to an array which is hashed. -2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. -3. This calculation is repeated for all remaining hash values of the original input. +1. The first and the second hash value are concatenated to an array which is hashed. +2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. +3. This calculation is repeated for all remaining hash values of the original input. **Arguments** diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 3de757bfa3f..b8122ebac85 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -91,12 +91,18 @@ namespace impl throw Exception(ErrorCodes::NOT_IMPLEMENTED, "wrong tuple size: key must be a tuple of 2 UInt64"); if (const auto * key0col = checkAndGetColumn(&(tuple->getColumn(0)))) - ret.key0 = key0col->get64(0); + { + const auto & key0col_data = key0col->getData(); + ret.key0 = key0col_data[0]; + } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); if (const auto * key1col = checkAndGetColumn(&(tuple->getColumn(1)))) - ret.key1 = key1col->get64(0); + { + const auto & key1col_data = key1col->getData(); + ret.key1 = key1col_data[0]; + } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 3606b9a41db..d4ab4a4e88f 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -194,3 +194,5 @@ E28DBDE7FE22E41C 1 E28DBDE7FE22E41C 1 +1CE422FEE7BD8DE20000000000000000 +7766709361750702608 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 9c914f586f0..09489be39c2 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -272,3 +272,7 @@ select hex(sipHash64()); SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; select hex(sipHash64Keyed()); SELECT hex(sipHash128Keyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; + +-- Crashed with memory sanitizer +SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); +SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); From 42e1e3ae208ed6488b7f30e6e87c88be38b07b17 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 May 2023 20:24:03 +0000 Subject: [PATCH 101/308] Fix working with secure socket after async connection --- src/Client/Connection.cpp | 5 ----- src/IO/ReadBufferFromPocoSocket.cpp | 28 +++++++++++++++++++++------- src/IO/ReadBufferFromPocoSocket.h | 2 ++ src/IO/WriteBufferFromPocoSocket.cpp | 27 +++++++++++++++++++++------ src/IO/WriteBufferFromPocoSocket.h | 2 ++ 5 files changed, 46 insertions(+), 18 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index d39148d3016..451d29d4091 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -138,11 +138,6 @@ void Connection::connect(const ConnectionTimeouts & timeouts) socket->impl()->error(err); // Throws an exception socket->setBlocking(true); - -#if USE_SSL - if (static_cast(secure)) - static_cast(socket.get())->completeHandshake(); -#endif } else { diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index d0fba2c28e8..4ceba347707 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace ProfileEvents { @@ -49,16 +50,18 @@ bool ReadBufferFromPocoSocket::nextImpl() { CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive); - /// If async_callback is specified, and read will block, run async_callback and try again later. - /// It is expected that file descriptor may be polled externally. - /// Note that receive timeout is not checked here. External code should check it while polling. - while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR)) - async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); - if (internal_buffer.size() > INT_MAX) throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + bytes_read = readFromSocket(); + + /// In case of non-blocking connect for secure socket receiveBytes can return ERR_SSL_WANT_READ, + /// in this case we should call receiveBytes again when socket is ready. + if (socket.secure()) + { + while (bytes_read == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ) + bytes_read = readFromSocket(); + } } catch (const Poco::Net::NetException & e) { @@ -86,6 +89,17 @@ bool ReadBufferFromPocoSocket::nextImpl() return true; } +ssize_t ReadBufferFromPocoSocket::readFromSocket() +{ + /// If async_callback is specified, and read will block, run async_callback and try again later. + /// It is expected that file descriptor may be polled externally. + /// Note that receive timeout is not checked here. External code should check it while polling. + while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR)) + async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); + + return socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); +} + ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) : BufferWithOwnMemory(buf_size) , socket(socket_) diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index dab4ac86295..3c4bc424334 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -30,6 +30,8 @@ public: void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); } private: + ssize_t readFromSocket(); + AsyncCallback async_callback; std::string socket_description; }; diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 039110dfb62..27fc78e5fe3 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace ProfileEvents @@ -62,13 +63,15 @@ void WriteBufferFromPocoSocket::nextImpl() if (size > INT_MAX) throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - /// If async_callback is specified, and write will block, run async_callback and try again later. - /// It is expected that file descriptor may be polled externally. - /// Note that send timeout is not checked here. External code should check it while polling. - while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_WRITE | Poco::Net::Socket::SELECT_ERROR)) - async_callback(socket.impl()->sockfd(), socket.getSendTimeout(), AsyncEventTimeoutType::SEND, socket_description, AsyncTaskExecutor::Event::WRITE | AsyncTaskExecutor::Event::ERROR); + res = writeToSocket(pos, size); - res = socket.impl()->sendBytes(pos, static_cast(size)); + /// In case of non-blocking connect for secure socket sendBytes can return ERR_SSL_WANT_READ, + /// in this case we should call sendBytes again when socket is ready. + if (socket.secure()) + { + while (res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE) + res = writeToSocket(pos, size); + } } catch (const Poco::Net::NetException & e) { @@ -95,6 +98,18 @@ void WriteBufferFromPocoSocket::nextImpl() } } +ssize_t WriteBufferFromPocoSocket::writeToSocket(char * data, size_t size) +{ + /// If async_callback is specified, and write will block, run async_callback and try again later. + /// It is expected that file descriptor may be polled externally. + /// Note that send timeout is not checked here. External code should check it while polling. + while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_WRITE | Poco::Net::Socket::SELECT_ERROR)) + async_callback(socket.impl()->sockfd(), socket.getSendTimeout(), AsyncEventTimeoutType::SEND, socket_description, AsyncTaskExecutor::Event::WRITE | AsyncTaskExecutor::Event::ERROR); + + return socket.impl()->sendBytes(data, static_cast(size)); + +} + WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) : BufferWithOwnMemory(buf_size) , socket(socket_) diff --git a/src/IO/WriteBufferFromPocoSocket.h b/src/IO/WriteBufferFromPocoSocket.h index ecb61020357..0f03e816af5 100644 --- a/src/IO/WriteBufferFromPocoSocket.h +++ b/src/IO/WriteBufferFromPocoSocket.h @@ -35,6 +35,8 @@ protected: Poco::Net::SocketAddress our_address; private: + ssize_t writeToSocket(char * data, size_t size); + AsyncCallback async_callback; std::string socket_description; }; From 1964d1bb7e55c5827837aa4ac083da3c2c8f39db Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 25 May 2023 22:30:16 +0200 Subject: [PATCH 102/308] Fix comment --- src/IO/WriteBufferFromPocoSocket.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 27fc78e5fe3..ed58e37ddee 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -65,7 +65,7 @@ void WriteBufferFromPocoSocket::nextImpl() res = writeToSocket(pos, size); - /// In case of non-blocking connect for secure socket sendBytes can return ERR_SSL_WANT_READ, + /// In case of non-blocking connect for secure socket sendBytes can return ERR_SSL_WANT_WRITE, /// in this case we should call sendBytes again when socket is ready. if (socket.secure()) { From 613568423d7b34a80d9c5a1688865122f1136a07 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuk Date: Fri, 26 May 2023 07:49:45 +1000 Subject: [PATCH 103/308] Update src/Processors/Formats/Impl/CSVRowInputFormat.cpp Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 9922bd41442..6593567a581 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -282,15 +282,7 @@ bool CSVFormatReader::readField( bool is_last_file_column, const String & /*column_name*/) { - if (format_settings.csv.trim_whitespaces) [[likely]] - skipWhitespacesAndTabs(*buf); - else if (type->isNullable()) - { - auto nested_type = typeid_cast(type.get())->getNestedType(); - if (!isStringOrFixedString(nested_type)) - skipWhitespacesAndTabs(*buf); - } - else if (!isStringOrFixedString(type)) + if (format_settings.csv.trim_whitespaces || !isStringOrFixedString(RemoveNullable(type))) [[likely]] skipWhitespacesAndTabs(*buf); const bool at_delimiter = !buf->eof() && *buf->position() == format_settings.csv.delimiter; From a9082b24b40a9b09ce057836b4b5d55bf7bb491b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 May 2023 00:20:20 +0200 Subject: [PATCH 104/308] Fix build --- src/IO/ReadBufferFromPocoSocket.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 4ceba347707..11acd3a9136 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -9,7 +9,10 @@ #include #include #include + +#if USE_SSL #include +#endif namespace ProfileEvents { From 67b78829fcc74407418db922f1ef58cfe8e1b6ad Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 May 2023 00:21:14 +0200 Subject: [PATCH 105/308] Fix build --- src/IO/ReadBufferFromPocoSocket.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 11acd3a9136..c051478afc5 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -58,6 +58,7 @@ bool ReadBufferFromPocoSocket::nextImpl() bytes_read = readFromSocket(); +#if USE_SSL /// In case of non-blocking connect for secure socket receiveBytes can return ERR_SSL_WANT_READ, /// in this case we should call receiveBytes again when socket is ready. if (socket.secure()) @@ -65,6 +66,7 @@ bool ReadBufferFromPocoSocket::nextImpl() while (bytes_read == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ) bytes_read = readFromSocket(); } +#endif } catch (const Poco::Net::NetException & e) { From f03ca41b081630061f57d3c3b1b1938c4ddcb759 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 May 2023 00:21:46 +0200 Subject: [PATCH 106/308] Fix build --- src/IO/WriteBufferFromPocoSocket.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index ed58e37ddee..e566c228534 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -10,8 +10,10 @@ #include #include #include -#include +#if USE_SSL +#include +#endif namespace ProfileEvents { @@ -65,6 +67,7 @@ void WriteBufferFromPocoSocket::nextImpl() res = writeToSocket(pos, size); +#if USE_SSL /// In case of non-blocking connect for secure socket sendBytes can return ERR_SSL_WANT_WRITE, /// in this case we should call sendBytes again when socket is ready. if (socket.secure()) @@ -72,6 +75,7 @@ void WriteBufferFromPocoSocket::nextImpl() while (res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE) res = writeToSocket(pos, size); } +#endif } catch (const Poco::Net::NetException & e) { From 01f3a46cf06d168452a8cbbca76b1c083355d63b Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 25 May 2023 22:49:36 +0000 Subject: [PATCH 107/308] fixed wrong case in removeNullable --- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 6593567a581..8b4dbbffe1d 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -282,7 +282,7 @@ bool CSVFormatReader::readField( bool is_last_file_column, const String & /*column_name*/) { - if (format_settings.csv.trim_whitespaces || !isStringOrFixedString(RemoveNullable(type))) [[likely]] + if (format_settings.csv.trim_whitespaces || !isStringOrFixedString(removeNullable(type))) [[likely]] skipWhitespacesAndTabs(*buf); const bool at_delimiter = !buf->eof() && *buf->position() == format_settings.csv.delimiter; From 03652efe58469351f87e87c0ca47a6789776710f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 May 2023 12:44:28 +0200 Subject: [PATCH 108/308] Add missing include --- src/IO/ReadBufferFromPocoSocket.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index c051478afc5..d6790439683 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -10,6 +10,8 @@ #include #include +#include "config.h" + #if USE_SSL #include #endif From 67c8c5c561668f0fca8d5ab2545d2ee82178fbdb Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 May 2023 12:44:43 +0200 Subject: [PATCH 109/308] Add missing include --- src/IO/WriteBufferFromPocoSocket.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index e566c228534..6e7c67cc054 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -11,6 +11,8 @@ #include #include +#include "config.h" + #if USE_SSL #include #endif From 050b93363b36e9dbb47e140f00ffc10c2c5b18ff Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 26 May 2023 12:25:11 +0000 Subject: [PATCH 110/308] Add unit tests --- src/Common/tests/gtest_DateLUTImpl.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index 6ae934b2296..04f63403ec2 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -148,6 +148,8 @@ TEST(DateLUTTest, TimeValuesInMiddleOfRange) EXPECT_EQ(lut.addYears(time, 10), 1884270011 /*time_t*/); EXPECT_EQ(lut.timeToString(time), "2019-09-16 19:20:11" /*std::string*/); EXPECT_EQ(lut.dateToString(time), "2019-09-16" /*std::string*/); + EXPECT_EQ(lut.toLastDayOfWeek(time), 1569099600 /*time_t*/); + EXPECT_EQ(lut.toLastDayNumOfWeek(time), DayNum(18161) /*DayNum*/); EXPECT_EQ(lut.toLastDayOfMonth(time), 1569790800 /*time_t*/); EXPECT_EQ(lut.toLastDayNumOfMonth(time), DayNum(18169) /*DayNum*/); } @@ -211,6 +213,8 @@ TEST(DateLUTTest, TimeValuesAtLeftBoderOfRange) EXPECT_EQ(lut.addYears(time, 10), 315532800 /*time_t*/); EXPECT_EQ(lut.timeToString(time), "1970-01-01 00:00:00" /*std::string*/); EXPECT_EQ(lut.dateToString(time), "1970-01-01" /*std::string*/); + EXPECT_EQ(lut.toLastDayOfWeek(time), 259200 /*time_t*/); + EXPECT_EQ(lut.toLastDayNumOfWeek(time), DayNum(3) /*DayNum*/); EXPECT_EQ(lut.toLastDayOfMonth(time), 2592000 /*time_t*/); EXPECT_EQ(lut.toLastDayNumOfMonth(time), DayNum(30) /*DayNum*/); } @@ -276,6 +280,8 @@ TEST(DateLUTTest, TimeValuesAtRightBoderOfRangeOfOldLUT) EXPECT_EQ(lut.timeToString(time), "2106-01-31 01:17:53" /*std::string*/); EXPECT_EQ(lut.dateToString(time), "2106-01-31" /*std::string*/); + EXPECT_EQ(lut.toLastDayOfWeek(time), 4294339200 /*time_t*/); + EXPECT_EQ(lut.toLastDayNumOfWeek(time), DayNum(49703) /*DayNum*/); EXPECT_EQ(lut.toLastDayOfMonth(time), 4294339200 /*time_t*/); // 2106-01-01 EXPECT_EQ(lut.toLastDayNumOfMonth(time), DayNum(49703)); } From ef9bae50b9cc83a885a6e0f8c0d82a88ee2c791b Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 26 May 2023 23:11:57 +0000 Subject: [PATCH 111/308] Fix bugs in Poco, use true non-blocking IO --- base/poco/Net/src/SocketImpl.cpp | 10 +++- .../include/Poco/Net/SecureSocketImpl.h | 10 ++++ .../include/Poco/Net/SecureStreamSocketImpl.h | 10 ++++ .../NetSSL_OpenSSL/src/SecureSocketImpl.cpp | 10 ++++ .../src/SecureStreamSocketImpl.cpp | 10 ++++ src/Common/checkSSLError.h | 8 +++ src/Common/checkSSLReturnCode.cpp | 29 ++++++++++ src/Common/checkSSLReturnCode.h | 12 ++++ src/IO/ReadBufferFromPocoSocket.cpp | 58 +++++++++---------- src/IO/ReadBufferFromPocoSocket.h | 2 - src/IO/WriteBufferFromPocoSocket.cpp | 58 +++++++++---------- src/IO/WriteBufferFromPocoSocket.h | 2 - 12 files changed, 154 insertions(+), 65 deletions(-) create mode 100644 src/Common/checkSSLError.h create mode 100644 src/Common/checkSSLReturnCode.cpp create mode 100644 src/Common/checkSSLReturnCode.h diff --git a/base/poco/Net/src/SocketImpl.cpp b/base/poco/Net/src/SocketImpl.cpp index 2aba413b322..484b8cfeec3 100644 --- a/base/poco/Net/src/SocketImpl.cpp +++ b/base/poco/Net/src/SocketImpl.cpp @@ -274,7 +274,9 @@ void SocketImpl::shutdown() int SocketImpl::sendBytes(const void* buffer, int length, int flags) { - if (_isBrokenTimeout) + bool blocking = _blocking && (flags & MSG_DONTWAIT) == 0; + + if (_isBrokenTimeout && blocking) { if (_sndTimeout.totalMicroseconds() != 0) { @@ -289,11 +291,13 @@ int SocketImpl::sendBytes(const void* buffer, int length, int flags) if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException(); rc = ::send(_sockfd, reinterpret_cast(buffer), length, flags); } - while (_blocking && rc < 0 && lastError() == POCO_EINTR); + while (blocking && rc < 0 && lastError() == POCO_EINTR); if (rc < 0) { int err = lastError(); - if (err == POCO_EAGAIN || err == POCO_ETIMEDOUT) + if ((err == POCO_EAGAIN || err == POCO_EWOULDBLOCK) && !blocking) + ; + else if (err == POCO_EAGAIN || err == POCO_ETIMEDOUT) throw TimeoutException(); else error(err); diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureSocketImpl.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureSocketImpl.h index 56c550decfe..49c12b6b45f 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureSocketImpl.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureSocketImpl.h @@ -183,6 +183,16 @@ namespace Net /// Returns true iff a reused session was negotiated during /// the handshake. + virtual void setBlocking(bool flag); + /// Sets the socket in blocking mode if flag is true, + /// disables blocking mode if flag is false. + + virtual bool getBlocking() const; + /// Returns the blocking mode of the socket. + /// This method will only work if the blocking modes of + /// the socket are changed via the setBlocking method! + + protected: void acceptSSL(); /// Assume per-object mutex is locked. diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureStreamSocketImpl.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureStreamSocketImpl.h index b41043769fe..99e2130d673 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureStreamSocketImpl.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureStreamSocketImpl.h @@ -201,6 +201,16 @@ namespace Net /// Returns true iff a reused session was negotiated during /// the handshake. + virtual void setBlocking(bool flag); + /// Sets the socket in blocking mode if flag is true, + /// disables blocking mode if flag is false. + + virtual bool getBlocking() const; + /// Returns the blocking mode of the socket. + /// This method will only work if the blocking modes of + /// the socket are changed via the setBlocking method! + + protected: void acceptSSL(); /// Performs a SSL server-side handshake. diff --git a/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp b/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp index 9631c7a401a..efe25f65909 100644 --- a/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp @@ -629,5 +629,15 @@ bool SecureSocketImpl::sessionWasReused() return false; } +void SecureSocketImpl::setBlocking(bool flag) +{ + _pSocket->setBlocking(flag); +} + +bool SecureSocketImpl::getBlocking() const +{ + return _pSocket->getBlocking(); +} + } } // namespace Poco::Net diff --git a/base/poco/NetSSL_OpenSSL/src/SecureStreamSocketImpl.cpp b/base/poco/NetSSL_OpenSSL/src/SecureStreamSocketImpl.cpp index aa1a96e1585..c00dd43b2ed 100644 --- a/base/poco/NetSSL_OpenSSL/src/SecureStreamSocketImpl.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SecureStreamSocketImpl.cpp @@ -237,5 +237,15 @@ int SecureStreamSocketImpl::completeHandshake() return _impl.completeHandshake(); } +bool SecureStreamSocketImpl::getBlocking() const +{ + return _impl.getBlocking(); +} + +void SecureStreamSocketImpl::setBlocking(bool flag) +{ + _impl.setBlocking(flag); +} + } } // namespace Poco::Net diff --git a/src/Common/checkSSLError.h b/src/Common/checkSSLError.h new file mode 100644 index 00000000000..05bca9f8b5f --- /dev/null +++ b/src/Common/checkSSLError.h @@ -0,0 +1,8 @@ +// +// Created by Павел Круглов on 27/05/2023. +// + +#ifndef CLICKHOUSE_CHECKSSLERROR_H +#define CLICKHOUSE_CHECKSSLERROR_H + +#endif //CLICKHOUSE_CHECKSSLERROR_H diff --git a/src/Common/checkSSLReturnCode.cpp b/src/Common/checkSSLReturnCode.cpp new file mode 100644 index 00000000000..8916a25e19c --- /dev/null +++ b/src/Common/checkSSLReturnCode.cpp @@ -0,0 +1,29 @@ +#include +#include "config.h" + +#if USE_SSL +#include +#endif + +namespace DB +{ + +bool checkSSLWantRead(ssize_t res) +{ +#if USE_SSL + return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ; +#else + return false; +#endif +} + +bool checkSSLWantWrite(ssize_t res) +{ +#if USE_SSL + return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE; +#else + return false; +#endif +} + +} diff --git a/src/Common/checkSSLReturnCode.h b/src/Common/checkSSLReturnCode.h new file mode 100644 index 00000000000..f30564137aa --- /dev/null +++ b/src/Common/checkSSLReturnCode.h @@ -0,0 +1,12 @@ +#pragma once + +namespace DB +{ + +/// Check if ret is ERR_SSL_WANT_READ. +bool checkSSLWantRead(ssize_t ret); + +/// CHeck if ret is ERR_SSL_WANT_WRITE. +bool checkSSLWantWrite(ssize_t ret); + +} diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index d6790439683..ff72dc5386c 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -9,12 +9,7 @@ #include #include #include - -#include "config.h" - -#if USE_SSL -#include -#endif +#include namespace ProfileEvents { @@ -27,7 +22,6 @@ namespace CurrentMetrics extern const Metric NetworkReceive; } - namespace DB { namespace ErrorCodes @@ -38,14 +32,13 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - bool ReadBufferFromPocoSocket::nextImpl() { ssize_t bytes_read = 0; Stopwatch watch; SCOPE_EXIT({ - // / NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one + /// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); }); @@ -58,17 +51,35 @@ bool ReadBufferFromPocoSocket::nextImpl() if (internal_buffer.size() > INT_MAX) throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - bytes_read = readFromSocket(); - -#if USE_SSL - /// In case of non-blocking connect for secure socket receiveBytes can return ERR_SSL_WANT_READ, - /// in this case we should call receiveBytes again when socket is ready. - if (socket.secure()) + /// If async_callback is specified, set socket to non-blocking mode + /// and try to read data from it, if socket is not ready for reading, + /// run async_callback and try again later. + /// It is expected that file descriptor may be polled externally. + /// Note that send timeout is not checked here. External code should check it while polling. + if (async_callback) { - while (bytes_read == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ) - bytes_read = readFromSocket(); + socket.setBlocking(false); + SCOPE_EXIT(socket.setBlocking(true)); + bool secure = socket.secure(); + bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + + /// Check EAGAIN and ERR_SSL_WANT_READ/ERR_SSL_WANT_WRITE for secure socket (reading from secure socket can write too). + while (bytes_read < 0 && (errno == EAGAIN || (secure && (checkSSLWantRead(bytes_read) || checkSSLWantWrite(bytes_read))))) + { + /// In case of ERR_SSL_WANT_WRITE we should wait for socket to be ready for writing, otherwise - for reading. + if (secure && checkSSLWantWrite(bytes_read)) + async_callback(socket.impl()->sockfd(), socket.getSendTimeout(), AsyncEventTimeoutType::SEND, socket_description, AsyncTaskExecutor::Event::WRITE | AsyncTaskExecutor::Event::ERROR); + else + async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); + + /// Try to read again. + bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + } + } + else + { + bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); } -#endif } catch (const Poco::Net::NetException & e) { @@ -96,17 +107,6 @@ bool ReadBufferFromPocoSocket::nextImpl() return true; } -ssize_t ReadBufferFromPocoSocket::readFromSocket() -{ - /// If async_callback is specified, and read will block, run async_callback and try again later. - /// It is expected that file descriptor may be polled externally. - /// Note that receive timeout is not checked here. External code should check it while polling. - while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR)) - async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); - - return socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); -} - ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) : BufferWithOwnMemory(buf_size) , socket(socket_) diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index 3c4bc424334..dab4ac86295 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -30,8 +30,6 @@ public: void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); } private: - ssize_t readFromSocket(); - AsyncCallback async_callback; std::string socket_description; }; diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 6e7c67cc054..df1041f0056 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -10,12 +10,7 @@ #include #include #include - -#include "config.h" - -#if USE_SSL -#include -#endif +#include namespace ProfileEvents { @@ -28,7 +23,6 @@ namespace CurrentMetrics extern const Metric NetworkSend; } - namespace DB { @@ -40,7 +34,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - void WriteBufferFromPocoSocket::nextImpl() { if (!offset()) @@ -67,17 +60,36 @@ void WriteBufferFromPocoSocket::nextImpl() if (size > INT_MAX) throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - res = writeToSocket(pos, size); - -#if USE_SSL - /// In case of non-blocking connect for secure socket sendBytes can return ERR_SSL_WANT_WRITE, - /// in this case we should call sendBytes again when socket is ready. - if (socket.secure()) + /// If async_callback is specified, set socket to non-blocking mode + /// and try to write data to it, if socket is not ready for writing, + /// run async_callback and try again later. + /// It is expected that file descriptor may be polled externally. + /// Note that send timeout is not checked here. External code should check it while polling. + if (async_callback) { - while (res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE) - res = writeToSocket(pos, size); + socket.setBlocking(false); + /// Set socket to blocking mode at the end. + SCOPE_EXIT(socket.setBlocking(true)); + bool secure = socket.secure(); + res = socket.impl()->sendBytes(pos, static_cast(size)); + + /// Check EAGAIN and ERR_SSL_WANT_WRITE/ERR_SSL_WANT_READ for secure socket (writing to secure socket can read too). + while (res < 0 && (errno == EAGAIN || (secure && (checkSSLWantRead(res) || checkSSLWantWrite(res))))) + { + /// In case of ERR_SSL_WANT_READ we should wait for socket to be ready for reading, otherwise - for writing. + if (secure && checkSSLWantRead(res)) + async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); + else + async_callback(socket.impl()->sockfd(), socket.getSendTimeout(), AsyncEventTimeoutType::SEND, socket_description, AsyncTaskExecutor::Event::WRITE | AsyncTaskExecutor::Event::ERROR); + + /// Try to write again. + res = socket.impl()->sendBytes(pos, static_cast(size)); + } + } + else + { + res = socket.impl()->sendBytes(pos, static_cast(size)); } -#endif } catch (const Poco::Net::NetException & e) { @@ -104,18 +116,6 @@ void WriteBufferFromPocoSocket::nextImpl() } } -ssize_t WriteBufferFromPocoSocket::writeToSocket(char * data, size_t size) -{ - /// If async_callback is specified, and write will block, run async_callback and try again later. - /// It is expected that file descriptor may be polled externally. - /// Note that send timeout is not checked here. External code should check it while polling. - while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_WRITE | Poco::Net::Socket::SELECT_ERROR)) - async_callback(socket.impl()->sockfd(), socket.getSendTimeout(), AsyncEventTimeoutType::SEND, socket_description, AsyncTaskExecutor::Event::WRITE | AsyncTaskExecutor::Event::ERROR); - - return socket.impl()->sendBytes(data, static_cast(size)); - -} - WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) : BufferWithOwnMemory(buf_size) , socket(socket_) diff --git a/src/IO/WriteBufferFromPocoSocket.h b/src/IO/WriteBufferFromPocoSocket.h index 0f03e816af5..ecb61020357 100644 --- a/src/IO/WriteBufferFromPocoSocket.h +++ b/src/IO/WriteBufferFromPocoSocket.h @@ -35,8 +35,6 @@ protected: Poco::Net::SocketAddress our_address; private: - ssize_t writeToSocket(char * data, size_t size); - AsyncCallback async_callback; std::string socket_description; }; From 65c34e73e7010bfb5c6298ed2a5bedcc434e7aa7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sun, 28 May 2023 15:45:40 +0200 Subject: [PATCH 112/308] Clean --- src/Common/checkSSLError.h | 8 -------- 1 file changed, 8 deletions(-) delete mode 100644 src/Common/checkSSLError.h diff --git a/src/Common/checkSSLError.h b/src/Common/checkSSLError.h deleted file mode 100644 index 05bca9f8b5f..00000000000 --- a/src/Common/checkSSLError.h +++ /dev/null @@ -1,8 +0,0 @@ -// -// Created by Павел Круглов on 27/05/2023. -// - -#ifndef CLICKHOUSE_CHECKSSLERROR_H -#define CLICKHOUSE_CHECKSSLERROR_H - -#endif //CLICKHOUSE_CHECKSSLERROR_H From 3aa795355f946e6638804124b03d0724bf1ba37b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sun, 28 May 2023 15:46:32 +0200 Subject: [PATCH 113/308] Fix build --- src/Common/checkSSLReturnCode.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/checkSSLReturnCode.h b/src/Common/checkSSLReturnCode.h index f30564137aa..77e51625daf 100644 --- a/src/Common/checkSSLReturnCode.h +++ b/src/Common/checkSSLReturnCode.h @@ -1,4 +1,5 @@ #pragma once +#include namespace DB { From 8574559e233b9319ffd6e15d317923fb1a84d22f Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Sun, 28 May 2023 22:25:40 +0000 Subject: [PATCH 114/308] Disallow usage of --query and --queries-file --- programs/client/Client.cpp | 3 +++ programs/local/LocalServer.cpp | 6 ++--- ...ent_local_queries_file_parameter.reference | 4 ++++ ...523_client_local_queries_file_parameter.sh | 23 +++++++++++++------ .../02751_multiquery_with_argument.reference | 2 ++ .../02751_multiquery_with_argument.sh | 4 ++++ 6 files changed, 32 insertions(+), 10 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 5870327c3b5..32a07284d26 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1180,6 +1180,9 @@ void Client::processOptions(const OptionsDescription & options_description, void Client::processConfig() { + if (config().has("query") && config().has("queries-file")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Options '--query' and '--queries-file' cannot be specified at the same time"); + /// Batch mode is enabled if one of the following is true: /// - -q (--query) command line option is present. /// The value of the option is used as the text of query (or of multiple queries). diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e026f87279a..944a60d4e4c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -516,12 +516,12 @@ void LocalServer::updateLoggerLevel(const String & logs_level) void LocalServer::processConfig() { + if (config().has("query") && config().has("queries-file")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Options '--query' and '--queries-file' cannot be specified at the same time"); + delayed_interactive = config().has("interactive") && (config().has("query") || config().has("queries-file")); if (is_interactive && !delayed_interactive) { - if (config().has("query") && config().has("queries-file")) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Specify either `query` or `queries-file` option"); - if (config().has("multiquery")) is_multiquery = true; } diff --git a/tests/queries/0_stateless/01523_client_local_queries_file_parameter.reference b/tests/queries/0_stateless/01523_client_local_queries_file_parameter.reference index 2a3eb745751..634bee3d288 100644 --- a/tests/queries/0_stateless/01523_client_local_queries_file_parameter.reference +++ b/tests/queries/0_stateless/01523_client_local_queries_file_parameter.reference @@ -4,3 +4,7 @@ 3 1 2 3 4 +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS diff --git a/tests/queries/0_stateless/01523_client_local_queries_file_parameter.sh b/tests/queries/0_stateless/01523_client_local_queries_file_parameter.sh index cf19efa9b7d..17dc04d7571 100755 --- a/tests/queries/0_stateless/01523_client_local_queries_file_parameter.sh +++ b/tests/queries/0_stateless/01523_client_local_queries_file_parameter.sh @@ -5,20 +5,29 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo "SELECT 1;" > 01523_client_local_queries_file_parameter_tmp.sql -$CLICKHOUSE_CLIENT --queries-file=01523_client_local_queries_file_parameter_tmp.sql 2>&1 +## Use process ID ($$) for uniqueness of file name +TEMP_SQL_FILE_NAME=$"01523_client_local_queries_file_parameter_tmp_$$.sql" +echo "SELECT 1;" > "$TEMP_SQL_FILE_NAME" +$CLICKHOUSE_CLIENT --queries-file="$TEMP_SQL_FILE_NAME" 2>&1 echo "CREATE TABLE 01523_test(value Int32) ENGINE=Log; INSERT INTO 01523_test VALUES (1), (2), (3); SELECT * FROM 01523_test; -DROP TABLE 01523_test;" > 01523_client_local_queries_file_parameter_tmp.sql -$CLICKHOUSE_CLIENT --queries-file=01523_client_local_queries_file_parameter_tmp.sql 2>&1 +DROP TABLE 01523_test;" > "$TEMP_SQL_FILE_NAME" +$CLICKHOUSE_CLIENT --queries-file="$TEMP_SQL_FILE_NAME" 2>&1 echo "CREATE TABLE 01523_test (a Int64, b Int64) ENGINE = File(CSV, stdin); SELECT a, b FROM 01523_test; -DROP TABLE 01523_test;" > 01523_client_local_queries_file_parameter_tmp.sql +DROP TABLE 01523_test;" > "$TEMP_SQL_FILE_NAME" -echo -e "1,2\n3,4" | $CLICKHOUSE_LOCAL --queries-file=01523_client_local_queries_file_parameter_tmp.sql 2>&1 +echo -e "1,2\n3,4" | $CLICKHOUSE_LOCAL --queries-file="$TEMP_SQL_FILE_NAME" 2>&1 -rm 01523_client_local_queries_file_parameter_tmp.sql +# Simultaneously passing --queries-file + --query is prohibited. +echo "SELECT 1;" > "$TEMP_SQL_FILE_NAME" +$CLICKHOUSE_LOCAL --queries-file="$TEMP_SQL_FILE_NAME" -q "SELECT 1;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_CLIENT --queries-file="$TEMP_SQL_FILE_NAME" -q "SELECT 2;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_LOCAL --queries-file="$TEMP_SQL_FILE_NAME" --query "SELECT 3;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_CLIENT --queries-file="$TEMP_SQL_FILE_NAME" --query "SELECT 4;" 2>&1 | grep -o 'BAD_ARGUMENTS' + +rm "$TEMP_SQL_FILE_NAME" diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.reference b/tests/queries/0_stateless/02751_multiquery_with_argument.reference index 33288ec5bcb..df9771b6bd3 100644 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.reference +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.reference @@ -8,6 +8,8 @@ Empty query BAD_ARGUMENTS BAD_ARGUMENTS BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS Bad arguments Bad arguments Bad arguments diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.sh b/tests/queries/0_stateless/02751_multiquery_with_argument.sh index 153004c752e..ce53ede3331 100755 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.sh +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.sh @@ -13,6 +13,10 @@ $CLICKHOUSE_LOCAL --multiquery "SELECT 200; S" 2>&1 | grep -o 'Syntax error' $CLICKHOUSE_LOCAL --multiquery "; SELECT 201;" 2>&1 | grep -o 'Empty query' $CLICKHOUSE_LOCAL --multiquery "; S; SELECT 202" 2>&1 | grep -o 'Empty query' +# Simultaneously passing --queries-file + --query (multiquery) is prohibited. +$CLICKHOUSE_LOCAL --queries-file "queries.csv" --multiquery "SELECT 250;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_CLIENT --queries-file "queries.csv" --multiquery "SELECT 251;" 2>&1 | grep -o 'BAD_ARGUMENTS' + # Error expectation cases. # -n is prohibited $CLICKHOUSE_LOCAL -n "SELECT 301" 2>&1 | grep -o 'BAD_ARGUMENTS' From be62c8faba774c8c594a6d0368efba0c7a540576 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 May 2023 15:27:39 +0200 Subject: [PATCH 115/308] fix build --- src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 05e6dad157b..ba8c2c6385f 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -82,7 +82,7 @@ MergeTreeReadPool::MergeTreeReadPool( const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; const auto avg_mark_bytes = std::max(total_compressed_bytes / total_marks, 1); /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. - const auto heuristic_min_marks = std::min(total_marks / threads_, min_bytes_per_task / avg_mark_bytes); + const auto heuristic_min_marks = std::min(total_marks / threads_, min_bytes_per_task / avg_mark_bytes); if (heuristic_min_marks > min_marks_for_concurrent_read) { min_marks_for_concurrent_read = heuristic_min_marks; From 437880d4c15a5cc9e3a541db58939ee7c507e10c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 May 2023 15:52:50 +0200 Subject: [PATCH 116/308] Enable `enable_memory_bound_merging_of_aggregation_results` by default --- 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 607be1522db..10d94f8d218 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -143,7 +143,7 @@ class IColumn; M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ M(UInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \ - M(Bool, enable_memory_bound_merging_of_aggregation_results, false, "Enable memory bound merging strategy for aggregation. Set it to true only if all nodes of your clusters have versions >= 22.12.", 0) \ + M(Bool, enable_memory_bound_merging_of_aggregation_results, true, "Enable memory bound merging strategy for aggregation.", 0) \ M(Bool, enable_positional_arguments, true, "Enable positional arguments in ORDER BY, GROUP BY and LIMIT BY", 0) \ M(Bool, enable_extended_results_for_datetime_functions, false, "Enable date functions like toLastDayOfMonth return Date32 results (instead of Date results) for Date32/DateTime64 arguments.", 0) \ \ From e10f951467390f5deb67b7fe12e38f546059ec9a Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 29 May 2023 19:09:07 +0000 Subject: [PATCH 117/308] Fix IS (NOT) NULL operator priority --- src/Parsers/ExpressionListParsers.cpp | 66 +++++++++---------- .../02752_is_null_priority.reference | 16 +++++ .../0_stateless/02752_is_null_priority.sql | 1 + 3 files changed, 50 insertions(+), 33 deletions(-) create mode 100644 tests/queries/0_stateless/02752_is_null_priority.reference create mode 100644 tests/queries/0_stateless/02752_is_null_priority.sql diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 1477f3d000d..8903b2b02b0 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2332,44 +2332,44 @@ const std::vector> ParserExpressionImpl::o {":", Operator("if", 3, 3, OperatorType::FinishIf)}, {"OR", Operator("or", 3, 2, OperatorType::Mergeable)}, {"AND", Operator("and", 4, 2, OperatorType::Mergeable)}, - {"BETWEEN", Operator("", 6, 0, OperatorType::StartBetween)}, - {"NOT BETWEEN", Operator("", 6, 0, OperatorType::StartNotBetween)}, - {"==", Operator("equals", 8, 2, OperatorType::Comparison)}, - {"!=", Operator("notEquals", 8, 2, OperatorType::Comparison)}, - {"<>", Operator("notEquals", 8, 2, OperatorType::Comparison)}, - {"<=", Operator("lessOrEquals", 8, 2, OperatorType::Comparison)}, - {">=", Operator("greaterOrEquals", 8, 2, OperatorType::Comparison)}, - {"<", Operator("less", 8, 2, OperatorType::Comparison)}, - {">", Operator("greater", 8, 2, OperatorType::Comparison)}, - {"=", Operator("equals", 8, 2, OperatorType::Comparison)}, - {"LIKE", Operator("like", 8, 2)}, - {"ILIKE", Operator("ilike", 8, 2)}, - {"NOT LIKE", Operator("notLike", 8, 2)}, - {"NOT ILIKE", Operator("notILike", 8, 2)}, - {"REGEXP", Operator("match", 8, 2)}, - {"IN", Operator("in", 8, 2)}, - {"NOT IN", Operator("notIn", 8, 2)}, - {"GLOBAL IN", Operator("globalIn", 8, 2)}, - {"GLOBAL NOT IN", Operator("globalNotIn", 8, 2)}, - {"||", Operator("concat", 9, 2, OperatorType::Mergeable)}, - {"+", Operator("plus", 10, 2)}, - {"-", Operator("minus", 10, 2)}, - {"*", Operator("multiply", 11, 2)}, - {"/", Operator("divide", 11, 2)}, - {"%", Operator("modulo", 11, 2)}, - {"MOD", Operator("modulo", 11, 2)}, - {"DIV", Operator("intDiv", 11, 2)}, - {".", Operator("tupleElement", 13, 2, OperatorType::TupleElement)}, - {"[", Operator("arrayElement", 13, 2, OperatorType::ArrayElement)}, - {"::", Operator("CAST", 13, 2, OperatorType::Cast)}, - {"IS NULL", Operator("isNull", 13, 1, OperatorType::IsNull)}, - {"IS NOT NULL", Operator("isNotNull", 13, 1, OperatorType::IsNull)}, + {"IS NULL", Operator("isNull", 6, 1, OperatorType::IsNull)}, + {"IS NOT NULL", Operator("isNotNull", 6, 1, OperatorType::IsNull)}, + {"BETWEEN", Operator("", 7, 0, OperatorType::StartBetween)}, + {"NOT BETWEEN", Operator("", 7, 0, OperatorType::StartNotBetween)}, + {"==", Operator("equals", 9, 2, OperatorType::Comparison)}, + {"!=", Operator("notEquals", 9, 2, OperatorType::Comparison)}, + {"<>", Operator("notEquals", 9, 2, OperatorType::Comparison)}, + {"<=", Operator("lessOrEquals", 9, 2, OperatorType::Comparison)}, + {">=", Operator("greaterOrEquals", 9, 2, OperatorType::Comparison)}, + {"<", Operator("less", 9, 2, OperatorType::Comparison)}, + {">", Operator("greater", 9, 2, OperatorType::Comparison)}, + {"=", Operator("equals", 9, 2, OperatorType::Comparison)}, + {"LIKE", Operator("like", 9, 2)}, + {"ILIKE", Operator("ilike", 9, 2)}, + {"NOT LIKE", Operator("notLike", 9, 2)}, + {"NOT ILIKE", Operator("notILike", 9, 2)}, + {"REGEXP", Operator("match", 9, 2)}, + {"IN", Operator("in", 9, 2)}, + {"NOT IN", Operator("notIn", 9, 2)}, + {"GLOBAL IN", Operator("globalIn", 9, 2)}, + {"GLOBAL NOT IN", Operator("globalNotIn", 9, 2)}, + {"||", Operator("concat", 10, 2, OperatorType::Mergeable)}, + {"+", Operator("plus", 11, 2)}, + {"-", Operator("minus", 11, 2)}, + {"*", Operator("multiply", 12, 2)}, + {"/", Operator("divide", 12, 2)}, + {"%", Operator("modulo", 12, 2)}, + {"MOD", Operator("modulo", 12, 2)}, + {"DIV", Operator("intDiv", 12, 2)}, + {".", Operator("tupleElement", 14, 2, OperatorType::TupleElement)}, + {"[", Operator("arrayElement", 14, 2, OperatorType::ArrayElement)}, + {"::", Operator("CAST", 14, 2, OperatorType::Cast)}, }; const std::vector> ParserExpressionImpl::unary_operators_table { {"NOT", Operator("not", 5, 1)}, - {"-", Operator("negate", 12, 1)} + {"-", Operator("negate", 13, 1)} }; const Operator ParserExpressionImpl::finish_between_operator("", 7, 0, OperatorType::FinishBetween); diff --git a/tests/queries/0_stateless/02752_is_null_priority.reference b/tests/queries/0_stateless/02752_is_null_priority.reference new file mode 100644 index 00000000000..54606882ddc --- /dev/null +++ b/tests/queries/0_stateless/02752_is_null_priority.reference @@ -0,0 +1,16 @@ +SelectWithUnionQuery (children 1) + ExpressionList (children 1) + SelectQuery (children 1) + ExpressionList (children 2) + Function isNull (children 1) + ExpressionList (children 1) + Function multiply (children 1) + ExpressionList (children 2) + Identifier a + Identifier b + Function isNotNull (children 1) + ExpressionList (children 1) + Function multiply (children 1) + ExpressionList (children 2) + Identifier a + Identifier b diff --git a/tests/queries/0_stateless/02752_is_null_priority.sql b/tests/queries/0_stateless/02752_is_null_priority.sql new file mode 100644 index 00000000000..a0a9741e752 --- /dev/null +++ b/tests/queries/0_stateless/02752_is_null_priority.sql @@ -0,0 +1 @@ +EXPLAIN AST SELECT a * b IS NULL, a * b IS NOT NULL; From 6c94632d47f9c8b123507da49da94b6b0cf727a0 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Mon, 29 May 2023 22:10:34 +0000 Subject: [PATCH 118/308] Deprive toStartOfWeek and toLastDayOfWeek functions of in-source documentation --- src/Functions/toCustomWeek.cpp | 26 ++----------------- ...new_functions_must_be_documented.reference | 2 ++ 2 files changed, 4 insertions(+), 24 deletions(-) diff --git a/src/Functions/toCustomWeek.cpp b/src/Functions/toCustomWeek.cpp index 0dbd60b3eff..98e7aaf1d6b 100644 --- a/src/Functions/toCustomWeek.cpp +++ b/src/Functions/toCustomWeek.cpp @@ -17,30 +17,8 @@ REGISTER_FUNCTION(ToCustomWeek) { factory.registerFunction(); factory.registerFunction(); - - factory.registerFunction(FunctionDocumentation{.description=R"( -Rounds a date or date with time down to the nearest Sunday or Monday. Returns the date. -Syntax: toStartOfWeek(t[, mode[, timezone]]) -The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, mode is assumed as 0. - -Example: -[example:typical] -)", - .examples{ - {"typical", "SELECT toStartOfWeek(today(), 1);", ""}}, - .categories{"Dates and Times"}}, FunctionFactory::CaseSensitive); - - factory.registerFunction(FunctionDocumentation{.description=R"( -Rounds a date or date with time up to the nearest Saturday or Sunday. Returns the date. -Syntax: toLastDayOfWeek(t[, mode[, timezone]]) -The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, mode is assumed as 0. - -Example: -[example:typical] -)", - .examples{ - {"typical", "SELECT toLastDayOfWeek(today(), 1);", ""}}, - .categories{"Dates and Times"}}, FunctionFactory::CaseSensitive); + factory.registerFunction(); + factory.registerFunction(); /// Compatibility aliases for mysql. factory.registerAlias("week", "toWeek", FunctionFactory::CaseInsensitive); diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index eae36604845..bf43288fe1d 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -784,6 +784,7 @@ toIntervalWeek toIntervalYear toJSONString toLastDayOfMonth +toLastDayOfWeek toLowCardinality toMinute toModifiedJulianDay @@ -815,6 +816,7 @@ toStartOfNanosecond toStartOfQuarter toStartOfSecond toStartOfTenMinutes +toStartOfWeek toStartOfYear toString toStringCutToZero From 9a8086a3775c89197fe29e609d8cda9eb32879b5 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Mon, 29 May 2023 22:12:44 +0000 Subject: [PATCH 119/308] Reorder toLastDayOf* functions in settings.md --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7cac9a30d5d..239b8960190 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3788,7 +3788,7 @@ Result: ## enable_extended_results_for_datetime_functions {#enable-extended-results-for-datetime-functions} Enables or disables returning results of type: -- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday), [toLastDayOfWeek](../../sql-reference/functions/date-time-functions.md#tolastdayofweek) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). +- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toLastDayOfWeek](../../sql-reference/functions/date-time-functions.md#tolastdayofweek) and [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday). - `DateTime64` with extended range (compared to type `DateTime`) for functions [toStartOfDay](../../sql-reference/functions/date-time-functions.md#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffifteenminutes) and [timeSlot](../../sql-reference/functions/date-time-functions.md#timeslot). Possible values: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 3b100729e4d..cda338cef75 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3790,7 +3790,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; ## enable_extended_results_for_datetime_functions {#enable-extended-results-for-datetime-functions} Включает или отключает возвращение результатов типа: -- `Date32` с расширенным диапазоном (по сравнению с типом `Date`) для функций [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday), [toLastDayOfWeek](../../sql-reference/functions/date-time-functions.md#tolastdayofweek) и [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). +- `Date32` с расширенным диапазоном (по сравнению с типом `Date`) для функций [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toLastDayOfWeek](../../sql-reference/functions/date-time-functions.md#tolastdayofweek) и [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday). - `DateTime64` с расширенным диапазоном (по сравнению с типом `DateTime`) для функций [toStartOfDay](../../sql-reference/functions/date-time-functions.md#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffifteenminutes) и [timeSlot](../../sql-reference/functions/date-time-functions.md#timeslot). Возможные значения: From ecea1ac090c9ac1749951b409f32e4911a9dc005 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Thu, 25 May 2023 14:25:08 +0200 Subject: [PATCH 120/308] Fix crash when Pool::Entry::disconnect() is called Many Pool::Entry objects can keep the same pointer to Pool::Connection. If Pool::Entry::disconnect() is called on one such object, Pool::removeConnection() is called to remove Pool::Connection from the pool, where connection->ref_count is cleared and connection->removed_from_pool is set. Next Pool::Entry::~Entry() calls decrementRefCount() with 1. const auto ref_count = data->ref_count.fetch_sub(1); where data->ref_count will be negative, since it was cleared 2. checks removed_from_pool and deletes Pool::Connection but there might be multiple Entry objects still keep pointer to this Pool::Connection Suggesting not to clear ref_count on disconnect() and delete Pool::Connection only on the last Pool::Entry is being destroyed. Fixes ea375ef9890f18be7038f66da2d731010ed4462f --- src/Common/mysqlxx/Pool.cpp | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/src/Common/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp index 6438d76cc3a..d10889d1f97 100644 --- a/src/Common/mysqlxx/Pool.cpp +++ b/src/Common/mysqlxx/Pool.cpp @@ -40,14 +40,10 @@ void Pool::Entry::decrementRefCount() { /// We were the last user of this thread, deinitialize it mysql_thread_end(); - } - else if (data->removed_from_pool) - { - /// data->ref_count == 0 in case we removed connection from pool (see Pool::removeConnection). - chassert(ref_count == 0); /// In Pool::Entry::disconnect() we remove connection from the list of pool's connections. /// So now we must deallocate the memory. - ::delete data; + if (data->removed_from_pool) + ::delete data; } } @@ -234,11 +230,8 @@ void Pool::removeConnection(Connection* connection) std::lock_guard lock(mutex); if (connection) { - if (connection->ref_count > 0) - { + if (!connection->removed_from_pool) connection->conn.disconnect(); - connection->ref_count = 0; - } connections.remove(connection); connection->removed_from_pool = true; } From fb6c4f280289215068f6e9c17a748b0c88c885fd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 May 2023 08:46:40 +0000 Subject: [PATCH 121/308] Fix msan issue, pt. II --- src/Functions/FunctionsHashing.h | 17 +++++++++++++---- .../0_stateless/02534_keyed_siphash.reference | 2 ++ .../queries/0_stateless/02534_keyed_siphash.sql | 2 ++ 3 files changed, 17 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index b8122ebac85..1fed80a66b8 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -92,16 +92,25 @@ namespace impl if (const auto * key0col = checkAndGetColumn(&(tuple->getColumn(0)))) { - const auto & key0col_data = key0col->getData(); - ret.key0 = key0col_data[0]; + if (!key0col->empty()) + { + const auto & key0col_data = key0col->getData(); + ret.key0 = key0col_data[0]; + } + else ret.key0 = 0; } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); if (const auto * key1col = checkAndGetColumn(&(tuple->getColumn(1)))) { - const auto & key1col_data = key1col->getData(); - ret.key1 = key1col_data[0]; + if (!key1col->empty()) + { + const auto & key1col_data = key1col->getData(); + ret.key1 = key1col_data[0]; + } + else + ret.key1 = 0; } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index d4ab4a4e88f..a891d01e99d 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -196,3 +196,5 @@ E28DBDE7FE22E41C 1 1CE422FEE7BD8DE20000000000000000 7766709361750702608 +20AF99D3A87829E0 +12489502208762728797 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 09489be39c2..7ce50583cdb 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -276,3 +276,5 @@ SELECT hex(sipHash128Keyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000 -- Crashed with memory sanitizer SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); +SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646)); +SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806); From 2d0694752212a60354986cc7e785f749fe79b230 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 May 2023 09:06:45 +0000 Subject: [PATCH 122/308] Better --- src/Functions/FunctionsHashing.h | 19 +++++-------------- 1 file changed, 5 insertions(+), 14 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 1fed80a66b8..d08a900c88a 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -92,25 +92,16 @@ namespace impl if (const auto * key0col = checkAndGetColumn(&(tuple->getColumn(0)))) { - if (!key0col->empty()) - { - const auto & key0col_data = key0col->getData(); - ret.key0 = key0col_data[0]; - } - else ret.key0 = 0; + const auto & key0col_data = key0col->getData(); + ret.key0 = key0col_data[0]; } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); if (const auto * key1col = checkAndGetColumn(&(tuple->getColumn(1)))) { - if (!key1col->empty()) - { - const auto & key1col_data = key1col->getData(); - ret.key1 = key1col_data[0]; - } - else - ret.key1 = 0; + const auto & key1col_data = key1col->getData(); + ret.key1 = key1col_data[0]; } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); @@ -1450,7 +1441,7 @@ public: KeyType key{}; if constexpr (Keyed) - if (!arguments.empty()) + if (!arguments.empty() && input_rows_count != 0) key = Impl::parseKey(arguments[0]); /// The function supports arbitrary number of arguments of arbitrary types. From 100b4d0969a5682e464d1de7f58685f3f4f635f3 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 26 May 2023 20:07:00 +0200 Subject: [PATCH 123/308] cope with finalize in d-tors --- src/IO/WriteBufferFromS3TaskTracker.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/WriteBufferFromS3TaskTracker.h b/src/IO/WriteBufferFromS3TaskTracker.h index 800e5239cc4..c3f4628b946 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.h +++ b/src/IO/WriteBufferFromS3TaskTracker.h @@ -49,6 +49,8 @@ private: /// waitTilInflightShrink waits til the number of in-flight tasks beyond the limit `max_tasks_inflight`. void waitTilInflightShrink() TSA_NO_THREAD_SAFETY_ANALYSIS; + void collectFinishedFutures(bool propagate_exceptions) TSA_REQUIRES(mutex); + const bool is_async; ThreadPoolCallbackRunner scheduler; const size_t max_tasks_inflight; From 0e019c8e83aead4cd01fa21b0f46164d4fd4fb50 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 26 May 2023 22:18:55 +0200 Subject: [PATCH 124/308] turn off some d-tor finalize --- src/Compression/CompressedWriteBuffer.cpp | 1 + .../WriteBufferFromNuraftBuffer.cpp | 1 + .../IO/CachedOnDiskWriteBufferFromFile.cpp | 1 + .../IO/WriteBufferFromAzureBlobStorage.cpp | 1 + .../IO/WriteBufferWithFinalizeCallback.cpp | 17 +++++++++-------- src/IO/Archives/ZipArchiveWriter.cpp | 7 ++++++- src/IO/BrotliWriteBuffer.cpp | 1 + src/IO/Bzip2WriteBuffer.cpp | 1 + src/IO/ForkWriteBuffer.cpp | 1 + src/IO/LZMADeflatingWriteBuffer.cpp | 1 + src/IO/Lz4DeflatingWriteBuffer.cpp | 1 + src/IO/SnappyWriteBuffer.cpp | 1 + src/IO/WriteBufferFromEncryptedFile.cpp | 1 + src/IO/WriteBufferFromFile.cpp | 1 + src/IO/WriteBufferFromFileDecorator.cpp | 17 +++++++++-------- src/IO/WriteBufferFromFileDescriptor.cpp | 1 + src/IO/WriteBufferFromOStream.cpp | 1 + src/IO/WriteBufferFromPocoSocket.cpp | 1 + src/IO/WriteBufferFromVector.h | 1 + src/IO/ZlibDeflatingWriteBuffer.cpp | 1 + src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 1 + src/IO/ZstdDeflatingWriteBuffer.cpp | 1 + .../Cache/WriteBufferToFileSegment.cpp | 1 + src/Interpreters/TemporaryDataOnDisk.cpp | 1 + .../HTTP/WriteBufferFromHTTPServerResponse.cpp | 1 + src/Storages/HDFS/WriteBufferFromHDFS.cpp | 1 + 26 files changed, 47 insertions(+), 17 deletions(-) diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index cb2ee1140d0..c3152c0adba 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -59,6 +59,7 @@ void CompressedWriteBuffer::nextImpl() CompressedWriteBuffer::~CompressedWriteBuffer() { + //! finalize(); } diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.cpp b/src/Coordination/WriteBufferFromNuraftBuffer.cpp index c955d3fdbbe..5bed2da2978 100644 --- a/src/Coordination/WriteBufferFromNuraftBuffer.cpp +++ b/src/Coordination/WriteBufferFromNuraftBuffer.cpp @@ -53,6 +53,7 @@ nuraft::ptr WriteBufferFromNuraftBuffer::getBuffer() WriteBufferFromNuraftBuffer::~WriteBufferFromNuraftBuffer() { + //! try { finalize(); diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 9153af90312..2abf3ab5203 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -123,6 +123,7 @@ void FileSegmentRangeWriter::finalize() FileSegmentRangeWriter::~FileSegmentRangeWriter() { + //! try { if (!finalized) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index b5d296bd865..56b45f1dc22 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -38,6 +38,7 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage() { + //! finalize(); } diff --git a/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp index 49e230b9dc3..63d7edf5437 100644 --- a/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp +++ b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp @@ -16,14 +16,15 @@ WriteBufferWithFinalizeCallback::WriteBufferWithFinalizeCallback( WriteBufferWithFinalizeCallback::~WriteBufferWithFinalizeCallback() { - try - { - finalize(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + //! +// try +// { +// finalize(); +// } +// catch (...) +// { +// tryLogCurrentException(__PRETTY_FUNCTION__); +// } } void WriteBufferWithFinalizeCallback::finalizeImpl() diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index 088d83cd29e..57bb6452914 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -115,6 +115,7 @@ public: ~WriteBufferFromZipArchive() override { + //! try { finalize(); @@ -191,7 +192,11 @@ namespace explicit StreamFromWriteBuffer(std::unique_ptr write_buffer_) : write_buffer(std::move(write_buffer_)), start_offset(write_buffer->count()) {} - ~StreamFromWriteBuffer() { write_buffer->finalize(); } + ~StreamFromWriteBuffer() + { + //! + write_buffer->finalize(); + } static int closeFileFunc(void *, void * stream) { diff --git a/src/IO/BrotliWriteBuffer.cpp b/src/IO/BrotliWriteBuffer.cpp index 47426d62a6e..0c434b4981e 100644 --- a/src/IO/BrotliWriteBuffer.cpp +++ b/src/IO/BrotliWriteBuffer.cpp @@ -44,6 +44,7 @@ BrotliWriteBuffer::BrotliWriteBuffer(std::unique_ptr out_, int comp BrotliWriteBuffer::~BrotliWriteBuffer() { + //! finalize(); } diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp index 4b6bed70d35..e5aab177f3a 100644 --- a/src/IO/Bzip2WriteBuffer.cpp +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -47,6 +47,7 @@ Bzip2WriteBuffer::Bzip2WriteBuffer(std::unique_ptr out_, int compre Bzip2WriteBuffer::~Bzip2WriteBuffer() { + //! finalize(); } diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp index 8e11b9ff590..b3481203757 100644 --- a/src/IO/ForkWriteBuffer.cpp +++ b/src/IO/ForkWriteBuffer.cpp @@ -53,6 +53,7 @@ void ForkWriteBuffer::finalizeImpl() ForkWriteBuffer::~ForkWriteBuffer() { + //! finalize(); } diff --git a/src/IO/LZMADeflatingWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp index 30e247b1016..3e1c9dc58b9 100644 --- a/src/IO/LZMADeflatingWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -46,6 +46,7 @@ LZMADeflatingWriteBuffer::LZMADeflatingWriteBuffer( LZMADeflatingWriteBuffer::~LZMADeflatingWriteBuffer() { + //! finalize(); } diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index c3a1b8282c3..2627643183d 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -42,6 +42,7 @@ Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer( Lz4DeflatingWriteBuffer::~Lz4DeflatingWriteBuffer() { + //! finalize(); } diff --git a/src/IO/SnappyWriteBuffer.cpp b/src/IO/SnappyWriteBuffer.cpp index ca40d0656d1..3ea6f831d5b 100644 --- a/src/IO/SnappyWriteBuffer.cpp +++ b/src/IO/SnappyWriteBuffer.cpp @@ -22,6 +22,7 @@ SnappyWriteBuffer::SnappyWriteBuffer(std::unique_ptr out_, size_t b SnappyWriteBuffer::~SnappyWriteBuffer() { + //! finish(); } diff --git a/src/IO/WriteBufferFromEncryptedFile.cpp b/src/IO/WriteBufferFromEncryptedFile.cpp index 5bca0dc68d5..7efdbc43238 100644 --- a/src/IO/WriteBufferFromEncryptedFile.cpp +++ b/src/IO/WriteBufferFromEncryptedFile.cpp @@ -21,6 +21,7 @@ WriteBufferFromEncryptedFile::WriteBufferFromEncryptedFile( WriteBufferFromEncryptedFile::~WriteBufferFromEncryptedFile() { + //! finalize(); } diff --git a/src/IO/WriteBufferFromFile.cpp b/src/IO/WriteBufferFromFile.cpp index e58f1e3a60c..964f7415803 100644 --- a/src/IO/WriteBufferFromFile.cpp +++ b/src/IO/WriteBufferFromFile.cpp @@ -77,6 +77,7 @@ WriteBufferFromFile::~WriteBufferFromFile() if (fd < 0) return; + //! finalize(); int err = ::close(fd); /// Everything except for EBADF should be ignored in dtor, since all of diff --git a/src/IO/WriteBufferFromFileDecorator.cpp b/src/IO/WriteBufferFromFileDecorator.cpp index 4cc881f177f..82e38eac72d 100644 --- a/src/IO/WriteBufferFromFileDecorator.cpp +++ b/src/IO/WriteBufferFromFileDecorator.cpp @@ -30,14 +30,15 @@ void WriteBufferFromFileDecorator::finalizeImpl() WriteBufferFromFileDecorator::~WriteBufferFromFileDecorator() { - try - { - finalize(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + //! +// try +// { +// finalize(); +// } +// catch (...) +// { +// tryLogCurrentException(__PRETTY_FUNCTION__); +// } /// It is not a mistake that swap is called here /// Swap has been called at constructor, it should be called at destructor diff --git a/src/IO/WriteBufferFromFileDescriptor.cpp b/src/IO/WriteBufferFromFileDescriptor.cpp index 135ff608967..0516171c051 100644 --- a/src/IO/WriteBufferFromFileDescriptor.cpp +++ b/src/IO/WriteBufferFromFileDescriptor.cpp @@ -105,6 +105,7 @@ WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor( WriteBufferFromFileDescriptor::~WriteBufferFromFileDescriptor() { + //! finalize(); } diff --git a/src/IO/WriteBufferFromOStream.cpp b/src/IO/WriteBufferFromOStream.cpp index 2d0d5976f85..d55aa054174 100644 --- a/src/IO/WriteBufferFromOStream.cpp +++ b/src/IO/WriteBufferFromOStream.cpp @@ -40,6 +40,7 @@ WriteBufferFromOStream::WriteBufferFromOStream( WriteBufferFromOStream::~WriteBufferFromOStream() { + //! finalize(); } diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 039110dfb62..a8b28445294 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -106,6 +106,7 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() { + //! finalize(); } diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index 4b2a3581625..b0ff3183291 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -63,6 +63,7 @@ public: ~WriteBufferFromVector() override { + //! finalize(); } diff --git a/src/IO/ZlibDeflatingWriteBuffer.cpp b/src/IO/ZlibDeflatingWriteBuffer.cpp index 43bb0405555..4440254d15f 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -74,6 +74,7 @@ void ZlibDeflatingWriteBuffer::nextImpl() ZlibDeflatingWriteBuffer::~ZlibDeflatingWriteBuffer() { + //! try { finalize(); diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index be739c0e654..05ee23158fd 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -87,6 +87,7 @@ void ZstdDeflatingAppendableWriteBuffer::nextImpl() ZstdDeflatingAppendableWriteBuffer::~ZstdDeflatingAppendableWriteBuffer() { + //! finalize(); } diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index c6d2ffc39f9..4ec3f5a3fce 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -33,6 +33,7 @@ ZstdDeflatingWriteBuffer::ZstdDeflatingWriteBuffer( ZstdDeflatingWriteBuffer::~ZstdDeflatingWriteBuffer() { + //! finalize(); } diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index 1eac87a804d..ed831350186 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -73,6 +73,7 @@ std::shared_ptr WriteBufferToFileSegment::getReadBufferImpl() WriteBufferToFileSegment::~WriteBufferToFileSegment() { + //! try { finalize(); diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 69fef21dbab..14e0bff6966 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -184,6 +184,7 @@ struct TemporaryFileStream::OutputWriter ~OutputWriter() { + //! try { finalize(); diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index c8015cfd185..03222fe7e0b 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -171,6 +171,7 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { + //! finalize(); } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index fad0447d2cf..48f35c378ba 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -147,6 +147,7 @@ void WriteBufferFromHDFS::finalizeImpl() WriteBufferFromHDFS::~WriteBufferFromHDFS() { + //! finalize(); } From fe3939287b476b7e0f2cec669dd9606f2fd0438f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 27 May 2023 01:02:48 +0200 Subject: [PATCH 125/308] add test, fix .gin_sid files --- src/IO/WriteBufferFromS3.cpp | 12 ++++++++++-- src/IO/WriteBufferFromS3TaskTracker.cpp | 15 ++++++++------- src/Storages/MergeTree/GinIndexStore.cpp | 7 +++++++ .../configs/config.d/storage_conf.xml | 19 +++++++++++++++++++ tests/integration/test_merge_tree_s3/test.py | 5 +++-- 5 files changed, 47 insertions(+), 11 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 954c996d929..4657a65f931 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -182,6 +182,14 @@ void WriteBufferFromS3::finalizeImpl() if (!is_prefinalized) preFinalize(); + if (std::uncaught_exception()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Detected buffer finalization when an exception is unwinding the stack." + " Do not call finalize buffer in destructors or when exception thrown." + " Details {}", + getLogDetails()); + chassert(offset() == 0); chassert(hidden_size == 0); @@ -521,7 +529,7 @@ void WriteBufferFromS3::completeMultipartUpload() if (multipart_tags.empty()) throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::S3_ERROR, "Failed to complete multipart upload. No parts have uploaded"); for (size_t i = 0; i < multipart_tags.size(); ++i) @@ -529,7 +537,7 @@ void WriteBufferFromS3::completeMultipartUpload() const auto tag = multipart_tags.at(i); if (tag.empty()) throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::S3_ERROR, "Failed to complete multipart upload. Part {} haven't been uploaded.", i); } diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 4abae90eeac..7cf5a89df86 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -162,15 +162,16 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() for (auto & it : finished_futures) { - SCOPE_EXIT({ - /// According to basic exception safety TaskTracker has to be destroyed after exception - /// If it would be true than this SCOPE_EXIT is superfluous - /// However WriteBufferWithFinalizeCallback, WriteBufferFromFileDecorator do call finalize in d-tor - /// TaskTracker has to cope this until the issue with finalizing in d-tor is addressed in #50274 - futures.erase(it); - }); +// SCOPE_EXIT({ +// /// According to basic exception safety TaskTracker has to be destroyed after exception +// /// If it would be true than this SCOPE_EXIT is superfluous +// /// However WriteBufferWithFinalizeCallback, WriteBufferFromFileDecorator do call finalize in d-tor +// /// TaskTracker has to cope this until the issue with finalizing in d-tor is addressed in #50274 +// futures.erase(it); +// }); it->get(); + futures.erase(it); } finished_futures.clear(); diff --git a/src/Storages/MergeTree/GinIndexStore.cpp b/src/Storages/MergeTree/GinIndexStore.cpp index 0904855755c..aa0c1fccbc3 100644 --- a/src/Storages/MergeTree/GinIndexStore.cpp +++ b/src/Storages/MergeTree/GinIndexStore.cpp @@ -166,6 +166,7 @@ UInt32 GinIndexStore::getNextSegmentIDRange(const String & file_name, size_t n) /// Write segment ID 1 writeVarUInt(1, *ostr); ostr->sync(); + ostr->finalize(); } /// Read id in file @@ -188,6 +189,7 @@ UInt32 GinIndexStore::getNextSegmentIDRange(const String & file_name, size_t n) writeVarUInt(result + n, *ostr); ostr->sync(); + ostr->finalize(); } return result; } @@ -317,8 +319,13 @@ void GinIndexStore::writeSegment() current_segment.segment_id = getNextSegmentID(); metadata_file_stream->sync(); + metadata_file_stream->finalize(); + dict_file_stream->sync(); + dict_file_stream->finalize(); + postings_file_stream->sync(); + postings_file_stream->finalize(); } GinIndexStoreDeserializer::GinIndexStoreDeserializer(const GinIndexStorePtr & store_) diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml index cca80143548..504280e4bed 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml @@ -33,6 +33,18 @@ 20000 1 + + s3 + http://resolver:8083/root/data/ + minio + minio123 + true + 0 + 20000 + 20000 + 0 + 1 + local / @@ -128,6 +140,13 @@ + + +
+ broken_s3_always_multi_part +
+
+
diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index f87644a6876..4dcc5805294 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -930,8 +930,9 @@ def test_merge_canceled_by_drop(cluster, node_name): ) +@pytest.mark.parametrize("storage_policy", ["broken_s3_always_multi_part", "broken_s3"]) @pytest.mark.parametrize("node_name", ["node"]) -def test_merge_canceled_by_s3_errors(cluster, node_name): +def test_merge_canceled_by_s3_errors(cluster, node_name, storage_policy): node = cluster.instances[node_name] node.query("DROP TABLE IF EXISTS test_merge_canceled_by_s3_errors NO DELAY") node.query( @@ -939,7 +940,7 @@ def test_merge_canceled_by_s3_errors(cluster, node_name): " (key UInt32, value String)" " Engine=MergeTree() " " ORDER BY value " - " SETTINGS storage_policy='broken_s3'" + f" SETTINGS storage_policy='{storage_policy}'" ) node.query("SYSTEM STOP MERGES test_merge_canceled_by_s3_errors") node.query( From 99ad481505798a6da906e6b96d10b641db902f68 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 27 May 2023 11:47:44 +0200 Subject: [PATCH 126/308] fix sizes.log writes --- src/Common/FileChecker.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index a6e37654ff1..876bc4e641c 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -138,7 +138,7 @@ void FileChecker::save() const std::string tmp_files_info_path = parentPath(files_info_path) + "tmp_" + fileName(files_info_path); { - std::unique_ptr out = disk ? disk->writeFile(tmp_files_info_path) : std::make_unique(tmp_files_info_path); + std::unique_ptr out = disk ? disk->writeFile(tmp_files_info_path) : std::make_unique(tmp_files_info_path); /// So complex JSON structure - for compatibility with the old format. writeCString("{\"clickhouse\":{", *out); @@ -157,7 +157,9 @@ void FileChecker::save() const } writeCString("}}", *out); - out->next(); + + out->sync(); + out->finalize(); } if (disk) From d95e5b51af32db90de8f2194e83c0015dfd0f4cb Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 27 May 2023 12:14:07 +0200 Subject: [PATCH 127/308] mark all finalize calls in buffers d-tors with issue id --- src/Compression/CompressedWriteBuffer.cpp | 2 +- src/Coordination/WriteBufferFromNuraftBuffer.cpp | 2 +- src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp | 2 +- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- src/Disks/IO/WriteBufferWithFinalizeCallback.cpp | 2 +- src/IO/Archives/ZipArchiveWriter.cpp | 4 ++-- src/IO/BrotliWriteBuffer.cpp | 2 +- src/IO/Bzip2WriteBuffer.cpp | 2 +- src/IO/ForkWriteBuffer.cpp | 2 +- src/IO/LZMADeflatingWriteBuffer.cpp | 2 +- src/IO/Lz4DeflatingWriteBuffer.cpp | 2 +- src/IO/SnappyWriteBuffer.cpp | 2 +- src/IO/WriteBufferFromEncryptedFile.cpp | 2 +- src/IO/WriteBufferFromFile.cpp | 2 +- src/IO/WriteBufferFromFileDecorator.cpp | 2 +- src/IO/WriteBufferFromFileDescriptor.cpp | 2 +- src/IO/WriteBufferFromOStream.cpp | 2 +- src/IO/WriteBufferFromPocoSocket.cpp | 2 +- src/IO/WriteBufferFromVector.h | 2 +- src/IO/ZlibDeflatingWriteBuffer.cpp | 2 +- src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 2 +- src/IO/ZstdDeflatingWriteBuffer.cpp | 2 +- src/Interpreters/Cache/WriteBufferToFileSegment.cpp | 2 +- src/Interpreters/TemporaryDataOnDisk.cpp | 2 +- src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp | 2 +- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 2 +- 26 files changed, 27 insertions(+), 27 deletions(-) diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index c3152c0adba..22c5f235a27 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -59,7 +59,7 @@ void CompressedWriteBuffer::nextImpl() CompressedWriteBuffer::~CompressedWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.cpp b/src/Coordination/WriteBufferFromNuraftBuffer.cpp index 5bed2da2978..41b933717c6 100644 --- a/src/Coordination/WriteBufferFromNuraftBuffer.cpp +++ b/src/Coordination/WriteBufferFromNuraftBuffer.cpp @@ -53,7 +53,7 @@ nuraft::ptr WriteBufferFromNuraftBuffer::getBuffer() WriteBufferFromNuraftBuffer::~WriteBufferFromNuraftBuffer() { - //! + /// ! #50274 try { finalize(); diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 2abf3ab5203..8da9ce73ae3 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -123,7 +123,7 @@ void FileSegmentRangeWriter::finalize() FileSegmentRangeWriter::~FileSegmentRangeWriter() { - //! + /// ! #50274 try { if (!finalized) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 56b45f1dc22..8ac2c8ca831 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -38,7 +38,7 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage() { - //! + /// ! #50274 finalize(); } diff --git a/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp index 63d7edf5437..47bf742fd5b 100644 --- a/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp +++ b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp @@ -16,7 +16,7 @@ WriteBufferWithFinalizeCallback::WriteBufferWithFinalizeCallback( WriteBufferWithFinalizeCallback::~WriteBufferWithFinalizeCallback() { - //! + /// ! #50274 // try // { // finalize(); diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index 57bb6452914..1c6ef18d372 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -115,7 +115,7 @@ public: ~WriteBufferFromZipArchive() override { - //! + /// ! #50274 try { finalize(); @@ -194,7 +194,7 @@ namespace ~StreamFromWriteBuffer() { - //! + /// ! #50274 write_buffer->finalize(); } diff --git a/src/IO/BrotliWriteBuffer.cpp b/src/IO/BrotliWriteBuffer.cpp index 0c434b4981e..98786313b09 100644 --- a/src/IO/BrotliWriteBuffer.cpp +++ b/src/IO/BrotliWriteBuffer.cpp @@ -44,7 +44,7 @@ BrotliWriteBuffer::BrotliWriteBuffer(std::unique_ptr out_, int comp BrotliWriteBuffer::~BrotliWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp index e5aab177f3a..c7159d4d002 100644 --- a/src/IO/Bzip2WriteBuffer.cpp +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -47,7 +47,7 @@ Bzip2WriteBuffer::Bzip2WriteBuffer(std::unique_ptr out_, int compre Bzip2WriteBuffer::~Bzip2WriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp index b3481203757..ea38f70745d 100644 --- a/src/IO/ForkWriteBuffer.cpp +++ b/src/IO/ForkWriteBuffer.cpp @@ -53,7 +53,7 @@ void ForkWriteBuffer::finalizeImpl() ForkWriteBuffer::~ForkWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/LZMADeflatingWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp index 3e1c9dc58b9..07597ce958b 100644 --- a/src/IO/LZMADeflatingWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -46,7 +46,7 @@ LZMADeflatingWriteBuffer::LZMADeflatingWriteBuffer( LZMADeflatingWriteBuffer::~LZMADeflatingWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index 2627643183d..988d9861a9b 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -42,7 +42,7 @@ Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer( Lz4DeflatingWriteBuffer::~Lz4DeflatingWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/SnappyWriteBuffer.cpp b/src/IO/SnappyWriteBuffer.cpp index 3ea6f831d5b..5203d8b053e 100644 --- a/src/IO/SnappyWriteBuffer.cpp +++ b/src/IO/SnappyWriteBuffer.cpp @@ -22,7 +22,7 @@ SnappyWriteBuffer::SnappyWriteBuffer(std::unique_ptr out_, size_t b SnappyWriteBuffer::~SnappyWriteBuffer() { - //! + /// ! #50274 finish(); } diff --git a/src/IO/WriteBufferFromEncryptedFile.cpp b/src/IO/WriteBufferFromEncryptedFile.cpp index 7efdbc43238..5831b3aee94 100644 --- a/src/IO/WriteBufferFromEncryptedFile.cpp +++ b/src/IO/WriteBufferFromEncryptedFile.cpp @@ -21,7 +21,7 @@ WriteBufferFromEncryptedFile::WriteBufferFromEncryptedFile( WriteBufferFromEncryptedFile::~WriteBufferFromEncryptedFile() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromFile.cpp b/src/IO/WriteBufferFromFile.cpp index 964f7415803..0777d596184 100644 --- a/src/IO/WriteBufferFromFile.cpp +++ b/src/IO/WriteBufferFromFile.cpp @@ -77,7 +77,7 @@ WriteBufferFromFile::~WriteBufferFromFile() if (fd < 0) return; - //! + /// ! #50274 finalize(); int err = ::close(fd); /// Everything except for EBADF should be ignored in dtor, since all of diff --git a/src/IO/WriteBufferFromFileDecorator.cpp b/src/IO/WriteBufferFromFileDecorator.cpp index 82e38eac72d..67a98d9e43f 100644 --- a/src/IO/WriteBufferFromFileDecorator.cpp +++ b/src/IO/WriteBufferFromFileDecorator.cpp @@ -30,7 +30,7 @@ void WriteBufferFromFileDecorator::finalizeImpl() WriteBufferFromFileDecorator::~WriteBufferFromFileDecorator() { - //! + /// ! #50274 // try // { // finalize(); diff --git a/src/IO/WriteBufferFromFileDescriptor.cpp b/src/IO/WriteBufferFromFileDescriptor.cpp index 0516171c051..03988c09f62 100644 --- a/src/IO/WriteBufferFromFileDescriptor.cpp +++ b/src/IO/WriteBufferFromFileDescriptor.cpp @@ -105,7 +105,7 @@ WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor( WriteBufferFromFileDescriptor::~WriteBufferFromFileDescriptor() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromOStream.cpp b/src/IO/WriteBufferFromOStream.cpp index d55aa054174..25886b9d10d 100644 --- a/src/IO/WriteBufferFromOStream.cpp +++ b/src/IO/WriteBufferFromOStream.cpp @@ -40,7 +40,7 @@ WriteBufferFromOStream::WriteBufferFromOStream( WriteBufferFromOStream::~WriteBufferFromOStream() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index a8b28445294..a0dcf69e67d 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -106,7 +106,7 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index b0ff3183291..ab4f27a7bcb 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -63,7 +63,7 @@ public: ~WriteBufferFromVector() override { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/ZlibDeflatingWriteBuffer.cpp b/src/IO/ZlibDeflatingWriteBuffer.cpp index 4440254d15f..7ae8f636f69 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -74,7 +74,7 @@ void ZlibDeflatingWriteBuffer::nextImpl() ZlibDeflatingWriteBuffer::~ZlibDeflatingWriteBuffer() { - //! + /// ! #50274 try { finalize(); diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index 05ee23158fd..8f66618f513 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -87,7 +87,7 @@ void ZstdDeflatingAppendableWriteBuffer::nextImpl() ZstdDeflatingAppendableWriteBuffer::~ZstdDeflatingAppendableWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index 4ec3f5a3fce..95e2e7d7716 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -33,7 +33,7 @@ ZstdDeflatingWriteBuffer::ZstdDeflatingWriteBuffer( ZstdDeflatingWriteBuffer::~ZstdDeflatingWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index ed831350186..cbbf22078c1 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -73,7 +73,7 @@ std::shared_ptr WriteBufferToFileSegment::getReadBufferImpl() WriteBufferToFileSegment::~WriteBufferToFileSegment() { - //! + /// ! #50274 try { finalize(); diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 14e0bff6966..2fc21e84be7 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -184,7 +184,7 @@ struct TemporaryFileStream::OutputWriter ~OutputWriter() { - //! + /// ! #50274 try { finalize(); diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 03222fe7e0b..7d6710e76e6 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -171,7 +171,7 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { - //! + /// ! #50274 finalize(); } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 48f35c378ba..169cf845d3b 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -147,7 +147,7 @@ void WriteBufferFromHDFS::finalizeImpl() WriteBufferFromHDFS::~WriteBufferFromHDFS() { - //! + /// ! #50274 finalize(); } From a5dcd8dabbd2ba9fd796b0ccacf447fc75ecc97e Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 27 May 2023 20:51:59 +0200 Subject: [PATCH 128/308] do not call finalize in d-tor MergedBlockOutputStream::Finalizer --- src/IO/WriteBufferFromS3.cpp | 7 +++++-- .../MergeTree/MergedBlockOutputStream.cpp | 17 +++++++++-------- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 4657a65f931..58d58e86ebe 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -109,8 +109,8 @@ void WriteBufferFromS3::nextImpl() if (is_prefinalized) throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot write to prefinalized buffer for S3, the file could have been created with PutObjectRequest"); + ErrorCodes::LOGICAL_ERROR, + "Cannot write to prefinalized buffer for S3, the file could have been created with PutObjectRequest"); /// Make sense to call waitIfAny before adding new async task to check if there is an exception /// The faster the exception is propagated the lesser time is spent for cancellation @@ -183,12 +183,15 @@ void WriteBufferFromS3::finalizeImpl() preFinalize(); if (std::uncaught_exception()) + { + tryLogCurrentException(__PRETTY_FUNCTION__); throw Exception( ErrorCodes::LOGICAL_ERROR, "Detected buffer finalization when an exception is unwinding the stack." " Do not call finalize buffer in destructors or when exception thrown." " Details {}", getLogDetails()); + } chassert(offset() == 0); chassert(hidden_size == 0); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index d97da5a0b50..7bff58fc411 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -121,14 +121,15 @@ void MergedBlockOutputStream::Finalizer::Impl::finish() MergedBlockOutputStream::Finalizer::~Finalizer() { - try - { - finish(); - } - catch (...) - { - tryLogCurrentException("MergedBlockOutputStream"); - } + /// ! #50274 +// try +// { +// finish(); +// } +// catch (...) +// { +// tryLogCurrentException("MergedBlockOutputStream"); +// } } MergedBlockOutputStream::Finalizer::Finalizer(Finalizer &&) noexcept = default; From 13dcb62ffb7abfe12bef0fb300e9c9e62610f754 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 28 May 2023 12:29:38 +0200 Subject: [PATCH 129/308] fix logs engins --- src/IO/WriteBufferFromS3.cpp | 8 +++++++- src/Storages/StorageLog.cpp | 3 +++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 58d58e86ebe..5bfc6b91946 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -253,7 +253,13 @@ WriteBufferFromS3::~WriteBufferFromS3() // That destructor could be call with finalized=false in case of exceptions if (!finalized) { - LOG_ERROR(log, "WriteBufferFromS3 is not finalized in destructor. It could be if an exception occurs. File is not written to S3. {}.", getLogDetails()); + LOG_INFO(log, + "WriteBufferFromS3 is not finalized in destructor. " + "It could be if an exception occurs. File is not written to S3. " + "{}. " + "Stack trace: {}", + getLogDetails(), + StackTrace().toString()); } task_tracker->safeWaitAll(); diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index f698f1881fa..02dc4843660 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -341,7 +341,10 @@ private: void finalize() { compressed.next(); + compressed.finalize(); + plain->next(); + plain->finalize(); } }; From b13990efcc184c17356a14b67ec5057718f76e4c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 28 May 2023 15:32:10 +0200 Subject: [PATCH 130/308] fix build --- src/IO/WriteBufferFromS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 5bfc6b91946..640763f793c 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -182,7 +182,7 @@ void WriteBufferFromS3::finalizeImpl() if (!is_prefinalized) preFinalize(); - if (std::uncaught_exception()) + if (std::uncaught_exceptions()) { tryLogCurrentException(__PRETTY_FUNCTION__); throw Exception( From 0b4ea3e2e1d5d91d96bbb13965b00a607664879d Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 29 May 2023 00:37:45 +0200 Subject: [PATCH 131/308] remove reminder comments --- src/Compression/CompressedWriteBuffer.cpp | 1 - src/Coordination/WriteBufferFromNuraftBuffer.cpp | 1 - src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp | 1 - src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 1 - src/Disks/IO/WriteBufferWithFinalizeCallback.cpp | 14 -------------- src/Disks/IO/WriteBufferWithFinalizeCallback.h | 2 -- src/IO/Archives/ZipArchiveWriter.cpp | 2 -- src/IO/BrotliWriteBuffer.cpp | 1 - src/IO/Bzip2WriteBuffer.cpp | 1 - src/IO/ForkWriteBuffer.cpp | 1 - src/IO/LZMADeflatingWriteBuffer.cpp | 1 - src/IO/Lz4DeflatingWriteBuffer.cpp | 1 - src/IO/SnappyWriteBuffer.cpp | 1 - src/IO/WriteBufferFromEncryptedFile.cpp | 1 - src/IO/WriteBufferFromFile.cpp | 1 - src/IO/WriteBufferFromFileDecorator.cpp | 10 ---------- src/IO/WriteBufferFromFileDescriptor.cpp | 1 - src/IO/WriteBufferFromOStream.cpp | 1 - src/IO/WriteBufferFromPocoSocket.cpp | 1 - src/IO/WriteBufferFromS3.cpp | 4 ++-- src/IO/WriteBufferFromS3TaskTracker.cpp | 8 -------- src/IO/WriteBufferFromVector.h | 1 - src/IO/ZlibDeflatingWriteBuffer.cpp | 1 - src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 1 - src/IO/ZstdDeflatingWriteBuffer.cpp | 1 - .../Cache/WriteBufferToFileSegment.cpp | 1 - src/Interpreters/TemporaryDataOnDisk.cpp | 1 - .../HTTP/WriteBufferFromHTTPServerResponse.cpp | 1 - src/Storages/HDFS/WriteBufferFromHDFS.cpp | 1 - .../MergeTree/MergedBlockOutputStream.cpp | 15 ++------------- src/Storages/MergeTree/MergedBlockOutputStream.h | 3 ++- tests/integration/test_merge_tree_s3/test.py | 8 ++++---- 32 files changed, 10 insertions(+), 79 deletions(-) diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index 22c5f235a27..cb2ee1140d0 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -59,7 +59,6 @@ void CompressedWriteBuffer::nextImpl() CompressedWriteBuffer::~CompressedWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.cpp b/src/Coordination/WriteBufferFromNuraftBuffer.cpp index 41b933717c6..c955d3fdbbe 100644 --- a/src/Coordination/WriteBufferFromNuraftBuffer.cpp +++ b/src/Coordination/WriteBufferFromNuraftBuffer.cpp @@ -53,7 +53,6 @@ nuraft::ptr WriteBufferFromNuraftBuffer::getBuffer() WriteBufferFromNuraftBuffer::~WriteBufferFromNuraftBuffer() { - /// ! #50274 try { finalize(); diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 8da9ce73ae3..9153af90312 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -123,7 +123,6 @@ void FileSegmentRangeWriter::finalize() FileSegmentRangeWriter::~FileSegmentRangeWriter() { - /// ! #50274 try { if (!finalized) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 8ac2c8ca831..b5d296bd865 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -38,7 +38,6 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage() { - /// ! #50274 finalize(); } diff --git a/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp index 47bf742fd5b..8703eae4913 100644 --- a/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp +++ b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp @@ -13,20 +13,6 @@ WriteBufferWithFinalizeCallback::WriteBufferWithFinalizeCallback( { } - -WriteBufferWithFinalizeCallback::~WriteBufferWithFinalizeCallback() -{ - /// ! #50274 -// try -// { -// finalize(); -// } -// catch (...) -// { -// tryLogCurrentException(__PRETTY_FUNCTION__); -// } -} - void WriteBufferWithFinalizeCallback::finalizeImpl() { WriteBufferFromFileDecorator::finalizeImpl(); diff --git a/src/Disks/IO/WriteBufferWithFinalizeCallback.h b/src/Disks/IO/WriteBufferWithFinalizeCallback.h index 73c1b8d25d4..2798582c336 100644 --- a/src/Disks/IO/WriteBufferWithFinalizeCallback.h +++ b/src/Disks/IO/WriteBufferWithFinalizeCallback.h @@ -19,8 +19,6 @@ public: FinalizeCallback && create_callback_, const String & remote_path_); - ~WriteBufferWithFinalizeCallback() override; - String getFileName() const override { return remote_path; } private: diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index 1c6ef18d372..ed4b3502b2f 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -115,7 +115,6 @@ public: ~WriteBufferFromZipArchive() override { - /// ! #50274 try { finalize(); @@ -194,7 +193,6 @@ namespace ~StreamFromWriteBuffer() { - /// ! #50274 write_buffer->finalize(); } diff --git a/src/IO/BrotliWriteBuffer.cpp b/src/IO/BrotliWriteBuffer.cpp index 98786313b09..47426d62a6e 100644 --- a/src/IO/BrotliWriteBuffer.cpp +++ b/src/IO/BrotliWriteBuffer.cpp @@ -44,7 +44,6 @@ BrotliWriteBuffer::BrotliWriteBuffer(std::unique_ptr out_, int comp BrotliWriteBuffer::~BrotliWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp index c7159d4d002..4b6bed70d35 100644 --- a/src/IO/Bzip2WriteBuffer.cpp +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -47,7 +47,6 @@ Bzip2WriteBuffer::Bzip2WriteBuffer(std::unique_ptr out_, int compre Bzip2WriteBuffer::~Bzip2WriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp index ea38f70745d..8e11b9ff590 100644 --- a/src/IO/ForkWriteBuffer.cpp +++ b/src/IO/ForkWriteBuffer.cpp @@ -53,7 +53,6 @@ void ForkWriteBuffer::finalizeImpl() ForkWriteBuffer::~ForkWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/IO/LZMADeflatingWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp index 07597ce958b..30e247b1016 100644 --- a/src/IO/LZMADeflatingWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -46,7 +46,6 @@ LZMADeflatingWriteBuffer::LZMADeflatingWriteBuffer( LZMADeflatingWriteBuffer::~LZMADeflatingWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index 988d9861a9b..c3a1b8282c3 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -42,7 +42,6 @@ Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer( Lz4DeflatingWriteBuffer::~Lz4DeflatingWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/IO/SnappyWriteBuffer.cpp b/src/IO/SnappyWriteBuffer.cpp index 5203d8b053e..ca40d0656d1 100644 --- a/src/IO/SnappyWriteBuffer.cpp +++ b/src/IO/SnappyWriteBuffer.cpp @@ -22,7 +22,6 @@ SnappyWriteBuffer::SnappyWriteBuffer(std::unique_ptr out_, size_t b SnappyWriteBuffer::~SnappyWriteBuffer() { - /// ! #50274 finish(); } diff --git a/src/IO/WriteBufferFromEncryptedFile.cpp b/src/IO/WriteBufferFromEncryptedFile.cpp index 5831b3aee94..5bca0dc68d5 100644 --- a/src/IO/WriteBufferFromEncryptedFile.cpp +++ b/src/IO/WriteBufferFromEncryptedFile.cpp @@ -21,7 +21,6 @@ WriteBufferFromEncryptedFile::WriteBufferFromEncryptedFile( WriteBufferFromEncryptedFile::~WriteBufferFromEncryptedFile() { - /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromFile.cpp b/src/IO/WriteBufferFromFile.cpp index 0777d596184..e58f1e3a60c 100644 --- a/src/IO/WriteBufferFromFile.cpp +++ b/src/IO/WriteBufferFromFile.cpp @@ -77,7 +77,6 @@ WriteBufferFromFile::~WriteBufferFromFile() if (fd < 0) return; - /// ! #50274 finalize(); int err = ::close(fd); /// Everything except for EBADF should be ignored in dtor, since all of diff --git a/src/IO/WriteBufferFromFileDecorator.cpp b/src/IO/WriteBufferFromFileDecorator.cpp index 67a98d9e43f..0e4e5e13a86 100644 --- a/src/IO/WriteBufferFromFileDecorator.cpp +++ b/src/IO/WriteBufferFromFileDecorator.cpp @@ -30,16 +30,6 @@ void WriteBufferFromFileDecorator::finalizeImpl() WriteBufferFromFileDecorator::~WriteBufferFromFileDecorator() { - /// ! #50274 -// try -// { -// finalize(); -// } -// catch (...) -// { -// tryLogCurrentException(__PRETTY_FUNCTION__); -// } - /// It is not a mistake that swap is called here /// Swap has been called at constructor, it should be called at destructor /// In oreder to provide valid buffer for impl's d-tor call diff --git a/src/IO/WriteBufferFromFileDescriptor.cpp b/src/IO/WriteBufferFromFileDescriptor.cpp index 03988c09f62..135ff608967 100644 --- a/src/IO/WriteBufferFromFileDescriptor.cpp +++ b/src/IO/WriteBufferFromFileDescriptor.cpp @@ -105,7 +105,6 @@ WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor( WriteBufferFromFileDescriptor::~WriteBufferFromFileDescriptor() { - /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromOStream.cpp b/src/IO/WriteBufferFromOStream.cpp index 25886b9d10d..2d0d5976f85 100644 --- a/src/IO/WriteBufferFromOStream.cpp +++ b/src/IO/WriteBufferFromOStream.cpp @@ -40,7 +40,6 @@ WriteBufferFromOStream::WriteBufferFromOStream( WriteBufferFromOStream::~WriteBufferFromOStream() { - /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index a0dcf69e67d..039110dfb62 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -106,7 +106,6 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() { - /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 640763f793c..44feb225471 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -538,7 +538,7 @@ void WriteBufferFromS3::completeMultipartUpload() if (multipart_tags.empty()) throw Exception( - ErrorCodes::S3_ERROR, + ErrorCodes::LOGICAL_ERROR, "Failed to complete multipart upload. No parts have uploaded"); for (size_t i = 0; i < multipart_tags.size(); ++i) @@ -546,7 +546,7 @@ void WriteBufferFromS3::completeMultipartUpload() const auto tag = multipart_tags.at(i); if (tag.empty()) throw Exception( - ErrorCodes::S3_ERROR, + ErrorCodes::LOGICAL_ERROR, "Failed to complete multipart upload. Part {} haven't been uploaded.", i); } diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 7cf5a89df86..7ae31044012 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -162,14 +162,6 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() for (auto & it : finished_futures) { -// SCOPE_EXIT({ -// /// According to basic exception safety TaskTracker has to be destroyed after exception -// /// If it would be true than this SCOPE_EXIT is superfluous -// /// However WriteBufferWithFinalizeCallback, WriteBufferFromFileDecorator do call finalize in d-tor -// /// TaskTracker has to cope this until the issue with finalizing in d-tor is addressed in #50274 -// futures.erase(it); -// }); - it->get(); futures.erase(it); } diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index ab4f27a7bcb..4b2a3581625 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -63,7 +63,6 @@ public: ~WriteBufferFromVector() override { - /// ! #50274 finalize(); } diff --git a/src/IO/ZlibDeflatingWriteBuffer.cpp b/src/IO/ZlibDeflatingWriteBuffer.cpp index 7ae8f636f69..43bb0405555 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -74,7 +74,6 @@ void ZlibDeflatingWriteBuffer::nextImpl() ZlibDeflatingWriteBuffer::~ZlibDeflatingWriteBuffer() { - /// ! #50274 try { finalize(); diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index 8f66618f513..be739c0e654 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -87,7 +87,6 @@ void ZstdDeflatingAppendableWriteBuffer::nextImpl() ZstdDeflatingAppendableWriteBuffer::~ZstdDeflatingAppendableWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index 95e2e7d7716..c6d2ffc39f9 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -33,7 +33,6 @@ ZstdDeflatingWriteBuffer::ZstdDeflatingWriteBuffer( ZstdDeflatingWriteBuffer::~ZstdDeflatingWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index cbbf22078c1..1eac87a804d 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -73,7 +73,6 @@ std::shared_ptr WriteBufferToFileSegment::getReadBufferImpl() WriteBufferToFileSegment::~WriteBufferToFileSegment() { - /// ! #50274 try { finalize(); diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 2fc21e84be7..69fef21dbab 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -184,7 +184,6 @@ struct TemporaryFileStream::OutputWriter ~OutputWriter() { - /// ! #50274 try { finalize(); diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 7d6710e76e6..c8015cfd185 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -171,7 +171,6 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { - /// ! #50274 finalize(); } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 169cf845d3b..fad0447d2cf 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -147,7 +147,6 @@ void WriteBufferFromHDFS::finalizeImpl() WriteBufferFromHDFS::~WriteBufferFromHDFS() { - /// ! #50274 finalize(); } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 7bff58fc411..7e69f03e959 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -119,23 +119,12 @@ void MergedBlockOutputStream::Finalizer::Impl::finish() part->getDataPartStorage().removeFile(file_name); } -MergedBlockOutputStream::Finalizer::~Finalizer() -{ - /// ! #50274 -// try -// { -// finish(); -// } -// catch (...) -// { -// tryLogCurrentException("MergedBlockOutputStream"); -// } -} - MergedBlockOutputStream::Finalizer::Finalizer(Finalizer &&) noexcept = default; MergedBlockOutputStream::Finalizer & MergedBlockOutputStream::Finalizer::operator=(Finalizer &&) noexcept = default; MergedBlockOutputStream::Finalizer::Finalizer(std::unique_ptr impl_) : impl(std::move(impl_)) {} +MergedBlockOutputStream::Finalizer::~Finalizer() {} + void MergedBlockOutputStream::finalizePart( const MergeTreeMutableDataPartPtr & new_part, bool sync, diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index ad1bb584788..f3a5653a880 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -44,9 +44,10 @@ public: std::unique_ptr impl; explicit Finalizer(std::unique_ptr impl_); - ~Finalizer(); Finalizer(Finalizer &&) noexcept; Finalizer & operator=(Finalizer &&) noexcept; + ~Finalizer(); + void finish(); }; diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 4dcc5805294..626a71f006e 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -1049,8 +1049,8 @@ def test_s3_engine_heavy_write_check_mem(cluster, node_name, in_flight_memory): " AND type!='QueryStart'" ).split() - assert int(memory_usage) < 1.1 * memory - assert int(memory_usage) > 0.9 * memory + assert int(memory_usage) < 1.2 * memory + assert int(memory_usage) > 0.8 * memory assert int(wait_inflight) > 10 * 1000 * 1000 @@ -1097,7 +1097,7 @@ def test_s3_disk_heavy_write_check_mem(cluster, node_name): " AND type!='QueryStart'" ) - assert int(result) < 1.1 * memory - assert int(result) > 0.9 * memory + assert int(result) < 1.2 * memory + assert int(result) > 0.8 * memory check_no_objects_after_drop(cluster, node_name=node_name) From 2d2b411c26fc4e64bce894a48684c144f3b689b1 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 May 2023 12:21:33 +0200 Subject: [PATCH 132/308] fix build --- src/Storages/MergeTree/MergedBlockOutputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 7e69f03e959..c93ad135835 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -123,7 +123,7 @@ MergedBlockOutputStream::Finalizer::Finalizer(Finalizer &&) noexcept = default; MergedBlockOutputStream::Finalizer & MergedBlockOutputStream::Finalizer::operator=(Finalizer &&) noexcept = default; MergedBlockOutputStream::Finalizer::Finalizer(std::unique_ptr impl_) : impl(std::move(impl_)) {} -MergedBlockOutputStream::Finalizer::~Finalizer() {} +MergedBlockOutputStream::Finalizer::~Finalizer() = default; void MergedBlockOutputStream::finalizePart( const MergeTreeMutableDataPartPtr & new_part, From 0a128cec6179fbc97c9329538f3e28c2ca1567d1 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 May 2023 12:28:58 +0200 Subject: [PATCH 133/308] remove tricky debug trap --- src/IO/WriteBufferFromS3.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 44feb225471..462cf2674c3 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -182,17 +182,6 @@ void WriteBufferFromS3::finalizeImpl() if (!is_prefinalized) preFinalize(); - if (std::uncaught_exceptions()) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Detected buffer finalization when an exception is unwinding the stack." - " Do not call finalize buffer in destructors or when exception thrown." - " Details {}", - getLogDetails()); - } - chassert(offset() == 0); chassert(hidden_size == 0); From fe82d2bbe24a98bf2192796f47a9b1e2b5d40b3a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 30 May 2023 12:58:37 +0200 Subject: [PATCH 134/308] Fix build --- src/Common/checkSSLReturnCode.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/checkSSLReturnCode.cpp b/src/Common/checkSSLReturnCode.cpp index 8916a25e19c..bc87af1d37d 100644 --- a/src/Common/checkSSLReturnCode.cpp +++ b/src/Common/checkSSLReturnCode.cpp @@ -8,7 +8,7 @@ namespace DB { -bool checkSSLWantRead(ssize_t res) +bool checkSSLWantRead([[maybe_unused]] ssize_t res) { #if USE_SSL return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ; @@ -17,7 +17,7 @@ bool checkSSLWantRead(ssize_t res) #endif } -bool checkSSLWantWrite(ssize_t res) +bool checkSSLWantWrite([[maybe_unused]] ssize_t res) { #if USE_SSL return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE; From fe944a55dc4f99bd5fd69da54427be83abe63ac9 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 30 May 2023 13:30:47 +0200 Subject: [PATCH 135/308] Updated to hold temporary_directory_lock till part is fetched, renamed and committed --- src/Storages/MergeTree/DataPartsExchange.cpp | 12 +++++----- src/Storages/MergeTree/DataPartsExchange.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 23 +++++++++++++++----- 3 files changed, 24 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 21fdda28767..357d48ae466 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -365,7 +365,7 @@ Fetcher::Fetcher(StorageReplicatedMergeTree & data_) , log(&Poco::Logger::get(data.getStorageID().getNameForLogs() + " (Fetcher)")) {} -MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( +std::pair Fetcher::fetchSelectedPart( const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const String & part_name, @@ -601,7 +601,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( return std::make_unique(full_path, std::min(DBMS_DEFAULT_BUFFER_SIZE, file_size)); }; - return downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix, disk, true, *in, output_buffer_getter, projections, throttler, sync); + return std::make_pair(downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix, disk, true, *in, output_buffer_getter, projections, throttler, sync), std::move(temporary_directory_lock)); } catch (const Exception & e) { @@ -667,11 +667,11 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( data.getRelativeDataPath(), part_name); - return downloadPartToMemory( + return std::make_pair(downloadPartToMemory( data_part_storage, part_name, MergeTreePartInfo::fromPartName(part_name, data.format_version), part_uuid, metadata_snapshot, context, *in, - projections, false, throttler); + projections, false, throttler), std::move(temporary_directory_lock)); } auto output_buffer_getter = [](IDataPartStorage & part_storage, const String & file_name, size_t file_size) @@ -679,10 +679,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( return part_storage.writeFile(file_name, std::min(file_size, DBMS_DEFAULT_BUFFER_SIZE), {}); }; - return downloadPartToDisk( + return std::make_pair(downloadPartToDisk( part_name, replica_path, to_detached, tmp_prefix, disk, false, *in, output_buffer_getter, - projections, throttler, sync); + projections, throttler, sync),std::move(temporary_directory_lock)); } MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index 704c1b0a226..07939a660a8 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -66,7 +66,7 @@ public: explicit Fetcher(StorageReplicatedMergeTree & data_); /// Downloads a part to tmp_directory. If to_detached - downloads to the `detached` directory. - MergeTreeData::MutableDataPartPtr fetchSelectedPart( + std::pair fetchSelectedPart( const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const String & part_name, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0f79e9f8f19..56896f88423 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2344,16 +2344,19 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) auto credentials = getContext()->getInterserverCredentials(); String interserver_scheme = getContext()->getInterserverScheme(); + scope_guard part_temp_directory_lock; if (interserver_scheme != address.scheme) throw Exception(ErrorCodes::LOGICAL_ERROR, "Interserver schemas are different '{}' != '{}', can't fetch part from {}", interserver_scheme, address.scheme, address.host); - part_desc->res_part = fetcher.fetchSelectedPart( + auto [fetched_part, lock] = fetcher.fetchSelectedPart( metadata_snapshot, getContext(), part_desc->found_new_part_name, zookeeper_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, false, TMP_PREFIX + "fetch_"); + part_desc->res_part = fetched_part; + part_temp_directory_lock = std::move(lock); /// TODO: check columns_version of fetched part @@ -2460,6 +2463,7 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr auto timeouts = getHTTPTimeouts(getContext()); auto credentials = getContext()->getInterserverCredentials(); String interserver_scheme = getContext()->getInterserverScheme(); + scope_guard part_temp_directory_lock; auto get_part = [&, address, timeouts, credentials, interserver_scheme]() { @@ -2467,11 +2471,13 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr throw Exception(ErrorCodes::LOGICAL_ERROR, "Interserver schemes are different: '{}' != '{}', can't fetch part from {}", interserver_scheme, address.scheme, address.host); - return fetcher.fetchSelectedPart( + auto [fetched_part, lock] = fetcher.fetchSelectedPart( metadata_snapshot, getContext(), entry.new_part_name, zookeeper_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, true); + part_temp_directory_lock = std::move(lock); + return fetched_part; }; part = get_part(); @@ -4170,14 +4176,14 @@ bool StorageReplicatedMergeTree::fetchPart( std::optional tagger_ptr; std::function get_part; MergeTreeData::HardlinkedFiles hardlinked_files; - scope_guard part_to_clone_lock; + scope_guard part_directory_lock; if (part_to_clone) { get_part = [&, part_to_clone]() { auto [cloned_part, lock] = cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot, NO_TRANSACTION_PTR, &hardlinked_files, false, {}); - part_to_clone_lock = std::move(lock); + part_directory_lock = std::move(lock); return cloned_part; }; } @@ -4195,7 +4201,7 @@ bool StorageReplicatedMergeTree::fetchPart( throw Exception(ErrorCodes::INTERSERVER_SCHEME_DOESNT_MATCH, "Interserver schemes are different: " "'{}' != '{}', can't fetch part from {}", interserver_scheme, address.scheme, address.host); - return fetcher.fetchSelectedPart( + auto [fetched_part, lock] = fetcher.fetchSelectedPart( metadata_snapshot, getContext(), part_name, @@ -4212,6 +4218,8 @@ bool StorageReplicatedMergeTree::fetchPart( "", &tagger_ptr, try_fetch_shared); + part_directory_lock = std::move(lock); + return fetched_part; }; } @@ -4355,6 +4363,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( auto timeouts = getHTTPTimeouts(getContext()); auto credentials = getContext()->getInterserverCredentials(); String interserver_scheme = getContext()->getInterserverScheme(); + scope_guard part_temp_directory_lock; get_part = [&, address, timeouts, interserver_scheme, credentials]() { @@ -4362,12 +4371,14 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( throw Exception(ErrorCodes::INTERSERVER_SCHEME_DOESNT_MATCH, "Interserver schemes are different: " "'{}' != '{}', can't fetch part from {}", interserver_scheme, address.scheme, address.host); - return fetcher.fetchSelectedPart( + auto [fetched_part, lock] = fetcher.fetchSelectedPart( metadata_snapshot, getContext(), part_name, zookeeper_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, false, "", nullptr, true, replaced_disk); + part_temp_directory_lock = std::move(lock); + return fetched_part; }; try From a69aa3f901d24a083f1be3d90eaa927138617f19 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Tue, 30 May 2023 13:39:32 +0200 Subject: [PATCH 136/308] Update README.md --- README.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index bbedea364fc..9c933540e01 100644 --- a/README.md +++ b/README.md @@ -23,10 +23,9 @@ curl https://clickhouse.com/ | sh ## Upcoming Events * [**v23.5 Release Webinar**](https://clickhouse.com/company/events/v23-5-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-05) - May 31 - 23.5 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. -* [**ClickHouse Meetup in Barcelona**](https://www.meetup.com/clickhouse-barcelona-user-group/events/292892669) - May 25 -* [**ClickHouse Meetup in London**](https://www.meetup.com/clickhouse-london-user-group/events/292892824) - May 25 +* [**ClickHouse Meetup in Bangalore**](https://www.meetup.com/clickhouse-bangalore-user-group/events/293740066/) - Jun 7 * [**ClickHouse Meetup in San Francisco**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/293426725/) - Jun 7 -* [**ClickHouse Meetup in Stockholm**](https://www.meetup.com/clickhouse-berlin-user-group/events/292892466) - Jun 13 + Also, keep an eye out for upcoming meetups in Amsterdam, Boston, NYC, Beijing, and Toronto. Somewhere else you want us to be? Please feel free to reach out to tyler clickhouse com. From 1cbce2196849dfb3dc8189a0f4c117b725c22fe3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 May 2023 11:46:23 +0000 Subject: [PATCH 137/308] Better, pt. II --- src/Functions/FunctionsHashing.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index d08a900c88a..14fad62bd85 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1426,6 +1426,9 @@ public: { auto col_to = ColumnVector::create(input_rows_count); + if (input_rows_count == 0) + return col_to; + typename ColumnVector::Container & vec_to = col_to->getData(); /// If using a "keyed" algorithm, the first argument is the key and @@ -1441,7 +1444,7 @@ public: KeyType key{}; if constexpr (Keyed) - if (!arguments.empty() && input_rows_count != 0) + if (!arguments.empty()) key = Impl::parseKey(arguments[0]); /// The function supports arbitrary number of arguments of arbitrary types. From bec8186b2f23aa0f7c4d0f714fce8e7f2b871fae Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 May 2023 13:59:31 +0200 Subject: [PATCH 138/308] do not fix the event list --- .../test_profile_events_s3/test.py | 62 ++----------------- 1 file changed, 6 insertions(+), 56 deletions(-) diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index 5c56b5c05cb..10c9385f865 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -31,61 +31,8 @@ def cluster(): cluster.shutdown() -init_list = { - "ReadBufferFromS3Bytes": 0, - "ReadBufferFromS3Microseconds": 0, - "ReadBufferFromS3InitMicroseconds": 0, - "ReadBufferFromS3RequestsErrors": 0, - "WriteBufferFromS3Bytes": 0, - "WriteBufferFromS3Microseconds": 0, - "WriteBufferFromS3RequestsErrors": 0, - "S3ReadMicroseconds": 0, - "S3ReadRequestsCount": 0, - "S3ReadRequestsErrorsTotal": 0, - "S3ReadRequestsErrors503": 0, - "S3ReadRequestsRedirects": 0, - "S3WriteMicroseconds": 0, - "S3WriteRequestsCount": 0, - "S3WriteRequestsErrorsTotal": 0, - "S3WriteRequestsErrors503": 0, - "S3WriteRequestsRedirects": 0, - "DiskS3ReadMicroseconds": 0, - "DiskS3ReadRequestsCount": 0, - "DiskS3ReadRequestsErrorsTotal": 0, - "DiskS3ReadRequestsErrors503": 0, - "DiskS3ReadRequestsRedirects": 0, - "DiskS3WriteMicroseconds": 0, - "DiskS3WriteRequestsCount": 0, - "DiskS3WriteRequestsErrorsTotal": 0, - "DiskS3WriteRequestsErrors503": 0, - "DiskS3WriteRequestsRedirects": 0, - "S3DeleteObjects": 0, - "S3CopyObject": 0, - "S3ListObjects": 0, - "S3HeadObject": 0, - "S3CreateMultipartUpload": 0, - "S3UploadPartCopy": 0, - "S3UploadPart": 0, - "S3AbortMultipartUpload": 0, - "S3CompleteMultipartUpload": 0, - "S3PutObject": 0, - "S3GetObject": 0, - "DiskS3DeleteObjects": 0, - "DiskS3CopyObject": 0, - "DiskS3ListObjects": 0, - "DiskS3HeadObject": 0, - "DiskS3CreateMultipartUpload": 0, - "DiskS3UploadPartCopy": 0, - "DiskS3UploadPart": 0, - "DiskS3AbortMultipartUpload": 0, - "DiskS3CompleteMultipartUpload": 0, - "DiskS3PutObject": 0, - "DiskS3GetObject": 0, -} - - def get_s3_events(instance): - result = init_list.copy() + result = dict() events = instance.query( "SELECT event, value FROM system.events WHERE event LIKE '%S3%'" ).split("\n") @@ -130,7 +77,7 @@ def get_minio_stat(cluster): def get_query_stat(instance, hint): - result = init_list.copy() + result = dict() instance.query("SYSTEM FLUSH LOGS") events = instance.query( """ @@ -146,7 +93,10 @@ def get_query_stat(instance, hint): ev = event.split("\t") if len(ev) == 2: if "S3" in ev[0]: - result[ev[0]] += int(ev[1]) + if ev[0] in result: + result[ev[0]] += int(ev[1]) + else: + result[ev[0]] = int(ev[1]) return result From c87e8c1f23bb88ac981ff59405e4f9a829e4fce6 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 30 May 2023 12:12:54 +0000 Subject: [PATCH 139/308] Fix bitShift* functions with both constant arguments --- src/Functions/FunctionBinaryArithmetic.h | 2 +- ...66_bitshift_with_const_arguments.reference | 7 ++++++ .../02766_bitshift_with_const_arguments.sql | 22 +++++++++++++++++++ 3 files changed, 30 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02766_bitshift_with_const_arguments.reference create mode 100644 tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index ec608883111..b205822aab5 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1741,7 +1741,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A OpImpl::template processString(in_vec.data(), col_left->getOffsets().data(), &value, out_vec, out_offsets, 1); } - return ColumnConst::create(std::move(col_res), col_left->size()); + return ColumnConst::create(std::move(col_res), col_left_const->size()); } else if (!col_left_const && !col_right_const && col_right) { diff --git a/tests/queries/0_stateless/02766_bitshift_with_const_arguments.reference b/tests/queries/0_stateless/02766_bitshift_with_const_arguments.reference new file mode 100644 index 00000000000..9a05aeb71aa --- /dev/null +++ b/tests/queries/0_stateless/02766_bitshift_with_const_arguments.reference @@ -0,0 +1,7 @@ +\0hdhdfhp +\0hdhdfhp +\0hdhdfhp +\0hdhdfhp +\0hdhdfhp +\0bdf +13000 diff --git a/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql b/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql new file mode 100644 index 00000000000..6b2961f0555 --- /dev/null +++ b/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql @@ -0,0 +1,22 @@ +SELECT bitShiftLeft(if(number = NULL, '14342', '4242348'), 1) FROM numbers(1); +SELECT bitShiftLeft(if(number = NULL, '14342', '4242348'), 1) FROM numbers(3); +SELECT bitShiftLeft(if(materialize(NULL), '14342', '4242348'), 1) FROM numbers(1); +SELECT bitShiftLeft(if(materialize(1), '123', '123'), 1) from numbers(1); + + +-- The next queries are from fuzzer that found the bug: +DROP TABLE IF EXISTS t0; +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); +DROP TABLE t0; +DROP TABLE t1; + +DROP TABLE IF EXISTS t5; +CREATE TABLE t5 (vkey UInt32, pkey UInt32, c18 Float32, c19 UInt32) ENGINE = Log; +INSERT INTO t5 VALUES (3, 13000, 73.90, 83); +SELECT subq_0.pkey as c_1_c1193_15 FROM t5 AS subq_0 WHERE sipHash128(0, subq_0.c18, bitShiftRight(case when false then (sipHash128(subq_0.pkey, subq_0.c18, 'S')) else '1' end, 0)) is not null; +DROP TABLE t5; + From 391e61549d59d338d475732704023092e34faf4a Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 30 May 2023 12:18:56 +0000 Subject: [PATCH 140/308] Fixes --- src/Parsers/ExpressionListParsers.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 8903b2b02b0..18a6de5b4f6 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2332,8 +2332,8 @@ const std::vector> ParserExpressionImpl::o {":", Operator("if", 3, 3, OperatorType::FinishIf)}, {"OR", Operator("or", 3, 2, OperatorType::Mergeable)}, {"AND", Operator("and", 4, 2, OperatorType::Mergeable)}, - {"IS NULL", Operator("isNull", 6, 1, OperatorType::IsNull)}, - {"IS NOT NULL", Operator("isNotNull", 6, 1, OperatorType::IsNull)}, + {"IS NULL", Operator("isNull", 6, 1, OperatorType::IsNull)}, + {"IS NOT NULL", Operator("isNotNull", 6, 1, OperatorType::IsNull)}, {"BETWEEN", Operator("", 7, 0, OperatorType::StartBetween)}, {"NOT BETWEEN", Operator("", 7, 0, OperatorType::StartNotBetween)}, {"==", Operator("equals", 9, 2, OperatorType::Comparison)}, @@ -2353,7 +2353,7 @@ const std::vector> ParserExpressionImpl::o {"NOT IN", Operator("notIn", 9, 2)}, {"GLOBAL IN", Operator("globalIn", 9, 2)}, {"GLOBAL NOT IN", Operator("globalNotIn", 9, 2)}, - {"||", Operator("concat", 10, 2, OperatorType::Mergeable)}, + {"||", Operator("concat", 10, 2, OperatorType::Mergeable)}, {"+", Operator("plus", 11, 2)}, {"-", Operator("minus", 11, 2)}, {"*", Operator("multiply", 12, 2)}, @@ -2372,7 +2372,7 @@ const std::vector> ParserExpressionImpl::u {"-", Operator("negate", 13, 1)} }; -const Operator ParserExpressionImpl::finish_between_operator("", 7, 0, OperatorType::FinishBetween); +const Operator ParserExpressionImpl::finish_between_operator("", 8, 0, OperatorType::FinishBetween); const std::array ParserExpressionImpl::overlapping_operators_to_skip { @@ -2392,6 +2392,7 @@ bool ParserExpressionImpl::parse(std::unique_ptr start, IParser::Pos & po { if (!layers.back()->parse(pos, expected, next)) break; + if (layers.back()->isFinished()) { if (layers.size() == 1) From 092b06f32eb7e4e8b0c4f379632d560810393299 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 30 May 2023 14:36:47 +0200 Subject: [PATCH 141/308] Fix special build --- src/Common/checkSSLReturnCode.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/checkSSLReturnCode.cpp b/src/Common/checkSSLReturnCode.cpp index bc87af1d37d..353c287813d 100644 --- a/src/Common/checkSSLReturnCode.cpp +++ b/src/Common/checkSSLReturnCode.cpp @@ -8,7 +8,7 @@ namespace DB { -bool checkSSLWantRead([[maybe_unused]] ssize_t res) +bool checkSSLWantRead([[maybe_unused]] ssize_t ret) { #if USE_SSL return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ; @@ -17,7 +17,7 @@ bool checkSSLWantRead([[maybe_unused]] ssize_t res) #endif } -bool checkSSLWantWrite([[maybe_unused]] ssize_t res) +bool checkSSLWantWrite([[maybe_unused]] ssize_t ret) { #if USE_SSL return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE; From a23af1b065e42f54a5c93ee96c3e4517d98611c2 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 30 May 2023 13:16:25 +0000 Subject: [PATCH 142/308] Integrate test 02763_last_day_of_week into existing test 00941_to_custom_week --- .../0_stateless/00941_to_custom_week.reference | 10 ++++++++++ .../0_stateless/00941_to_custom_week.sql | 18 ++++++++++++++++++ .../02763_last_day_of_week.reference | 10 ---------- .../0_stateless/02763_last_day_of_week.sql | 17 ----------------- 4 files changed, 28 insertions(+), 27 deletions(-) delete mode 100644 tests/queries/0_stateless/02763_last_day_of_week.reference delete mode 100644 tests/queries/0_stateless/02763_last_day_of_week.sql diff --git a/tests/queries/0_stateless/00941_to_custom_week.reference b/tests/queries/0_stateless/00941_to_custom_week.reference index 6171bc3937f..1650f955989 100644 --- a/tests/queries/0_stateless/00941_to_custom_week.reference +++ b/tests/queries/0_stateless/00941_to_custom_week.reference @@ -71,3 +71,13 @@ 2019-01-01 2019-01-01 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 2019-01-02 2019-01-02 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 2019-01-03 2019-01-03 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 +2018-12-25 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-26 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-27 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-28 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-29 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-30 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-31 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 +2019-01-01 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 +2019-01-02 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 +2019-01-03 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 diff --git a/tests/queries/0_stateless/00941_to_custom_week.sql b/tests/queries/0_stateless/00941_to_custom_week.sql index 4dd5d209306..58a81c03ef8 100644 --- a/tests/queries/0_stateless/00941_to_custom_week.sql +++ b/tests/queries/0_stateless/00941_to_custom_week.sql @@ -50,3 +50,21 @@ SELECT toStartOfWeek(x_t, 3) AS wt3 FROM numbers(10); +-- toLastDayOfWeek +WITH + toDate('2018-12-25') + number AS d, + toDate32(d) AS d32, + toDateTime(d) AS dt, + toDateTime64(d, 0) AS dt64 +SELECT + dt64, + toLastDayOfWeek(d) AS wd_sun, + toLastDayOfWeek(d32) AS wd32_sun, + toLastDayOfWeek(dt) AS wdt_sun, + toLastDayOfWeek(dt64) AS wdt64_sun, + toLastDayOfWeek(d, 1) AS wd_mon, + toLastDayOfWeek(d32, 1) AS wd32_mon, + toLastDayOfWeek(dt, 1) AS wdt_mon, + toLastDayOfWeek(dt64, 1) AS wdt64_mon +FROM numbers(10); + diff --git a/tests/queries/0_stateless/02763_last_day_of_week.reference b/tests/queries/0_stateless/02763_last_day_of_week.reference deleted file mode 100644 index 140ee9bfe24..00000000000 --- a/tests/queries/0_stateless/02763_last_day_of_week.reference +++ /dev/null @@ -1,10 +0,0 @@ -2018-12-25 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 -2018-12-26 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 -2018-12-27 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 -2018-12-28 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 -2018-12-29 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 -2018-12-30 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2018-12-30 2018-12-30 2018-12-30 2018-12-30 -2018-12-31 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 -2019-01-01 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 -2019-01-02 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 -2019-01-03 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 diff --git a/tests/queries/0_stateless/02763_last_day_of_week.sql b/tests/queries/0_stateless/02763_last_day_of_week.sql deleted file mode 100644 index 533ecfaa5b2..00000000000 --- a/tests/queries/0_stateless/02763_last_day_of_week.sql +++ /dev/null @@ -1,17 +0,0 @@ -WITH - toDate('2018-12-25') + number AS d, - toDate32(d) AS d32, - toDateTime(d) AS dt, - toDateTime64(d, 0) AS dt64 -SELECT - dt64, - toLastDayOfWeek(d) AS wd_0, - toLastDayOfWeek(d32) AS wd32_0, - toLastDayOfWeek(dt) AS wdt_0, - toLastDayOfWeek(dt64) AS wdt64_0, - toLastDayOfWeek(d, 3) AS wd_3, - toLastDayOfWeek(d32, 3) AS wd32_3, - toLastDayOfWeek(dt, 3) AS wdt_3, - toLastDayOfWeek(dt64, 3) AS wdt64_3 -FROM numbers(10); - From 1c904ecc8ef5886ec90221f52c02758a26a5b81e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 30 May 2023 16:25:17 +0200 Subject: [PATCH 143/308] Fix typo --- src/Common/checkSSLReturnCode.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/checkSSLReturnCode.cpp b/src/Common/checkSSLReturnCode.cpp index 353c287813d..ffb912da18a 100644 --- a/src/Common/checkSSLReturnCode.cpp +++ b/src/Common/checkSSLReturnCode.cpp @@ -11,7 +11,7 @@ namespace DB bool checkSSLWantRead([[maybe_unused]] ssize_t ret) { #if USE_SSL - return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ; + return ret == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ; #else return false; #endif @@ -20,7 +20,7 @@ bool checkSSLWantRead([[maybe_unused]] ssize_t ret) bool checkSSLWantWrite([[maybe_unused]] ssize_t ret) { #if USE_SSL - return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE; + return ret == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE; #else return false; #endif From bcd57078cebbadd00b3f51fd17aba2f67a571996 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 30 May 2023 16:36:54 +0200 Subject: [PATCH 144/308] Update the runner version --- tests/ci/worker/ubuntu_ami_for_ci.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/ubuntu_ami_for_ci.sh b/tests/ci/worker/ubuntu_ami_for_ci.sh index f8d1645a54d..22e5ada2932 100644 --- a/tests/ci/worker/ubuntu_ami_for_ci.sh +++ b/tests/ci/worker/ubuntu_ami_for_ci.sh @@ -3,7 +3,7 @@ set -xeuo pipefail echo "Running prepare script" export DEBIAN_FRONTEND=noninteractive -export RUNNER_VERSION=2.298.2 +export RUNNER_VERSION=2.304.0 export RUNNER_HOME=/home/ubuntu/actions-runner deb_arch() { From 40d658e467549d589116da449f2ac8edbfe54368 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 1 May 2023 15:09:26 +0000 Subject: [PATCH 145/308] Fix join_use_nulls in analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 81 +++- .../02722_matcher_join_use_nulls.reference | 430 ++++++++++++++++++ .../02722_matcher_join_use_nulls.sql.j2 | 119 +++++ 3 files changed, 612 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/02722_matcher_join_use_nulls.reference create mode 100644 tests/queries/0_stateless/02722_matcher_join_use_nulls.sql.j2 diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index aa915e48d35..36a85739339 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1205,6 +1205,29 @@ private: static std::string rewriteAggregateFunctionNameIfNeeded(const std::string & aggregate_function_name, const ContextPtr & context); + static std::optional getColumnSideFromJoinTree(QueryTreeNodePtr & resolved_identifier, const JoinNode & join_node) + { + const auto * column_src = resolved_identifier->as().getColumnSource().get(); + + if (join_node.getLeftTableExpression().get() == column_src) + return JoinTableSide::Left; + if (join_node.getRightTableExpression().get() == column_src) + return JoinTableSide::Right; + return {}; + } + + static void convertJoinedColumnTypeToNullIfNeeded(QueryTreeNodePtr & resolved_identifier, const JoinKind & join_kind, std::optional resolved_side) + { + if (resolved_identifier->getNodeType() == QueryTreeNodeType::COLUMN && + (isFull(join_kind) || + (isLeft(join_kind) && resolved_side && *resolved_side == JoinTableSide::Right) || + (isRight(join_kind) && resolved_side && *resolved_side == JoinTableSide::Left))) + { + auto & resolved_column = resolved_identifier->as(); + resolved_column.setColumnType(makeNullableOrLowCardinalityNullable(resolved_column.getColumnType())); + } + } + /// Resolve identifier functions static QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier, ContextPtr context); @@ -2982,6 +3005,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo QueryTreeNodePtr resolved_identifier; JoinKind join_kind = from_join_node.getKind(); + bool join_use_nulls = scope.context->getSettingsRef().join_use_nulls; if (left_resolved_identifier && right_resolved_identifier) { @@ -3027,19 +3051,31 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo * * Otherwise we prefer column from left table. */ - if (identifier_path_part == right_column_source_alias) - return right_resolved_identifier; - else if (!left_column_source_alias.empty() && - right_column_source_alias.empty() && - identifier_path_part != left_column_source_alias) - return right_resolved_identifier; + bool column_resolved_using_right_alias = identifier_path_part == right_column_source_alias; + bool column_resolved_without_using_left_alias = !left_column_source_alias.empty() + && right_column_source_alias.empty() + && identifier_path_part != left_column_source_alias; + if (column_resolved_using_right_alias || column_resolved_without_using_left_alias) + { + resolved_side = JoinTableSide::Right; + resolved_identifier = right_resolved_identifier; + } + else + { + resolved_side = JoinTableSide::Left; + resolved_identifier = left_resolved_identifier; + } + } + else + { + resolved_side = JoinTableSide::Left; + resolved_identifier = left_resolved_identifier; } - - return left_resolved_identifier; } else if (scope.joins_count == 1 && scope.context->getSettingsRef().single_join_prefer_left_table) { - return left_resolved_identifier; + resolved_side = JoinTableSide::Left; + resolved_identifier = left_resolved_identifier; } else { @@ -3092,17 +3128,10 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo if (join_node_in_resolve_process || !resolved_identifier) return resolved_identifier; - bool join_use_nulls = scope.context->getSettingsRef().join_use_nulls; - - if (join_use_nulls && - resolved_identifier->getNodeType() == QueryTreeNodeType::COLUMN && - (isFull(join_kind) || - (isLeft(join_kind) && resolved_side && *resolved_side == JoinTableSide::Right) || - (isRight(join_kind) && resolved_side && *resolved_side == JoinTableSide::Left))) + if (join_use_nulls) { resolved_identifier = resolved_identifier->clone(); - auto & resolved_column = resolved_identifier->as(); - resolved_column.setColumnType(makeNullableOrLowCardinalityNullable(resolved_column.getColumnType())); + convertJoinedColumnTypeToNullIfNeeded(resolved_identifier, join_kind, resolved_side); } return resolved_identifier; @@ -4001,6 +4030,22 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I else matched_expression_nodes_with_names = resolveUnqualifiedMatcher(matcher_node, scope); + if (scope.context->getSettingsRef().join_use_nulls) + { + const auto * nearest_query_scope = scope.getNearestQueryScope(); + const QueryNode * nearest_scope_query_node = nearest_query_scope ? nearest_query_scope->scope_node->as() : nullptr; + const QueryTreeNodePtr & nearest_scope_join_tree = nearest_scope_query_node ? nearest_scope_query_node->getJoinTree() : nullptr; + const JoinNode * nearest_scope_join_node = nearest_scope_join_tree ? nearest_scope_join_tree->as() : nullptr; + if (nearest_scope_join_node) + { + for (auto & [node, node_name] : matched_expression_nodes_with_names) + { + auto join_identifier_side = getColumnSideFromJoinTree(node, *nearest_scope_join_node); + convertJoinedColumnTypeToNullIfNeeded(node, nearest_scope_join_node->getKind(), join_identifier_side); + } + } + } + std::unordered_map> strict_transformer_to_used_column_names; for (const auto & transformer : matcher_node_typed.getColumnTransformers().getNodes()) { diff --git a/tests/queries/0_stateless/02722_matcher_join_use_nulls.reference b/tests/queries/0_stateless/02722_matcher_join_use_nulls.reference new file mode 100644 index 00000000000..746d02dc381 --- /dev/null +++ b/tests/queries/0_stateless/02722_matcher_join_use_nulls.reference @@ -0,0 +1,430 @@ +-- { echoOn } + +SELECT '============ LEFT JOIN ============' FORMAT Null; +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 \N Int32 Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +\N Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + LEFT JOIN t2 + ON t1.a = t2.a +) ORDER BY 1; +\N \N Nullable(UInt32) Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +1 Int64 +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +1 Int64 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +\N Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM t1 + LEFT JOIN t2 + USING (a) +) ORDER BY 1; +1 \N Int64 Nullable(UInt32) +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 \N Int32 Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +\N Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + LEFT JOIN t2 + ON t1.a = t2.key +) ORDER BY 1; +\N \N Nullable(UInt32) Nullable(UInt32) +SELECT '============ RIGHT JOIN ============' FORMAT Null; +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +\N 2 Nullable(Int32) UInt32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +2 UInt32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + RIGHT JOIN t2 + ON t1.a = t2.a +) ORDER BY 1; +2 2 UInt32 UInt32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +2 Int64 +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +2 Int64 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +2 Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +2 UInt32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM t1 + RIGHT JOIN t2 + USING (a) +) ORDER BY 1; +2 2 Int64 UInt32 +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +\N 2 Nullable(Int32) UInt32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +2 UInt32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + RIGHT JOIN t2 + ON t1.a = t2.key +) ORDER BY 1; +2 2 UInt32 UInt32 +SELECT '============ FULL JOIN ============' FORMAT Null; +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 Nullable(Int32) +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 Nullable(Int32) +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 \N Nullable(Int32) Nullable(UInt32) +\N 2 Nullable(Int32) Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 Nullable(Int32) +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +2 Nullable(UInt32) +\N Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + FULL JOIN t2 + ON t1.a = t2.a +) ORDER BY 1; +2 2 Nullable(UInt32) Nullable(UInt32) +\N \N Nullable(UInt32) Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +1 Nullable(Int64) +2 Nullable(Int64) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +1 Nullable(Int64) +2 Nullable(Int64) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +1 Nullable(Int32) +2 Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +2 Nullable(UInt32) +\N Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM t1 + FULL JOIN t2 + USING (a) +) ORDER BY 1; +1 \N Nullable(Int64) Nullable(UInt32) +2 2 Nullable(Int64) Nullable(UInt32) +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 Nullable(Int32) +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 Nullable(Int32) +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 \N Nullable(Int32) Nullable(UInt32) +\N 2 Nullable(Int32) Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 Nullable(Int32) +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +2 Nullable(UInt32) +\N Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + FULL JOIN t2 + ON t1.a = t2.key +) ORDER BY 1; +2 2 Nullable(UInt32) Nullable(UInt32) +\N \N Nullable(UInt32) Nullable(UInt32) diff --git a/tests/queries/0_stateless/02722_matcher_join_use_nulls.sql.j2 b/tests/queries/0_stateless/02722_matcher_join_use_nulls.sql.j2 new file mode 100644 index 00000000000..25451a34867 --- /dev/null +++ b/tests/queries/0_stateless/02722_matcher_join_use_nulls.sql.j2 @@ -0,0 +1,119 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (a Int32) ENGINE = TinyLog; +CREATE TABLE t2 (a UInt32, key UInt32) ENGINE = TinyLog; + +INSERT INTO t1 VALUES (1); +INSERT INTO t2 VALUES (2, 2); + +SET join_use_nulls = 1; +SET allow_experimental_analyzer = 1; + +-- { echoOn } + +{% for KIND in ('LEFT', 'RIGHT', 'FULL') -%} + +SELECT '============ {{ KIND }} JOIN ============' FORMAT Null; + +{% for right_column_name in ['a', 'key'] -%} + +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + ON t1.a = t2.{{ right_column_name }} +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + ON t1.a = t2.{{ right_column_name }} +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + ON t1.a = t2.{{ right_column_name }} +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + ON t1.a = t2.{{ right_column_name }} +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + ON t1.a = t2.{{ right_column_name }} +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + {{ KIND }} JOIN t2 + ON t1.a = t2.{{ right_column_name }} +) ORDER BY 1; + +{% if right_column_name == 'a' -%} + +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + USING (a) +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + USING (a) +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + USING (a) +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + USING (a) +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM t1 + {{ KIND }} JOIN t2 + USING (a) +) ORDER BY 1; + +{% endif -%} + +{% endfor -%} +{% endfor -%} + +-- { echoOff } + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; From 12141bb6cdd42e8a2f9f985549d1c970dbb0295f Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 May 2023 16:43:32 +0000 Subject: [PATCH 146/308] Check can become nullable before applying join_use_nulls --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 36a85739339..8bacdfd7bd2 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1219,6 +1219,7 @@ private: static void convertJoinedColumnTypeToNullIfNeeded(QueryTreeNodePtr & resolved_identifier, const JoinKind & join_kind, std::optional resolved_side) { if (resolved_identifier->getNodeType() == QueryTreeNodeType::COLUMN && + JoinCommon::canBecomeNullable(resolved_identifier->getResultType()) && (isFull(join_kind) || (isLeft(join_kind) && resolved_side && *resolved_side == JoinTableSide::Right) || (isRight(join_kind) && resolved_side && *resolved_side == JoinTableSide::Left))) From 6fb836a4b207aaa4cf174e03b33e8dd41b671952 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 10 May 2023 13:22:57 +0200 Subject: [PATCH 147/308] Update broken_tests.txt --- tests/broken_tests.txt | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index faee1c5b295..02935712325 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -21,8 +21,6 @@ 01072_optimize_skip_unused_shards_const_expr_eval 01083_expressions_in_engine_arguments 01086_odbc_roundtrip -01142_join_lc_and_nullable_in_key -01142_merge_join_lc_and_nullable_in_key 01152_cross_replication 01155_rename_move_materialized_view 01173_transaction_control_queries @@ -39,8 +37,6 @@ 01319_optimize_skip_unused_shards_nesting 01353_low_cardinality_join_types 01455_shard_leaf_max_rows_bytes_to_read -01476_right_full_join_switch -01477_lc_in_merge_join_left_key 01487_distributed_in_not_default_db 01495_subqueries_in_with_statement 01504_rocksdb From d72789369329661b15fc0b1f1ae1d24b8b2b545f Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 15 May 2023 16:40:46 +0000 Subject: [PATCH 148/308] Handle function nodes in getColumnSideFromJoinTree --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 28 ++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 8bacdfd7bd2..637f8a03a76 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1205,8 +1205,34 @@ private: static std::string rewriteAggregateFunctionNameIfNeeded(const std::string & aggregate_function_name, const ContextPtr & context); - static std::optional getColumnSideFromJoinTree(QueryTreeNodePtr & resolved_identifier, const JoinNode & join_node) + static std::optional getColumnSideFromJoinTree(const QueryTreeNodePtr & resolved_identifier, const JoinNode & join_node) { + if (resolved_identifier->getNodeType() == QueryTreeNodeType::CONSTANT) + return {}; + + if (resolved_identifier->getNodeType() == QueryTreeNodeType::FUNCTION) + { + const auto & resolved_function = resolved_identifier->as(); + + const auto & argument_nodes = resolved_function.getArguments().getNodes(); + + std::optional result; + for (const auto & argument_node : argument_nodes) + { + auto table_side = getColumnSideFromJoinTree(argument_node, join_node); + if (table_side && result && *table_side != *result) + { + throw Exception(ErrorCodes::AMBIGUOUS_IDENTIFIER, + "Ambiguous identifier {}. In scope {}", + resolved_identifier->formatASTForErrorMessage(), + join_node.formatASTForErrorMessage()); + } + if (table_side) + result = *table_side; + } + return result; + } + const auto * column_src = resolved_identifier->as().getColumnSource().get(); if (join_node.getLeftTableExpression().get() == column_src) From 783b54624b7122a591e48f72d69a03f2c6e87978 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 30 May 2023 16:48:56 +0000 Subject: [PATCH 149/308] add comment to join_use_nulls in QueryAnalyzer::resolveMatcher --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 637f8a03a76..b2bfa648435 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4059,6 +4059,11 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I if (scope.context->getSettingsRef().join_use_nulls) { + /** If we are resolving matcher came from the result of JOIN and `join_use_nulls` is set, + * we need to convert joined column type to Nullable. + * We are taking the nearest JoinNode to check to which table column belongs, + * because for LEFT/RIGHT join, we convert only the corresponding side. + */ const auto * nearest_query_scope = scope.getNearestQueryScope(); const QueryNode * nearest_scope_query_node = nearest_query_scope ? nearest_query_scope->scope_node->as() : nullptr; const QueryTreeNodePtr & nearest_scope_join_tree = nearest_scope_query_node ? nearest_scope_query_node->getJoinTree() : nullptr; From 1a6517d4a0904908dbc09fc5537c1bf08a49a5c9 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 30 May 2023 17:13:28 +0000 Subject: [PATCH 150/308] Fix priority of the operators after IS NULL --- src/Parsers/ExpressionListParsers.cpp | 12 ++++++++++-- .../0_stateless/02477_is_null_parser.reference | 4 ++-- tests/queries/0_stateless/02477_is_null_parser.sql | 4 ++-- 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 18a6de5b4f6..cd399531064 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2736,11 +2736,19 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po } } - layers.back()->pushOperator(op); - /// isNull & isNotNull are postfix unary operators if (op.type == OperatorType::IsNull) + { + ASTPtr function = makeASTFunction(op); + + if (!layers.back()->popLastNOperands(function->children[0]->children, 1)) + return Action::NONE; + + layers.back()->pushOperand(std::move(function)); return Action::OPERATOR; + } + + layers.back()->pushOperator(op); if (op.type == OperatorType::Cast) { diff --git a/tests/queries/0_stateless/02477_is_null_parser.reference b/tests/queries/0_stateless/02477_is_null_parser.reference index 57d96862011..2820f5ec2db 100644 --- a/tests/queries/0_stateless/02477_is_null_parser.reference +++ b/tests/queries/0_stateless/02477_is_null_parser.reference @@ -1,3 +1,3 @@ -SELECT (\'a\' IS NULL) + (\'b\' IS NOT NULL) -SELECT (\'a\' IS NULL) = 0 +SELECT ((1 IS NULL) + 1) IS NOT NULL +SELECT (1 IS NULL) = 0 SELECT CAST(1 IS NULL, \'Int32\') diff --git a/tests/queries/0_stateless/02477_is_null_parser.sql b/tests/queries/0_stateless/02477_is_null_parser.sql index b95a35fde21..f3ec0affd85 100644 --- a/tests/queries/0_stateless/02477_is_null_parser.sql +++ b/tests/queries/0_stateless/02477_is_null_parser.sql @@ -1,3 +1,3 @@ -EXPLAIN SYNTAX SELECT 'a' IS NULL + 'b' IS NOT NULL; -EXPLAIN SYNTAX SELECT 'a' IS NULL = 0; +EXPLAIN SYNTAX SELECT 1 IS NULL + 1 IS NOT NULL; +EXPLAIN SYNTAX SELECT 1 IS NULL = 0; EXPLAIN SYNTAX SELECT 1 IS NULL :: Int32; From 7fcafaabd515b9453b3ce7b5f5453e386d70ad67 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 May 2023 20:56:48 +0200 Subject: [PATCH 151/308] Fix --- .../IO/AsynchronousBoundedReadBuffer.cpp | 2 +- .../02770_async_buffer_ignore.reference | 1 + .../0_stateless/02770_async_buffer_ignore.sh | 29 +++++++++++++++++++ 3 files changed, 31 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02770_async_buffer_ignore.reference create mode 100755 tests/queries/0_stateless/02770_async_buffer_ignore.sh diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp index aed4c2f82f2..4aff69505a1 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -298,7 +298,7 @@ off_t AsynchronousBoundedReadBuffer::seek(off_t offset, int whence) * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. * Note: we read in range [file_offset_of_buffer_end, read_until_position). */ - if (read_until_position && new_pos < *read_until_position + if (file_offset_of_buffer_end && read_until_position && new_pos < *read_until_position && new_pos > file_offset_of_buffer_end && new_pos < file_offset_of_buffer_end + read_settings.remote_read_min_bytes_for_seek) { diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.reference b/tests/queries/0_stateless/02770_async_buffer_ignore.reference new file mode 100644 index 00000000000..9a9d6d6febf --- /dev/null +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.reference @@ -0,0 +1 @@ +4 66446 66446 diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.sh b/tests/queries/0_stateless/02770_async_buffer_ignore.sh new file mode 100755 index 00000000000..f8ccce2b5bb --- /dev/null +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test_optimize_exception" +${CLICKHOUSE_CLIENT} -nm --query " +DROP TABLE IF EXISTS test_s3; + +CREATE TABLE test_s3 (a UInt64, b UInt64) +ENGINE = MergeTree ORDER BY a +SETTINGS storage_policy = 's3', min_bytes_for_wide_part = 0; + +INSERT INTO test_s3 SELECT number, number FROM numbers(1000000); +" +query="SELECT sum(b) FROM test_s3 WHERE a >= 100000 AND a <= 102000" +query_id=$(${CLICKHOUSE_CLIENT} --query "select queryID() from ($query) limit 1" 2>&1) +${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -nm --query " +SELECT + ProfileEvents['S3ReadRequestsCount'], + ProfileEvents['ReadBufferFromS3Bytes'], + ProfileEvents['ReadCompressedBytes'] +FROM system.query_log +WHERE type = 'QueryFinish' + AND current_database = currentDatabase() + AND query_id='$query_id'; +" From 07467639a21ecad0e7744da898a6ff2682241f65 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 30 May 2023 21:00:15 +0200 Subject: [PATCH 152/308] Update 02770_async_buffer_ignore.sh --- tests/queries/0_stateless/02770_async_buffer_ignore.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.sh b/tests/queries/0_stateless/02770_async_buffer_ignore.sh index f8ccce2b5bb..779d5b6729e 100755 --- a/tests/queries/0_stateless/02770_async_buffer_ignore.sh +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.sh @@ -10,7 +10,7 @@ DROP TABLE IF EXISTS test_s3; CREATE TABLE test_s3 (a UInt64, b UInt64) ENGINE = MergeTree ORDER BY a -SETTINGS storage_policy = 's3', min_bytes_for_wide_part = 0; +SETTINGS disk = 's3_disk', min_bytes_for_wide_part = 0; INSERT INTO test_s3 SELECT number, number FROM numbers(1000000); " From a2c7c843464be20f48ecde46306b00992b27ce4c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 30 May 2023 23:37:27 +0200 Subject: [PATCH 153/308] Update 02770_async_buffer_ignore.sh --- tests/queries/0_stateless/02770_async_buffer_ignore.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.sh b/tests/queries/0_stateless/02770_async_buffer_ignore.sh index 779d5b6729e..ca8909a8d0d 100755 --- a/tests/queries/0_stateless/02770_async_buffer_ignore.sh +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.sh @@ -4,7 +4,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test_optimize_exception" ${CLICKHOUSE_CLIENT} -nm --query " DROP TABLE IF EXISTS test_s3; From db3a05091185284cdc60685c5ed077845f710303 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 31 May 2023 00:22:12 +0200 Subject: [PATCH 154/308] Seems like buildx needs explicit install since some time --- tests/ci/worker/ubuntu_ami_for_ci.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/ubuntu_ami_for_ci.sh b/tests/ci/worker/ubuntu_ami_for_ci.sh index 22e5ada2932..2bb8f01535f 100644 --- a/tests/ci/worker/ubuntu_ami_for_ci.sh +++ b/tests/ci/worker/ubuntu_ami_for_ci.sh @@ -56,7 +56,7 @@ echo "deb [arch=$(deb_arch) signed-by=/usr/share/keyrings/docker-archive-keyring apt-get update -apt-get install --yes --no-install-recommends docker-ce docker-ce-cli containerd.io +apt-get install --yes --no-install-recommends docker-ce docker-buildx-plugin docker-ce-cli containerd.io usermod -aG docker ubuntu From dbd3766f5f662e3338c30fd5408e4818598b9660 Mon Sep 17 00:00:00 2001 From: pufit Date: Sat, 27 May 2023 01:33:07 -0400 Subject: [PATCH 155/308] Specify roles in users.xml --- src/Access/UsersConfigAccessStorage.cpp | 159 +++++++++++++----- src/Parsers/Access/ParserGrantQuery.h | 2 +- .../configs/another_user.xml | 25 +++ .../test_user_grants_from_config/test.py | 18 +- 4 files changed, 163 insertions(+), 41 deletions(-) diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index df0e4584709..187258d0fcd 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -13,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -52,11 +54,64 @@ namespace UUID generateID(const IAccessEntity & entity) { return generateID(entity.getType(), entity.getName()); } + template + void parseGrant(T & entity, const String & string_query, const std::unordered_set & allowed_role_ids) + { + ParserGrantQuery parser; + parser.setParseWithoutGrantees(); + + String error_message; + const char * pos = string_query.data(); + auto ast = tryParseQuery(parser, pos, pos + string_query.size(), error_message, false, "", false, 0, 0); + + if (!ast) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse grant query. Error: {}", error_message); + + auto & query = ast->as(); + + if (query.roles && query.is_revoke) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Roles can't be revoked in config file"); + + if (!query.cluster.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Can't grant on cluster using config file"); + + if (query.grantees) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "You can't specify grantees in query using config file"); + + for (auto & element : query.access_rights_elements) + { + if (query.is_revoke) + entity.access.revoke(element); + else + entity.access.grant(element); + } + + if (query.roles) + { + std::vector roles_to_grant; + roles_to_grant.reserve(query.roles->size()); + + for (const auto & role_name : query.roles->names) + { + auto role_id = generateID(AccessEntityType::ROLE, role_name); + if (!allowed_role_ids.contains(role_id)) + throw Exception(ErrorCodes::THERE_IS_NO_PROFILE, "Role {} was not found", role_name); + + roles_to_grant.push_back(role_id); + } + + if (query.admin_option) + entity.granted_roles.grantWithAdminOption(roles_to_grant); + else + entity.granted_roles.grant(roles_to_grant); + } + } UserPtr parseUser( const Poco::Util::AbstractConfiguration & config, const String & user_name, const std::unordered_set & allowed_profile_ids, + const std::unordered_set & allowed_role_ids, bool allow_no_password, bool allow_plaintext_password) { @@ -241,37 +296,8 @@ namespace if (grant_queries) { - ParserGrantQuery parser; - parser.parseWithoutGrantees(); - for (const auto & string_query : *grant_queries) - { - String error_message; - const char * pos = string_query.data(); - auto ast = tryParseQuery(parser, pos, pos + string_query.size(), error_message, false, "", false, 0, 0); - - if (!ast) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse grant query. Error: {}", error_message); - - auto & query = ast->as(); - - if (query.roles) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Roles can't be granted in config file"); - - if (!query.cluster.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Can't grant on cluster using config file"); - - if (query.grantees) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "You can't specify grantees in query using config file"); - - for (auto & element : query.access_rights_elements) - { - if (query.is_revoke) - user->access.revoke(element); - else - user->access.grant(element); - } - } + parseGrant(*user, string_query, allowed_role_ids); } else { @@ -321,6 +347,7 @@ namespace std::vector parseUsers( const Poco::Util::AbstractConfiguration & config, const std::unordered_set & allowed_profile_ids, + const std::unordered_set & allowed_role_ids, bool allow_no_password, bool allow_plaintext_password) { @@ -333,7 +360,7 @@ namespace { try { - users.push_back(parseUser(config, user_name, allowed_profile_ids, allow_no_password, allow_plaintext_password)); + users.push_back(parseUser(config, user_name, allowed_profile_ids, allowed_role_ids, allow_no_password, allow_plaintext_password)); } catch (Exception & e) { @@ -345,6 +372,55 @@ namespace return users; } + RolePtr parseRole( + const Poco::Util::AbstractConfiguration & config, + const String & role_name, + const std::unordered_set & allowed_role_ids) + { + auto role = std::make_shared(); + role->setName(role_name); + String role_config = "roles." + role_name; + + const auto grants_config = role_config + ".grants"; + if (config.has(grants_config)) + { + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(grants_config, keys); + for (const auto & key : keys) + { + const auto query = config.getString(grants_config + "." + key); + parseGrant(*role, query, allowed_role_ids); + } + } + + return role; + } + + std::vector parseRoles( + const Poco::Util::AbstractConfiguration & config, + const std::unordered_set & allowed_role_ids) + { + Poco::Util::AbstractConfiguration::Keys role_names; + config.keys("roles", role_names); + + std::vector roles; + roles.reserve(role_names.size()); + for (const auto & role_name : role_names) + { + try + { + roles.push_back(parseRole(config, role_name, allowed_role_ids)); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while parsing roles '{}' in users configuration file", role_name)); + throw; + } + } + + return roles; + } + QuotaPtr parseQuota(const Poco::Util::AbstractConfiguration & config, const String & quota_name, const std::vector & user_ids) { @@ -635,14 +711,16 @@ namespace return profiles; } - - std::unordered_set getAllowedSettingsProfileIDs(const Poco::Util::AbstractConfiguration & config) + std::unordered_set getAllowedIDs( + const Poco::Util::AbstractConfiguration & config, + const String & configuration_key, + const AccessEntityType type) { - Poco::Util::AbstractConfiguration::Keys profile_names; - config.keys("profiles", profile_names); + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(configuration_key, keys); std::unordered_set ids; - for (const auto & profile_name : profile_names) - ids.emplace(generateID(AccessEntityType::SETTINGS_PROFILE, profile_name)); + for (const auto & key : keys) + ids.emplace(generateID(type, key)); return ids; } } @@ -693,12 +771,13 @@ void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfigu { try { - auto allowed_profile_ids = getAllowedSettingsProfileIDs(config); + auto allowed_profile_ids = getAllowedIDs(config, "profiles", AccessEntityType::SETTINGS_PROFILE); + auto allowed_role_ids = getAllowedIDs(config, "roles", AccessEntityType::ROLE); bool no_password_allowed = access_control.isNoPasswordAllowed(); bool plaintext_password_allowed = access_control.isPlaintextPasswordAllowed(); std::vector> all_entities; - for (const auto & entity : parseUsers(config, allowed_profile_ids, no_password_allowed, plaintext_password_allowed)) + for (const auto & entity : parseUsers(config, allowed_profile_ids, allowed_role_ids, no_password_allowed, plaintext_password_allowed)) all_entities.emplace_back(generateID(*entity), entity); for (const auto & entity : parseQuotas(config)) all_entities.emplace_back(generateID(*entity), entity); @@ -706,6 +785,8 @@ void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfigu all_entities.emplace_back(generateID(*entity), entity); for (const auto & entity : parseSettingsProfiles(config, allowed_profile_ids, access_control)) all_entities.emplace_back(generateID(*entity), entity); + for (const auto & entity : parseRoles(config, allowed_role_ids)) + all_entities.emplace_back(generateID(*entity), entity); memory_storage.setAll(all_entities); } catch (Exception & e) diff --git a/src/Parsers/Access/ParserGrantQuery.h b/src/Parsers/Access/ParserGrantQuery.h index 58c2be433d5..0ecfef916f5 100644 --- a/src/Parsers/Access/ParserGrantQuery.h +++ b/src/Parsers/Access/ParserGrantQuery.h @@ -14,7 +14,7 @@ class ParserGrantQuery : public IParserBase public: ParserGrantQuery & useAttachMode(bool attach_mode_ = true) { attach_mode = attach_mode_; return *this; } - ParserGrantQuery & parseWithoutGrantees(bool allow_no_grantees_ = true) { allow_no_grantees = allow_no_grantees_; return *this; } + ParserGrantQuery & setParseWithoutGrantees(bool allow_no_grantees_ = true) { allow_no_grantees = allow_no_grantees_; return *this; } protected: const char * getName() const override { return "GRANT or REVOKE query"; } diff --git a/tests/integration/test_user_grants_from_config/configs/another_user.xml b/tests/integration/test_user_grants_from_config/configs/another_user.xml index 16c026e81d0..0b0b2473142 100644 --- a/tests/integration/test_user_grants_from_config/configs/another_user.xml +++ b/tests/integration/test_user_grants_from_config/configs/another_user.xml @@ -14,5 +14,30 @@ REVOKE CREATE ON system.* + + + + ::/0 + + default + default + + GRANT admin_role + + + + + + GRANT SHOW ON *.* + REVOKE SHOW ON system.* + GRANT CREATE ON *.* WITH GRANT OPTION + + + + + GRANT ALL ON *.* WITH GRANT OPTION + + + diff --git a/tests/integration/test_user_grants_from_config/test.py b/tests/integration/test_user_grants_from_config/test.py index a4d5c0c904b..d2bd0b0facd 100644 --- a/tests/integration/test_user_grants_from_config/test.py +++ b/tests/integration/test_user_grants_from_config/test.py @@ -42,7 +42,7 @@ def test_allow_read_from_system_tables(): ) -def test_grants_from_config(): +def test_user_grants_from_config(): assert node.query("SHOW GRANTS FOR another") == TSV( [ "GRANT SHOW ON *.* TO another", @@ -51,3 +51,19 @@ def test_grants_from_config(): "REVOKE CREATE DATABASE, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY ON system.* FROM another", ] ) + + assert node.query("SHOW GRANTS FOR admin_user") == TSV( + [ + "GRANT admin_role TO admin_user", + ] + ) + + +def test_role_grants_from_config(): + assert node.query("SHOW GRANTS FOR test_role") == TSV( + [ + "GRANT SHOW ON *.* TO test_role", + "GRANT CREATE ON *.* TO test_role WITH GRANT OPTION", + "REVOKE SHOW ON system.* FROM test_role", + ] + ) From 28ee307e75490840c85b16e934e6789e7569707a Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 31 May 2023 06:21:23 +0000 Subject: [PATCH 156/308] Integrate toLastDayOfWeek() to 01921_datatype_date32 test --- tests/queries/0_stateless/01921_datatype_date32.reference | 6 ++++++ tests/queries/0_stateless/01921_datatype_date32.sql | 2 ++ 2 files changed, 8 insertions(+) diff --git a/tests/queries/0_stateless/01921_datatype_date32.reference b/tests/queries/0_stateless/01921_datatype_date32.reference index 14079b906cb..cc3dba12069 100644 --- a/tests/queries/0_stateless/01921_datatype_date32.reference +++ b/tests/queries/0_stateless/01921_datatype_date32.reference @@ -84,6 +84,12 @@ 2120-07-05 2120-07-26 2021-06-20 +-------toLastDayOfWeek--------- +2079-06-12 +2079-06-12 +2120-07-11 +2120-08-01 +2021-06-26 -------toStartOfMonth--------- 2079-06-07 2079-06-07 diff --git a/tests/queries/0_stateless/01921_datatype_date32.sql b/tests/queries/0_stateless/01921_datatype_date32.sql index 8b65f82825f..717afc483aa 100644 --- a/tests/queries/0_stateless/01921_datatype_date32.sql +++ b/tests/queries/0_stateless/01921_datatype_date32.sql @@ -36,6 +36,8 @@ select '-------toYearWeek---------'; select toYearWeek(x1) from t1; select '-------toStartOfWeek---------'; select toStartOfWeek(x1) from t1; +select '-------toLastDayOfWeek---------'; +select toLastDayOfWeek(x1) from t1; select '-------toStartOfMonth---------'; select toStartOfMonth(x1) from t1; select '-------toStartOfQuarter---------'; From dfd7d8b59eda5f2f1b7230641a5b3b9409f5eda6 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 31 May 2023 07:19:12 +0000 Subject: [PATCH 157/308] Align toStartOfWeek() and toLastDayOfWeek() test cases in 01921_datatype_date32.sql test --- .../00941_to_custom_week.reference | 20 +++++++++---------- .../0_stateless/00941_to_custom_week.sql | 20 +++++++++++++------ 2 files changed, 24 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/00941_to_custom_week.reference b/tests/queries/0_stateless/00941_to_custom_week.reference index 1650f955989..48be1dbb072 100644 --- a/tests/queries/0_stateless/00941_to_custom_week.reference +++ b/tests/queries/0_stateless/00941_to_custom_week.reference @@ -61,16 +61,16 @@ 2017-01-09 00:00:00 2 3 201702 201703 2017-01-10 00:00:00 2 3 201702 201703 2017-01-11 00:00:00 2 3 201702 201703 -2018-12-25 2018-12-25 00:00:00 2018-12-23 2018-12-23 2018-12-24 2018-12-24 -2018-12-26 2018-12-26 00:00:00 2018-12-23 2018-12-23 2018-12-24 2018-12-24 -2018-12-27 2018-12-27 00:00:00 2018-12-23 2018-12-23 2018-12-24 2018-12-24 -2018-12-28 2018-12-28 00:00:00 2018-12-23 2018-12-23 2018-12-24 2018-12-24 -2018-12-29 2018-12-29 00:00:00 2018-12-23 2018-12-23 2018-12-24 2018-12-24 -2018-12-30 2018-12-30 00:00:00 2018-12-30 2018-12-30 2018-12-24 2018-12-24 -2018-12-31 2018-12-31 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 -2019-01-01 2019-01-01 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 -2019-01-02 2019-01-02 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 -2019-01-03 2019-01-03 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 +2018-12-25 00:00:00 2018-12-23 2018-12-23 2018-12-23 2018-12-23 2018-12-24 2018-12-24 2018-12-24 2018-12-24 +2018-12-26 00:00:00 2018-12-23 2018-12-23 2018-12-23 2018-12-23 2018-12-24 2018-12-24 2018-12-24 2018-12-24 +2018-12-27 00:00:00 2018-12-23 2018-12-23 2018-12-23 2018-12-23 2018-12-24 2018-12-24 2018-12-24 2018-12-24 +2018-12-28 00:00:00 2018-12-23 2018-12-23 2018-12-23 2018-12-23 2018-12-24 2018-12-24 2018-12-24 2018-12-24 +2018-12-29 00:00:00 2018-12-23 2018-12-23 2018-12-23 2018-12-23 2018-12-24 2018-12-24 2018-12-24 2018-12-24 +2018-12-30 00:00:00 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-24 2018-12-24 2018-12-24 2018-12-24 +2018-12-31 00:00:00 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-31 2018-12-31 2018-12-31 2018-12-31 +2019-01-01 00:00:00 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-31 2018-12-31 2018-12-31 2018-12-31 +2019-01-02 00:00:00 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-31 2018-12-31 2018-12-31 2018-12-31 +2019-01-03 00:00:00 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-31 2018-12-31 2018-12-31 2018-12-31 2018-12-25 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-26 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-27 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 diff --git a/tests/queries/0_stateless/00941_to_custom_week.sql b/tests/queries/0_stateless/00941_to_custom_week.sql index 58a81c03ef8..3281ed23fe7 100644 --- a/tests/queries/0_stateless/00941_to_custom_week.sql +++ b/tests/queries/0_stateless/00941_to_custom_week.sql @@ -41,13 +41,21 @@ SELECT toDateTime(toDate('2016-12-22') + number, 'Asia/Istanbul' ) AS d, FROM numbers(21); -- toStartOfWeek +WITH + toDate('2018-12-25') + number AS d, + toDate32(d) AS d32, + toDateTime(d) AS dt, + toDateTime64(d, 0) AS dt64 SELECT - toDate('2018-12-25') + number AS x, - toDateTime(x) AS x_t, - toStartOfWeek(x) AS w0, - toStartOfWeek(x_t) AS wt0, - toStartOfWeek(x, 3) AS w3, - toStartOfWeek(x_t, 3) AS wt3 + dt64, + toStartOfWeek(d) AS wd_sun, + toStartOfWeek(d32) AS wd32_sun, + toStartOfWeek(dt) AS wdt_sun, + toStartOfWeek(dt64) AS wdt64_sun, + toStartOfWeek(d, 1) AS wd_mon, + toStartOfWeek(d32, 1) AS wd32_mon, + toStartOfWeek(dt, 1) AS wdt_mon, + toStartOfWeek(dt64, 1) AS wdt64_mon FROM numbers(10); -- toLastDayOfWeek From 65586c50f500dd5daa51de60087c54acfdf5f914 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Wed, 31 May 2023 00:21:13 -0700 Subject: [PATCH 158/308] Minor improvements in CGroup awareness. 1. Support CGroup2 in getMemoryAmountOrZero(). 2. Report CFS period and quota in asynchronous metric log. --- base/base/getMemoryAmount.cpp | 22 +++++++++--- src/Common/AsynchronousMetrics.cpp | 56 ++++++++++++++++++++++++++++-- src/Common/AsynchronousMetrics.h | 3 ++ 3 files changed, 75 insertions(+), 6 deletions(-) diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index 9e1d2ac3279..6a5470a0549 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -28,14 +28,28 @@ uint64_t getMemoryAmountOrZero() #if defined(OS_LINUX) // Try to lookup at the Cgroup limit - std::ifstream cgroup_limit("/sys/fs/cgroup/memory/memory.limit_in_bytes"); - if (cgroup_limit.is_open()) + + // v2 + std::ifstream cgroupv2_limit("/sys/fs/cgroup/memory.max"); + if (cgroupv2_limit.is_open()) { - uint64_t memory_limit = 0; // in case of read error - cgroup_limit >> memory_limit; + uint64_t memory_limit = 0; + cgroupv2_limit >> memory_limit; if (memory_limit > 0 && memory_limit < memory_amount) memory_amount = memory_limit; } + else + { + // v1 + std::ifstream cgroup_limit("/sys/fs/cgroup/memory/memory.limit_in_bytes"); + if (cgroup_limit.is_open()) + { + uint64_t memory_limit = 0; // in case of read error + cgroup_limit >> memory_limit; + if (memory_limit > 0 && memory_limit < memory_amount) + memory_amount = memory_limit; + } + } #endif return memory_amount; diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index ac2180103c5..e1e99a3f7c7 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -69,13 +69,23 @@ AsynchronousMetrics::AsynchronousMetrics( /// CGroups v2 openFileIfExists("/sys/fs/cgroup/memory.max", cgroupmem_limit_in_bytes); - openFileIfExists("/sys/fs/cgroup/memory.current", cgroupmem_usage_in_bytes); + if (cgroupmem_limit_in_bytes) + { + openFileIfExists("/sys/fs/cgroup/memory.current", cgroupmem_usage_in_bytes); + } + openFileIfExists("/sys/fs/cgroup/cpu.max", cgroupcpu_max); /// CGroups v1 if (!cgroupmem_limit_in_bytes) + { openFileIfExists("/sys/fs/cgroup/memory/memory.limit_in_bytes", cgroupmem_limit_in_bytes); - if (!cgroupmem_usage_in_bytes) openFileIfExists("/sys/fs/cgroup/memory/memory.usage_in_bytes", cgroupmem_usage_in_bytes); + } + if (!cgroupcpu_max) + { + openFileIfExists("/sys/fs/cgroup/cpu/cpu.cfs_period_us", cgroupcpu_cfs_period); + openFileIfExists("/sys/fs/cgroup/cpu/cpu.cfs_quota_us", cgroupcpu_cfs_quota); + } openSensors(); openBlockDevices(); @@ -926,6 +936,48 @@ void AsynchronousMetrics::update(TimePoint update_time) tryLogCurrentException(__PRETTY_FUNCTION__); } } + + if (cgroupcpu_max) + { + try { + cgroupcpu_max->rewind(); + + uint64_t quota = 0; + uint64_t period = 0; + + readText(quota, *cgroupcpu_max); + skipWhitespaceIfAny(*cgroupcpu_max); + readText(period, *cgroupcpu_max); + + new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."}; + new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup."}; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + else if (cgroupcpu_cfs_quota && cgroupcpu_cfs_period) + { + try { + cgroupcpu_cfs_quota->rewind(); + cgroupcpu_cfs_period->rewind(); + + uint64_t quota = 0; + uint64_t period = 0; + + tryReadText(quota, *cgroupcpu_cfs_quota); + tryReadText(period, *cgroupcpu_cfs_period); + + new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."}; + new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup."}; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + if (meminfo) { try diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index d104b872f52..e3b5142553b 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -110,6 +110,9 @@ private: std::optional cgroupmem_limit_in_bytes; std::optional cgroupmem_usage_in_bytes; + std::optional cgroupcpu_cfs_period; + std::optional cgroupcpu_cfs_quota; + std::optional cgroupcpu_max; std::vector> thermal; From 45e2fb4b3eefca48b27e90d23147660d3da46e39 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Wed, 31 May 2023 10:58:05 +0200 Subject: [PATCH 159/308] Update README.md changing release webinar date --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 9c933540e01..9561458ba37 100644 --- a/README.md +++ b/README.md @@ -22,7 +22,7 @@ curl https://clickhouse.com/ | sh ## Upcoming Events -* [**v23.5 Release Webinar**](https://clickhouse.com/company/events/v23-5-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-05) - May 31 - 23.5 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. +* [**v23.5 Release Webinar**](https://clickhouse.com/company/events/v23-5-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-05) - Jun 8 - 23.5 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. * [**ClickHouse Meetup in Bangalore**](https://www.meetup.com/clickhouse-bangalore-user-group/events/293740066/) - Jun 7 * [**ClickHouse Meetup in San Francisco**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/293426725/) - Jun 7 From cd4c4e316991ad2d13cd41ee8feea3e6cc65322f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 31 May 2023 11:57:51 +0200 Subject: [PATCH 160/308] Update 02770_async_buffer_ignore.sh --- tests/queries/0_stateless/02770_async_buffer_ignore.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.sh b/tests/queries/0_stateless/02770_async_buffer_ignore.sh index ca8909a8d0d..22b432db178 100755 --- a/tests/queries/0_stateless/02770_async_buffer_ignore.sh +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 8ce56dfe4616dcaf2638ffc8f1ef7c35718fea06 Mon Sep 17 00:00:00 2001 From: lihaibo42 Date: Wed, 31 May 2023 17:53:00 +0800 Subject: [PATCH 161/308] Link boost::context library to clickhouse_common_io --- src/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 622e18d4ff7..e1359a5a8aa 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -346,6 +346,7 @@ target_link_libraries(clickhouse_common_io PUBLIC boost::program_options boost::system + boost::context ch_contrib::cityhash ch_contrib::re2 ch_contrib::re2_st From 08d38878326fb9a2793e708008268a6eff000749 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 31 May 2023 07:04:29 -0400 Subject: [PATCH 162/308] Add re-creation for cherry-pick PRs (#50373) * Add recreation for cherry-pick PRs. Small PR comment fix. * Automatic style fix --------- Co-authored-by: robot-clickhouse Co-authored-by: Nikita Mikhaylov --- tests/ci/cherry_pick.py | 24 +++++++++++++++++++----- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 2fa562a1386..d36315151aa 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -70,9 +70,12 @@ This pull-request will be merged automatically as it reaches the mergeable state ### If the PR was closed and then reopened -If it stuck, check {pr_url} for `{label_backports_created}` and delete it if \ +If it stuck, check {pr_url} for `{backport_created_label}` and delete it if \ necessary. Manually merging will do nothing, since `{label_backports_created}` \ prevents the original PR {pr_url} from being processed. + +If you want to recreate the PR: delete the `{label_cherrypick}` label and delete this branch. +You may also need to delete the `{label_backports_created}` label from the original PR. """ BACKPORT_DESCRIPTION = """This pull-request is a last step of an automated \ backporting. @@ -82,7 +85,13 @@ close it. """ REMOTE = "" - def __init__(self, name: str, pr: PullRequest, repo: Repository): + def __init__( + self, + name: str, + pr: PullRequest, + repo: Repository, + backport_created_label: str = Labels.BACKPORTS_CREATED, + ): self.name = name self.pr = pr self.repo = repo @@ -93,6 +102,8 @@ close it. self.backport_pr = None # type: Optional[PullRequest] self._backported = False + self.backport_created_label = backport_created_label + self.git_prefix = ( # All commits to cherrypick are done as robot-clickhouse "git -c user.email=robot-clickhouse@users.noreply.github.com " "-c user.name=robot-clickhouse -c commit.gpgsign=false" @@ -226,7 +237,8 @@ close it. body=self.CHERRYPICK_DESCRIPTION.format( pr_number=self.pr.number, pr_url=self.pr.html_url, - label_backports_created=Labels.BACKPORTS_CREATED, + backport_created_label=self.backport_created_label, + label_cherrypick=Labels.CHERRYPICK, ), base=self.backport_branch, head=self.cherrypick_branch, @@ -459,11 +471,12 @@ class Backport: pr_labels = [label.name for label in pr.labels] if self.must_create_backport_label in pr_labels: branches = [ - ReleaseBranch(br, pr, self.repo) for br in self.release_branches + ReleaseBranch(br, pr, self.repo, self.backport_created_label) + for br in self.release_branches ] # type: List[ReleaseBranch] else: branches = [ - ReleaseBranch(br, pr, self.repo) + ReleaseBranch(br, pr, self.repo, self.backport_created_label) for br in [ label.split("-", 1)[0][1:] # v21.8-must-backport for label in pr_labels @@ -492,6 +505,7 @@ class Backport: ) bp_cp_prs = self.gh.get_pulls_from_search( query=f"type:pr repo:{self._repo_name} {query_suffix}", + label=f"{Labels.BACKPORT},{Labels.CHERRYPICK}", ) for br in branches: br.pop_prs(bp_cp_prs) From 6d45d0c37404f4d3a7bd03c92e6a2f04adef7beb Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 31 May 2023 13:11:10 +0200 Subject: [PATCH 163/308] Use fingerprints instead of key IDs in encrypted disks (#49882) * Use fingerprints instead of key IDs to find keys in encrypted disks. Always use little endian in the headers of encryption files. * Add tests. * Fix copying binary files to test containers. * Fix ownership for copied files in test containers. * Add comments after review. --------- Co-authored-by: Nikita Mikhaylov --- src/Backups/tests/gtest_backup_entries.cpp | 11 +- src/Disks/DiskEncrypted.cpp | 251 +++++++++++----- src/Disks/DiskEncryptedTransaction.cpp | 43 +-- src/Disks/DiskEncryptedTransaction.h | 8 +- src/Disks/tests/gtest_disk_encrypted.cpp | 8 +- src/IO/FileEncryptionCommon.cpp | 108 +++++-- src/IO/FileEncryptionCommon.h | 42 ++- src/IO/tests/gtest_file_encryption.cpp | 3 +- tests/integration/helpers/cluster.py | 5 +- .../old_versions/version_1be/__marks.mrk | Bin 0 -> 96 bytes .../old_versions/version_1be/data.bin | Bin 0 -> 99 bytes .../old_versions/version_1be/id.bin | Bin 0 -> 102 bytes .../old_versions/version_1be/sizes.json | Bin 0 -> 162 bytes .../old_versions/version_1le/__marks.mrk | Bin 0 -> 96 bytes .../old_versions/version_1le/data.bin | Bin 0 -> 99 bytes .../old_versions/version_1le/id.bin | Bin 0 -> 102 bytes .../old_versions/version_1le/sizes.json | Bin 0 -> 162 bytes .../old_versions/version_2/__marks.mrk | Bin 0 -> 96 bytes .../old_versions/version_2/data.bin | Bin 0 -> 99 bytes .../old_versions/version_2/id.bin | Bin 0 -> 102 bytes .../old_versions/version_2/sizes.json | Bin 0 -> 162 bytes tests/integration/test_encrypted_disk/test.py | 284 ++++++++++++++---- 22 files changed, 543 insertions(+), 220 deletions(-) create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1be/__marks.mrk create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1be/data.bin create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1be/id.bin create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1be/sizes.json create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1le/__marks.mrk create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1le/data.bin create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1le/id.bin create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1le/sizes.json create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_2/__marks.mrk create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_2/data.bin create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_2/id.bin create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_2/sizes.json diff --git a/src/Backups/tests/gtest_backup_entries.cpp b/src/Backups/tests/gtest_backup_entries.cpp index 3077bedad0e..ca603d20787 100644 --- a/src/Backups/tests/gtest_backup_entries.cpp +++ b/src/Backups/tests/gtest_backup_entries.cpp @@ -29,10 +29,15 @@ protected: /// Make encrypted disk. auto settings = std::make_unique(); settings->wrapped_disk = local_disk; - settings->current_algorithm = FileEncryption::Algorithm::AES_128_CTR; - settings->keys[0] = "1234567890123456"; - settings->current_key_id = 0; settings->disk_path = "encrypted/"; + + settings->current_algorithm = FileEncryption::Algorithm::AES_128_CTR; + String key = "1234567890123456"; + UInt128 fingerprint = FileEncryption::calculateKeyFingerprint(key); + settings->all_keys[fingerprint] = key; + settings->current_key = key; + settings->current_key_fingerprint = fingerprint; + encrypted_disk = std::make_shared("encrypted_disk", std::move(settings), true); } diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 2415b432e01..6b515b100c9 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -19,7 +19,6 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int INCORRECT_DISK_INDEX; - extern const int DATA_ENCRYPTION_ERROR; extern const int NOT_IMPLEMENTED; } @@ -42,87 +41,201 @@ namespace } } + /// Reads encryption keys from the configuration. + void getKeysFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, + std::map & out_keys_by_id, Strings & out_keys_without_id) + { + Strings config_keys; + config.keys(config_prefix, config_keys); + + for (const std::string & config_key : config_keys) + { + String key; + std::optional key_id; + + if ((config_key == "key") || config_key.starts_with("key[")) + { + String key_path = config_prefix + "." + config_key; + key = config.getString(key_path); + String key_id_path = key_path + "[@id]"; + if (config.has(key_id_path)) + key_id = config.getUInt64(key_id_path); + } + else if ((config_key == "key_hex") || config_key.starts_with("key_hex[")) + { + String key_path = config_prefix + "." + config_key; + key = unhexKey(config.getString(key_path)); + String key_id_path = key_path + "[@id]"; + if (config.has(key_id_path)) + key_id = config.getUInt64(key_id_path); + } + else + continue; + + if (key_id) + { + if (!out_keys_by_id.contains(*key_id)) + out_keys_by_id[*key_id] = key; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple keys specified for same ID {}", *key_id); + } + else + out_keys_without_id.push_back(key); + } + + if (out_keys_by_id.empty() && out_keys_without_id.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No encryption keys found"); + + if (out_keys_by_id.empty() && (out_keys_without_id.size() == 1)) + { + out_keys_by_id[0] = out_keys_without_id.front(); + out_keys_without_id.clear(); + } + } + + /// Reads the current encryption key from the configuration. + String getCurrentKeyFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, + const std::map & keys_by_id, const Strings & keys_without_id) + { + String key_path = config_prefix + ".current_key"; + String key_hex_path = config_prefix + ".current_key_hex"; + String key_id_path = config_prefix + ".current_key_id"; + + if (config.has(key_path) + config.has(key_hex_path) + config.has(key_id_path) > 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The current key is specified multiple times"); + + auto check_current_key_found = [&](const String & current_key_) + { + for (const auto & [_, key] : keys_by_id) + { + if (key == current_key_) + return; + } + for (const auto & key : keys_without_id) + { + if (key == current_key_) + return; + } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The current key is not found in keys"); + }; + + if (config.has(key_path)) + { + String current_key = config.getString(key_path); + check_current_key_found(current_key); + return current_key; + } + else if (config.has(key_hex_path)) + { + String current_key = unhexKey(config.getString(key_hex_path)); + check_current_key_found(current_key); + return current_key; + } + else if (config.has(key_id_path)) + { + UInt64 current_key_id = config.getUInt64(key_id_path); + auto it = keys_by_id.find(current_key_id); + if (it == keys_by_id.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found a key with the current ID {}", current_key_id); + return it->second; + } + else if (keys_by_id.size() == 1 && keys_without_id.empty() && keys_by_id.begin()->first == 0) + { + /// There is only a single key defined with id=0, so we can choose it as current. + return keys_by_id.begin()->second; + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The current key is not specified"); + } + } + + /// Reads the current encryption algorithm from the configuration. + Algorithm getCurrentAlgorithmFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) + { + String path = config_prefix + ".algorithm"; + if (!config.has(path)) + return DEFAULT_ENCRYPTION_ALGORITHM; + return parseAlgorithmFromString(config.getString(path)); + } + + /// Reads the name of a wrapped disk & the path on the wrapped disk and then finds that disk in a disk map. + void getDiskAndPathFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const DisksMap & map, + DiskPtr & out_disk, String & out_path) + { + String disk_name = config.getString(config_prefix + ".disk", ""); + if (disk_name.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Name of the wrapped disk must not be empty. Encrypted disk is a wrapper over another disk"); + + auto disk_it = map.find(disk_name); + if (disk_it == map.end()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "The wrapped disk must have been announced earlier. No disk with name {}", disk_name); + + out_disk = disk_it->second; + + out_path = config.getString(config_prefix + ".path", ""); + if (!out_path.empty() && (out_path.back() != '/')) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk path must ends with '/', but '{}' doesn't.", quoteString(out_path)); + } + + /// Parses the settings of an ecnrypted disk from the configuration. std::unique_ptr parseDiskEncryptedSettings( - const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const DisksMap & map) + const String & disk_name, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + const DisksMap & disk_map) { try { auto res = std::make_unique(); - res->current_algorithm = DEFAULT_ENCRYPTION_ALGORITHM; - if (config.has(config_prefix + ".algorithm")) - parseFromString(res->current_algorithm, config.getString(config_prefix + ".algorithm")); - Strings config_keys; - config.keys(config_prefix, config_keys); - for (const std::string & config_key : config_keys) + std::map keys_by_id; + Strings keys_without_id; + getKeysFromConfig(config, config_prefix, keys_by_id, keys_without_id); + + for (const auto & [key_id, key] : keys_by_id) { - String key; - UInt64 key_id; + auto fingerprint = calculateKeyFingerprint(key); + res->all_keys[fingerprint] = key; - if ((config_key == "key") || config_key.starts_with("key[")) - { - key = config.getString(config_prefix + "." + config_key, ""); - key_id = config.getUInt64(config_prefix + "." + config_key + "[@id]", 0); - } - else if ((config_key == "key_hex") || config_key.starts_with("key_hex[")) - { - key = unhexKey(config.getString(config_prefix + "." + config_key, "")); - key_id = config.getUInt64(config_prefix + "." + config_key + "[@id]", 0); - } - else - continue; - - if (res->keys.contains(key_id)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple keys have the same ID {}", key_id); - res->keys[key_id] = key; + /// Version 1 used key fingerprints based on the key id. + /// We have to add such fingerprints to the map too to support reading files encrypted by version 1. + auto v1_fingerprint = calculateV1KeyFingerprint(key, key_id); + res->all_keys[v1_fingerprint] = key; } - if (res->keys.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "No keys, an encrypted disk needs keys to work"); - - if (!config.has(config_prefix + ".current_key_id")) + for (const auto & key : keys_without_id) { - /// In case of multiple keys, current_key_id is mandatory - if (res->keys.size() > 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "There are multiple keys in config. current_key_id is required"); - - /// If there is only one key with non zero ID, curren_key_id should be defined. - if (res->keys.size() == 1 && !res->keys.contains(0)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config has one key with non zero id. сurrent_key_id is required"); + auto fingerprint = calculateKeyFingerprint(key); + res->all_keys[fingerprint] = key; } - res->current_key_id = config.getUInt64(config_prefix + ".current_key_id", 0); - if (!res->keys.contains(res->current_key_id)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found a key with the current ID {}", res->current_key_id); - FileEncryption::checkKeySize(res->current_algorithm, res->keys[res->current_key_id].size()); + String current_key = getCurrentKeyFromConfig(config, config_prefix, keys_by_id, keys_without_id); + res->current_key = current_key; + res->current_key_fingerprint = calculateKeyFingerprint(current_key); - String wrapped_disk_name = config.getString(config_prefix + ".disk", ""); - if (wrapped_disk_name.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Name of the wrapped disk must not be empty. Encrypted disk is a wrapper over another disk"); + res->current_algorithm = getCurrentAlgorithmFromConfig(config, config_prefix); - auto wrapped_disk_it = map.find(wrapped_disk_name); - if (wrapped_disk_it == map.end()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "The wrapped disk must have been announced earlier. No disk with name {}", - wrapped_disk_name); - res->wrapped_disk = wrapped_disk_it->second; + FileEncryption::checkKeySize(res->current_key.size(), res->current_algorithm); - res->disk_path = config.getString(config_prefix + ".path", ""); - if (!res->disk_path.empty() && (res->disk_path.back() != '/')) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk path must ends with '/', but '{}' doesn't.", quoteString(res->disk_path)); + DiskPtr wrapped_disk; + String disk_path; + getDiskAndPathFromConfig(config, config_prefix, disk_map, wrapped_disk, disk_path); + res->wrapped_disk = wrapped_disk; + res->disk_path = disk_path; return res; } catch (Exception & e) { - e.addMessage("Disk " + name); + e.addMessage("Disk " + disk_name); throw; } } + /// Reads the header of an encrypted file. FileEncryption::Header readHeader(ReadBufferFromFileBase & read_buffer) { try @@ -138,24 +251,6 @@ namespace } } - String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings) - { - auto it = settings.keys.find(header.key_id); - if (it == settings.keys.end()) - throw Exception( - ErrorCodes::DATA_ENCRYPTION_ERROR, - "Not found a key with ID {} required to decipher file {}", - header.key_id, - quoteString(path)); - - String key = it->second; - if (calculateKeyHash(key) != header.key_hash) - throw Exception( - ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong key with ID {}, could not decipher file {}", header.key_id, quoteString(path)); - - return key; - } - bool inline isSameDiskType(const IDisk & one, const IDisk & another) { return typeid(one) == typeid(another); @@ -225,7 +320,7 @@ void DiskEncrypted::copy(const String & from_path, const std::shared_ptr { auto from_settings = current_settings.get(); auto to_settings = to_disk_enc->current_settings.get(); - if (from_settings->keys == to_settings->keys) + if (from_settings->all_keys == to_settings->all_keys) { /// Keys are the same so we can simply copy the encrypted file. auto wrapped_from_path = wrappedPath(from_path); @@ -252,7 +347,7 @@ void DiskEncrypted::copyDirectoryContent(const String & from_dir, const std::sha { auto from_settings = current_settings.get(); auto to_settings = to_disk_enc->current_settings.get(); - if (from_settings->keys == to_settings->keys) + if (from_settings->all_keys == to_settings->all_keys) { /// Keys are the same so we can simply copy the encrypted file. auto wrapped_from_path = wrappedPath(from_dir); @@ -293,7 +388,7 @@ std::unique_ptr DiskEncrypted::readFile( } auto encryption_settings = current_settings.get(); FileEncryption::Header header = readHeader(*buffer); - String key = getKey(path, header, *encryption_settings); + String key = encryption_settings->findKeyByFingerprint(header.key_fingerprint, path); return std::make_unique(settings.local_fs_buffer_size, std::move(buffer), key, header); } diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 4a613374ccf..40df94b309a 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -38,39 +38,21 @@ FileEncryption::Header readHeader(ReadBufferFromFileBase & read_buffer) } } -String getCurrentKey(const String & path, const DiskEncryptedSettings & settings) +} + +String DiskEncryptedSettings::findKeyByFingerprint(UInt128 key_fingerprint, const String & path_for_logs) const { - auto it = settings.keys.find(settings.current_key_id); - if (it == settings.keys.end()) + auto it = all_keys.find(key_fingerprint); + if (it == all_keys.end()) + { throw Exception( ErrorCodes::DATA_ENCRYPTION_ERROR, - "Not found a key with the current ID {} required to cipher file {}", - settings.current_key_id, - quoteString(path)); - + "Not found an encryption key required to decipher file {}", + quoteString(path_for_logs)); + } return it->second; } -String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings) -{ - auto it = settings.keys.find(header.key_id); - if (it == settings.keys.end()) - throw Exception( - ErrorCodes::DATA_ENCRYPTION_ERROR, - "Not found a key with ID {} required to decipher file {}", - header.key_id, - quoteString(path)); - - String key = it->second; - if (FileEncryption::calculateKeyHash(key) != header.key_hash) - throw Exception( - ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong key with ID {}, could not decipher file {}", header.key_id, quoteString(path)); - - return key; -} - -} - void DiskEncryptedTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path) { auto wrapped_from_path = wrappedPath(from_file_path); @@ -98,16 +80,15 @@ std::unique_ptr DiskEncryptedTransaction::writeFile( // /// Append mode: we continue to use the same header. auto read_buffer = delegate_disk->readFile(wrapped_path, ReadSettings().adjustBufferSize(FileEncryption::Header::kSize)); header = readHeader(*read_buffer); - key = getKey(path, header, current_settings); + key = current_settings.findKeyByFingerprint(header.key_fingerprint, path); } } if (!old_file_size) { /// Rewrite mode: we generate a new header. - key = getCurrentKey(path, current_settings); header.algorithm = current_settings.current_algorithm; - header.key_id = current_settings.current_key_id; - header.key_hash = FileEncryption::calculateKeyHash(key); + key = current_settings.current_key; + header.key_fingerprint = current_settings.current_key_fingerprint; header.init_vector = FileEncryption::InitVector::random(); } auto buffer = delegate_transaction->writeFile(wrapped_path, buf_size, mode, settings, autocommit); diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index bae3f2c728c..04cc63f1671 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -18,9 +18,13 @@ struct DiskEncryptedSettings { DiskPtr wrapped_disk; String disk_path; - std::unordered_map keys; - UInt64 current_key_id; + String current_key; + UInt128 current_key_fingerprint; FileEncryption::Algorithm current_algorithm; + std::unordered_map all_keys; + + /// Returns an encryption key found by its fingerprint. + String findKeyByFingerprint(UInt128 key_fingerprint, const String & path_for_logs) const; }; diff --git a/src/Disks/tests/gtest_disk_encrypted.cpp b/src/Disks/tests/gtest_disk_encrypted.cpp index 80a10e8680b..ee9e284d409 100644 --- a/src/Disks/tests/gtest_disk_encrypted.cpp +++ b/src/Disks/tests/gtest_disk_encrypted.cpp @@ -37,8 +37,10 @@ protected: auto settings = std::make_unique(); settings->wrapped_disk = local_disk; settings->current_algorithm = algorithm; - settings->keys[0] = key; - settings->current_key_id = 0; + auto fingerprint = FileEncryption::calculateKeyFingerprint(key); + settings->all_keys[fingerprint] = key; + settings->current_key = key; + settings->current_key_fingerprint = fingerprint; settings->disk_path = path; encrypted_disk = std::make_shared("encrypted_disk", std::move(settings), true); } @@ -255,7 +257,7 @@ TEST_F(DiskEncryptedTest, RandomIV) String bina = getBinaryRepresentation(getDirectory() + "a.txt"); String binb = getBinaryRepresentation(getDirectory() + "b.txt"); - constexpr size_t iv_offset = 16; + constexpr size_t iv_offset = 23; /// See the description of the format in the comment for FileEncryption::Header. constexpr size_t iv_size = FileEncryption::InitVector::kSize; EXPECT_EQ(bina.substr(0, iv_offset), binb.substr(0, iv_offset)); /// Part of the header before IV is the same. EXPECT_NE(bina.substr(iv_offset, iv_size), binb.substr(iv_offset, iv_size)); /// IV differs. diff --git a/src/IO/FileEncryptionCommon.cpp b/src/IO/FileEncryptionCommon.cpp index 4ac4d289b32..5529c813c40 100644 --- a/src/IO/FileEncryptionCommon.cpp +++ b/src/IO/FileEncryptionCommon.cpp @@ -34,6 +34,7 @@ namespace case Algorithm::AES_128_CTR: return EVP_aes_128_ctr(); case Algorithm::AES_192_CTR: return EVP_aes_192_ctr(); case Algorithm::AES_256_CTR: return EVP_aes_256_ctr(); + case Algorithm::MAX: break; } throw Exception( ErrorCodes::BAD_ARGUMENTS, @@ -187,10 +188,14 @@ namespace return plaintext_size; } - constexpr const char kHeaderSignature[] = "ENC"; - constexpr const UInt16 kHeaderCurrentVersion = 1; -} + constexpr const std::string_view kHeaderSignature = "ENC"; + UInt128 calculateV1KeyFingerprint(UInt8 small_key_hash, UInt64 key_id) + { + /// In the version 1 we stored {key_id, very_small_hash(key)} instead of a fingerprint. + return static_cast(key_id) | (static_cast(small_key_hash) << 64); + } +} String toString(Algorithm algorithm) { @@ -199,6 +204,7 @@ String toString(Algorithm algorithm) case Algorithm::AES_128_CTR: return "aes_128_ctr"; case Algorithm::AES_192_CTR: return "aes_192_ctr"; case Algorithm::AES_256_CTR: return "aes_256_ctr"; + case Algorithm::MAX: break; } throw Exception( ErrorCodes::BAD_ARGUMENTS, @@ -206,14 +212,14 @@ String toString(Algorithm algorithm) static_cast(algorithm)); } -void parseFromString(Algorithm & algorithm, const String & str) +Algorithm parseAlgorithmFromString(const String & str) { if (boost::iequals(str, "aes_128_ctr")) - algorithm = Algorithm::AES_128_CTR; + return Algorithm::AES_128_CTR; else if (boost::iequals(str, "aes_192_ctr")) - algorithm = Algorithm::AES_192_CTR; + return Algorithm::AES_192_CTR; else if (boost::iequals(str, "aes_256_ctr")) - algorithm = Algorithm::AES_256_CTR; + return Algorithm::AES_256_CTR; else throw Exception( ErrorCodes::BAD_ARGUMENTS, @@ -221,7 +227,7 @@ void parseFromString(Algorithm & algorithm, const String & str) str); } -void checkKeySize(Algorithm algorithm, size_t key_size) { checkKeySize(getCipher(algorithm), key_size); } +void checkKeySize(size_t key_size, Algorithm algorithm) { checkKeySize(getCipher(algorithm), key_size); } String InitVector::toString() const @@ -364,54 +370,92 @@ void Encryptor::decrypt(const char * data, size_t size, char * out) void Header::read(ReadBuffer & in) { - constexpr size_t header_signature_size = std::size(kHeaderSignature) - 1; - char signature[std::size(kHeaderSignature)] = {}; - in.readStrict(signature, header_signature_size); - if (strcmp(signature, kHeaderSignature) != 0) + char signature[kHeaderSignature.length()]; + in.readStrict(signature, kHeaderSignature.length()); + if (memcmp(signature, kHeaderSignature.data(), kHeaderSignature.length()) != 0) throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong signature, this is not an encrypted file"); - UInt16 version; - readPODBinary(version, in); - if (version != kHeaderCurrentVersion) + /// The endianness of how the header is written. + /// Starting from version 2 the header is always in little endian. + std::endian endian = std::endian::little; + + readBinaryLittleEndian(version, in); + + if (version == 0x0100ULL) + { + /// Version 1 could write the header of an encrypted file in either little-endian or big-endian. + /// So now if we read the version as little-endian and it's 256 that means two things: the version is actually 1 and the whole header is in big endian. + endian = std::endian::big; + version = 1; + } + + if (version < 1 || version > kCurrentVersion) throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Version {} of the header is not supported", version); UInt16 algorithm_u16; readPODBinary(algorithm_u16, in); + if (std::endian::native != endian) + algorithm_u16 = std::byteswap(algorithm_u16); + if (algorithm_u16 >= static_cast(Algorithm::MAX)) + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Algorithm {} is not supported", algorithm_u16); algorithm = static_cast(algorithm_u16); - readPODBinary(key_id, in); - readPODBinary(key_hash, in); + size_t bytes_to_skip = kSize - kHeaderSignature.length() - sizeof(version) - sizeof(algorithm_u16) - InitVector::kSize; + + if (version < 2) + { + UInt64 key_id; + UInt8 small_key_hash; + readPODBinary(key_id, in); + readPODBinary(small_key_hash, in); + bytes_to_skip -= sizeof(key_id) + sizeof(small_key_hash); + if (std::endian::native != endian) + key_id = std::byteswap(key_id); + key_fingerprint = calculateV1KeyFingerprint(small_key_hash, key_id); + } + else + { + readBinaryLittleEndian(key_fingerprint, in); + bytes_to_skip -= sizeof(key_fingerprint); + } + init_vector.read(in); - constexpr size_t reserved_size = kSize - header_signature_size - sizeof(version) - sizeof(algorithm_u16) - sizeof(key_id) - sizeof(key_hash) - InitVector::kSize; - static_assert(reserved_size < kSize); - in.ignore(reserved_size); + chassert(bytes_to_skip < kSize); + in.ignore(bytes_to_skip); } void Header::write(WriteBuffer & out) const { - constexpr size_t header_signature_size = std::size(kHeaderSignature) - 1; - out.write(kHeaderSignature, header_signature_size); + writeString(kHeaderSignature, out); - UInt16 version = kHeaderCurrentVersion; - writePODBinary(version, out); + writeBinaryLittleEndian(version, out); UInt16 algorithm_u16 = static_cast(algorithm); - writePODBinary(algorithm_u16, out); + writeBinaryLittleEndian(algorithm_u16, out); + + writeBinaryLittleEndian(key_fingerprint, out); - writePODBinary(key_id, out); - writePODBinary(key_hash, out); init_vector.write(out); - constexpr size_t reserved_size = kSize - header_signature_size - sizeof(version) - sizeof(algorithm_u16) - sizeof(key_id) - sizeof(key_hash) - InitVector::kSize; + constexpr size_t reserved_size = kSize - kHeaderSignature.length() - sizeof(version) - sizeof(algorithm_u16) - sizeof(key_fingerprint) - InitVector::kSize; static_assert(reserved_size < kSize); - char reserved_zero_bytes[reserved_size] = {}; - out.write(reserved_zero_bytes, reserved_size); + char zero_bytes[reserved_size] = {}; + out.write(zero_bytes, reserved_size); } -UInt8 calculateKeyHash(const String & key) +UInt128 calculateKeyFingerprint(const String & key) { - return static_cast(sipHash64(key.data(), key.size())) & 0x0F; + const UInt64 seed0 = 0x4368456E63727970ULL; // ChEncryp + const UInt64 seed1 = 0x7465644469736B46ULL; // tedDiskF + return sipHash128Keyed(seed0, seed1, key.data(), key.size()); +} + +UInt128 calculateV1KeyFingerprint(const String & key, UInt64 key_id) +{ + /// In the version 1 we stored {key_id, very_small_hash(key)} instead of a fingerprint. + UInt8 small_key_hash = sipHash64(key.data(), key.size()) & 0x0F; + return calculateV1KeyFingerprint(small_key_hash, key_id); } } diff --git a/src/IO/FileEncryptionCommon.h b/src/IO/FileEncryptionCommon.h index efc0194da52..87aa1194273 100644 --- a/src/IO/FileEncryptionCommon.h +++ b/src/IO/FileEncryptionCommon.h @@ -23,13 +23,14 @@ enum class Algorithm AES_128_CTR, /// Size of key is 16 bytes. AES_192_CTR, /// Size of key is 24 bytes. AES_256_CTR, /// Size of key is 32 bytes. + MAX }; String toString(Algorithm algorithm); -void parseFromString(Algorithm & algorithm, const String & str); +Algorithm parseAlgorithmFromString(const String & str); /// Throws an exception if a specified key size doesn't correspond a specified encryption algorithm. -void checkKeySize(Algorithm algorithm, size_t key_size); +void checkKeySize(size_t key_size, Algorithm algorithm); /// Initialization vector. Its size is always 16 bytes. @@ -103,15 +104,34 @@ private: /// File header which is stored at the beginning of encrypted files. +/// +/// The format of that header is following: +/// +--------+------+--------------------------------------------------------------------------+ +/// | offset | size | description | +/// +--------+------+--------------------------------------------------------------------------+ +/// | 0 | 3 | 'E', 'N', 'C' (file's signature) | +/// | 3 | 2 | version of this header (1..2) | +/// | 5 | 2 | encryption algorithm (0..2, 0=AES_128_CTR, 1=AES_192_CTR, 2=AES_256_CTR) | +/// | 7 | 16 | fingerprint of encryption key (SipHash) | +/// | 23 | 16 | initialization vector (randomly generated) | +/// | 39 | 25 | reserved for future use | +/// +--------+------+--------------------------------------------------------------------------+ +/// struct Header { + /// Versions: + /// 1 - Initial version + /// 2 - The header of an encrypted file contains the fingerprint of a used encryption key instead of a pair {key_id, very_small_hash(key)}. + /// The header is always stored in little endian. + static constexpr const UInt16 kCurrentVersion = 2; + + UInt16 version = kCurrentVersion; + + /// Encryption algorithm. Algorithm algorithm = Algorithm::AES_128_CTR; - /// Identifier of the key to encrypt or decrypt this file. - UInt64 key_id = 0; - - /// Hash of the key to encrypt or decrypt this file. - UInt8 key_hash = 0; + /// Fingerprint of a key. + UInt128 key_fingerprint = 0; InitVector init_vector; @@ -122,9 +142,11 @@ struct Header void write(WriteBuffer & out) const; }; -/// Calculates the hash of a passed key. -/// 1 byte is enough because this hash is used only for the first check. -UInt8 calculateKeyHash(const String & key); +/// Calculates the fingerprint of a passed encryption key. +UInt128 calculateKeyFingerprint(const String & key); + +/// Calculates kind of the fingerprint of a passed encryption key & key ID as it was implemented in version 1. +UInt128 calculateV1KeyFingerprint(const String & key, UInt64 key_id); } } diff --git a/src/IO/tests/gtest_file_encryption.cpp b/src/IO/tests/gtest_file_encryption.cpp index 6a090ff0810..2b3d7ce81c5 100644 --- a/src/IO/tests/gtest_file_encryption.cpp +++ b/src/IO/tests/gtest_file_encryption.cpp @@ -226,8 +226,7 @@ TEST(FileEncryptionPositionUpdateTest, Decryption) String key = "1234567812345678"; FileEncryption::Header header; header.algorithm = Algorithm::AES_128_CTR; - header.key_id = 1; - header.key_hash = calculateKeyHash(key); + header.key_fingerprint = calculateKeyFingerprint(key); header.init_vector = InitVector::random(); auto lwb = std::make_unique(tmp_path); diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 950663cb429..f57ebf40e54 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1963,9 +1963,9 @@ class ClickHouseCluster: return output def copy_file_to_container(self, container_id, local_path, dest_path): - with open(local_path, "r") as fdata: + with open(local_path, "rb") as fdata: data = fdata.read() - encodedBytes = base64.b64encode(data.encode("utf-8")) + encodedBytes = base64.b64encode(data) encodedStr = str(encodedBytes, "utf-8") self.exec_in_container( container_id, @@ -1974,7 +1974,6 @@ class ClickHouseCluster: "-c", "echo {} | base64 --decode > {}".format(encodedStr, dest_path), ], - user="root", ) def wait_for_url( diff --git a/tests/integration/test_encrypted_disk/old_versions/version_1be/__marks.mrk b/tests/integration/test_encrypted_disk/old_versions/version_1be/__marks.mrk new file mode 100644 index 0000000000000000000000000000000000000000..88ad4b6cf3a1b43e3a27eff0696429c94bee8597 GIT binary patch literal 96 zcmZ?ub7o*F)A02 literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/old_versions/version_1be/data.bin b/tests/integration/test_encrypted_disk/old_versions/version_1be/data.bin new file mode 100644 index 0000000000000000000000000000000000000000..f82db60e077eec4aec1ff5e7885e71e4fb67587c GIT binary patch literal 99 zcmZ?ub7o*n;quo=<- literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/old_versions/version_1be/id.bin b/tests/integration/test_encrypted_disk/old_versions/version_1be/id.bin new file mode 100644 index 0000000000000000000000000000000000000000..0c2426599a7cdafed1b0835df4962a6b4be8b0d2 GIT binary patch literal 102 zcmZ?ub7o*1(U%=%e;^La?0M$MLunY#jeu5XR2z7 YJrWis`8aqU?pvBM-}Zcv!kO9M0LYjcNB{r; literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/old_versions/version_1be/sizes.json b/tests/integration/test_encrypted_disk/old_versions/version_1be/sizes.json new file mode 100644 index 0000000000000000000000000000000000000000..6d610f2da837e016e0dd58162e3fa47b3fdd8723 GIT binary patch literal 162 zcmZ?ub7o*(<1jvv&|4{XdzVvOGW)@xdOrrw;qjl6pl*Dbh_wq@spu#6KD7ua3}on$qCxAl-@ l_Vn!wPt8i^(JEOzCwNxJMXuhz7YoltPj!>n7vP&`4FHC@I$i(( literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/old_versions/version_1le/__marks.mrk b/tests/integration/test_encrypted_disk/old_versions/version_1le/__marks.mrk new file mode 100644 index 0000000000000000000000000000000000000000..919c95234e5c54930c013a7f7f602b8a82d40841 GIT binary patch literal 96 zcmZ?ub7o{TRIz3A{p@GOcv~$@NmHm+R`0m20NkJvON8<9y_ m|1Z5Bz;phefdB6v(eAx#0#kB11CA=HobP^ZT^9CnZZ80`Q8xDg literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/old_versions/version_2/__marks.mrk b/tests/integration/test_encrypted_disk/old_versions/version_2/__marks.mrk new file mode 100644 index 0000000000000000000000000000000000000000..e85676068f7e6ebe4952560368177d9e87987834 GIT binary patch literal 96 zcmZ?ub7o>-V3^Y8qto>FLP|yPKkMk--(;PlWBJu1r|n273|JHCH~j@r33jj{EcuQo h=ldzYWq4Js8YDihd0xh_KPBk}-{vE;FTS5M0RXtmAgurZ literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/old_versions/version_2/data.bin b/tests/integration/test_encrypted_disk/old_versions/version_2/data.bin new file mode 100644 index 0000000000000000000000000000000000000000..b0b5e06a28009caccea335a822ccae153c432902 GIT binary patch literal 99 zcmZ?ub7o>-V3^Y8qto>FLP|yPKkMk--(>H;TzD?zhE~IueU93t?-V3^Y8qto>FLP|yPKkMk--(=k*J-Rk77E*mA^(d`dd*)G~66}D#>VZt6 n_SLQDubuKL4;Aq~InCh8nW`(@UduI>7Ajm%Ymk(EGx;-V3^Y8qto>FLP|yPKkMk--(>r(e!mW4uVz^qyi_D;+M2IGCD?)9(#c#4 zVrtY(^m|UdpT6?Ks#O7(CO>%Mf77C0pwZU-c!b-Gd8MhGw{`a1KJ$FGW}(;0*D3aY y8zy{XoSm7pbNSjGDs44hF*$6tMpc3ucAK_0__1@XivP~0@bL|UZ^+TM<|F{?^gwX{ literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/test.py b/tests/integration/test_encrypted_disk/test.py index 66ff073f02b..9f5415f4bea 100644 --- a/tests/integration/test_encrypted_disk/test.py +++ b/tests/integration/test_encrypted_disk/test.py @@ -1,9 +1,11 @@ import pytest +import os.path from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException import os.path from helpers.test_tools import assert_eq_with_retry +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) FIRST_PART_NAME = "all_1_1_0" @@ -170,53 +172,62 @@ def test_optimize_table(policy, encrypted_disk): assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" -# Test adding encryption key on the fly. -def test_add_key(): - def make_storage_policy_with_keys(policy_name, keys): - node.exec_in_container( - [ - "bash", - "-c", - """cat > /etc/clickhouse-server/config.d/storage_policy_{policy_name}.xml << EOF +def make_storage_policy_with_keys( + policy_name, keys, check_system_storage_policies=False +): + if check_system_storage_policies: + node.query("SELECT policy_name FROM system.storage_policies") + + node.exec_in_container( + [ + "bash", + "-c", + """cat > /etc/clickhouse-server/config.d/storage_policy_{policy_name}.xml << EOF - - - <{policy_name}_disk> - encrypted - disk_local - {policy_name}_dir/ - {keys} - - - - <{policy_name}> - -
- {policy_name}_disk -
-
- -
-
+ + + <{policy_name}_disk> + encrypted + disk_local + {policy_name}_dir/ + {keys} + + + + <{policy_name}> + +
+ {policy_name}_disk +
+
+ +
+
EOF""".format( - policy_name=policy_name, keys=keys - ), - ] + policy_name=policy_name, keys=keys + ), + ] + ) + + node.query("SYSTEM RELOAD CONFIG") + + if check_system_storage_policies: + assert_eq_with_retry( + node, + f"SELECT policy_name FROM system.storage_policies WHERE policy_name='{policy_name}'", + policy_name, ) - node.query("SYSTEM RELOAD CONFIG") + + +# Test adding encryption key on the fly. +def test_add_keys(): + keys = "firstfirstfirstf" + make_storage_policy_with_keys( + "encrypted_policy_multikeys", keys, check_system_storage_policies=True + ) # Add some data to an encrypted disk. - node.query("SELECT policy_name FROM system.storage_policies") - make_storage_policy_with_keys( - "encrypted_policy_multikeys", "firstfirstfirstf" - ) - assert_eq_with_retry( - node, - "SELECT policy_name FROM system.storage_policies WHERE policy_name='encrypted_policy_multikeys'", - "encrypted_policy_multikeys", - ) - node.query( """ CREATE TABLE encrypted_test ( @@ -233,31 +244,39 @@ EOF""".format( assert node.query(select_query) == "(0,'data'),(1,'data')" # Add a second key and start using it. - make_storage_policy_with_keys( - "encrypted_policy_multikeys", + keys = """ + firstfirstfirstf + secondsecondseco + secondsecondseco """ - firstfirstfirstf - secondsecondseco - 1 - """, - ) + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + node.query("INSERT INTO encrypted_test VALUES (2,'data'),(3,'data')") # Now "(0,'data'),(1,'data')" is encrypted with the first key and "(2,'data'),(3,'data')" is encrypted with the second key. # All data are accessible. assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" - # Try to replace the first key with something wrong, and check that "(0,'data'),(1,'data')" cannot be read. - make_storage_policy_with_keys( - "encrypted_policy_multikeys", - """ - wrongwrongwrongw + # Keys can be reordered. + keys = """ secondsecondseco + firstfirstfirstf 1 - """, - ) + """ + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) - expected_error = "Wrong key" + # All data are still accessible. + assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" + + # Try to replace the first key with something wrong, and check that "(0,'data'),(1,'data')" cannot be read. + keys = """ + secondsecondseco + wrongwrongwrongw + secondsecondseco + """ + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + + expected_error = "Not found an encryption key required to decipher" assert expected_error in node.query_and_get_error(select_query) # Detach the part encrypted with the wrong key and check that another part containing "(2,'data'),(3,'data')" still can be read. @@ -265,6 +284,159 @@ EOF""".format( assert node.query(select_query) == "(2,'data'),(3,'data')" +# Test adding encryption key on the fly. +def test_add_keys_with_id(): + keys = "firstfirstfirstf" + make_storage_policy_with_keys( + "encrypted_policy_multikeys", keys, check_system_storage_policies=True + ) + + # Add some data to an encrypted disk. + node.query( + """ + CREATE TABLE encrypted_test ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='encrypted_policy_multikeys' + """ + ) + + node.query("INSERT INTO encrypted_test VALUES (0,'data'),(1,'data')") + select_query = "SELECT * FROM encrypted_test ORDER BY id FORMAT Values" + assert node.query(select_query) == "(0,'data'),(1,'data')" + + # Add a second key and start using it. + keys = """ + firstfirstfirstf + secondsecondseco + 1 + """ + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + + node.query("INSERT INTO encrypted_test VALUES (2,'data'),(3,'data')") + + # Now "(0,'data'),(1,'data')" is encrypted with the first key and "(2,'data'),(3,'data')" is encrypted with the second key. + # All data are accessible. + assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" + + # Keys can be reordered. + keys = """ + secondsecondseco + firstfirstfirstf + 1 + """ + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + + # All data are still accessible. + assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" + + # Try to replace the first key with something wrong, and check that "(0,'data'),(1,'data')" cannot be read. + keys = """ + secondsecondseco + wrongwrongwrongw + 1 + """ + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + + expected_error = "Not found an encryption key required to decipher" + assert expected_error in node.query_and_get_error(select_query) + + # Detach the part encrypted with the wrong key and check that another part containing "(2,'data'),(3,'data')" still can be read. + node.query("ALTER TABLE encrypted_test DETACH PART '{}'".format(FIRST_PART_NAME)) + assert node.query(select_query) == "(2,'data'),(3,'data')" + + +# Test appending of encrypted files. +def test_log_family(): + keys = "firstfirstfirstf" + make_storage_policy_with_keys( + "encrypted_policy_multikeys", keys, check_system_storage_policies=True + ) + + # Add some data to an encrypted disk. + node.query( + """ + CREATE TABLE encrypted_test ( + id Int64, + data String + ) ENGINE=Log + SETTINGS storage_policy='encrypted_policy_multikeys' + """ + ) + + node.query("INSERT INTO encrypted_test VALUES (0,'data'),(1,'data')") + select_query = "SELECT * FROM encrypted_test ORDER BY id FORMAT Values" + assert node.query(select_query) == "(0,'data'),(1,'data')" + + # Add a second key and start using it. + keys = """ + firstfirstfirstf + secondsecondseco + secondsecondseco + """ + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + + node.query("INSERT INTO encrypted_test VALUES (2,'data'),(3,'data')") + assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" + + # Everything is still encrypted with the first key (because the Log engine appends files), so the second key can be removed. + keys = "firstfirstfirstf" + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + + assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" + + +@pytest.mark.parametrize( + "old_version", + ["version_1le", "version_1be", "version_2"], +) +def test_migration_from_old_version(old_version): + keys = """ + first_key_first_ + second_key_secon + third_key_third_ + 3 + """ + make_storage_policy_with_keys( + "migration_from_old_version", keys, check_system_storage_policies=True + ) + + # Create a table without data. + node.query( + """ + CREATE TABLE encrypted_test ( + id Int64, + data String + ) ENGINE=Log + SETTINGS storage_policy='migration_from_old_version' + """ + ) + + # Copy table's data from an old version. + data_path = node.query( + "SELECT data_paths[1] FROM system.tables WHERE table = 'encrypted_test'" + ).splitlines()[0] + node.query("DETACH TABLE encrypted_test") + + old_version_dir = os.path.join(SCRIPT_DIR, "old_versions", old_version) + for file_name in os.listdir(old_version_dir): + src_path = os.path.join(old_version_dir, file_name) + dest_path = os.path.join(data_path, file_name) + node.copy_file_to_container(src_path, dest_path) + + node.query("ATTACH TABLE encrypted_test") + + # We can read from encrypted disk after migration. + select_query = "SELECT * FROM encrypted_test ORDER BY id FORMAT Values" + assert node.query(select_query) == "(0,'ab'),(1,'cdefg')" + + # We can append files on encrypted disk after migration. + node.query("INSERT INTO encrypted_test VALUES (2,'xyz')") + assert node.query(select_query) == "(0,'ab'),(1,'cdefg'),(2,'xyz')" + + def test_read_in_order(): node.query( "CREATE TABLE encrypted_test(`a` UInt64, `b` String(150)) ENGINE = MergeTree() ORDER BY (a, b) SETTINGS storage_policy='encrypted_policy'" From 74dc37cf610746a8814e3cc3195c44f2f3926650 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 May 2023 14:15:28 +0200 Subject: [PATCH 164/308] Fix assertion --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index e244f61ae2e..dc4a2599d1d 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -925,16 +925,16 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() LOG_TEST( log, - "Read {} bytes, read type {}, position: {}, offset: {}, segment end: {}", - size, toString(read_type), implementation_buffer->getPosition(), - implementation_buffer->getFileOffsetOfBufferEnd(), file_segment.range().right); + "Read {} bytes, read type {}, file offset: {}, impl offset: {}/{}, segment: {}", + size, toString(read_type), file_offset_of_buffer_end, + implementation_buffer->getFileOffsetOfBufferEnd(), read_until_position, file_segment.range().toString()); if (read_type == ReadType::CACHED) { ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed); - chassert(file_offset_of_buffer_end + size <= file_segment.range().size()); + chassert(file_offset_of_buffer_end + size - 1 <= file_segment.range().right); } else { From 9b8159e08135ccd65e2b4dd5c3000b89c6d09350 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 31 May 2023 14:20:28 +0200 Subject: [PATCH 165/308] Update 02770_async_buffer_ignore.sh --- tests/queries/0_stateless/02770_async_buffer_ignore.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.sh b/tests/queries/0_stateless/02770_async_buffer_ignore.sh index 22b432db178..6eaf655324d 100755 --- a/tests/queries/0_stateless/02770_async_buffer_ignore.sh +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 4d4112ff536f819514973dfd0cb8274cf044bb3e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 31 May 2023 15:26:56 +0300 Subject: [PATCH 166/308] Revert "less logs in WriteBufferFromS3" (#50390) --- src/IO/WriteBufferFromS3.cpp | 8 ++++++++ src/IO/WriteBufferFromS3TaskTracker.cpp | 11 +++++++++++ 2 files changed, 19 insertions(+) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 6992c3ea4ac..462cf2674c3 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -195,14 +195,18 @@ void WriteBufferFromS3::finalizeImpl() if (request_settings.check_objects_after_upload) { + LOG_TRACE(log, "Checking object {} exists after upload", key); S3::checkObjectExists(*client_ptr, bucket, key, {}, request_settings, /* for_disk_s3= */ write_settings.for_object_storage, "Immediately after upload"); + LOG_TRACE(log, "Checking object {} has size as expected {}", key, total_size); size_t actual_size = S3::getObjectSize(*client_ptr, bucket, key, {}, request_settings, /* for_disk_s3= */ write_settings.for_object_storage); if (actual_size != total_size) throw Exception( ErrorCodes::S3_ERROR, "Object {} from bucket {} has unexpected size {} after upload, expected size {}, it's a bug in S3 or S3 API.", key, bucket, actual_size, total_size); + + LOG_TRACE(log, "Object {} exists after upload", key); } } @@ -288,6 +292,8 @@ void WriteBufferFromS3::reallocateFirstBuffer() WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size); chassert(offset() == 0); + + LOG_TRACE(log, "Reallocated first buffer with size {}. {}", memory.size(), getLogDetails()); } void WriteBufferFromS3::detachBuffer() @@ -310,6 +316,8 @@ void WriteBufferFromS3::allocateFirstBuffer() const auto size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), max_first_buffer); memory = Memory(size); WriteBuffer::set(memory.data(), memory.size()); + + LOG_TRACE(log, "Allocated first buffer with size {}. {}", memory.size(), getLogDetails()); } void WriteBufferFromS3::allocateBuffer() diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index c10af5d0672..7ae31044012 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -36,6 +36,8 @@ ThreadPoolCallbackRunner WriteBufferFromS3::TaskTracker::syncRunner() void WriteBufferFromS3::TaskTracker::waitAll() { + LOG_TEST(log, "waitAll, in queue {}", futures.size()); + /// Exceptions are propagated for (auto & future : futures) { @@ -49,6 +51,8 @@ void WriteBufferFromS3::TaskTracker::waitAll() void WriteBufferFromS3::TaskTracker::safeWaitAll() { + LOG_TEST(log, "safeWaitAll, wait in queue {}", futures.size()); + for (auto & future : futures) { if (future.valid()) @@ -72,6 +76,7 @@ void WriteBufferFromS3::TaskTracker::safeWaitAll() void WriteBufferFromS3::TaskTracker::waitIfAny() { + LOG_TEST(log, "waitIfAny, in queue {}", futures.size()); if (futures.empty()) return; @@ -96,6 +101,8 @@ void WriteBufferFromS3::TaskTracker::waitIfAny() watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); + + LOG_TEST(log, "waitIfAny ended, in queue {}", futures.size()); } void WriteBufferFromS3::TaskTracker::add(Callback && func) @@ -140,6 +147,8 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() if (!max_tasks_inflight) return; + LOG_TEST(log, "waitTilInflightShrink, in queue {}", futures.size()); + Stopwatch watch; /// Alternative approach is to wait until at least futures.size() - max_tasks_inflight element are finished @@ -162,6 +171,8 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); + + LOG_TEST(log, "waitTilInflightShrink ended, in queue {}", futures.size()); } bool WriteBufferFromS3::TaskTracker::isAsync() const From 7d077f6130c61b529b13403b8a744220002f2ef0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 May 2023 14:30:27 +0200 Subject: [PATCH 167/308] Add one more assertion --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index dc4a2599d1d..5cb9d3bbf6f 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -934,7 +934,9 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed); - chassert(file_offset_of_buffer_end + size - 1 <= file_segment.range().right); + [[maybe_unused]] size_t new_file_offset = file_offset_of_buffer_end + size; + chassert(new_file_offset - 1 <= file_segment.range().right); + chassert(new_file_offset <= file_segment.getCurrentWriteOffset(true)); } else { From a59effcc88314cfd8e4c7972c67d2dffde2114bb Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 May 2023 14:40:13 +0200 Subject: [PATCH 168/308] Minor improvements --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 52 +++++++++---------- 1 file changed, 25 insertions(+), 27 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 5cb9d3bbf6f..202914a0774 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -51,8 +51,8 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( std::optional read_until_position_, std::shared_ptr cache_log_) : ReadBufferFromFileBase(use_external_buffer_ ? 0 : settings_.remote_fs_buffer_size, nullptr, 0, file_size_) -#ifndef NDEBUG - , log(&Poco::Logger::get("CachedOnDiskReadBufferFromFile(" + source_file_path_ + ")")) +#ifdef ABORT_ON_LOGICAL_ERROR + , log(&Poco::Logger::get(fmt::format("CachedOnDiskReadBufferFromFile({})", cache_key_))) #else , log(&Poco::Logger::get("CachedOnDiskReadBufferFromFile")) #endif @@ -75,6 +75,9 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog( const FileSegment::Range & file_segment_range, CachedOnDiskReadBufferFromFile::ReadType type) { + if (!cache_log) + return; + FilesystemCacheLogElement elem { .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), @@ -104,8 +107,7 @@ void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog( break; } - if (cache_log) - cache_log->add(elem); + cache_log->add(elem); } void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) @@ -411,7 +413,7 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegment & file_segme { case ReadType::CACHED: { -#ifndef NDEBUG +#ifdef ABORT_ON_LOGICAL_ERROR size_t file_size = getFileSizeFromReadBuffer(*read_buffer_for_file_segment); if (file_size == 0 || range.left + file_size <= file_offset_of_buffer_end) throw Exception( @@ -456,7 +458,7 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegment & file_segme { read_buffer_for_file_segment->seek(file_offset_of_buffer_end, SEEK_SET); - assert(read_buffer_for_file_segment->getFileOffsetOfBufferEnd() == file_offset_of_buffer_end); + chassert(read_buffer_for_file_segment->getFileOffsetOfBufferEnd() == file_offset_of_buffer_end); } const auto current_write_offset = file_segment.getCurrentWriteOffset(false); @@ -887,28 +889,24 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (!result) { - auto debug_check = [&]() +#ifdef ABORT_ON_LOGICAL_ERROR + if (read_type == ReadType::CACHED) { - if (read_type == ReadType::CACHED) + size_t cache_file_size = getFileSizeFromReadBuffer(*implementation_buffer); + if (cache_file_size == 0) { - size_t cache_file_size = getFileSizeFromReadBuffer(*implementation_buffer); - if (cache_file_size == 0) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Attempt to read from an empty cache file: {} (just before actual read)", - cache_file_size); - } + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Attempt to read from an empty cache file: {} (just before actual read)", + cache_file_size); } - else - { - chassert(file_offset_of_buffer_end == static_cast(implementation_buffer->getFileOffsetOfBufferEnd())); - } - chassert(!implementation_buffer->hasPendingData()); - return true; - }; - - chassert(debug_check()); + } + else + { + chassert(file_offset_of_buffer_end == static_cast(implementation_buffer->getFileOffsetOfBufferEnd())); + } + chassert(!implementation_buffer->hasPendingData()); +#endif Stopwatch watch(CLOCK_MONOTONIC); @@ -1093,8 +1091,8 @@ off_t CachedOnDiskReadBufferFromFile::seek(off_t offset, int whence) if (file_offset_of_buffer_end - working_buffer.size() <= new_pos && new_pos <= file_offset_of_buffer_end) { pos = working_buffer.end() - file_offset_of_buffer_end + new_pos; - assert(pos >= working_buffer.begin()); - assert(pos <= working_buffer.end()); + chassert(pos >= working_buffer.begin()); + chassert(pos <= working_buffer.end()); return new_pos; } } From dbae50b6db6f885daef4d97f00d3ef81b6104741 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 May 2023 14:50:59 +0200 Subject: [PATCH 169/308] Better logging --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 202914a0774..a60f5dffa96 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -365,8 +365,8 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegment & file_s else { LOG_TRACE( - log, - "Bypassing cache because file segment state is `PARTIALLY_DOWNLOADED_NO_CONTINUATION` and downloaded part already used"); + log, "Bypassing cache because file segment state is " + "`PARTIALLY_DOWNLOADED_NO_CONTINUATION` and downloaded part already used"); read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; return getRemoteReadBuffer(file_segment, read_type); } @@ -466,8 +466,8 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegment & file_segme { throw Exception( ErrorCodes::LOGICAL_ERROR, - "Buffer's offsets mismatch. Cached buffer offset: {}, current_write_offset: {}, implementation buffer position: {}, " - "implementation buffer end position: {}, file segment info: {}", + "Buffer's offsets mismatch. Cached buffer offset: {}, current_write_offset: {}, " + "implementation buffer position: {}, implementation buffer end position: {}, file segment info: {}", file_offset_of_buffer_end, current_write_offset, read_buffer_for_file_segment->getPosition(), @@ -932,9 +932,18 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed); - [[maybe_unused]] size_t new_file_offset = file_offset_of_buffer_end + size; +#ifdef ABORT_ON_LOGICAL_ERROR + const size_t new_file_offset = file_offset_of_buffer_end + size; chassert(new_file_offset - 1 <= file_segment.range().right); - chassert(new_file_offset <= file_segment.getCurrentWriteOffset(true)); + const size_t file_segment_write_offset = file_segment.getCurrentWriteOffset(true); + if (new_file_offset > file_segment_write_offset) + { + LOG_TRACE( + log, "Read {} bytes, file offset: {}, segment: {}, segment write offset: {}", + size, file_offset_of_buffer_end, file_segment.range().toString(), file_segment_write_offset); + chassert(false); + } +#endif } else { From f74d7474d2a95b7fb4856c5a0e541855b5cd7fdd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 31 May 2023 15:04:41 +0200 Subject: [PATCH 170/308] fix poll timeout in MaterializedMySQL --- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index aaf6b00dc4f..a01ab2a15a8 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -262,9 +262,13 @@ void MaterializedMySQLSyncThread::synchronization() try { - BinlogEventPtr binlog_event = client.readOneBinlogEvent(std::max(UInt64(1), max_flush_time - watch.elapsedMilliseconds())); - if (binlog_event) - onEvent(buffers, binlog_event, metadata); + UInt64 elapsed_ms = watch.elapsedMilliseconds(); + if (elapsed_ms < max_flush_time) + { + BinlogEventPtr binlog_event = client.readOneBinlogEvent(max_flush_time - elapsed_ms); + if (binlog_event) + onEvent(buffers, binlog_event, metadata); + } } catch (const Exception & e) { From 2efebee5a37b25b26898705ecd833a2c3091eaf9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 31 May 2023 16:24:36 +0300 Subject: [PATCH 171/308] Compare functions NaN update test (#50366) Co-authored-by: Nikita Mikhaylov --- ...e => 02769_compare_functions_nan.reference} | 1 + ...son.sql => 02769_compare_functions_nan.sql} | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+) rename tests/queries/0_stateless/{02769_nan_equality_comparison.reference => 02769_compare_functions_nan.reference} (96%) rename tests/queries/0_stateless/{02769_nan_equality_comparison.sql => 02769_compare_functions_nan.sql} (78%) diff --git a/tests/queries/0_stateless/02769_nan_equality_comparison.reference b/tests/queries/0_stateless/02769_compare_functions_nan.reference similarity index 96% rename from tests/queries/0_stateless/02769_nan_equality_comparison.reference rename to tests/queries/0_stateless/02769_compare_functions_nan.reference index a8ba06cfce6..81f0ee6da73 100644 --- a/tests/queries/0_stateless/02769_nan_equality_comparison.reference +++ b/tests/queries/0_stateless/02769_compare_functions_nan.reference @@ -8,3 +8,4 @@ nan 1 1 1 1 nan nan 1 1 1 1 -- nan +-- diff --git a/tests/queries/0_stateless/02769_nan_equality_comparison.sql b/tests/queries/0_stateless/02769_compare_functions_nan.sql similarity index 78% rename from tests/queries/0_stateless/02769_nan_equality_comparison.sql rename to tests/queries/0_stateless/02769_compare_functions_nan.sql index 6cce19a2204..1e1a9df9ce2 100644 --- a/tests/queries/0_stateless/02769_nan_equality_comparison.sql +++ b/tests/queries/0_stateless/02769_compare_functions_nan.sql @@ -7,11 +7,13 @@ SELECT nan AS lhs, cast(nan, 'Float32') AS rhs, lhs = rhs, lhs = materialize(rhs SELECT '--'; +DROP TABLE IF EXISTS test_table; CREATE TABLE test_table ( id UInt32, value UInt32 ) ENGINE = MergeTree ORDER BY id; + INSERT INTO test_table VALUES (76, 57); SELECT value FROM (SELECT stddevSamp(id) AS value FROM test_table) as subquery @@ -33,6 +35,7 @@ CREATE TABLE test_table value_1 UInt32, value_2 Float32 ) ENGINE = MergeTree ORDER BY id; + INSERT INTO test_table VALUES (12000, 36, 77.94); SELECT value @@ -40,3 +43,18 @@ FROM (SELECT (corr(value_1, value_1) OVER test_window) AS value FROM test_table WHERE not (not (value <> value)); DROP TABLE test_table; + +SELECT '--'; + +CREATE TABLE test_table +( + id Float32, + value Float32 +) ENGINE=MergeTree ORDER BY id; + +INSERT INTO test_table VALUES (-10.75, 95.57); + +SELECT * FROM (SELECT corr(id, id) as corr_value FROM test_table GROUP BY value) AS subquery LEFT ANTI JOIN test_table ON (subquery.corr_value = test_table.id) +WHERE (test_table.id >= test_table.id) AND (NOT (test_table.id >= test_table.id)); + +DROP TABLE test_table; From a96ee7411b0fdd28d9a77d127f74848b889a73f6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 31 May 2023 14:28:36 +0000 Subject: [PATCH 172/308] Bump From cf9954f17a6363c9bd4211d4797cd33772c3a54f Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 31 May 2023 14:55:51 +0000 Subject: [PATCH 173/308] Remove pointless duplication of some queries in 00189_time_zones_long test --- .../00189_time_zones_long.reference | 16 --------------- .../0_stateless/00189_time_zones_long.sql | 20 ++++--------------- 2 files changed, 4 insertions(+), 32 deletions(-) diff --git a/tests/queries/0_stateless/00189_time_zones_long.reference b/tests/queries/0_stateless/00189_time_zones_long.reference index d41c925bbe5..11aebd71417 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.reference +++ b/tests/queries/0_stateless/00189_time_zones_long.reference @@ -16,10 +16,6 @@ toMonday 2014-12-29 2014-12-22 2014-12-29 -2014-12-29 -2014-12-29 -2014-12-29 -2014-12-29 toStartOfMonth 2014-12-01 2014-12-01 @@ -27,10 +23,6 @@ toStartOfMonth 2014-12-01 2014-12-01 2014-12-01 -2014-12-01 -2014-12-01 -2014-12-01 -2014-12-01 toStartOfQuarter 2014-07-01 2014-07-01 @@ -38,10 +30,6 @@ toStartOfQuarter 2014-10-01 2014-07-01 2014-07-01 -2014-07-01 -2014-07-01 -2014-07-01 -2014-07-01 toStartOfYear 2014-01-01 2014-01-01 @@ -49,10 +37,6 @@ toStartOfYear 2014-01-01 2014-01-01 2014-01-01 -2014-01-01 -2014-01-01 -2014-01-01 -2014-01-01 toTime 1970-01-02 11:00:00 1970-01-02 12:00:00 1970-01-02 10:00:00 1970-01-02 11:00:00 diff --git a/tests/queries/0_stateless/00189_time_zones_long.sql b/tests/queries/0_stateless/00189_time_zones_long.sql index 5760f6c0447..8c8de50abe7 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.sql +++ b/tests/queries/0_stateless/00189_time_zones_long.sql @@ -33,10 +33,7 @@ SELECT toMonday(toDateTime(1419800400), 'Europe/Paris'); SELECT toMonday(toDateTime(1419800400), 'Europe/London'); SELECT toMonday(toDateTime(1419800400), 'Asia/Tokyo'); SELECT toMonday(toDateTime(1419800400), 'Pacific/Pitcairn'); -SELECT toMonday(toDate(16433)); -SELECT toMonday(toDate(16433)); -SELECT toMonday(toDate(16433)); -SELECT toMonday(toDate(16433)); +SELECT toMonday(toDate(16433), 'Asia/Istanbul'); -- { serverError 43 } SELECT toMonday(toDate(16433)); /* toStartOfMonth */ @@ -47,10 +44,7 @@ SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Paris'); SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/London'); SELECT toStartOfMonth(toDateTime(1419800400), 'Asia/Tokyo'); SELECT toStartOfMonth(toDateTime(1419800400), 'Pacific/Pitcairn'); -SELECT toStartOfMonth(toDate(16433)); -SELECT toStartOfMonth(toDate(16433)); -SELECT toStartOfMonth(toDate(16433)); -SELECT toStartOfMonth(toDate(16433)); +SELECT toStartOfMonth(toDate(16433), 'Asia/Istanbul'); -- { serverError 43 } SELECT toStartOfMonth(toDate(16433)); /* toStartOfQuarter */ @@ -61,10 +55,7 @@ SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/Paris'); SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/London'); SELECT toStartOfQuarter(toDateTime(1412106600), 'Asia/Tokyo'); SELECT toStartOfQuarter(toDateTime(1412106600), 'Pacific/Pitcairn'); -SELECT toStartOfQuarter(toDate(16343)); -SELECT toStartOfQuarter(toDate(16343)); -SELECT toStartOfQuarter(toDate(16343)); -SELECT toStartOfQuarter(toDate(16343)); +SELECT toStartOfQuarter(toDate(16343), 'Asia/Istanbul'); -- { serverError 43 } SELECT toStartOfQuarter(toDate(16343)); /* toStartOfYear */ @@ -75,10 +66,7 @@ SELECT toStartOfYear(toDateTime(1419800400), 'Europe/Paris'); SELECT toStartOfYear(toDateTime(1419800400), 'Europe/London'); SELECT toStartOfYear(toDateTime(1419800400), 'Asia/Tokyo'); SELECT toStartOfYear(toDateTime(1419800400), 'Pacific/Pitcairn'); -SELECT toStartOfYear(toDate(16433)); -SELECT toStartOfYear(toDate(16433)); -SELECT toStartOfYear(toDate(16433)); -SELECT toStartOfYear(toDate(16433)); +SELECT toStartOfYear(toDate(16433), 'Asia/Istanbul'); -- { serverError 43 } SELECT toStartOfYear(toDate(16433)); /* toTime */ From 570778dee0cd06ff74558e12ef55ebc117a4b305 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 31 May 2023 15:08:25 +0000 Subject: [PATCH 174/308] Integrated toStartOfWeekand and toLastDayOfWeek in 00189_time_zones_long --- .../00189_time_zones_long.reference | 28 ++++++++++++ .../0_stateless/00189_time_zones_long.sql | 44 +++++++++++++++++++ 2 files changed, 72 insertions(+) diff --git a/tests/queries/0_stateless/00189_time_zones_long.reference b/tests/queries/0_stateless/00189_time_zones_long.reference index 11aebd71417..a4287217a19 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.reference +++ b/tests/queries/0_stateless/00189_time_zones_long.reference @@ -16,6 +16,34 @@ toMonday 2014-12-29 2014-12-22 2014-12-29 +toStartOfWeek (Sunday) +2014-12-28 +2014-12-28 +2014-12-28 +2014-12-28 +2014-12-28 +2014-12-28 +toStartOfWeek (Monday) +2014-12-22 +2014-12-22 +2014-12-22 +2014-12-29 +2014-12-22 +2014-12-29 +toLastDayOfWeek (Sunday) +2015-01-03 +2015-01-03 +2015-01-03 +2015-01-03 +2015-01-03 +2015-01-03 +toLastDayOfWeek (Monday) +2014-12-28 +2014-12-28 +2014-12-28 +2015-01-04 +2014-12-28 +2015-01-04 toStartOfMonth 2014-12-01 2014-12-01 diff --git a/tests/queries/0_stateless/00189_time_zones_long.sql b/tests/queries/0_stateless/00189_time_zones_long.sql index 8c8de50abe7..4785bee1482 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.sql +++ b/tests/queries/0_stateless/00189_time_zones_long.sql @@ -36,6 +36,50 @@ SELECT toMonday(toDateTime(1419800400), 'Pacific/Pitcairn'); SELECT toMonday(toDate(16433), 'Asia/Istanbul'); -- { serverError 43 } SELECT toMonday(toDate(16433)); +/* toStartOfWeek (Sunday) */ + +SELECT 'toStartOfWeek (Sunday)'; +SELECT toStartOfWeek(toDateTime(1419800400), 0, 'Asia/Istanbul'); +SELECT toStartOfWeek(toDateTime(1419800400), 0, 'Europe/Paris'); +SELECT toStartOfWeek(toDateTime(1419800400), 0, 'Europe/London'); +SELECT toStartOfWeek(toDateTime(1419800400), 0, 'Asia/Tokyo'); +SELECT toStartOfWeek(toDateTime(1419800400), 0, 'Pacific/Pitcairn'); +SELECT toStartOfWeek(toDate(16433), 0, 'Asia/Istanbul'); -- { serverError 43 } +SELECT toStartOfWeek(toDate(16433), 0); + +/* toStartOfWeek (Monday) */ + +SELECT 'toStartOfWeek (Monday)'; +SELECT toStartOfWeek(toDateTime(1419800400), 1, 'Asia/Istanbul'); +SELECT toStartOfWeek(toDateTime(1419800400), 1, 'Europe/Paris'); +SELECT toStartOfWeek(toDateTime(1419800400), 1, 'Europe/London'); +SELECT toStartOfWeek(toDateTime(1419800400), 1, 'Asia/Tokyo'); +SELECT toStartOfWeek(toDateTime(1419800400), 1, 'Pacific/Pitcairn'); +SELECT toStartOfWeek(toDate(16433), 1, 'Asia/Istanbul'); -- { serverError 43 } +SELECT toStartOfWeek(toDate(16433), 1); + +/* toLastDayOfWeek (Sunday) */ + +SELECT 'toLastDayOfWeek (Sunday)'; +SELECT toLastDayOfWeek(toDateTime(1419800400), 0, 'Asia/Istanbul'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 0, 'Europe/Paris'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 0, 'Europe/London'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 0, 'Asia/Tokyo'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 0, 'Pacific/Pitcairn'); +SELECT toLastDayOfWeek(toDate(16433), 0, 'Asia/Istanbul'); -- { serverError 43 } +SELECT toLastDayOfWeek(toDate(16433), 0); + +/* toLastDayOfWeek (Monday) */ + +SELECT 'toLastDayOfWeek (Monday)'; +SELECT toLastDayOfWeek(toDateTime(1419800400), 1, 'Asia/Istanbul'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 1, 'Europe/Paris'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 1, 'Europe/London'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 1, 'Asia/Tokyo'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 1, 'Pacific/Pitcairn'); +SELECT toLastDayOfWeek(toDate(16433), 1, 'Asia/Istanbul'); -- { serverError 43 } +SELECT toLastDayOfWeek(toDate(16433), 1); + /* toStartOfMonth */ SELECT 'toStartOfMonth'; From a96a845e232c05601119df156972fba783696b21 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 31 May 2023 17:12:49 +0200 Subject: [PATCH 175/308] Add comment to prepare-ci-ami.sh, add a success at the end --- tests/ci/worker/{ubuntu_ami_for_ci.sh => prepare-ci-ami.sh} | 5 +++++ 1 file changed, 5 insertions(+) rename tests/ci/worker/{ubuntu_ami_for_ci.sh => prepare-ci-ami.sh} (93%) diff --git a/tests/ci/worker/ubuntu_ami_for_ci.sh b/tests/ci/worker/prepare-ci-ami.sh similarity index 93% rename from tests/ci/worker/ubuntu_ami_for_ci.sh rename to tests/ci/worker/prepare-ci-ami.sh index 2bb8f01535f..fca13266f09 100644 --- a/tests/ci/worker/ubuntu_ami_for_ci.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# The script is downloaded the AWS image builder Task Orchestrator and Executor (AWSTOE) +# We can't use `user data script` because cloud-init does not check the exit code set -xeuo pipefail echo "Running prepare script" @@ -121,3 +123,6 @@ gpg --verify /tmp/amazon-cloudwatch-agent.deb.sig dpkg -i /tmp/amazon-cloudwatch-agent.deb aws ssm get-parameter --region us-east-1 --name AmazonCloudWatch-github-runners --query 'Parameter.Value' --output text > /opt/aws/amazon-cloudwatch-agent/etc/amazon-cloudwatch-agent.json systemctl enable amazon-cloudwatch-agent.service + +# The following line is used in aws TOE check. +touch /var/tmp/clickhouse-ci-ami.success From bed7443181eb543d8ebd4b6a47e85f9d400931df Mon Sep 17 00:00:00 2001 From: rfraposa Date: Wed, 31 May 2023 09:31:46 -0600 Subject: [PATCH 176/308] Fixes --- .../example-datasets/reddit-comments.md | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/docs/en/getting-started/example-datasets/reddit-comments.md b/docs/en/getting-started/example-datasets/reddit-comments.md index e1e372746c9..aa5c474103d 100644 --- a/docs/en/getting-started/example-datasets/reddit-comments.md +++ b/docs/en/getting-started/example-datasets/reddit-comments.md @@ -5,7 +5,7 @@ sidebar_label: Reddit comments # Reddit comments dataset -This dataset contains publicly-available comments on Reddit that go back to December, 2005, to March, 2023, and contains over 7B rows of data. The raw data is in JSON format in compressed `.zst` files and the rows look like the following: +This dataset contains publicly-available comments on Reddit that go back to December, 2005, to March, 2023, and contains over 14B rows of data. The raw data is in JSON format in compressed files and the rows look like the following: ```json {"controversiality":0,"body":"A look at Vietnam and Mexico exposes the myth of market liberalisation.","subreddit_id":"t5_6","link_id":"t3_17863","stickied":false,"subreddit":"reddit.com","score":2,"ups":2,"author_flair_css_class":null,"created_utc":1134365188,"author_flair_text":null,"author":"frjo","id":"c13","edited":false,"parent_id":"t3_17863","gilded":0,"distinguished":null,"retrieved_on":1473738411} @@ -75,18 +75,6 @@ The names of the files in S3 start with `RC_YYYY-MM` where `YYYY-MM` goes from ` 2. We are going to start with one month of data, but if you want to simply insert every row - skip ahead to step 8 below. The following file has 86M records from December, 2017: -```sql -INSERT INTO reddit - SELECT * - FROM s3Cluster( - 'default', - 'https://clickhouse-public-datasets.s3.eu-central-1.amazonaws.com/reddit/original/RC_2017-12.xz', - 'JSONEachRow' - ); -``` - -If you do not have a cluster, use `s3` instead of `s3Cluster`: - ```sql INSERT INTO reddit SELECT * @@ -94,6 +82,7 @@ INSERT INTO reddit 'https://clickhouse-public-datasets.s3.eu-central-1.amazonaws.com/reddit/original/RC_2017-12.xz', 'JSONEachRow' ); + ``` 3. It will take a while depending on your resources, but when it's done verify it worked: @@ -211,6 +200,18 @@ FROM s3Cluster( SETTINGS zstd_window_log_max = 31; ``` +If you do not have a cluster, use `s3` instead of `s3Cluster`: + +```sql +INSERT INTO reddit +SELECT * +FROM s3( + 'https://clickhouse-public-datasets.s3.amazonaws.com/reddit/original/RC*', + 'JSONEachRow' + ) +SETTINGS zstd_window_log_max = 31; +``` + The response looks like: ```response From aedd3afb8aa6127d24a7a84146be113f88936bed Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Wed, 31 May 2023 18:20:58 +0200 Subject: [PATCH 177/308] fix hung in unit tests (#50391) * fix hung in unit tests * Update gtest_writebuffer_s3.cpp * Update gtest_writebuffer_s3.cpp --------- Co-authored-by: Alexander Tokmakov --- src/IO/tests/gtest_writebuffer_s3.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index bc16af7f779..cd38291fb31 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -609,9 +609,16 @@ protected: test_with_pool = GetParam(); client = MockS3::Client::CreateClient(bucket); if (test_with_pool) + { + /// Do not block the main thread awaiting the others task. + /// This test use the only one thread at all + getSettings().s3_max_inflight_parts_for_one_file = 0; async_policy = std::make_unique(); + } else + { async_policy = std::make_unique(); + } } }; From 653da5f00219c088af66d97a8f1ea3e35e798268 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 31 May 2023 18:23:21 +0200 Subject: [PATCH 178/308] Reduce the unzip verbosity --- tests/ci/worker/prepare-ci-ami.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index fca13266f09..57ab149237a 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -102,7 +102,7 @@ chown -R ubuntu:ubuntu $RUNNER_HOME cd /home/ubuntu curl "https://awscli.amazonaws.com/awscli-exe-linux-$(uname -m).zip" -o "awscliv2.zip" -unzip awscliv2.zip +unzip -q awscliv2.zip ./aws/install rm -rf /home/ubuntu/awscliv2.zip /home/ubuntu/aws From adf71e706eab3f0bd2e2fc23aba13d6a0b9fb782 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 31 May 2023 16:34:55 +0000 Subject: [PATCH 179/308] Intergate week related functions to 01472_toStartOfInterval_disallow_empty_tz_field test --- ...rtOfInterval_disallow_empty_tz_field.reference | 5 +++++ ..._toStartOfInterval_disallow_empty_tz_field.sql | 15 +++++++++++++++ 2 files changed, 20 insertions(+) diff --git a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.reference b/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.reference index c5218102aa4..83a1ac4f193 100644 --- a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.reference +++ b/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.reference @@ -1,4 +1,9 @@ 2017-12-31 00:00:00 +2017-12-25 +2017-12-31 +2017-12-25 +2018-01-06 +2017-12-31 2017-12-01 2017-10-01 2017-01-01 diff --git a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql b/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql index c2bdac2b279..47e81653b25 100644 --- a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql +++ b/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql @@ -1,6 +1,21 @@ SELECT toStartOfDay(toDateTime('2017-12-31 00:00:00', 'UTC'), ''); -- {serverError 43} SELECT toStartOfDay(toDateTime('2017-12-31 03:45:00', 'UTC'), 'UTC'); -- success +SELECT toMonday(toDateTime('2017-12-31 00:00:00', 'UTC'), ''); -- {serverError 43} +SELECT toMonday(toDateTime('2017-12-31 00:00:00', 'UTC'), 'UTC'); -- success + +SELECT toStartOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 0, ''); -- {serverError 43} +SELECT toStartOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 0, 'UTC'); -- success + +SELECT toStartOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 1, ''); -- {serverError 43} +SELECT toStartOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 1, 'UTC'); -- success + +SELECT toLastDayOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 0, ''); -- {serverError 43} +SELECT toLastDayOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 0, 'UTC'); -- success + +SELECT toLastDayOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 1, ''); -- {serverError 43} +SELECT toLastDayOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 1, 'UTC'); -- success + SELECT toStartOfMonth(toDateTime('2017-12-31 00:00:00', 'UTC'), ''); -- {serverError 43} SELECT toStartOfMonth(toDateTime('2017-12-31 00:00:00', 'UTC'), 'UTC'); -- success From 8c1f579c481747c1257c53adf674493aae35b2c1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 16:41:30 +0000 Subject: [PATCH 180/308] Add google-protobuf submodule --- .gitmodules | 3 +++ contrib/google-protobuf | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/google-protobuf diff --git a/.gitmodules b/.gitmodules index f0984fec4db..30777a42a9d 100644 --- a/.gitmodules +++ b/.gitmodules @@ -344,3 +344,6 @@ [submodule "contrib/isa-l"] path = contrib/isa-l url = https://github.com/ClickHouse/isa-l.git +[submodule "contrib/google-protobuf"] + path = contrib/google-protobuf + url = https://github.com/ClickHouse/google-protobuf.git diff --git a/contrib/google-protobuf b/contrib/google-protobuf new file mode 160000 index 00000000000..315ffb5be89 --- /dev/null +++ b/contrib/google-protobuf @@ -0,0 +1 @@ +Subproject commit 315ffb5be89460f2857387d20aefc59b76b8bdc3 From 8c3e256caedbbf1dae3bd52cdddbd1b1a315e8ee Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 16:42:56 +0000 Subject: [PATCH 181/308] Switch protobuf to v3.18.x --- contrib/google-protobuf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/google-protobuf b/contrib/google-protobuf index 315ffb5be89..3b3d8fe1913 160000 --- a/contrib/google-protobuf +++ b/contrib/google-protobuf @@ -1 +1 @@ -Subproject commit 315ffb5be89460f2857387d20aefc59b76b8bdc3 +Subproject commit 3b3d8fe191314ea903ea6b072f0e73ef18e15faa From 7d8c1ff3cca5a59749b839e7fe23dc1e3bd9cac8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 16:43:36 +0000 Subject: [PATCH 182/308] Move protobuf entry in .gitmodules --- .gitmodules | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index 30777a42a9d..d28f205b65c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -39,6 +39,9 @@ path = contrib/protobuf url = https://github.com/ClickHouse/protobuf branch = v3.13.0.1 +[submodule "contrib/google-protobuf"] + path = contrib/google-protobuf + url = https://github.com/ClickHouse/google-protobuf.git [submodule "contrib/boost"] path = contrib/boost url = https://github.com/ClickHouse/boost @@ -344,6 +347,3 @@ [submodule "contrib/isa-l"] path = contrib/isa-l url = https://github.com/ClickHouse/isa-l.git -[submodule "contrib/google-protobuf"] - path = contrib/google-protobuf - url = https://github.com/ClickHouse/google-protobuf.git From ab6fe946bd4554d08f70808ee4c13699e9862069 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 16:47:26 +0000 Subject: [PATCH 183/308] Switch build to google-protobuf-cmake --- contrib/CMakeLists.txt | 4 +- contrib/google-protobuf-cmake/CMakeLists.txt | 329 ++++++++++++++++++ .../protobuf_generate.cmake | 198 +++++++++++ 3 files changed, 529 insertions(+), 2 deletions(-) create mode 100644 contrib/google-protobuf-cmake/CMakeLists.txt create mode 100644 contrib/google-protobuf-cmake/protobuf_generate.cmake diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 020fe1e1c5a..4a4ff9982ea 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -88,7 +88,7 @@ add_contrib (thrift-cmake thrift) # parquet/arrow/orc add_contrib (arrow-cmake arrow) # requires: snappy, thrift, double-conversion add_contrib (avro-cmake avro) # requires: snappy -add_contrib (protobuf-cmake protobuf) +add_contrib (google-protobuf-cmake google-protobuf) add_contrib (openldap-cmake openldap) add_contrib (grpc-cmake grpc) add_contrib (msgpack-c-cmake msgpack-c) @@ -156,7 +156,7 @@ add_contrib (libgsasl-cmake libgsasl) # requires krb5 add_contrib (librdkafka-cmake librdkafka) # requires: libgsasl add_contrib (nats-io-cmake nats-io) add_contrib (isa-l-cmake isa-l) -add_contrib (libhdfs3-cmake libhdfs3) # requires: protobuf, krb5, isa-l +add_contrib (libhdfs3-cmake libhdfs3) # requires: google-protobuf, krb5, isa-l add_contrib (hive-metastore-cmake hive-metastore) # requires: thrift/avro/arrow/libhdfs3 add_contrib (cppkafka-cmake cppkafka) add_contrib (libpqxx-cmake libpqxx) diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt new file mode 100644 index 00000000000..e2d38acb51d --- /dev/null +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -0,0 +1,329 @@ +option(ENABLE_PROTOBUF "Enable protobuf" ${ENABLE_LIBRARIES}) + +if(NOT ENABLE_PROTOBUF) + message(STATUS "Not using protobuf") + return() +endif() + +set(Protobuf_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/google-protobuf/src") +if(OS_FREEBSD AND SANITIZE STREQUAL "address") + # ../contrib/protobuf/src/google/protobuf/arena_impl.h:45:10: fatal error: 'sanitizer/asan_interface.h' file not found + # #include + if(LLVM_INCLUDE_DIRS) + set(Protobuf_INCLUDE_DIR "${Protobuf_INCLUDE_DIR}" ${LLVM_INCLUDE_DIRS}) + else() + message(${RECONFIGURE_MESSAGE_LEVEL} "Can't use protobuf on FreeBSD with address sanitizer without LLVM") + return() + endif() +endif() + +set(protobuf_source_dir "${ClickHouse_SOURCE_DIR}/contrib/google-protobuf") +set(protobuf_binary_dir "${ClickHouse_BINARY_DIR}/contrib/google-protobuf") + + +add_definitions(-DGOOGLE_PROTOBUF_CMAKE_BUILD) + +add_definitions(-DHAVE_PTHREAD) +add_definitions(-DHAVE_ZLIB) + +include_directories( + ${protobuf_binary_dir} + ${protobuf_source_dir}/src) + +set(libprotobuf_lite_files + ${protobuf_source_dir}/src/google/protobuf/any_lite.cc + ${protobuf_source_dir}/src/google/protobuf/arena.cc + ${protobuf_source_dir}/src/google/protobuf/arenastring.cc + ${protobuf_source_dir}/src/google/protobuf/extension_set.cc + ${protobuf_source_dir}/src/google/protobuf/field_access_listener.cc + ${protobuf_source_dir}/src/google/protobuf/generated_enum_util.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven_lite.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_util.cc + ${protobuf_source_dir}/src/google/protobuf/implicit_weak_message.cc + ${protobuf_source_dir}/src/google/protobuf/io/coded_stream.cc + ${protobuf_source_dir}/src/google/protobuf/io/io_win32.cc + ${protobuf_source_dir}/src/google/protobuf/io/strtod.cc + ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream.cc + ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream_impl.cc + ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream_impl_lite.cc + ${protobuf_source_dir}/src/google/protobuf/map.cc + ${protobuf_source_dir}/src/google/protobuf/message_lite.cc + ${protobuf_source_dir}/src/google/protobuf/parse_context.cc + ${protobuf_source_dir}/src/google/protobuf/repeated_field.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/bytestream.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/common.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/int128.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/status.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/statusor.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/stringpiece.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/stringprintf.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/structurally_valid.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/strutil.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/time.cc + ${protobuf_source_dir}/src/google/protobuf/wire_format_lite.cc +) + +add_library(_libprotobuf-lite ${libprotobuf_lite_files}) +target_link_libraries(_libprotobuf-lite pthread) +if(${CMAKE_SYSTEM_NAME} STREQUAL "Android") + target_link_libraries(_libprotobuf-lite log) +endif() +target_include_directories(_libprotobuf-lite SYSTEM PUBLIC ${protobuf_source_dir}/src) +add_library(protobuf::libprotobuf-lite ALIAS _libprotobuf-lite) + + +set(libprotobuf_files + ${protobuf_source_dir}/src/google/protobuf/any.cc + ${protobuf_source_dir}/src/google/protobuf/any.pb.cc + ${protobuf_source_dir}/src/google/protobuf/api.pb.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/importer.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/parser.cc + ${protobuf_source_dir}/src/google/protobuf/descriptor.cc + ${protobuf_source_dir}/src/google/protobuf/descriptor.pb.cc + ${protobuf_source_dir}/src/google/protobuf/descriptor_database.cc + ${protobuf_source_dir}/src/google/protobuf/duration.pb.cc + ${protobuf_source_dir}/src/google/protobuf/dynamic_message.cc + ${protobuf_source_dir}/src/google/protobuf/empty.pb.cc + ${protobuf_source_dir}/src/google/protobuf/extension_set_heavy.cc + ${protobuf_source_dir}/src/google/protobuf/field_mask.pb.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_reflection.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven.cc + ${protobuf_source_dir}/src/google/protobuf/io/gzip_stream.cc + ${protobuf_source_dir}/src/google/protobuf/io/printer.cc + ${protobuf_source_dir}/src/google/protobuf/io/tokenizer.cc + ${protobuf_source_dir}/src/google/protobuf/map_field.cc + ${protobuf_source_dir}/src/google/protobuf/message.cc + ${protobuf_source_dir}/src/google/protobuf/reflection_ops.cc + ${protobuf_source_dir}/src/google/protobuf/service.cc + ${protobuf_source_dir}/src/google/protobuf/source_context.pb.cc + ${protobuf_source_dir}/src/google/protobuf/struct.pb.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/substitute.cc + ${protobuf_source_dir}/src/google/protobuf/text_format.cc + ${protobuf_source_dir}/src/google/protobuf/timestamp.pb.cc + ${protobuf_source_dir}/src/google/protobuf/type.pb.cc + ${protobuf_source_dir}/src/google/protobuf/unknown_field_set.cc + ${protobuf_source_dir}/src/google/protobuf/util/delimited_message_util.cc + ${protobuf_source_dir}/src/google/protobuf/util/field_comparator.cc + ${protobuf_source_dir}/src/google/protobuf/util/field_mask_util.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/datapiece.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/default_value_objectwriter.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/error_listener.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/field_mask_utility.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/json_escaping.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/json_objectwriter.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/json_stream_parser.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/object_writer.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/proto_writer.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/protostream_objectsource.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/protostream_objectwriter.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/type_info.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/type_info_test_helper.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/utility.cc + ${protobuf_source_dir}/src/google/protobuf/util/json_util.cc + ${protobuf_source_dir}/src/google/protobuf/util/message_differencer.cc + ${protobuf_source_dir}/src/google/protobuf/util/time_util.cc + ${protobuf_source_dir}/src/google/protobuf/util/type_resolver_util.cc + ${protobuf_source_dir}/src/google/protobuf/wire_format.cc + ${protobuf_source_dir}/src/google/protobuf/wrappers.pb.cc +) + +add_library(_libprotobuf ${libprotobuf_lite_files} ${libprotobuf_files}) +if (ENABLE_FUZZING) + target_compile_options(_libprotobuf PRIVATE "-fsanitize-recover=all") +endif() +target_link_libraries(_libprotobuf pthread) +target_link_libraries(_libprotobuf ch_contrib::zlib) +if(${CMAKE_SYSTEM_NAME} STREQUAL "Android") + target_link_libraries(_libprotobuf log) +endif() +target_include_directories(_libprotobuf SYSTEM PUBLIC ${protobuf_source_dir}/src) +add_library(protobuf::libprotobuf ALIAS _libprotobuf) + + +set(libprotoc_files + ${protobuf_source_dir}/src/google/protobuf/compiler/code_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/command_line_interface.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_enum.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_extension.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_file.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_helpers.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_map_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_message.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_padding_optimizer.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_parse_function_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_service.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_string_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_doc_comment.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_enum.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_field_base.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_helpers.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_map_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_message.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_reflection_class.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_source_generator_base.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_wrapper_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_context.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_doc_comment.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_extension.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_extension_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_file.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_generator_factory.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_helpers.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_kotlin_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_map_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_map_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_builder.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_builder_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_name_resolver.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_primitive_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_service.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_shared_code_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_string_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_string_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/js/js_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/js/well_known_types_embed.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_enum.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_extension.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_file.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_helpers.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_map_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_message.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_oneof.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/php/php_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/plugin.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/plugin.pb.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/python/python_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/ruby/ruby_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/subprocess.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/zip_writer.cc +) + +add_library(_libprotoc ${libprotoc_files}) +target_link_libraries(_libprotoc _libprotobuf) +add_library(protobuf::libprotoc ALIAS _libprotoc) + +set(protoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/main.cc) + +if (CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME + AND CMAKE_HOST_SYSTEM_PROCESSOR STREQUAL CMAKE_SYSTEM_PROCESSOR) + + add_executable(protoc ${protoc_files}) + target_link_libraries(protoc _libprotoc _libprotobuf pthread) + add_executable(protobuf::protoc ALIAS protoc) + + if (ENABLE_FUZZING) + # `protoc` will be built with sanitizer and it could fail during ClickHouse build + # It easily reproduces in oss-fuzz building pipeline + # To avoid this we can try to build `protoc` without any sanitizer with option `-fno-sanitize=all`, but + # it this case we will face with linker errors, because libcxx still will be built with sanitizer + # So, we can simply suppress all of these failures with a combination this flag and an environment variable + # export MSAN_OPTIONS=exit_code=0 + target_compile_options(protoc PRIVATE "-fsanitize-recover=all") + endif() +else () + # Build 'protoc' for host arch + set (PROTOC_BUILD_DIR "${protobuf_binary_dir}/build") + + if (NOT EXISTS "${PROTOC_BUILD_DIR}/protoc") + + # This is quite ugly but I cannot make dependencies work propery. + + execute_process( + COMMAND mkdir -p ${PROTOC_BUILD_DIR} + COMMAND_ECHO STDOUT) + + execute_process( + COMMAND ${CMAKE_COMMAND} + "-G${CMAKE_GENERATOR}" + "-DCMAKE_MAKE_PROGRAM=${CMAKE_MAKE_PROGRAM}" + "-DCMAKE_C_COMPILER=${CMAKE_C_COMPILER}" + "-DCMAKE_CXX_COMPILER=${CMAKE_CXX_COMPILER}" + "-Dprotobuf_BUILD_TESTS=0" + "-Dprotobuf_BUILD_CONFORMANCE=0" + "-Dprotobuf_BUILD_EXAMPLES=0" + "-Dprotobuf_BUILD_PROTOC_BINARIES=1" + "${protobuf_source_dir}/cmake" + WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" + COMMAND_ECHO STDOUT) + + execute_process( + COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" + COMMAND_ECHO STDOUT) + endif () + +# add_custom_command ( +# OUTPUT ${PROTOC_BUILD_DIR} +# COMMAND mkdir -p ${PROTOC_BUILD_DIR}) +# +# add_custom_command ( +# OUTPUT "${PROTOC_BUILD_DIR}/CMakeCache.txt" +# +# COMMAND ${CMAKE_COMMAND} +# -G"${CMAKE_GENERATOR}" +# -DCMAKE_MAKE_PROGRAM="${CMAKE_MAKE_PROGRAM}" +# -DCMAKE_C_COMPILER="${CMAKE_C_COMPILER}" +# -DCMAKE_CXX_COMPILER="${CMAKE_CXX_COMPILER}" +# -Dprotobuf_BUILD_TESTS=0 +# -Dprotobuf_BUILD_CONFORMANCE=0 +# -Dprotobuf_BUILD_EXAMPLES=0 +# -Dprotobuf_BUILD_PROTOC_BINARIES=1 +# "${protobuf_source_dir}/cmake" +# +# DEPENDS "${PROTOC_BUILD_DIR}" +# WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" +# COMMENT "Configuring 'protoc' for host architecture." +# USES_TERMINAL) +# +# add_custom_command ( +# OUTPUT "${PROTOC_BUILD_DIR}/protoc" +# COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" +# DEPENDS "${PROTOC_BUILD_DIR}/CMakeCache.txt" +# COMMENT "Building 'protoc' for host architecture." +# USES_TERMINAL) +# +# add_custom_target (protoc-host DEPENDS "${PROTOC_BUILD_DIR}/protoc") + + add_executable(protoc IMPORTED GLOBAL) + set_target_properties (protoc PROPERTIES IMPORTED_LOCATION "${PROTOC_BUILD_DIR}/protoc") + add_dependencies(protoc "${PROTOC_BUILD_DIR}/protoc") +endif () + +include("${ClickHouse_SOURCE_DIR}/contrib/google-protobuf-cmake/protobuf_generate.cmake") + +add_library(_protobuf INTERFACE) +target_link_libraries(_protobuf INTERFACE _libprotobuf) +target_include_directories(_protobuf INTERFACE "${Protobuf_INCLUDE_DIR}") +add_library(ch_contrib::protobuf ALIAS _protobuf) + +add_library(_protoc INTERFACE) +target_link_libraries(_protoc INTERFACE _libprotoc _libprotobuf) +target_include_directories(_protoc INTERFACE "${Protobuf_INCLUDE_DIR}") +add_library(ch_contrib::protoc ALIAS _protoc) diff --git a/contrib/google-protobuf-cmake/protobuf_generate.cmake b/contrib/google-protobuf-cmake/protobuf_generate.cmake new file mode 100644 index 00000000000..3e30b4e40fd --- /dev/null +++ b/contrib/google-protobuf-cmake/protobuf_generate.cmake @@ -0,0 +1,198 @@ +# The code in this file was copied from https://github.com/Kitware/CMake/blob/master/Modules/FindProtobuf.cmake + +#[[ +Add custom commands to process ``.proto`` files to C++:: + +protobuf_generate_cpp ( + [DESCRIPTORS ] [EXPORT_MACRO ] [...]) + +``SRCS`` + Variable to define with autogenerated source files +``HDRS`` + Variable to define with autogenerated header files +``DESCRIPTORS`` + Variable to define with autogenerated descriptor files, if requested. +``EXPORT_MACRO`` + is a macro which should expand to ``__declspec(dllexport)`` or + ``__declspec(dllimport)`` depending on what is being compiled. +``ARGN`` + ``.proto`` files +#]] + +function(PROTOBUF_GENERATE_CPP SRCS HDRS) + cmake_parse_arguments(protobuf_generate_cpp "" "EXPORT_MACRO;DESCRIPTORS" "" ${ARGN}) + + set(_proto_files "${protobuf_generate_cpp_UNPARSED_ARGUMENTS}") + if(NOT _proto_files) + message(SEND_ERROR "Error: PROTOBUF_GENERATE_CPP() called without any proto files") + return() + endif() + + if(PROTOBUF_GENERATE_CPP_APPEND_PATH) + set(_append_arg APPEND_PATH) + endif() + + if(protobuf_generate_cpp_DESCRIPTORS) + set(_descriptors DESCRIPTORS) + endif() + + if(DEFINED PROTOBUF_IMPORT_DIRS AND NOT DEFINED Protobuf_IMPORT_DIRS) + set(Protobuf_IMPORT_DIRS "${PROTOBUF_IMPORT_DIRS}") + endif() + + if(DEFINED Protobuf_IMPORT_DIRS) + set(_import_arg IMPORT_DIRS ${Protobuf_IMPORT_DIRS}) + endif() + + set(_outvar) + protobuf_generate(${_append_arg} ${_descriptors} LANGUAGE cpp EXPORT_MACRO ${protobuf_generate_cpp_EXPORT_MACRO} OUT_VAR _outvar ${_import_arg} PROTOS ${_proto_files}) + + set(${SRCS}) + set(${HDRS}) + if(protobuf_generate_cpp_DESCRIPTORS) + set(${protobuf_generate_cpp_DESCRIPTORS}) + endif() + + foreach(_file ${_outvar}) + if(_file MATCHES "cc$") + list(APPEND ${SRCS} ${_file}) + elseif(_file MATCHES "desc$") + list(APPEND ${protobuf_generate_cpp_DESCRIPTORS} ${_file}) + else() + list(APPEND ${HDRS} ${_file}) + endif() + endforeach() + set(${SRCS} ${${SRCS}} PARENT_SCOPE) + set(${HDRS} ${${HDRS}} PARENT_SCOPE) + if(protobuf_generate_cpp_DESCRIPTORS) + set(${protobuf_generate_cpp_DESCRIPTORS} "${${protobuf_generate_cpp_DESCRIPTORS}}" PARENT_SCOPE) + endif() +endfunction() + +# By default have PROTOBUF_GENERATE_CPP macro pass -I to protoc +# for each directory where a proto file is referenced. +if(NOT DEFINED PROTOBUF_GENERATE_CPP_APPEND_PATH) + set(PROTOBUF_GENERATE_CPP_APPEND_PATH TRUE) +endif() + +function(protobuf_generate) + set(_options APPEND_PATH DESCRIPTORS) + set(_singleargs LANGUAGE OUT_VAR EXPORT_MACRO PROTOC_OUT_DIR) + if(COMMAND target_sources) + list(APPEND _singleargs TARGET) + endif() + set(_multiargs PROTOS IMPORT_DIRS GENERATE_EXTENSIONS) + + cmake_parse_arguments(protobuf_generate "${_options}" "${_singleargs}" "${_multiargs}" "${ARGN}") + + if(NOT protobuf_generate_PROTOS AND NOT protobuf_generate_TARGET) + message(SEND_ERROR "Error: protobuf_generate called without any targets or source files") + return() + endif() + + if(NOT protobuf_generate_OUT_VAR AND NOT protobuf_generate_TARGET) + message(SEND_ERROR "Error: protobuf_generate called without a target or output variable") + return() + endif() + + if(NOT protobuf_generate_LANGUAGE) + set(protobuf_generate_LANGUAGE cpp) + endif() + string(TOLOWER ${protobuf_generate_LANGUAGE} protobuf_generate_LANGUAGE) + + if(NOT protobuf_generate_PROTOC_OUT_DIR) + set(protobuf_generate_PROTOC_OUT_DIR ${CMAKE_CURRENT_BINARY_DIR}) + endif() + + if(protobuf_generate_EXPORT_MACRO AND protobuf_generate_LANGUAGE STREQUAL cpp) + set(_dll_export_decl "dllexport_decl=${protobuf_generate_EXPORT_MACRO}:") + endif() + + if(NOT protobuf_generate_GENERATE_EXTENSIONS) + if(protobuf_generate_LANGUAGE STREQUAL cpp) + set(protobuf_generate_GENERATE_EXTENSIONS .pb.h .pb.cc) + elseif(protobuf_generate_LANGUAGE STREQUAL python) + set(protobuf_generate_GENERATE_EXTENSIONS _pb2.py) + else() + message(SEND_ERROR "Error: protobuf_generate given unknown Language ${LANGUAGE}, please provide a value for GENERATE_EXTENSIONS") + return() + endif() + endif() + + if(protobuf_generate_TARGET) + get_target_property(_source_list ${protobuf_generate_TARGET} SOURCES) + foreach(_file ${_source_list}) + if(_file MATCHES "proto$") + list(APPEND protobuf_generate_PROTOS ${_file}) + endif() + endforeach() + endif() + + if(NOT protobuf_generate_PROTOS) + message(SEND_ERROR "Error: protobuf_generate could not find any .proto files") + return() + endif() + + if(protobuf_generate_APPEND_PATH) + # Create an include path for each file specified + foreach(_file ${protobuf_generate_PROTOS}) + get_filename_component(_abs_file ${_file} ABSOLUTE) + get_filename_component(_abs_path ${_abs_file} PATH) + list(FIND _protobuf_include_path ${_abs_path} _contains_already) + if(${_contains_already} EQUAL -1) + list(APPEND _protobuf_include_path -I ${_abs_path}) + endif() + endforeach() + else() + set(_protobuf_include_path -I ${CMAKE_CURRENT_SOURCE_DIR}) + endif() + + foreach(DIR ${protobuf_generate_IMPORT_DIRS}) + get_filename_component(ABS_PATH ${DIR} ABSOLUTE) + list(FIND _protobuf_include_path ${ABS_PATH} _contains_already) + if(${_contains_already} EQUAL -1) + list(APPEND _protobuf_include_path -I ${ABS_PATH}) + endif() + endforeach() + + 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() + + set(_generated_srcs) + foreach(_ext ${protobuf_generate_GENERATE_EXTENSIONS}) + list(APPEND _generated_srcs "${protobuf_generate_PROTOC_OUT_DIR}/${_possible_rel_dir}${_basename}${_ext}") + endforeach() + + if(protobuf_generate_DESCRIPTORS AND protobuf_generate_LANGUAGE STREQUAL cpp) + set(_descriptor_file "${CMAKE_CURRENT_BINARY_DIR}/${_basename}.desc") + set(_dll_desc_out "--descriptor_set_out=${_descriptor_file}") + list(APPEND _generated_srcs ${_descriptor_file}) + endif() + list(APPEND _generated_srcs_all ${_generated_srcs}) + + add_custom_command( + OUTPUT ${_generated_srcs} + COMMAND $ + ARGS --${protobuf_generate_LANGUAGE}_out ${_dll_export_decl}${protobuf_generate_PROTOC_OUT_DIR} ${_dll_desc_out} ${_protobuf_include_path} ${_abs_file} + DEPENDS ${_abs_file} protoc + COMMENT "Running ${protobuf_generate_LANGUAGE} protocol buffer compiler on ${_proto}" + VERBATIM) + endforeach() + + set_source_files_properties(${_generated_srcs_all} PROPERTIES GENERATED TRUE) + if(protobuf_generate_OUT_VAR) + set(${protobuf_generate_OUT_VAR} ${_generated_srcs_all} PARENT_SCOPE) + endif() + if(protobuf_generate_TARGET) + target_sources(${protobuf_generate_TARGET} PRIVATE ${_generated_srcs_all}) + endif() +endfunction() From 9d4c8c30ab7dc00f0f5f0930177856c933a767f5 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 31 May 2023 19:57:37 +0300 Subject: [PATCH 184/308] Rename 01472_toStartOfInterval_disallow_empty_tz_field test to 01472_toBoundsOfInterval_disallow_empty_tz_field --- ...=> 01472_toBoundsOfInterval_disallow_empty_tz_field.reference} | 0 ...d.sql => 01472_toBoundsOfInterval_disallow_empty_tz_field.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01472_toStartOfInterval_disallow_empty_tz_field.reference => 01472_toBoundsOfInterval_disallow_empty_tz_field.reference} (100%) rename tests/queries/0_stateless/{01472_toStartOfInterval_disallow_empty_tz_field.sql => 01472_toBoundsOfInterval_disallow_empty_tz_field.sql} (100%) diff --git a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.reference b/tests/queries/0_stateless/01472_toBoundsOfInterval_disallow_empty_tz_field.reference similarity index 100% rename from tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.reference rename to tests/queries/0_stateless/01472_toBoundsOfInterval_disallow_empty_tz_field.reference diff --git a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql b/tests/queries/0_stateless/01472_toBoundsOfInterval_disallow_empty_tz_field.sql similarity index 100% rename from tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql rename to tests/queries/0_stateless/01472_toBoundsOfInterval_disallow_empty_tz_field.sql From 495580918a285ff46ddb3fb91f3b66885b6e2138 Mon Sep 17 00:00:00 2001 From: alekar Date: Wed, 31 May 2023 10:00:19 -0700 Subject: [PATCH 185/308] Update base/base/getMemoryAmount.cpp Co-authored-by: Sergei Trifonov --- base/base/getMemoryAmount.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index 6a5470a0549..8a2fcd092d2 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -29,7 +29,7 @@ uint64_t getMemoryAmountOrZero() #if defined(OS_LINUX) // Try to lookup at the Cgroup limit - // v2 + // CGroups v2 std::ifstream cgroupv2_limit("/sys/fs/cgroup/memory.max"); if (cgroupv2_limit.is_open()) { From cb85e5a01ea1e823229c312279c67b12a5d6d3db Mon Sep 17 00:00:00 2001 From: alekar Date: Wed, 31 May 2023 10:00:43 -0700 Subject: [PATCH 186/308] Update base/base/getMemoryAmount.cpp Co-authored-by: Sergei Trifonov --- base/base/getMemoryAmount.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index 8a2fcd092d2..a46e964c5a3 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -40,7 +40,7 @@ uint64_t getMemoryAmountOrZero() } else { - // v1 + // CGroups v1 std::ifstream cgroup_limit("/sys/fs/cgroup/memory/memory.limit_in_bytes"); if (cgroup_limit.is_open()) { From 57c88e664c856d44eb144416947d30ef19f5d073 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 17:01:00 +0000 Subject: [PATCH 187/308] Remove old protobuf --- .gitmodules | 4 - contrib/protobuf | 1 - contrib/protobuf-cmake/CMakeLists.txt | 329 ------------------ .../protobuf-cmake/protobuf_generate.cmake | 198 ----------- 4 files changed, 532 deletions(-) delete mode 160000 contrib/protobuf delete mode 100644 contrib/protobuf-cmake/CMakeLists.txt delete mode 100644 contrib/protobuf-cmake/protobuf_generate.cmake diff --git a/.gitmodules b/.gitmodules index d28f205b65c..e28d8257465 100644 --- a/.gitmodules +++ b/.gitmodules @@ -35,10 +35,6 @@ [submodule "contrib/unixodbc"] path = contrib/unixodbc url = https://github.com/ClickHouse/UnixODBC -[submodule "contrib/protobuf"] - path = contrib/protobuf - url = https://github.com/ClickHouse/protobuf - branch = v3.13.0.1 [submodule "contrib/google-protobuf"] path = contrib/google-protobuf url = https://github.com/ClickHouse/google-protobuf.git diff --git a/contrib/protobuf b/contrib/protobuf deleted file mode 160000 index 6bb70196c53..00000000000 --- a/contrib/protobuf +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 6bb70196c5360268d9f021bb7936fb0b551724c2 diff --git a/contrib/protobuf-cmake/CMakeLists.txt b/contrib/protobuf-cmake/CMakeLists.txt deleted file mode 100644 index 5e22136fc1f..00000000000 --- a/contrib/protobuf-cmake/CMakeLists.txt +++ /dev/null @@ -1,329 +0,0 @@ -option(ENABLE_PROTOBUF "Enable protobuf" ${ENABLE_LIBRARIES}) - -if(NOT ENABLE_PROTOBUF) - message(STATUS "Not using protobuf") - return() -endif() - -set(Protobuf_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") -if(OS_FREEBSD AND SANITIZE STREQUAL "address") - # ../contrib/protobuf/src/google/protobuf/arena_impl.h:45:10: fatal error: 'sanitizer/asan_interface.h' file not found - # #include - if(LLVM_INCLUDE_DIRS) - set(Protobuf_INCLUDE_DIR "${Protobuf_INCLUDE_DIR}" ${LLVM_INCLUDE_DIRS}) - else() - message(${RECONFIGURE_MESSAGE_LEVEL} "Can't use protobuf on FreeBSD with address sanitizer without LLVM") - return() - endif() -endif() - -set(protobuf_source_dir "${ClickHouse_SOURCE_DIR}/contrib/protobuf") -set(protobuf_binary_dir "${ClickHouse_BINARY_DIR}/contrib/protobuf") - - -add_definitions(-DGOOGLE_PROTOBUF_CMAKE_BUILD) - -add_definitions(-DHAVE_PTHREAD) -add_definitions(-DHAVE_ZLIB) - -include_directories( - ${protobuf_binary_dir} - ${protobuf_source_dir}/src) - -set(libprotobuf_lite_files - ${protobuf_source_dir}/src/google/protobuf/any_lite.cc - ${protobuf_source_dir}/src/google/protobuf/arena.cc - ${protobuf_source_dir}/src/google/protobuf/arenastring.cc - ${protobuf_source_dir}/src/google/protobuf/extension_set.cc - ${protobuf_source_dir}/src/google/protobuf/field_access_listener.cc - ${protobuf_source_dir}/src/google/protobuf/generated_enum_util.cc - ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven_lite.cc - ${protobuf_source_dir}/src/google/protobuf/generated_message_util.cc - ${protobuf_source_dir}/src/google/protobuf/implicit_weak_message.cc - ${protobuf_source_dir}/src/google/protobuf/io/coded_stream.cc - ${protobuf_source_dir}/src/google/protobuf/io/io_win32.cc - ${protobuf_source_dir}/src/google/protobuf/io/strtod.cc - ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream.cc - ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream_impl.cc - ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream_impl_lite.cc - ${protobuf_source_dir}/src/google/protobuf/map.cc - ${protobuf_source_dir}/src/google/protobuf/message_lite.cc - ${protobuf_source_dir}/src/google/protobuf/parse_context.cc - ${protobuf_source_dir}/src/google/protobuf/repeated_field.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/bytestream.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/common.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/int128.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/status.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/statusor.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/stringpiece.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/stringprintf.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/structurally_valid.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/strutil.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/time.cc - ${protobuf_source_dir}/src/google/protobuf/wire_format_lite.cc -) - -add_library(_libprotobuf-lite ${libprotobuf_lite_files}) -target_link_libraries(_libprotobuf-lite pthread) -if(${CMAKE_SYSTEM_NAME} STREQUAL "Android") - target_link_libraries(_libprotobuf-lite log) -endif() -target_include_directories(_libprotobuf-lite SYSTEM PUBLIC ${protobuf_source_dir}/src) -add_library(protobuf::libprotobuf-lite ALIAS _libprotobuf-lite) - - -set(libprotobuf_files - ${protobuf_source_dir}/src/google/protobuf/any.cc - ${protobuf_source_dir}/src/google/protobuf/any.pb.cc - ${protobuf_source_dir}/src/google/protobuf/api.pb.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/importer.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/parser.cc - ${protobuf_source_dir}/src/google/protobuf/descriptor.cc - ${protobuf_source_dir}/src/google/protobuf/descriptor.pb.cc - ${protobuf_source_dir}/src/google/protobuf/descriptor_database.cc - ${protobuf_source_dir}/src/google/protobuf/duration.pb.cc - ${protobuf_source_dir}/src/google/protobuf/dynamic_message.cc - ${protobuf_source_dir}/src/google/protobuf/empty.pb.cc - ${protobuf_source_dir}/src/google/protobuf/extension_set_heavy.cc - ${protobuf_source_dir}/src/google/protobuf/field_mask.pb.cc - ${protobuf_source_dir}/src/google/protobuf/generated_message_reflection.cc - ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven.cc - ${protobuf_source_dir}/src/google/protobuf/io/gzip_stream.cc - ${protobuf_source_dir}/src/google/protobuf/io/printer.cc - ${protobuf_source_dir}/src/google/protobuf/io/tokenizer.cc - ${protobuf_source_dir}/src/google/protobuf/map_field.cc - ${protobuf_source_dir}/src/google/protobuf/message.cc - ${protobuf_source_dir}/src/google/protobuf/reflection_ops.cc - ${protobuf_source_dir}/src/google/protobuf/service.cc - ${protobuf_source_dir}/src/google/protobuf/source_context.pb.cc - ${protobuf_source_dir}/src/google/protobuf/struct.pb.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/substitute.cc - ${protobuf_source_dir}/src/google/protobuf/text_format.cc - ${protobuf_source_dir}/src/google/protobuf/timestamp.pb.cc - ${protobuf_source_dir}/src/google/protobuf/type.pb.cc - ${protobuf_source_dir}/src/google/protobuf/unknown_field_set.cc - ${protobuf_source_dir}/src/google/protobuf/util/delimited_message_util.cc - ${protobuf_source_dir}/src/google/protobuf/util/field_comparator.cc - ${protobuf_source_dir}/src/google/protobuf/util/field_mask_util.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/datapiece.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/default_value_objectwriter.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/error_listener.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/field_mask_utility.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/json_escaping.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/json_objectwriter.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/json_stream_parser.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/object_writer.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/proto_writer.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/protostream_objectsource.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/protostream_objectwriter.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/type_info.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/type_info_test_helper.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/utility.cc - ${protobuf_source_dir}/src/google/protobuf/util/json_util.cc - ${protobuf_source_dir}/src/google/protobuf/util/message_differencer.cc - ${protobuf_source_dir}/src/google/protobuf/util/time_util.cc - ${protobuf_source_dir}/src/google/protobuf/util/type_resolver_util.cc - ${protobuf_source_dir}/src/google/protobuf/wire_format.cc - ${protobuf_source_dir}/src/google/protobuf/wrappers.pb.cc -) - -add_library(_libprotobuf ${libprotobuf_lite_files} ${libprotobuf_files}) -if (ENABLE_FUZZING) - target_compile_options(_libprotobuf PRIVATE "-fsanitize-recover=all") -endif() -target_link_libraries(_libprotobuf pthread) -target_link_libraries(_libprotobuf ch_contrib::zlib) -if(${CMAKE_SYSTEM_NAME} STREQUAL "Android") - target_link_libraries(_libprotobuf log) -endif() -target_include_directories(_libprotobuf SYSTEM PUBLIC ${protobuf_source_dir}/src) -add_library(protobuf::libprotobuf ALIAS _libprotobuf) - - -set(libprotoc_files - ${protobuf_source_dir}/src/google/protobuf/compiler/code_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/command_line_interface.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_enum.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_extension.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_file.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_helpers.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_map_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_message.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_padding_optimizer.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_parse_function_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_primitive_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_service.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_string_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_doc_comment.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_enum.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_field_base.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_helpers.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_map_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_message.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_primitive_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_reflection_class.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_primitive_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_source_generator_base.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_wrapper_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_context.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_doc_comment.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_extension.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_extension_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_file.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_generator_factory.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_helpers.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_kotlin_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_map_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_map_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_builder.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_builder_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_name_resolver.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_primitive_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_primitive_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_service.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_shared_code_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_string_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_string_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/js/js_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/js/well_known_types_embed.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_enum.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_extension.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_file.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_helpers.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_map_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_message.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_oneof.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_primitive_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/php/php_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/plugin.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/plugin.pb.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/python/python_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/ruby/ruby_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/subprocess.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/zip_writer.cc -) - -add_library(_libprotoc ${libprotoc_files}) -target_link_libraries(_libprotoc _libprotobuf) -add_library(protobuf::libprotoc ALIAS _libprotoc) - -set(protoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/main.cc) - -if (CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME - AND CMAKE_HOST_SYSTEM_PROCESSOR STREQUAL CMAKE_SYSTEM_PROCESSOR) - - add_executable(protoc ${protoc_files}) - target_link_libraries(protoc _libprotoc _libprotobuf pthread) - add_executable(protobuf::protoc ALIAS protoc) - - if (ENABLE_FUZZING) - # `protoc` will be built with sanitizer and it could fail during ClickHouse build - # It easily reproduces in oss-fuzz building pipeline - # To avoid this we can try to build `protoc` without any sanitizer with option `-fno-sanitize=all`, but - # it this case we will face with linker errors, because libcxx still will be built with sanitizer - # So, we can simply suppress all of these failures with a combination this flag and an environment variable - # export MSAN_OPTIONS=exit_code=0 - target_compile_options(protoc PRIVATE "-fsanitize-recover=all") - endif() -else () - # Build 'protoc' for host arch - set (PROTOC_BUILD_DIR "${protobuf_binary_dir}/build") - - if (NOT EXISTS "${PROTOC_BUILD_DIR}/protoc") - - # This is quite ugly but I cannot make dependencies work propery. - - execute_process( - COMMAND mkdir -p ${PROTOC_BUILD_DIR} - COMMAND_ECHO STDOUT) - - execute_process( - COMMAND ${CMAKE_COMMAND} - "-G${CMAKE_GENERATOR}" - "-DCMAKE_MAKE_PROGRAM=${CMAKE_MAKE_PROGRAM}" - "-DCMAKE_C_COMPILER=${CMAKE_C_COMPILER}" - "-DCMAKE_CXX_COMPILER=${CMAKE_CXX_COMPILER}" - "-Dprotobuf_BUILD_TESTS=0" - "-Dprotobuf_BUILD_CONFORMANCE=0" - "-Dprotobuf_BUILD_EXAMPLES=0" - "-Dprotobuf_BUILD_PROTOC_BINARIES=1" - "${protobuf_source_dir}/cmake" - WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" - COMMAND_ECHO STDOUT) - - execute_process( - COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" - COMMAND_ECHO STDOUT) - endif () - -# add_custom_command ( -# OUTPUT ${PROTOC_BUILD_DIR} -# COMMAND mkdir -p ${PROTOC_BUILD_DIR}) -# -# add_custom_command ( -# OUTPUT "${PROTOC_BUILD_DIR}/CMakeCache.txt" -# -# COMMAND ${CMAKE_COMMAND} -# -G"${CMAKE_GENERATOR}" -# -DCMAKE_MAKE_PROGRAM="${CMAKE_MAKE_PROGRAM}" -# -DCMAKE_C_COMPILER="${CMAKE_C_COMPILER}" -# -DCMAKE_CXX_COMPILER="${CMAKE_CXX_COMPILER}" -# -Dprotobuf_BUILD_TESTS=0 -# -Dprotobuf_BUILD_CONFORMANCE=0 -# -Dprotobuf_BUILD_EXAMPLES=0 -# -Dprotobuf_BUILD_PROTOC_BINARIES=1 -# "${protobuf_source_dir}/cmake" -# -# DEPENDS "${PROTOC_BUILD_DIR}" -# WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" -# COMMENT "Configuring 'protoc' for host architecture." -# USES_TERMINAL) -# -# add_custom_command ( -# OUTPUT "${PROTOC_BUILD_DIR}/protoc" -# COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" -# DEPENDS "${PROTOC_BUILD_DIR}/CMakeCache.txt" -# COMMENT "Building 'protoc' for host architecture." -# USES_TERMINAL) -# -# add_custom_target (protoc-host DEPENDS "${PROTOC_BUILD_DIR}/protoc") - - add_executable(protoc IMPORTED GLOBAL) - set_target_properties (protoc PROPERTIES IMPORTED_LOCATION "${PROTOC_BUILD_DIR}/protoc") - add_dependencies(protoc "${PROTOC_BUILD_DIR}/protoc") -endif () - -include("${ClickHouse_SOURCE_DIR}/contrib/protobuf-cmake/protobuf_generate.cmake") - -add_library(_protobuf INTERFACE) -target_link_libraries(_protobuf INTERFACE _libprotobuf) -target_include_directories(_protobuf INTERFACE "${Protobuf_INCLUDE_DIR}") -add_library(ch_contrib::protobuf ALIAS _protobuf) - -add_library(_protoc INTERFACE) -target_link_libraries(_protoc INTERFACE _libprotoc _libprotobuf) -target_include_directories(_protoc INTERFACE "${Protobuf_INCLUDE_DIR}") -add_library(ch_contrib::protoc ALIAS _protoc) diff --git a/contrib/protobuf-cmake/protobuf_generate.cmake b/contrib/protobuf-cmake/protobuf_generate.cmake deleted file mode 100644 index 3e30b4e40fd..00000000000 --- a/contrib/protobuf-cmake/protobuf_generate.cmake +++ /dev/null @@ -1,198 +0,0 @@ -# The code in this file was copied from https://github.com/Kitware/CMake/blob/master/Modules/FindProtobuf.cmake - -#[[ -Add custom commands to process ``.proto`` files to C++:: - -protobuf_generate_cpp ( - [DESCRIPTORS ] [EXPORT_MACRO ] [...]) - -``SRCS`` - Variable to define with autogenerated source files -``HDRS`` - Variable to define with autogenerated header files -``DESCRIPTORS`` - Variable to define with autogenerated descriptor files, if requested. -``EXPORT_MACRO`` - is a macro which should expand to ``__declspec(dllexport)`` or - ``__declspec(dllimport)`` depending on what is being compiled. -``ARGN`` - ``.proto`` files -#]] - -function(PROTOBUF_GENERATE_CPP SRCS HDRS) - cmake_parse_arguments(protobuf_generate_cpp "" "EXPORT_MACRO;DESCRIPTORS" "" ${ARGN}) - - set(_proto_files "${protobuf_generate_cpp_UNPARSED_ARGUMENTS}") - if(NOT _proto_files) - message(SEND_ERROR "Error: PROTOBUF_GENERATE_CPP() called without any proto files") - return() - endif() - - if(PROTOBUF_GENERATE_CPP_APPEND_PATH) - set(_append_arg APPEND_PATH) - endif() - - if(protobuf_generate_cpp_DESCRIPTORS) - set(_descriptors DESCRIPTORS) - endif() - - if(DEFINED PROTOBUF_IMPORT_DIRS AND NOT DEFINED Protobuf_IMPORT_DIRS) - set(Protobuf_IMPORT_DIRS "${PROTOBUF_IMPORT_DIRS}") - endif() - - if(DEFINED Protobuf_IMPORT_DIRS) - set(_import_arg IMPORT_DIRS ${Protobuf_IMPORT_DIRS}) - endif() - - set(_outvar) - protobuf_generate(${_append_arg} ${_descriptors} LANGUAGE cpp EXPORT_MACRO ${protobuf_generate_cpp_EXPORT_MACRO} OUT_VAR _outvar ${_import_arg} PROTOS ${_proto_files}) - - set(${SRCS}) - set(${HDRS}) - if(protobuf_generate_cpp_DESCRIPTORS) - set(${protobuf_generate_cpp_DESCRIPTORS}) - endif() - - foreach(_file ${_outvar}) - if(_file MATCHES "cc$") - list(APPEND ${SRCS} ${_file}) - elseif(_file MATCHES "desc$") - list(APPEND ${protobuf_generate_cpp_DESCRIPTORS} ${_file}) - else() - list(APPEND ${HDRS} ${_file}) - endif() - endforeach() - set(${SRCS} ${${SRCS}} PARENT_SCOPE) - set(${HDRS} ${${HDRS}} PARENT_SCOPE) - if(protobuf_generate_cpp_DESCRIPTORS) - set(${protobuf_generate_cpp_DESCRIPTORS} "${${protobuf_generate_cpp_DESCRIPTORS}}" PARENT_SCOPE) - endif() -endfunction() - -# By default have PROTOBUF_GENERATE_CPP macro pass -I to protoc -# for each directory where a proto file is referenced. -if(NOT DEFINED PROTOBUF_GENERATE_CPP_APPEND_PATH) - set(PROTOBUF_GENERATE_CPP_APPEND_PATH TRUE) -endif() - -function(protobuf_generate) - set(_options APPEND_PATH DESCRIPTORS) - set(_singleargs LANGUAGE OUT_VAR EXPORT_MACRO PROTOC_OUT_DIR) - if(COMMAND target_sources) - list(APPEND _singleargs TARGET) - endif() - set(_multiargs PROTOS IMPORT_DIRS GENERATE_EXTENSIONS) - - cmake_parse_arguments(protobuf_generate "${_options}" "${_singleargs}" "${_multiargs}" "${ARGN}") - - if(NOT protobuf_generate_PROTOS AND NOT protobuf_generate_TARGET) - message(SEND_ERROR "Error: protobuf_generate called without any targets or source files") - return() - endif() - - if(NOT protobuf_generate_OUT_VAR AND NOT protobuf_generate_TARGET) - message(SEND_ERROR "Error: protobuf_generate called without a target or output variable") - return() - endif() - - if(NOT protobuf_generate_LANGUAGE) - set(protobuf_generate_LANGUAGE cpp) - endif() - string(TOLOWER ${protobuf_generate_LANGUAGE} protobuf_generate_LANGUAGE) - - if(NOT protobuf_generate_PROTOC_OUT_DIR) - set(protobuf_generate_PROTOC_OUT_DIR ${CMAKE_CURRENT_BINARY_DIR}) - endif() - - if(protobuf_generate_EXPORT_MACRO AND protobuf_generate_LANGUAGE STREQUAL cpp) - set(_dll_export_decl "dllexport_decl=${protobuf_generate_EXPORT_MACRO}:") - endif() - - if(NOT protobuf_generate_GENERATE_EXTENSIONS) - if(protobuf_generate_LANGUAGE STREQUAL cpp) - set(protobuf_generate_GENERATE_EXTENSIONS .pb.h .pb.cc) - elseif(protobuf_generate_LANGUAGE STREQUAL python) - set(protobuf_generate_GENERATE_EXTENSIONS _pb2.py) - else() - message(SEND_ERROR "Error: protobuf_generate given unknown Language ${LANGUAGE}, please provide a value for GENERATE_EXTENSIONS") - return() - endif() - endif() - - if(protobuf_generate_TARGET) - get_target_property(_source_list ${protobuf_generate_TARGET} SOURCES) - foreach(_file ${_source_list}) - if(_file MATCHES "proto$") - list(APPEND protobuf_generate_PROTOS ${_file}) - endif() - endforeach() - endif() - - if(NOT protobuf_generate_PROTOS) - message(SEND_ERROR "Error: protobuf_generate could not find any .proto files") - return() - endif() - - if(protobuf_generate_APPEND_PATH) - # Create an include path for each file specified - foreach(_file ${protobuf_generate_PROTOS}) - get_filename_component(_abs_file ${_file} ABSOLUTE) - get_filename_component(_abs_path ${_abs_file} PATH) - list(FIND _protobuf_include_path ${_abs_path} _contains_already) - if(${_contains_already} EQUAL -1) - list(APPEND _protobuf_include_path -I ${_abs_path}) - endif() - endforeach() - else() - set(_protobuf_include_path -I ${CMAKE_CURRENT_SOURCE_DIR}) - endif() - - foreach(DIR ${protobuf_generate_IMPORT_DIRS}) - get_filename_component(ABS_PATH ${DIR} ABSOLUTE) - list(FIND _protobuf_include_path ${ABS_PATH} _contains_already) - if(${_contains_already} EQUAL -1) - list(APPEND _protobuf_include_path -I ${ABS_PATH}) - endif() - endforeach() - - 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() - - set(_generated_srcs) - foreach(_ext ${protobuf_generate_GENERATE_EXTENSIONS}) - list(APPEND _generated_srcs "${protobuf_generate_PROTOC_OUT_DIR}/${_possible_rel_dir}${_basename}${_ext}") - endforeach() - - if(protobuf_generate_DESCRIPTORS AND protobuf_generate_LANGUAGE STREQUAL cpp) - set(_descriptor_file "${CMAKE_CURRENT_BINARY_DIR}/${_basename}.desc") - set(_dll_desc_out "--descriptor_set_out=${_descriptor_file}") - list(APPEND _generated_srcs ${_descriptor_file}) - endif() - list(APPEND _generated_srcs_all ${_generated_srcs}) - - add_custom_command( - OUTPUT ${_generated_srcs} - COMMAND $ - ARGS --${protobuf_generate_LANGUAGE}_out ${_dll_export_decl}${protobuf_generate_PROTOC_OUT_DIR} ${_dll_desc_out} ${_protobuf_include_path} ${_abs_file} - DEPENDS ${_abs_file} protoc - COMMENT "Running ${protobuf_generate_LANGUAGE} protocol buffer compiler on ${_proto}" - VERBATIM) - endforeach() - - set_source_files_properties(${_generated_srcs_all} PROPERTIES GENERATED TRUE) - if(protobuf_generate_OUT_VAR) - set(${protobuf_generate_OUT_VAR} ${_generated_srcs_all} PARENT_SCOPE) - endif() - if(protobuf_generate_TARGET) - target_sources(${protobuf_generate_TARGET} PRIVATE ${_generated_srcs_all}) - endif() -endfunction() From f57c5105f6318573107ab6ae0d01fdccb757cf73 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 17:02:05 +0000 Subject: [PATCH 188/308] Remove file deleted upstream from build description --- contrib/google-protobuf-cmake/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index e2d38acb51d..2d72ac90f49 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -35,7 +35,6 @@ set(libprotobuf_lite_files ${protobuf_source_dir}/src/google/protobuf/arena.cc ${protobuf_source_dir}/src/google/protobuf/arenastring.cc ${protobuf_source_dir}/src/google/protobuf/extension_set.cc - ${protobuf_source_dir}/src/google/protobuf/field_access_listener.cc ${protobuf_source_dir}/src/google/protobuf/generated_enum_util.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven_lite.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_util.cc From 2e2f98ced69f0c6205292cee1290cab6795914c5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 17:26:31 +0000 Subject: [PATCH 189/308] Add new source file --- contrib/google-protobuf-cmake/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index 2d72ac90f49..8afb86b25dd 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -85,6 +85,7 @@ set(libprotobuf_files ${protobuf_source_dir}/src/google/protobuf/empty.pb.cc ${protobuf_source_dir}/src/google/protobuf/extension_set_heavy.cc ${protobuf_source_dir}/src/google/protobuf/field_mask.pb.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_bases.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_reflection.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven.cc ${protobuf_source_dir}/src/google/protobuf/io/gzip_stream.cc From 95cbc11d7b440bb22a3b9118685213cffd41862c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 17:54:31 +0000 Subject: [PATCH 190/308] Remove c-ares submodule --- .gitmodules | 3 --- contrib/c-ares | 1 - 2 files changed, 4 deletions(-) delete mode 160000 contrib/c-ares diff --git a/.gitmodules b/.gitmodules index f0984fec4db..0915280e08c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -268,9 +268,6 @@ [submodule "contrib/vectorscan"] path = contrib/vectorscan url = https://github.com/VectorCamp/vectorscan.git -[submodule "contrib/c-ares"] - path = contrib/c-ares - url = https://github.com/ClickHouse/c-ares [submodule "contrib/llvm-project"] path = contrib/llvm-project url = https://github.com/ClickHouse/llvm-project diff --git a/contrib/c-ares b/contrib/c-ares deleted file mode 160000 index afee6748b0b..00000000000 --- a/contrib/c-ares +++ /dev/null @@ -1 +0,0 @@ -Subproject commit afee6748b0b99acf4509d42fa37ac8422262f91b From 3a1e07d4c18c276ba70fbb803ffa5a6e89d78c4b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 17:54:47 +0000 Subject: [PATCH 191/308] Add submodule c-ares --- .gitmodules | 3 +++ contrib/c-ares | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/c-ares diff --git a/.gitmodules b/.gitmodules index 0915280e08c..67f181414e5 100644 --- a/.gitmodules +++ b/.gitmodules @@ -341,3 +341,6 @@ [submodule "contrib/isa-l"] path = contrib/isa-l url = https://github.com/ClickHouse/isa-l.git +[submodule "contrib/c-ares"] + path = contrib/c-ares + url = https://github.com/c-ares/c-ares.git diff --git a/contrib/c-ares b/contrib/c-ares new file mode 160000 index 00000000000..903a613f9f7 --- /dev/null +++ b/contrib/c-ares @@ -0,0 +1 @@ +Subproject commit 903a613f9f72e1509bb04cd57b5f320fe7e4d39e From 5eb592acc57d4fdda62907935b5468bf390defc9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 17:57:50 +0000 Subject: [PATCH 192/308] Bump libxml2 from 2.10.3 to 2.10.4 CVE-2023-28484 CVE-2023-29469 --- contrib/libxml2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libxml2 b/contrib/libxml2 index f507d167f17..223cb03a5d2 160000 --- a/contrib/libxml2 +++ b/contrib/libxml2 @@ -1 +1 @@ -Subproject commit f507d167f1755b7eaea09fb1a44d29aab828b6d1 +Subproject commit 223cb03a5d27b1b2393b266a8657443d046139d6 From eeab9fb3d4a3be717159795d608b8776430e1c7b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 18:08:20 +0000 Subject: [PATCH 193/308] Add new file --- contrib/c-ares-cmake/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/c-ares-cmake/CMakeLists.txt b/contrib/c-ares-cmake/CMakeLists.txt index 63e145bec18..86ab6f90260 100644 --- a/contrib/c-ares-cmake/CMakeLists.txt +++ b/contrib/c-ares-cmake/CMakeLists.txt @@ -48,6 +48,7 @@ SET(SRCS "${LIBRARY_DIR}/src/lib/ares_platform.c" "${LIBRARY_DIR}/src/lib/ares_process.c" "${LIBRARY_DIR}/src/lib/ares_query.c" + "${LIBRARY_DIR}/src/lib/ares_rand.c" "${LIBRARY_DIR}/src/lib/ares_search.c" "${LIBRARY_DIR}/src/lib/ares_send.c" "${LIBRARY_DIR}/src/lib/ares_strcasecmp.c" From 2bf867c1cf325e237aae63f96003bd9bf37a0b59 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 18:13:18 +0000 Subject: [PATCH 194/308] Bump to v1.19.1 --- contrib/c-ares | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/c-ares b/contrib/c-ares index 903a613f9f7..6360e96b5cf 160000 --- a/contrib/c-ares +++ b/contrib/c-ares @@ -1 +1 @@ -Subproject commit 903a613f9f72e1509bb04cd57b5f320fe7e4d39e +Subproject commit 6360e96b5cf8e5980c887ce58ef727e53d77243a From 5b32c026b91f02ab0a1c22ab5c77c16681233d99 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 18:48:41 +0000 Subject: [PATCH 195/308] Fix CVE-2022-2469 --- contrib/libgsasl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libgsasl b/contrib/libgsasl index f4e7bf0bb06..0fb79e7609a 160000 --- a/contrib/libgsasl +++ b/contrib/libgsasl @@ -1 +1 @@ -Subproject commit f4e7bf0bb068030d57266f87ccac4c8c012fb5c4 +Subproject commit 0fb79e7609ae5a5e015a41d24bcbadd48f8f5469 From 3543d95980d5426fdc496bb3cafb572b8a62cf71 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 31 May 2023 22:10:33 +0200 Subject: [PATCH 196/308] Parallel replicas always skip unavailable ones (#50293) --- programs/server/config.xml | 5 ++ src/Interpreters/InterpreterSelectQuery.cpp | 6 +++ src/QueryPipeline/RemoteQueryExecutor.cpp | 39 ++++++++++----- src/QueryPipeline/RemoteQueryExecutor.h | 4 +- .../ParallelReplicasReadingCoordinator.cpp | 49 ++++++++++++++++--- .../ParallelReplicasReadingCoordinator.h | 7 ++- ...llel_replicas_unavailable_shards.reference | 2 + ...9_parallel_replicas_unavailable_shards.sql | 15 ++++++ 8 files changed, 105 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.reference create mode 100644 tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql diff --git a/programs/server/config.xml b/programs/server/config.xml index 5b69d9f6283..d18b4cb2ac9 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -909,6 +909,11 @@ 127.0.0.10 9000 + + + 127.0.0.11 + 1234 + diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a4ea474e4e9..d2be48dafb3 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -468,6 +468,12 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } + /// Set skip_unavailable_shards to true only if it wasn't disabled explicitly + if (settings.allow_experimental_parallel_reading_from_replicas > 0 && !settings.skip_unavailable_shards && !settings.isChanged("skip_unavailable_shards")) + { + context->setSetting("skip_unavailable_shards", true); + } + /// Check support for JOIN for parallel replicas with custom key if (joined_tables.tablesCount() > 1 && !settings.parallel_replicas_custom_key.value.empty()) { diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 991876eccbd..3f9f945fd45 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -47,8 +47,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( QueryProcessingStage::Enum stage_, std::optional extension_) : header(header_), query(query_), context(context_), scalars(scalars_) , external_tables(external_tables_), stage(stage_) - , task_iterator(extension_ ? extension_->task_iterator : nullptr) - , parallel_reading_coordinator(extension_ ? extension_->parallel_reading_coordinator : nullptr) + , extension(extension_) {} RemoteQueryExecutor::RemoteQueryExecutor( @@ -90,8 +89,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( QueryProcessingStage::Enum stage_, std::optional extension_) : header(header_), query(query_), context(context_) , scalars(scalars_), external_tables(external_tables_), stage(stage_) - , task_iterator(extension_ ? extension_->task_iterator : nullptr) - , parallel_reading_coordinator(extension_ ? extension_->parallel_reading_coordinator : nullptr) + , extension(extension_) { create_connections = [this, connections_, throttler, extension_](AsyncCallback) mutable { auto res = std::make_unique(std::move(connections_), context->getSettingsRef(), throttler); @@ -108,8 +106,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( QueryProcessingStage::Enum stage_, std::optional extension_) : header(header_), query(query_), context(context_) , scalars(scalars_), external_tables(external_tables_), stage(stage_) - , task_iterator(extension_ ? extension_->task_iterator : nullptr) - , parallel_reading_coordinator(extension_ ? extension_->parallel_reading_coordinator : nullptr) + , extension(extension_) { create_connections = [this, pool, throttler, extension_](AsyncCallback async_callback)->std::unique_ptr { @@ -247,6 +244,13 @@ void RemoteQueryExecutor::sendQueryUnlocked(ClientInfo::QueryKind query_kind, As finished = true; sent_query = true; + /// We need to tell the coordinator not to wait for this replica. + if (extension && extension->parallel_reading_coordinator) + { + chassert(extension->replica_info); + extension->parallel_reading_coordinator->markReplicaAsUnavailable(extension->replica_info->number_of_current_replica); + } + return; } @@ -360,7 +364,18 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync() read_context->resume(); if (needToSkipUnavailableShard()) + { + /// We need to tell the coordinator not to wait for this replica. + /// But at this point it may lead to an incomplete result set, because + /// this replica committed to read some part of there data and then died. + if (extension && extension->parallel_reading_coordinator) + { + chassert(extension->parallel_reading_coordinator); + extension->parallel_reading_coordinator->markReplicaAsUnavailable(extension->replica_info->number_of_current_replica); + } + return ReadResult(Block()); + } /// Check if packet is not ready yet. if (read_context->isInProgress()) @@ -524,30 +539,30 @@ bool RemoteQueryExecutor::setPartUUIDs(const std::vector & uuids) void RemoteQueryExecutor::processReadTaskRequest() { - if (!task_iterator) + if (!extension || !extension->task_iterator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Distributed task iterator is not initialized"); ProfileEvents::increment(ProfileEvents::ReadTaskRequestsReceived); - auto response = (*task_iterator)(); + auto response = (*extension->task_iterator)(); connections->sendReadTaskResponse(response); } void RemoteQueryExecutor::processMergeTreeReadTaskRequest(ParallelReadRequest request) { - if (!parallel_reading_coordinator) + if (!extension || !extension->parallel_reading_coordinator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Coordinator for parallel reading from replicas is not initialized"); ProfileEvents::increment(ProfileEvents::MergeTreeReadTaskRequestsReceived); - auto response = parallel_reading_coordinator->handleRequest(std::move(request)); + auto response = extension->parallel_reading_coordinator->handleRequest(std::move(request)); connections->sendMergeTreeReadTaskResponse(response); } void RemoteQueryExecutor::processMergeTreeInitialReadAnnounecement(InitialAllRangesAnnouncement announcement) { - if (!parallel_reading_coordinator) + if (!extension || !extension->parallel_reading_coordinator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Coordinator for parallel reading from replicas is not initialized"); - parallel_reading_coordinator->handleInitialAllRangesAnnouncement(announcement); + extension->parallel_reading_coordinator->handleInitialAllRangesAnnouncement(announcement); } void RemoteQueryExecutor::finish() diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 96dc5510bf4..a843ce520de 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -212,11 +212,11 @@ private: /// Temporary tables needed to be sent to remote servers Tables external_tables; QueryProcessingStage::Enum stage; + + std::optional extension; /// Initiator identifier for distributed task processing std::shared_ptr task_iterator; - std::shared_ptr parallel_reading_coordinator; - /// This is needed only for parallel reading from replicas, because /// we create a RemoteQueryExecutor per replica and have to store additional info /// about the number of the current replica or the count of replicas at all. diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index ee38cecb9c4..bb044d15ba2 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -19,6 +19,7 @@ #include "Storages/MergeTree/RequestResponse.h" #include #include +#include #include namespace DB @@ -61,18 +62,22 @@ public: { size_t number_of_requests{0}; size_t sum_marks{0}; + bool is_unavailable{false}; }; using Stats = std::vector; static String toString(Stats stats) { String result = "Statistics: "; + std::vector stats_by_replica; for (size_t i = 0; i < stats.size(); ++i) - result += fmt::format("-- replica {}, requests: {} marks: {} ", i, stats[i].number_of_requests, stats[i].sum_marks); + stats_by_replica.push_back(fmt::format("replica {}{} - {{requests: {} marks: {}}}", i, stats[i].is_unavailable ? " is unavailable" : "", stats[i].number_of_requests, stats[i].sum_marks)); + result += fmt::format("{}", fmt::join(stats_by_replica, "; ")); return result; } Stats stats; - size_t replicas_count; + size_t replicas_count{0}; + size_t unavailable_replicas_count{0}; explicit ImplInterface(size_t replicas_count_) : stats{replicas_count_} @@ -82,6 +87,7 @@ public: virtual ~ImplInterface() = default; virtual ParallelReadResponse handleRequest(ParallelReadRequest request) = 0; virtual void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) = 0; + virtual void markReplicaAsUnavailable(size_t replica_number) = 0; }; using Parts = std::set; @@ -128,6 +134,7 @@ public: ParallelReadResponse handleRequest(ParallelReadRequest request) override; void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) override; + void markReplicaAsUnavailable(size_t replica_number) override; void updateReadingState(const InitialAllRangesAnnouncement & announcement); void finalizeReadingState(); @@ -199,6 +206,17 @@ void DefaultCoordinator::updateReadingState(const InitialAllRangesAnnouncement & } } +void DefaultCoordinator::markReplicaAsUnavailable(size_t replica_number) +{ + LOG_DEBUG(log, "Replica number {} is unavailable", replica_number); + + ++unavailable_replicas_count; + stats[replica_number].is_unavailable = true; + + if (sent_initial_requests == replicas_count - unavailable_replicas_count) + finalizeReadingState(); +} + void DefaultCoordinator::finalizeReadingState() { /// Clear all the delayed queue @@ -345,12 +363,23 @@ public: ParallelReadResponse handleRequest([[ maybe_unused ]] ParallelReadRequest request) override; void handleInitialAllRangesAnnouncement([[ maybe_unused ]] InitialAllRangesAnnouncement announcement) override; + void markReplicaAsUnavailable(size_t replica_number) override; Parts all_parts_to_read; Poco::Logger * log = &Poco::Logger::get(fmt::format("{}{}", magic_enum::enum_name(mode), "Coordinator")); }; +template +void InOrderCoordinator::markReplicaAsUnavailable(size_t replica_number) +{ + LOG_DEBUG(log, "Replica number {} is unavailable", replica_number); + + stats[replica_number].is_unavailable = true; + ++unavailable_replicas_count; + + /// There is nothing to do else. +} template void InOrderCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) @@ -388,7 +417,6 @@ void InOrderCoordinator::handleInitialAllRangesAnnouncement(InitialAllRang } } - template ParallelReadResponse InOrderCoordinator::handleRequest(ParallelReadRequest request) { @@ -486,7 +514,7 @@ void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(Init if (!pimpl) { - setMode(announcement.mode); + mode = announcement.mode; initialize(); } @@ -500,16 +528,23 @@ ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelR if (!pimpl) { - setMode(request.mode); + mode = request.mode; initialize(); } return pimpl->handleRequest(std::move(request)); } -void ParallelReplicasReadingCoordinator::setMode(CoordinationMode mode_) +void ParallelReplicasReadingCoordinator::markReplicaAsUnavailable(size_t replica_number) { - mode = mode_; + std::lock_guard lock(mutex); + + if (!pimpl) + { + initialize(); + } + + return pimpl->markReplicaAsUnavailable(replica_number); } void ParallelReplicasReadingCoordinator::initialize() diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h index 0f41d24a9c6..ad8229be2d0 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h @@ -18,10 +18,15 @@ public: explicit ParallelReplicasReadingCoordinator(size_t replicas_count_); ~ParallelReplicasReadingCoordinator(); - void setMode(CoordinationMode mode); void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement); ParallelReadResponse handleRequest(ParallelReadRequest request); + /// Called when some replica is unavailable and we skipped it. + /// This is needed to "finalize" reading state e.g. spread all the marks using + /// consistent hashing, because otherwise coordinator will continue working in + /// "pending" state waiting for the unavailable replica to send the announcement. + void markReplicaAsUnavailable(size_t replica_number); + private: void initialize(); diff --git a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.reference b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.reference new file mode 100644 index 00000000000..af81158ecae --- /dev/null +++ b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.reference @@ -0,0 +1,2 @@ +10 +1 diff --git a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql new file mode 100644 index 00000000000..ecc243b9c89 --- /dev/null +++ b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test_parallel_replicas_unavailable_shards; +CREATE TABLE test_parallel_replicas_unavailable_shards (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO test_parallel_replicas_unavailable_shards SELECT * FROM numbers(10); + +SYSTEM FLUSH LOGS; + +SET skip_unavailable_shards=1, allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=11, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1; +SET send_logs_level='error'; +SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*); + +SYSTEM FLUSH LOGS; + +SELECT count() > 0 FROM system.text_log WHERE yesterday() <= event_date AND message LIKE '%Replica number 10 is unavailable%'; + +DROP TABLE test_parallel_replicas_unavailable_shards; From da09823ecebe67d2d6983ac58ee38fe51d5d9e7f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 31 May 2023 23:14:39 +0300 Subject: [PATCH 197/308] Compile aggregate expressions enable by default --- 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 607be1522db..6fed70a9303 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -135,7 +135,7 @@ class IColumn; M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \ M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ - M(Bool, compile_aggregate_expressions, false, "Compile aggregate functions to native code. This feature has a bug and should not be used.", 0) \ + M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code. This feature has a bug and should not be used.", 0) \ M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \ M(Bool, compile_sort_description, true, "Compile sort description to native code.", 0) \ M(UInt64, min_count_to_compile_sort_description, 3, "The number of identical sort descriptions before they are JIT-compiled", 0) \ From 801a4d574294b76a7f660bdf2e698d534f861ab3 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 31 May 2023 20:08:22 -0300 Subject: [PATCH 198/308] test for #42610 --- ...10_view_dictionary_check_refresh.reference | 4 ++ .../01910_view_dictionary_check_refresh.sql | 54 +++++++++++++++++++ 2 files changed, 58 insertions(+) create mode 100644 tests/queries/0_stateless/01910_view_dictionary_check_refresh.reference create mode 100644 tests/queries/0_stateless/01910_view_dictionary_check_refresh.sql diff --git a/tests/queries/0_stateless/01910_view_dictionary_check_refresh.reference b/tests/queries/0_stateless/01910_view_dictionary_check_refresh.reference new file mode 100644 index 00000000000..c1be003ebef --- /dev/null +++ b/tests/queries/0_stateless/01910_view_dictionary_check_refresh.reference @@ -0,0 +1,4 @@ +view 1 2022-10-20 first +dict 1 2022-10-20 first +view 1 2022-10-21 second +dict 1 2022-10-21 second diff --git a/tests/queries/0_stateless/01910_view_dictionary_check_refresh.sql b/tests/queries/0_stateless/01910_view_dictionary_check_refresh.sql new file mode 100644 index 00000000000..b36a378d827 --- /dev/null +++ b/tests/queries/0_stateless/01910_view_dictionary_check_refresh.sql @@ -0,0 +1,54 @@ +-- Tags: long + +DROP DICTIONARY IF EXISTS TestTblDict; +DROP VIEW IF EXISTS TestTbl_view; +DROP TABLE IF EXISTS TestTbl; + +CREATE TABLE TestTbl +( + `id` UInt16, + `dt` Date, + `val` String +) +ENGINE = MergeTree +PARTITION BY dt +ORDER BY (id); + +CREATE VIEW TestTbl_view +AS +SELECT * +FROM TestTbl +WHERE dt = ( SELECT max(dt) FROM TestTbl ); + +CREATE DICTIONARY IF NOT EXISTS TestTblDict +( + `id` UInt16, + `dt` Date, + `val` String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE TestTbl_view DB currentDatabase())) +LIFETIME(1) +LAYOUT(COMPLEX_KEY_HASHED()); + +select 'view' src,* FROM TestTbl_view; +select 'dict' src,* FROM TestTblDict ; + +insert into TestTbl values(1, '2022-10-20', 'first'); + +SELECT sleep(3) from numbers(4) settings max_block_size= 1 format Null; + +select 'view' src,* FROM TestTbl_view; +select 'dict' src,* FROM TestTblDict ; + +insert into TestTbl values(1, '2022-10-21', 'second'); + +SELECT sleep(3) from numbers(4) settings max_block_size= 1 format Null; + +select 'view' src,* FROM TestTbl_view; +select 'dict' src,* FROM TestTblDict ; + +DROP DICTIONARY IF EXISTS TestTblDict; +DROP VIEW IF EXISTS TestTbl_view; +DROP TABLE IF EXISTS TestTbl; + From ad19d2142f86d9b205d3635685e092e8d61eebe4 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 31 May 2023 20:17:54 -0300 Subject: [PATCH 199/308] ping ci From 0aa30ef129ff0cc920c2e8c2880625d4e1dc6c1e Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 31 May 2023 21:23:01 +0200 Subject: [PATCH 200/308] reuse s3_mocks, rewrite test test_paranoid_check_in_logs --- tests/integration/helpers/mock_servers.py | 26 ++++ .../integration/helpers/s3_mocks/__init__.py | 0 .../s3_mocks/broken_s3.py | 122 ++++++++++++++---- .../configs/storage_conf.xml | 14 +- .../test_checking_s3_blobs_paranoid/test.py | 31 +++-- tests/integration/test_merge_tree_s3/test.py | 111 ++++------------ 6 files changed, 175 insertions(+), 129 deletions(-) create mode 100644 tests/integration/helpers/s3_mocks/__init__.py rename tests/integration/{test_merge_tree_s3 => helpers}/s3_mocks/broken_s3.py (64%) diff --git a/tests/integration/helpers/mock_servers.py b/tests/integration/helpers/mock_servers.py index ea1f4e07a9f..e4655ffeeaf 100644 --- a/tests/integration/helpers/mock_servers.py +++ b/tests/integration/helpers/mock_servers.py @@ -1,6 +1,7 @@ import logging import os import time +import importlib # Starts simple HTTP servers written in Python. @@ -65,3 +66,28 @@ def start_mock_servers(cluster, script_dir, mocks, timeout=100): attempt += 1 logging.info(f"Mock {server_names_with_desc} started") + + +# The same as start_mock_servers, but +# import servers from central directory tests/integration/helpers +# and return the control instance +def start_s3_mock(cluster, mock_name, port, timeout=100): + script_dir = os.path.join(os.path.dirname(__file__), "s3_mocks") + registered_servers = [ + mock + for mock in os.listdir(script_dir) + if os.path.isfile(os.path.join(script_dir, mock)) + ] + + file_name = mock_name + ".py" + if file_name not in registered_servers: + raise KeyError( + f"Can't run s3 mock `{mock_name}`. No file `{file_name}` in directory `{script_dir}`" + ) + + start_mock_servers(cluster, script_dir, [(file_name, "resolver", port)], timeout) + + fmt = importlib.import_module("." + mock_name, "helpers.s3_mocks") + control = getattr(fmt, "MockControl")(cluster, "resolver", port) + + return control diff --git a/tests/integration/helpers/s3_mocks/__init__.py b/tests/integration/helpers/s3_mocks/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_merge_tree_s3/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py similarity index 64% rename from tests/integration/test_merge_tree_s3/s3_mocks/broken_s3.py rename to tests/integration/helpers/s3_mocks/broken_s3.py index 3adf3ba0047..026a3c6f515 100644 --- a/tests/integration/test_merge_tree_s3/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -12,7 +12,75 @@ UPSTREAM_HOST = "minio1" UPSTREAM_PORT = 9001 -class ServerRuntime: +class MockControl: + def __init__(self, cluster, container, port): + self._cluster = cluster + self._container = container + self._port = port + + def reset(self): + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + [ + "curl", + "-s", + f"http://localhost:{self._port}/mock_settings/reset", + ], + nothrow=True, + ) + assert response == "OK" + + def setup_fail_upload(self, part_length): + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + [ + "curl", + "-s", + f"http://localhost:{self._port}/mock_settings/error_at_put?when_length_bigger={part_length}", + ], + nothrow=True, + ) + assert response == "OK" + + def setup_fake_upload(self, part_length): + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + [ + "curl", + "-s", + f"http://localhost:{self._port}/mock_settings/fake_put?when_length_bigger={part_length}", + ], + nothrow=True, + ) + assert response == "OK" + + def setup_slow_answers( + self, minimal_length=0, timeout=None, probability=None, count=None + ): + url = ( + f"http://localhost:{self._port}/" + f"mock_settings/slow_put" + f"?minimal_length={minimal_length}" + ) + + if timeout is not None: + url += f"&timeout={timeout}" + + if probability is not None: + url += f"&probability={probability}" + + if count is not None: + url += f"&count={count}" + + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + ["curl", "-s", url], + nothrow=True, + ) + assert response == "OK" + + +class _ServerRuntime: class SlowPut: def __init__( self, probability_=None, timeout_=None, minimal_length_=None, count_=None @@ -34,11 +102,11 @@ class ServerRuntime: if content_length > self.minimal_length: if self.count > 0: if ( - runtime.slow_put.probability == 1 - or random.random() <= runtime.slow_put.probability + _runtime.slow_put.probability == 1 + or random.random() <= _runtime.slow_put.probability ): self.count -= 1 - return runtime.slow_put.timeout + return _runtime.slow_put.timeout return None def __init__(self): @@ -65,10 +133,10 @@ class ServerRuntime: self.slow_put = None -runtime = ServerRuntime() +_runtime = _ServerRuntime() -def and_then(value, func): +def _and_then(value, func): assert callable(func) return None if value is None else func(value) @@ -153,28 +221,28 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "error_at_put": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - runtime.error_at_put_when_length_bigger = int( + _runtime.error_at_put_when_length_bigger = int( params.get("when_length_bigger", [1024 * 1024])[0] ) return self._ok() if path[1] == "fake_put": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - runtime.fake_put_when_length_bigger = int( + _runtime.fake_put_when_length_bigger = int( params.get("when_length_bigger", [1024 * 1024])[0] ) return self._ok() if path[1] == "slow_put": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - runtime.slow_put = ServerRuntime.SlowPut( - minimal_length_=and_then(params.get("minimal_length", [None])[0], int), - probability_=and_then(params.get("probability", [None])[0], float), - timeout_=and_then(params.get("timeout", [None])[0], float), - count_=and_then(params.get("count", [None])[0], int), + _runtime.slow_put = _ServerRuntime.SlowPut( + minimal_length_=_and_then(params.get("minimal_length", [None])[0], int), + probability_=_and_then(params.get("probability", [None])[0], float), + timeout_=_and_then(params.get("timeout", [None])[0], float), + count_=_and_then(params.get("count", [None])[0], int), ) - self.log_message("set slow put %s", runtime.slow_put) + self.log_message("set slow put %s", _runtime.slow_put) return self._ok() if path[1] == "reset": - runtime.reset() + _runtime.reset() return self._ok() return self._error("_mock_settings: wrong command") @@ -191,14 +259,14 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): def do_PUT(self): content_length = int(self.headers.get("Content-Length", 0)) - if runtime.slow_put is not None: - timeout = runtime.slow_put.get_timeout(content_length) + if _runtime.slow_put is not None: + timeout = _runtime.slow_put.get_timeout(content_length) if timeout is not None: self.log_message("slow put %s", timeout) time.sleep(timeout) - if runtime.error_at_put_when_length_bigger is not None: - if content_length > runtime.error_at_put_when_length_bigger: + if _runtime.error_at_put_when_length_bigger is not None: + if content_length > _runtime.error_at_put_when_length_bigger: return self._error( '' "" @@ -211,9 +279,10 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): parts = urllib.parse.urlsplit(self.path) params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) upload_id = params.get("uploadId", [None])[0] - if runtime.fake_put_when_length_bigger is not None and upload_id is not None: - if content_length > runtime.fake_put_when_length_bigger: - runtime.register_fake_upload(upload_id, parts.path) + if _runtime.fake_put_when_length_bigger is not None: + if content_length > _runtime.fake_put_when_length_bigger: + if upload_id is not None: + _runtime.register_fake_upload(upload_id, parts.path) return self._fake_put_ok() return self._redirect() @@ -223,7 +292,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) upload_id = params.get("uploadId", [None])[0] - if runtime.is_fake_upload(upload_id, parts.path): + if _runtime.is_fake_upload(upload_id, parts.path): return self._fake_post_ok(parts.path) return self._redirect() @@ -235,9 +304,10 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self._redirect() -class ThreadedHTTPServer(socketserver.ThreadingMixIn, http.server.HTTPServer): +class _ThreadedHTTPServer(socketserver.ThreadingMixIn, http.server.HTTPServer): """Handle requests in a separate thread.""" -httpd = ThreadedHTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) -httpd.serve_forever() +if __name__ == "__main__": + httpd = _ThreadedHTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) + httpd.serve_forever() diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml index 733205ce3e1..b77e72d808b 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml @@ -7,25 +7,25 @@ - + s3 - http://minio1:9001/root/data/ + http://resolver:8083/root/data/ minio minio123 - + - +
- s3 + broken_s3
-
+
- s3 + broken_s3 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index adb56b1899c..042d57a0c43 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -1,12 +1,10 @@ #!/usr/bin/env python3 import logging -import os -import time - +import pytest from helpers.cluster import ClickHouseCluster -import pytest +from helpers.mock_servers import start_s3_mock @pytest.fixture(scope="module") @@ -32,12 +30,23 @@ def cluster(): cluster.shutdown() -def test_paranoid_check_in_logs(cluster): +@pytest.fixture(scope="module") +def init_broken_s3(cluster): + yield start_s3_mock(cluster, "broken_s3", "8083") + + +@pytest.fixture(scope="function") +def broken_s3(init_broken_s3): + init_broken_s3.reset() + yield init_broken_s3 + + +def test_upload_after_check_works(cluster, broken_s3): node = cluster.instances["node"] node.query( """ - CREATE TABLE s3_failover_test ( + CREATE TABLE s3_upload_after_check_works ( id Int64, data String ) ENGINE=MergeTree() @@ -45,8 +54,12 @@ def test_paranoid_check_in_logs(cluster): """ ) - node.query("INSERT INTO s3_failover_test VALUES (1, 'Hello')") + broken_s3.setup_fake_upload(1) - assert node.contains_in_log("exists after upload") + error = node.query_and_get_error( + "INSERT INTO s3_upload_after_check_works VALUES (1, 'Hello')" + ) - assert node.query("SELECT * FROM s3_failover_test ORDER BY id") == "1\tHello\n" + assert "Code: 499" in error, error + assert "Immediately after upload" in error, error + assert "suddenly disappeared" in error, error diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 626a71f006e..7730bfcf7b2 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -4,12 +4,11 @@ import os import pytest from helpers.cluster import ClickHouseCluster -from helpers.mock_servers import start_mock_servers +from helpers.mock_servers import start_s3_mock, start_mock_servers from helpers.utility import generate_values, replace_config, SafeThread from helpers.wait_for_helpers import wait_for_delete_inactive_parts from helpers.wait_for_helpers import wait_for_delete_empty_parts from helpers.wait_for_helpers import wait_for_merges -from helpers.test_tools import assert_eq_with_retry SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -94,6 +93,17 @@ def create_table(node, table_name, **additional_settings): node.query(create_table_statement) +@pytest.fixture(scope="module") +def init_broken_s3(cluster): + yield start_s3_mock(cluster, "broken_s3", "8083") + + +@pytest.fixture(scope="function") +def broken_s3(init_broken_s3): + init_broken_s3.reset() + yield init_broken_s3 + + def run_s3_mocks(cluster): script_dir = os.path.join(os.path.dirname(__file__), "s3_mocks") start_mock_servers( @@ -102,7 +112,6 @@ def run_s3_mocks(cluster): [ ("unstable_proxy.py", "resolver", "8081"), ("no_delete_objects.py", "resolver", "8082"), - ("broken_s3.py", "resolver", "8083"), ], ) @@ -142,80 +151,6 @@ def clear_minio(cluster): yield -class BrokenS3: - @staticmethod - def reset(cluster): - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - [ - "curl", - "-s", - f"http://localhost:8083/mock_settings/reset", - ], - nothrow=True, - ) - assert response == "OK" - - @staticmethod - def setup_fail_upload(cluster, part_length): - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - [ - "curl", - "-s", - f"http://localhost:8083/mock_settings/error_at_put?when_length_bigger={part_length}", - ], - nothrow=True, - ) - assert response == "OK" - - @staticmethod - def setup_fake_upload(cluster, part_length): - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - [ - "curl", - "-s", - f"http://localhost:8083/mock_settings/fake_put?when_length_bigger={part_length}", - ], - nothrow=True, - ) - assert response == "OK" - - @staticmethod - def setup_slow_answers( - cluster, minimal_length=0, timeout=None, probability=None, count=None - ): - url = ( - f"http://localhost:8083/" - f"mock_settings/slow_put" - f"?minimal_length={minimal_length}" - ) - - if timeout is not None: - url += f"&timeout={timeout}" - - if probability is not None: - url += f"&probability={probability}" - - if count is not None: - url += f"&count={count}" - - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - ["curl", "-s", url], - nothrow=True, - ) - assert response == "OK" - - -@pytest.fixture(autouse=True, scope="function") -def reset_broken_s3(cluster): - BrokenS3.reset(cluster) - - yield - - def check_no_objects_after_drop(cluster, table_name="s3_test", node_name="node"): node = cluster.instances[node_name] node.query(f"DROP TABLE IF EXISTS {table_name} SYNC") @@ -932,7 +867,7 @@ def test_merge_canceled_by_drop(cluster, node_name): @pytest.mark.parametrize("storage_policy", ["broken_s3_always_multi_part", "broken_s3"]) @pytest.mark.parametrize("node_name", ["node"]) -def test_merge_canceled_by_s3_errors(cluster, node_name, storage_policy): +def test_merge_canceled_by_s3_errors(cluster, broken_s3, node_name, storage_policy): node = cluster.instances[node_name] node.query("DROP TABLE IF EXISTS test_merge_canceled_by_s3_errors NO DELAY") node.query( @@ -952,7 +887,7 @@ def test_merge_canceled_by_s3_errors(cluster, node_name, storage_policy): min_key = node.query("SELECT min(key) FROM test_merge_canceled_by_s3_errors") assert int(min_key) == 0, min_key - BrokenS3.setup_fail_upload(cluster, 50000) + broken_s3.setup_fail_upload(50000) node.query("SYSTEM START MERGES test_merge_canceled_by_s3_errors") @@ -969,7 +904,7 @@ def test_merge_canceled_by_s3_errors(cluster, node_name, storage_policy): @pytest.mark.parametrize("node_name", ["node"]) -def test_merge_canceled_by_s3_errors_when_move(cluster, node_name): +def test_merge_canceled_by_s3_errors_when_move(cluster, broken_s3, node_name): node = cluster.instances[node_name] settings = { "storage_policy": "external_broken_s3", @@ -995,7 +930,7 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, node_name): settings={"materialize_ttl_after_modify": 0}, ) - BrokenS3.setup_fail_upload(cluster, 10000) + broken_s3.setup_fail_upload(10000) node.query("SYSTEM START MERGES merge_canceled_by_s3_errors_when_move") @@ -1015,7 +950,9 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, node_name): @pytest.mark.parametrize( "in_flight_memory", [(10, 245918115), (5, 156786752), (1, 106426187)] ) -def test_s3_engine_heavy_write_check_mem(cluster, node_name, in_flight_memory): +def test_s3_engine_heavy_write_check_mem( + cluster, broken_s3, node_name, in_flight_memory +): in_flight = in_flight_memory[0] memory = in_flight_memory[1] @@ -1029,8 +966,8 @@ def test_s3_engine_heavy_write_check_mem(cluster, node_name, in_flight_memory): " ENGINE S3('http://resolver:8083/root/data/test-upload.csv', 'minio', 'minio123', 'CSV')", ) - BrokenS3.setup_fake_upload(cluster, 1000) - BrokenS3.setup_slow_answers(cluster, 10 * 1024 * 1024, timeout=15, count=10) + broken_s3.setup_fake_upload(1000) + broken_s3.setup_slow_answers(10 * 1024 * 1024, timeout=15, count=10) query_id = f"INSERT_INTO_S3_ENGINE_QUERY_ID_{in_flight}" node.query( @@ -1058,7 +995,7 @@ def test_s3_engine_heavy_write_check_mem(cluster, node_name, in_flight_memory): @pytest.mark.parametrize("node_name", ["node"]) -def test_s3_disk_heavy_write_check_mem(cluster, node_name): +def test_s3_disk_heavy_write_check_mem(cluster, broken_s3, node_name): memory = 2279055040 node = cluster.instances[node_name] @@ -1075,8 +1012,8 @@ def test_s3_disk_heavy_write_check_mem(cluster, node_name): ) node.query("SYSTEM STOP MERGES s3_test") - BrokenS3.setup_fake_upload(cluster, 1000) - BrokenS3.setup_slow_answers(cluster, 10 * 1024 * 1024, timeout=10, count=50) + broken_s3.setup_fake_upload(1000) + broken_s3.setup_slow_answers(10 * 1024 * 1024, timeout=10, count=50) query_id = f"INSERT_INTO_S3_DISK_QUERY_ID" node.query( From c54abdb4f8ad952f0402d10807fe65fb78a5a0b3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Jun 2023 01:32:03 +0200 Subject: [PATCH 201/308] Support clang-17 --- src/IO/WriteBufferFromS3TaskTracker.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 4abae90eeac..01455fc7fa3 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -121,17 +121,17 @@ void WriteBufferFromS3::TaskTracker::add(Callback && func) /// preallocation for the second issue FinishedList pre_allocated_finished {future_placeholder}; - Callback func_with_notification = [&, func=std::move(func), pre_allocated_finished=std::move(pre_allocated_finished)] () mutable + Callback func_with_notification = [&, my_func = std::move(func), my_pre_allocated_finished = std::move(pre_allocated_finished)]() mutable { SCOPE_EXIT({ DENY_ALLOCATIONS_IN_SCOPE; std::lock_guard lock(mutex); - finished_futures.splice(finished_futures.end(), pre_allocated_finished); + finished_futures.splice(finished_futures.end(), my_pre_allocated_finished); has_finished.notify_one(); }); - func(); + my_func(); }; /// this move is nothrow From 5dbce62ec365f8d4801faef4ef5a0cd8eb117615 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 31 May 2023 17:37:20 +0800 Subject: [PATCH 202/308] fixed. without apply prepareRightBlock will cause mismatch block structure --- src/Interpreters/GraceHashJoin.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index f54ee9d85c7..0dd61ff2793 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -623,6 +623,7 @@ Block GraceHashJoin::prepareRightBlock(const Block & block) void GraceHashJoin::addJoinedBlockImpl(Block block) { + block = prepareRightBlock(block); Buckets buckets_snapshot = getCurrentBuckets(); size_t bucket_index = current_bucket->idx; Block current_block; From 6334b6d5698f81429f5ae1351172ae89f5d7d2ea Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Wed, 31 May 2023 17:54:30 -0700 Subject: [PATCH 203/308] Address case where cpu cgroup is set to max. --- src/Common/AsynchronousMetrics.cpp | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 168dd3f0c4a..a4cb18249b6 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -945,12 +945,22 @@ void AsynchronousMetrics::update(TimePoint update_time) uint64_t quota = 0; uint64_t period = 0; - readText(quota, *cgroupcpu_max); - skipWhitespaceIfAny(*cgroupcpu_max); - readText(period, *cgroupcpu_max); + std::string line; + readText(line, *cgroupcpu_max); + + auto space = line.find_first_of(" "); + + if (line.rfind("max", 0) == std::string::npos) + { + auto field1 = line.substr(0, space); + quota = std::stoull(field1); + } + + auto field2 = line.substr(space + 1); + period = std::stoull(field2); new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."}; - new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup."}; + new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."}; } catch (...) { @@ -970,7 +980,7 @@ void AsynchronousMetrics::update(TimePoint update_time) tryReadText(period, *cgroupcpu_cfs_period); new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."}; - new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup."}; + new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."}; } catch (...) { From 956c399b2af27e6d70c2e4894657f010efb2594d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Jun 2023 03:04:29 +0200 Subject: [PATCH 204/308] Remove useless code --- src/Common/ProgressIndication.cpp | 5 +++-- src/Databases/DatabaseFactory.cpp | 1 - src/Storages/StorageDistributed.cpp | 7 ------- src/Storages/StorageS3.h | 1 - 4 files changed, 3 insertions(+), 11 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index df8778eb0d1..61b60060430 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -9,12 +9,13 @@ #include "Common/formatReadable.h" #include #include -#include "IO/WriteBufferFromString.h" -#include +#include +#include /// http://en.wikipedia.org/wiki/ANSI_escape_code #define CLEAR_TO_END_OF_LINE "\033[K" + namespace DB { diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index f346205ea79..e1c8afa52c0 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index fc552a5ab6d..bcf6f68d00d 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1137,13 +1137,6 @@ SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadata auto cluster = getCluster(); const auto & settings = local_context->getSettingsRef(); - /// Ban an attempt to make async insert into the table belonging to DatabaseMemory - if (!storage_policy && !owned_cluster && !settings.insert_distributed_sync && !settings.insert_shard_id) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage {} must have own data directory to enable asynchronous inserts", - getName()); - } - auto shard_num = cluster->getLocalShardCount() + cluster->getRemoteShardCount(); /// If sharding key is not specified, then you can only write to a shard containing only one shard diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 12573ab513f..fae8d8eb3cb 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -31,7 +31,6 @@ namespace DB { class PullingPipelineExecutor; -class StorageS3SequentialSource; class NamedCollection; class StorageS3Source : public ISource, WithContext From fb86fe8f9dfb4fb96cca399b4967959424c8e959 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Jun 2023 03:08:05 +0200 Subject: [PATCH 205/308] Remove useless code --- src/Storages/StorageS3.h | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index fae8d8eb3cb..bca804a7f53 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -247,11 +247,6 @@ public: String getPath() const { return url.key; } - void appendToPath(const String & suffix) - { - url = S3::URI{std::filesystem::path(url.uri.toString()) / suffix}; - } - bool update(ContextPtr context); void connect(ContextPtr context); From c9d0d217f5d180f5c78286b1b819534dd8cc1b59 Mon Sep 17 00:00:00 2001 From: frinkr Date: Wed, 31 May 2023 19:53:06 +0800 Subject: [PATCH 206/308] fix Keeper deadlock on exception when preprocessing requests --- src/Coordination/KeeperStateMachine.cpp | 11 ++++++++++- src/Coordination/KeeperStateMachine.h | 2 ++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 6635c74149a..f787cc8722e 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -272,7 +272,8 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req } catch (...) { - rollbackRequest(request_for_session, true); + tryLogCurrentException(__PRETTY_FUNCTION__); + rollbackRequestNoLock(request_for_session, true); throw; } @@ -411,6 +412,14 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession storage->rollbackRequest(request_for_session.zxid, allow_missing); } +void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing) +{ + if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) + return; + + storage->rollbackRequest(request_for_session.zxid, allow_missing); +} + nuraft::ptr KeeperStateMachine::last_snapshot() { /// Just return the latest snapshot. diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index fbd97fd8631..f6d81d23056 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -68,6 +68,8 @@ public: // (can happen in case of exception during preprocessing) void rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); + void rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); + uint64_t last_commit_index() override { return last_committed_idx; } /// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state. From 3d0f33664390607258eecd6982e3d787347c298d Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Mon, 29 May 2023 14:31:54 +0800 Subject: [PATCH 207/308] Optimize predicate with toYYYYMM converter Like toYear(PR #50062), the predicate with toYYYYMM could also be optimized with its preimage. For example, we could transform the predicate "toYYYYMM(c) = 202305" to "c >= '2023-05-01' AND c <= '2023-05-31'", so that the invocation of toYYYYMM is avoided. The similar transformation also applies to other comparisons. This commit extends OptimizeDateFilterInPlaceVisitor by having it recognize the toYYYYMM converter in the WHERE/PREWHERE clauses and replace it with the above tranformed predicate in-place at the AST level. --- .../OptimizeDateFilterVisitor.cpp | 33 +++++++-- ..._date_filter_predicate_optimizer.reference | 69 +++++++++++++++++++ .../02764_date_filter_predicate_optimizer.sql | 23 +++++++ 3 files changed, 120 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/OptimizeDateFilterVisitor.cpp b/src/Interpreters/OptimizeDateFilterVisitor.cpp index 58e1b3335f9..aec2dec19c8 100644 --- a/src/Interpreters/OptimizeDateFilterVisitor.cpp +++ b/src/Interpreters/OptimizeDateFilterVisitor.cpp @@ -10,14 +10,37 @@ namespace DB { -ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 year) +ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 compare_to) { const DateLUTImpl & date_lut = DateLUT::instance(); - if (converter != "toYear") return {}; + String start_date; + String end_date; - String start_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); - String end_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); + if (converter == "toYear") + { + UInt64 year = compare_to; + start_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); + end_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); + } + else if (converter == "toYYYYMM") + { + UInt64 year = compare_to / 100; + UInt64 month = compare_to % 100; + + if (month == 0 || month > 12) return {}; + + static constexpr UInt8 days_of_month[] = {31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; + + bool leap_year = (year & 3) == 0 && (year % 100 || (year % 400 == 0 && year)); + + start_date = date_lut.dateToString(date_lut.makeDayNum(year, month, 1)); + end_date = date_lut.dateToString(date_lut.makeDayNum(year, month, days_of_month[month - 1] + (leap_year && month == 2))); + } + else + { + return {}; + } if (comparator == "equals") { @@ -82,7 +105,7 @@ bool rewritePredicateInPlace(ASTFunction & function, ASTPtr & ast) { if (const auto * func = function.arguments->children[i]->as(); func) { - if (func->name == "toYear") + if (func->name == "toYear" || func->name == "toYYYYMM") { func_id = i; } diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference index e5c608ddc1a..54704fb3b3e 100644 --- a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference +++ b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference @@ -37,3 +37,72 @@ WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) A SELECT value1 FROM t WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) +SELECT value1 +FROM t +WHERE ((date1 >= \'1900-02-01\') AND (date1 <= \'1900-02-28\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1992-02-01\') AND (date1 <= \'1992-02-29\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'2000-02-01\') AND (date1 <= \'2000-02-29\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (toYYYYMM(date1) = 199300) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-01-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-02-01\') AND (date1 <= \'1993-02-28\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-03-01\') AND (date1 <= \'1993-03-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-04-01\') AND (date1 <= \'1993-04-30\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-05-01\') AND (date1 <= \'1993-05-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-06-01\') AND (date1 <= \'1993-06-30\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-07-01\') AND (date1 <= \'1993-07-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-08-01\') AND (date1 <= \'1993-08-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-09-01\') AND (date1 <= \'1993-09-30\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-10-01\') AND (date1 <= \'1993-10-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-11-01\') AND (date1 <= \'1993-11-30\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-12-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (toYYYYMM(date1) = 199313) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 < \'1992-03-01\') OR (date1 > \'1992-03-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 < \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 > \'1992-03-31\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 <= \'1992-03-31\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 >= \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1992-03-01\') OR ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\'))) AND ((id >= 1) AND (id <= 3)) diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql index 563468d4f82..a04273bbb18 100644 --- a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql +++ b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql @@ -13,5 +13,28 @@ EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM t WHERE year1 = 1993 A EXPLAIN SYNTAX SELECT value1 FROM t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; EXPLAIN SYNTAX SELECT value1 FROM t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; EXPLAIN SYNTAX SELECT value1 FROM t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 190002 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199202 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 200002 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199300 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199301 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199302 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199303 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199304 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199305 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199306 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199307 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199308 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199309 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199310 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199311 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199313 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) <> 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) < 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) > 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) <= 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) >= 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE (toYYYYMM(date1) >= 199203 OR toYear(date1) = 1993) AND id BETWEEN 1 AND 3; DROP TABLE t; From bcbce15cde19eb1ec3705a572451812cb6e18c2c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Jun 2023 08:51:44 +0200 Subject: [PATCH 208/308] Mark the builds without results as pending --- tests/ci/build_report_check.py | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 82c04ce82c5..1362f3c8934 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -23,6 +23,7 @@ from get_robot_token import get_best_robot_token from pr_info import NeedsDataType, PRInfo from commit_status_helper import ( RerunHelper, + format_description, get_commit, post_commit_status, update_mergeable_check, @@ -269,14 +270,20 @@ def main(): if build_result.status == "success": ok_groups += 1 - if ok_groups == 0 or some_builds_are_missing: - summary_status = "error" + # Check if there are no builds at all, do not override bad status + if summary_status == "success": + if some_builds_are_missing: + summary_status = "pending" + elif ok_groups == 0: + summary_status = "error" addition = "" if some_builds_are_missing: - addition = f"({len(build_reports)} of {required_builds} builds are OK)" + addition = f" ({len(build_reports)} of {required_builds} builds are OK)" - description = f"{ok_groups}/{total_groups} artifact groups are OK {addition}" + description = format_description( + f"{ok_groups}/{total_groups} artifact groups are OK{addition}" + ) post_commit_status( commit, summary_status, url, description, build_check_name, pr_info From 08d98329b07ff772812999059a45af03352be030 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 10:12:09 +0200 Subject: [PATCH 209/308] Revert "Add SQL functions for Entropy Learned Hashing" --- .../sql-reference/functions/hash-functions.md | 71 ---- src/Functions/EntropyLearnedHash.cpp | 395 ------------------ ...new_functions_must_be_documented.reference | 3 - .../02734_entropy_learned_hashing.reference | 18 - .../02734_entropy_learned_hashing.sql | 30 -- 5 files changed, 517 deletions(-) delete mode 100644 src/Functions/EntropyLearnedHash.cpp delete mode 100644 tests/queries/0_stateless/02734_entropy_learned_hashing.reference delete mode 100644 tests/queries/0_stateless/02734_entropy_learned_hashing.sql diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 89afcca3799..8dfa03ceaf2 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -560,77 +560,6 @@ Result: └───────────────────────────┘ ``` -## Entropy-learned hashing (experimental) - -Entropy-learned hashing is disabled by default, to enable: `SET allow_experimental_hash_functions=1`. - -Entropy-learned hashing is not a standalone hash function like `metroHash64`, `cityHash64`, `sipHash64` etc. Instead, it aims to preprocess -the data to be hashed in a way that a standalone hash function can be computed more efficiently while not compromising the hash quality, -i.e. the randomness of the hashes. For that, entropy-based hashing chooses a subset of the bytes in a training data set of Strings which has -the same randomness (entropy) as the original Strings. For example, if the Strings are in average 100 bytes long, and we pick a subset of 5 -bytes, then a hash function will be 95% less expensive to evaluate. For details of the method, refer to [Entropy-Learned Hashing: Constant -Time Hashing with Controllable Uniformity](https://doi.org/10.1145/3514221.3517894). - -Entropy-learned hashing has two phases: - -1. A training phase on a representative but typically small set of Strings to be hashed. Training consists of two steps: - - - Function `prepareTrainEntropyLearnedHash(data, id)` caches the training data in a global state under a given `id`. It returns dummy - value `0` on every row. - - Function `trainEntropyLearnedHash(id)` computes a minimal partial sub-key of the training data stored stored under `id` in the global - state. The cached training data in the global state is replaced by the partial key. Dummy value `0` is returned on every row. - -2. An evaluation phase where hashes are computed using the previously calculated partial sub-keys. Function `entropyLearnedHash(data, id)` - hashes `data` using the partial subkey stored as `id`. CityHash64 is used as hash function. - -The reason that the training phase comprises two steps is that ClickHouse processes data at chunk granularity but entropy-learned hashing -needs to process the entire training set at once. - -Since functions `prepareTrainEntropyLearnedHash()` and `trainEntropyLearnedHash()` access global state, they should not be called in -parallel with the same `id`. - -**Syntax** - -``` sql -prepareTrainEntropyLearnedHash(data, id); -trainEntropyLearnedHash(id); -entropyLearnedHash(data, id); -``` - -**Example** - -```sql -SET allow_experimental_hash_functions=1; -CREATE TABLE tab (col String) ENGINE=Memory; -INSERT INTO tab VALUES ('aa'), ('ba'), ('ca'); - -SELECT prepareTrainEntropyLearnedHash(col, 'id1') AS prepared FROM tab; -SELECT trainEntropyLearnedHash('id1') AS trained FROM tab; -SELECT entropyLearnedHash(col, 'id1') as hashes FROM tab; -``` - -Result: - -``` response -┌─prepared─┐ -│ 0 │ -│ 0 │ -│ 0 │ -└──────────┘ - -┌─trained─┐ -│ 0 │ -│ 0 │ -│ 0 │ -└─────────┘ - -┌───────────────hashes─┐ -│ 2603192927274642682 │ -│ 4947675599669400333 │ -│ 10783339242466472992 │ -└──────────────────────┘ -``` - ## metroHash64 Produces a 64-bit [MetroHash](http://www.jandrewrogers.com/2015/05/27/metrohash/) hash value. diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp deleted file mode 100644 index 854379bbb9d..00000000000 --- a/src/Functions/EntropyLearnedHash.cpp +++ /dev/null @@ -1,395 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -/// Implementation of entropy-learned hashing: https://doi.org/10.1145/3514221.3517894 -/// If you change something in this file, please don't deviate too much from the pseudocode in the paper! - -/// TODOs for future work: -/// - allow to specify an arbitrary hash function (currently always CityHash is used) -/// - allow function chaining a la entropyLearnedHash(trainEntropyLearnedHash()) -/// - support more datatypes for data (besides String) - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; - extern const int SUPPORT_IS_DISABLED; -} - -namespace -{ - -using PartialKeyPositions = std::vector; -using Entropies = std::vector; - -void getPartialKey(std::string_view key, const PartialKeyPositions & partial_key_positions, String & result) -{ - result.clear(); - result.reserve(partial_key_positions.size()); - - for (auto partial_key_position : partial_key_positions) - if (partial_key_position < key.size()) - result.push_back(key[partial_key_position]); -} - -bool allPartialKeysAreUnique(const std::vector & keys, const PartialKeyPositions & partial_key_positions) -{ - std::unordered_set unique_partial_keys; - unique_partial_keys.reserve(keys.size()); - String partial_key; - - for (const auto & key : keys) - { - getPartialKey(key, partial_key_positions, partial_key); - if (!unique_partial_keys.insert(partial_key).second) - return false; - } - - return true; -} - -// NextByte returns position of byte which adds the most entropy and the new entropy -std::pair nextByte(const std::vector & keys, size_t max_len, PartialKeyPositions & partial_key_positions) -{ - size_t min_collisions = std::numeric_limits::max(); - size_t best_position = 0; - - std::unordered_map count_table; - count_table.reserve(keys.size()); - - String partial_key; - - for (size_t i = 0; i < max_len; ++i) - { - count_table.clear(); - - partial_key_positions.push_back(i); - size_t collisions = 0; - for (const auto & key : keys) - { - getPartialKey(key, partial_key_positions, partial_key); - collisions += count_table[partial_key]++; - } - - if (collisions < min_collisions) - { - min_collisions = collisions; - best_position = i; - } - partial_key_positions.pop_back(); - } - - return {best_position, min_collisions}; -} - -std::pair chooseBytes(const std::vector & train_data) -{ - if (train_data.size() <= 1) - return {}; - - PartialKeyPositions partial_key_positions; - Entropies entropies; - - size_t max_len = 0; /// length of the longest key in training data - for (const auto & key : train_data) - max_len = std::max(max_len, key.size()); - - while (!allPartialKeysAreUnique(train_data, partial_key_positions)) - { - auto [new_position, new_entropy] = nextByte(train_data, max_len, partial_key_positions); - if (!entropies.empty() && new_entropy == entropies.back()) - break; - partial_key_positions.push_back(new_position); - entropies.push_back(new_entropy); - } - return {partial_key_positions, entropies}; -} - -/// Contains global state to convey information between SQL functions -/// - prepareTrainEntropyLearnedHash(), -/// - trainEntropyLearnedHash() and -/// - entropyLearnedHash(). -/// -/// The reason this machinery is necessary is that ClickHouse processes data in chunks of unpredictable size, yet the training step of -/// entropy-learned hashing needs to process *all* training data in one go. The downside is that the training step becomes quite expensive :-( -class EntropyLearnedHashGlobalState -{ -public: - static EntropyLearnedHashGlobalState & instance() - { - static EntropyLearnedHashGlobalState instance; - return instance; - } - - /// Called by prepareTrainEntropyLearnedHash() - void cacheTrainingSample(const String & user_name, const String & id, IColumn::MutablePtr column) - { - std::lock_guard lock(mutex); - auto & ids_for_user = global_state[user_name]; - auto & training_samples_for_id = ids_for_user[id].training_samples; - training_samples_for_id.push_back(std::move(column)); - } - - void train(const String & user_name, const String & id) - { - std::lock_guard lock(mutex); - auto & ids_for_user = global_state[user_name]; - auto & training_samples = ids_for_user[id].training_samples; - - if (training_samples.empty()) - return; - - auto & concatenated_training_sample = training_samples[0]; - for (size_t i = 1; i < training_samples.size(); ++i) - { - auto & other_training_sample = training_samples[i]; - concatenated_training_sample->insertRangeFrom(*other_training_sample, 0, other_training_sample->size()); - } - - const ColumnString * concatenated_training_sample_string = checkAndGetColumn(*concatenated_training_sample); - if (!concatenated_training_sample_string) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column"); - - const size_t num_rows = concatenated_training_sample_string->size(); - std::vector training_data; - for (size_t i = 0; i < num_rows; ++i) - { - std::string_view string_view = concatenated_training_sample_string->getDataAt(i).toView(); - training_data.emplace_back(string_view); - } - - PartialKeyPositions partial_key_positions = chooseBytes(training_data).first; - - ids_for_user[id].partial_key_positions = partial_key_positions; - training_samples.clear(); - } - - const PartialKeyPositions & getPartialKeyPositions(const String & user_name, const String & id) const - { - std::lock_guard lock(mutex); - auto it_user = global_state.find(user_name); - if (it_user == global_state.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Id {} not registered for user in entropy learned hashing", id); - auto it_id = it_user->second.find(id); - if (it_id == it_user->second.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Id {} not registered for user in entropy learned hashing", id); - return it_id->second.partial_key_positions; - } - -private: - mutable std::mutex mutex; - - /// The state. - struct ColumnsAndPartialKeyPositions - { - /// Caches training data chunks. Filled by prepareTrainEntropyLearnedHash(), cleared by trainEntropyLearnedHash(). - MutableColumns training_samples; - /// The result of the training phase. Filled by trainEntropyLearnedHash(). - PartialKeyPositions partial_key_positions; - }; - - /// Maps a state id to the state. - using IdToColumnsAndPartialKeyPositions = std::map; - - /// Maps the user name to a state id. As a result, the state id is unique at user scope. - using UserNameToId = std::map; - - UserNameToId global_state TSA_GUARDED_BY(mutex); -}; - -} - - -/// Copies all chunks of the training sample column into the global state under a given id. -class FunctionPrepareTrainEntropyLearnedHash : public IFunction -{ -public: - static constexpr auto name = "prepareTrainEntropyLearnedHash"; - static FunctionPtr create(ContextPtr context) - { - if (!context->getSettings().allow_experimental_hash_functions) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Entropy-learned hashing is experimental. Set `allow_experimental_hash_functions` setting to enable it"); - - return std::make_shared(context->getUserName()); - } - explicit FunctionPrepareTrainEntropyLearnedHash(const String & user_name_) : IFunction(), user_name(user_name_) {} - - String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 2; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - FunctionArgumentDescriptors args{ - {"data", &isString, nullptr, "String"}, - {"id", &isString, nullptr, "String"} - }; - - validateFunctionArgumentTypes(*this, arguments, args); - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override - { - const IColumn * id_col = arguments[1].column.get(); - const ColumnConst * id_col_const = checkAndGetColumn(id_col); - const String id = id_col_const->getValue(); - - IColumn::Ptr data_col = arguments[0].column; - IColumn::MutablePtr data_col_mutable = IColumn::mutate(data_col); - - auto & global_state = EntropyLearnedHashGlobalState::instance(); - global_state.cacheTrainingSample(user_name, id, std::move(data_col_mutable)); - - const size_t num_rows = data_col->size(); - return result_type->createColumnConst(num_rows, 0u); /// dummy output - } -private: - const String user_name; -}; - - -/// 1. Concatenates the training samples of a given id in the global state. -/// 2. Computes the partial key positions from the concatenated training samples and stores that in the global state. -/// 3. clear()-s the training samples in the global state. -class FunctionTrainEntropyLearnedHash : public IFunction -{ -public: - static constexpr auto name = "trainEntropyLearnedHash"; - static FunctionPtr create(ContextPtr context) - { - if (!context->getSettings().allow_experimental_hash_functions) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Entropy-learned hashing is experimental. Set `allow_experimental_hash_functions` setting to enable it"); - return std::make_shared(context->getUserName()); - } - explicit FunctionTrainEntropyLearnedHash(const String & user_name_) : IFunction(), user_name(user_name_) {} - - String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 1; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - bool useDefaultImplementationForConstants() const override { return false; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - FunctionArgumentDescriptors args{ - {"id", &isString, nullptr, "String"} - }; - - validateFunctionArgumentTypes(*this, arguments, args); - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override - { - const IColumn * id_col = arguments[0].column.get(); - const ColumnConst * id_col_const = checkAndGetColumn(id_col); - if (!id_col_const) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", - arguments.begin()->column->getName(), getName()); - - auto & global_state = EntropyLearnedHashGlobalState::instance(); - - const String id = id_col_const->getValue(); - global_state.train(user_name, id); - - const size_t num_rows = id_col->size(); - return result_type->createColumnConst(num_rows, 0u); /// dummy output - } -private: - const String user_name; -}; - - -/// Hashes input strings using partial key positions stored in the global state. -class FunctionEntropyLearnedHash : public IFunction -{ -public: - static constexpr auto name = "entropyLearnedHash"; - static FunctionPtr create(ContextPtr context) - { - if (!context->getSettings().allow_experimental_hash_functions) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Entropy-learned hashing experimental. Set `allow_experimental_hash_functions` setting to enable it"); - return std::make_shared(context->getUserName()); - } - explicit FunctionEntropyLearnedHash(const String & user_name_) : IFunction(), user_name(user_name_) {} - - String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 2; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - FunctionArgumentDescriptors args{ - {"data", &isString, nullptr, "String"}, - {"id", &isString, nullptr, "String"} - }; - - validateFunctionArgumentTypes(*this, arguments, args); - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override - { - const IColumn * id_col = arguments.back().column.get(); - const ColumnConst * id_col_const = checkAndGetColumn(id_col); - const String id = id_col_const->getValue(); - - const auto & global_state = EntropyLearnedHashGlobalState::instance(); - const auto & partial_key_positions = global_state.getPartialKeyPositions(user_name, id); - - const auto * data_col = arguments[0].column.get(); - if (const auto * col_data_string = checkAndGetColumn(data_col)) - { - const size_t num_rows = col_data_string->size(); - auto col_res = ColumnUInt64::create(num_rows); - - auto & col_res_vec = col_res->getData(); - String partial_key; - for (size_t i = 0; i < num_rows; ++i) - { - std::string_view string_ref = col_data_string->getDataAt(i).toView(); - getPartialKey(string_ref, partial_key_positions, partial_key); - col_res_vec[i] = CityHash_v1_0_2::CityHash64(partial_key.data(), partial_key.size()); - } - - return col_res; - } - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", - arguments.begin()->column->getName(), getName()); - } -private: - const String user_name; -}; - -REGISTER_FUNCTION(EntropyLearnedHash) -{ - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); -} - -} diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index b2c9073648e..7ab26982402 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -266,7 +266,6 @@ encodeURLComponent encodeURLFormComponent encodeXMLComponent endsWith -entropyLearnedHash equals erf erfc @@ -559,7 +558,6 @@ positionCaseInsensitive positionCaseInsensitiveUTF8 positionUTF8 pow -prepareTrainEntropyLearnedHash proportionsZTest protocol queryID @@ -864,7 +862,6 @@ toYear toYearWeek today tokens -trainEntropyLearnedHash transactionID transactionLatestSnapshot transactionOldestSnapshot diff --git a/tests/queries/0_stateless/02734_entropy_learned_hashing.reference b/tests/queries/0_stateless/02734_entropy_learned_hashing.reference deleted file mode 100644 index f558e3cd444..00000000000 --- a/tests/queries/0_stateless/02734_entropy_learned_hashing.reference +++ /dev/null @@ -1,18 +0,0 @@ -0 -0 -0 -0 -0 -0 -2603192927274642682 -4947675599669400333 -10783339242466472992 -0 -0 -0 -0 -0 -0 -2603192927274642682 -4947675599669400333 -10783339242466472992 diff --git a/tests/queries/0_stateless/02734_entropy_learned_hashing.sql b/tests/queries/0_stateless/02734_entropy_learned_hashing.sql deleted file mode 100644 index ae829fa03c3..00000000000 --- a/tests/queries/0_stateless/02734_entropy_learned_hashing.sql +++ /dev/null @@ -1,30 +0,0 @@ --- Tags: no-parallel --- no-parallel because entropy-learned hash uses global state - -SET allow_experimental_hash_functions = 1; - --- no commonalities between keys -DROP TABLE IF EXISTS tbl1; -CREATE TABLE tbl1 (x String) ENGINE=Memory; -INSERT INTO tbl1 VALUES ('a'), ('b'), ('c'); -SELECT prepareTrainEntropyLearnedHash(x, 'id1') FROM tbl1; -SELECT trainEntropyLearnedHash('id1') FROM tbl1; -SELECT entropyLearnedHash(x, 'id1') FROM tbl1; - --- with commonalities between keys -DROP TABLE IF EXISTS tbl2; -CREATE TABLE tbl2 (x String) ENGINE=Memory; -INSERT INTO tbl2 VALUES ('aa'), ('ba'), ('ca'); -SELECT prepareTrainEntropyLearnedHash(x, 'id2') FROM tbl2; -SELECT trainEntropyLearnedHash('id2') FROM tbl2; -SELECT entropyLearnedHash(x, 'id2') FROM tbl2; - --- negative tests -SELECT prepareTrainEntropyLearnedHash(x, 1) FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT prepareTrainEntropyLearnedHash(1, 'id1') FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT trainEntropyLearnedHash(1) FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT entropyLearnedHash(1, 'id1') FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT entropyLearnedHash(x, 'non-existing id') FROM tbl1; -- { serverError BAD_ARGUMENTS } - -DROP TABLE tbl1; -DROP TABLE tbl2; From 86e97f5f5c80ae036655cf201dd50feac3563ad5 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Thu, 1 Jun 2023 03:19:23 -0600 Subject: [PATCH 210/308] Update reddit-comments.md --- .../example-datasets/reddit-comments.md | 616 ++++++++++-------- 1 file changed, 350 insertions(+), 266 deletions(-) diff --git a/docs/en/getting-started/example-datasets/reddit-comments.md b/docs/en/getting-started/example-datasets/reddit-comments.md index aa5c474103d..02961a07730 100644 --- a/docs/en/getting-started/example-datasets/reddit-comments.md +++ b/docs/en/getting-started/example-datasets/reddit-comments.md @@ -18,7 +18,7 @@ This dataset contains publicly-available comments on Reddit that go back to Dece A shoutout to Percona for the [motivation behind ingesting this dataset](https://www.percona.com/blog/big-data-set-reddit-comments-analyzing-clickhouse/), which we have downloaded and stored in an S3 bucket. :::note -The following commands were executed on ClickHouse Cloud. To run this on your own cluster, replace `default` in the `s3Cluster` function call with the name of your cluster. If you do not have a cluster, then replace the `s3Cluster` function with the `s3` function. +The following commands were executed on a Production instance of ClickHouse Cloud with the minimum memory set to 720GB. To run this on your own cluster, replace `default` in the `s3Cluster` function call with the name of your cluster. If you do not have a cluster, then replace the `s3Cluster` function with the `s3` function. ::: 1. Let's create a table for the Reddit data: @@ -187,14 +187,28 @@ LIMIT 10; TRUNCATE TABLE reddit; ``` -8. This is a fun dataset and it looks like we can find some great information, so let's go ahead and insert the entire dataset from 2005 to 2023. When you're ready, run this command to insert all the rows. (It takes a while - up to 17 hours!) +8. This is a fun dataset and it looks like we can find some great information, so let's go ahead and insert the entire dataset from 2005 to 2023. For practical reasons, it works well to insert the data by years starting with... + + +```sql +INSERT INTO reddit + SELECT * + FROM s3Cluster( + 'default', + 'https://clickhouse-public-datasets.s3.eu-central-1.amazonaws.com/reddit/original/RC_2005*', + 'JSONEachRow' + ) + SETTINGS zstd_window_log_max = 31; +``` + +...and ending with: ```sql INSERT INTO reddit SELECT * FROM s3Cluster( 'default', - 'https://clickhouse-public-datasets.s3.amazonaws.com/reddit/original/RC*', + 'https://clickhouse-public-datasets.s3.amazonaws.com/reddit/original/RC_2023*', 'JSONEachRow' ) SETTINGS zstd_window_log_max = 31; @@ -206,19 +220,48 @@ If you do not have a cluster, use `s3` instead of `s3Cluster`: INSERT INTO reddit SELECT * FROM s3( - 'https://clickhouse-public-datasets.s3.amazonaws.com/reddit/original/RC*', + 'https://clickhouse-public-datasets.s3.amazonaws.com/reddit/original/RC_2005*', 'JSONEachRow' ) SETTINGS zstd_window_log_max = 31; ``` -The response looks like: +8. To verify it worked, here are the number of rows per year (as of February, 2023): -```response -0 rows in set. Elapsed: 61187.839 sec. Processed 6.74 billion rows, 2.06 TB (110.17 thousand rows/s., 33.68 MB/s.) +```sql +SELECT + toYear(created_utc) AS year, + formatReadableQuantity(count()) +FROM reddit +GROUP BY year; ``` -8. Let's see how many rows were inserted and how much disk space the table is using: +```response + +┌─year─┬─formatReadableQuantity(count())─┐ +│ 2005 │ 1.07 thousand │ +│ 2006 │ 417.18 thousand │ +│ 2007 │ 2.46 million │ +│ 2008 │ 7.24 million │ +│ 2009 │ 18.86 million │ +│ 2010 │ 42.93 million │ +│ 2011 │ 28.91 million │ +│ 2012 │ 260.31 million │ +│ 2013 │ 402.21 million │ +│ 2014 │ 531.80 million │ +│ 2015 │ 667.76 million │ +│ 2016 │ 799.90 million │ +│ 2017 │ 972.86 million │ +│ 2018 │ 1.24 billion │ +│ 2019 │ 1.66 billion │ +│ 2020 │ 2.16 billion │ +│ 2021 │ 2.59 billion │ +│ 2022 │ 2.82 billion │ +│ 2023 │ 474.86 million │ +└──────┴─────────────────────────────────┘ +``` + +9. Let's see how many rows were inserted and how much disk space the table is using: ```sql @@ -228,17 +271,17 @@ SELECT formatReadableSize(sum(bytes)) AS disk_size, formatReadableSize(sum(data_uncompressed_bytes)) AS uncompressed_size FROM system.parts -WHERE (table = 'reddit') AND active +WHERE (table = 'reddit') AND active; ``` Notice the compression of disk storage is about 1/3 of the uncompressed size: ```response -┌──────count─┬─formatReadableQuantity(sum(rows))─┬─disk_size──┬─uncompressed_size─┐ -│ 6739503568 │ 6.74 billion │ 501.10 GiB │ 1.51 TiB │ -└────────────┴───────────────────────────────────┴────────────┴───────────────────┘ +┌───────count─┬─formatReadableQuantity(sum(rows))─┬─disk_size─┬─uncompressed_size─┐ +│ 14688534662 │ 14.69 billion │ 1.03 TiB │ 3.26 TiB │ +└─────────────┴───────────────────────────────────┴───────────┴───────────────────┘ -1 row in set. Elapsed: 0.010 sec. +1 row in set. Elapsed: 0.005 sec. ``` 9. The following query shows how many comments, authors and subreddits we have for each month: @@ -257,185 +300,216 @@ GROUP BY firstOfMonth ORDER BY firstOfMonth ASC; ``` -This is a substantial query that has to process all 6.74 billion rows, but we still get an impressive response time (about 3 minutes): +This is a substantial query that has to process all 14.69 billion rows, but we still get an impressive response time (about 48 seconds): ```response -┌─firstOfMonth─┬─────────c─┬─bar_count─────────────────┬─authors─┬─bar_authors───────────────┬─subreddits─┬─bar_subreddits────────────┐ -│ 2005-12-01 │ 1075 │ │ 394 │ │ 1 │ │ -│ 2006-01-01 │ 3666 │ │ 791 │ │ 2 │ │ -│ 2006-02-01 │ 9095 │ │ 1464 │ │ 18 │ │ -│ 2006-03-01 │ 13859 │ │ 1958 │ │ 15 │ │ -│ 2006-04-01 │ 19090 │ │ 2334 │ │ 21 │ │ -│ 2006-05-01 │ 26859 │ │ 2698 │ │ 21 │ │ -│ 2006-06-01 │ 29163 │ │ 3043 │ │ 19 │ │ -│ 2006-07-01 │ 37031 │ │ 3532 │ │ 22 │ │ -│ 2006-08-01 │ 50559 │ │ 4750 │ │ 24 │ │ -│ 2006-09-01 │ 50675 │ │ 4908 │ │ 21 │ │ -│ 2006-10-01 │ 54148 │ │ 5654 │ │ 31 │ │ -│ 2006-11-01 │ 62021 │ │ 6490 │ │ 23 │ │ -│ 2006-12-01 │ 61018 │ │ 6707 │ │ 24 │ │ -│ 2007-01-01 │ 81341 │ │ 7931 │ │ 23 │ │ -│ 2007-02-01 │ 95634 │ │ 9020 │ │ 21 │ │ -│ 2007-03-01 │ 112444 │ │ 10842 │ │ 23 │ │ -│ 2007-04-01 │ 126773 │ │ 10701 │ │ 26 │ │ -│ 2007-05-01 │ 170097 │ │ 11365 │ │ 25 │ │ -│ 2007-06-01 │ 178800 │ │ 11267 │ │ 22 │ │ -│ 2007-07-01 │ 203319 │ │ 12482 │ │ 25 │ │ -│ 2007-08-01 │ 225111 │ │ 14124 │ │ 30 │ │ -│ 2007-09-01 │ 259497 │ ▏ │ 15416 │ │ 33 │ │ -│ 2007-10-01 │ 274170 │ ▏ │ 15302 │ │ 36 │ │ -│ 2007-11-01 │ 372983 │ ▏ │ 15134 │ │ 43 │ │ -│ 2007-12-01 │ 363390 │ ▏ │ 15915 │ │ 31 │ │ -│ 2008-01-01 │ 452990 │ ▏ │ 18857 │ │ 126 │ │ -│ 2008-02-01 │ 441768 │ ▏ │ 18266 │ │ 173 │ │ -│ 2008-03-01 │ 463728 │ ▏ │ 18947 │ │ 292 │ │ -│ 2008-04-01 │ 468317 │ ▏ │ 18590 │ │ 323 │ │ -│ 2008-05-01 │ 536380 │ ▎ │ 20861 │ │ 375 │ │ -│ 2008-06-01 │ 577684 │ ▎ │ 22557 │ │ 575 │ ▏ │ -│ 2008-07-01 │ 592610 │ ▎ │ 23123 │ │ 657 │ ▏ │ -│ 2008-08-01 │ 595959 │ ▎ │ 23729 │ │ 707 │ ▏ │ -│ 2008-09-01 │ 680892 │ ▎ │ 26374 │ ▏ │ 801 │ ▏ │ -│ 2008-10-01 │ 789874 │ ▍ │ 28970 │ ▏ │ 893 │ ▏ │ -│ 2008-11-01 │ 792310 │ ▍ │ 30272 │ ▏ │ 1024 │ ▎ │ -│ 2008-12-01 │ 850359 │ ▍ │ 34073 │ ▏ │ 1103 │ ▎ │ -│ 2009-01-01 │ 1051649 │ ▌ │ 38978 │ ▏ │ 1316 │ ▎ │ -│ 2009-02-01 │ 944711 │ ▍ │ 43390 │ ▏ │ 1132 │ ▎ │ -│ 2009-03-01 │ 1048643 │ ▌ │ 46516 │ ▏ │ 1203 │ ▎ │ -│ 2009-04-01 │ 1094599 │ ▌ │ 48284 │ ▏ │ 1334 │ ▎ │ -│ 2009-05-01 │ 1201257 │ ▌ │ 52512 │ ▎ │ 1395 │ ▎ │ -│ 2009-06-01 │ 1258750 │ ▋ │ 57728 │ ▎ │ 1473 │ ▎ │ -│ 2009-07-01 │ 1470290 │ ▋ │ 60098 │ ▎ │ 1686 │ ▍ │ -│ 2009-08-01 │ 1750688 │ ▉ │ 67347 │ ▎ │ 1777 │ ▍ │ -│ 2009-09-01 │ 2032276 │ █ │ 78051 │ ▍ │ 1784 │ ▍ │ -│ 2009-10-01 │ 2242017 │ █ │ 93409 │ ▍ │ 2071 │ ▌ │ -│ 2009-11-01 │ 2207444 │ █ │ 95940 │ ▍ │ 2141 │ ▌ │ -│ 2009-12-01 │ 2560510 │ █▎ │ 104239 │ ▌ │ 2141 │ ▌ │ -│ 2010-01-01 │ 2884096 │ █▍ │ 114314 │ ▌ │ 2313 │ ▌ │ -│ 2010-02-01 │ 2687779 │ █▎ │ 115683 │ ▌ │ 2522 │ ▋ │ -│ 2010-03-01 │ 3228254 │ █▌ │ 125775 │ ▋ │ 2890 │ ▋ │ -│ 2010-04-01 │ 3209898 │ █▌ │ 128936 │ ▋ │ 3170 │ ▊ │ -│ 2010-05-01 │ 3267363 │ █▋ │ 131851 │ ▋ │ 3166 │ ▊ │ -│ 2010-06-01 │ 3532867 │ █▊ │ 139522 │ ▋ │ 3301 │ ▊ │ -│ 2010-07-01 │ 4032737 │ ██ │ 153451 │ ▊ │ 3662 │ ▉ │ -│ 2010-08-01 │ 4247982 │ ██ │ 164071 │ ▊ │ 3653 │ ▉ │ -│ 2010-09-01 │ 4704069 │ ██▎ │ 186613 │ ▉ │ 4009 │ █ │ -│ 2010-10-01 │ 5032368 │ ██▌ │ 203800 │ █ │ 4154 │ █ │ -│ 2010-11-01 │ 5689002 │ ██▊ │ 226134 │ █▏ │ 4383 │ █ │ -│ 2010-12-01 │ 5972642 │ ██▉ │ 245824 │ █▏ │ 4692 │ █▏ │ -│ 2011-01-01 │ 6603329 │ ███▎ │ 270025 │ █▎ │ 5141 │ █▎ │ -│ 2011-02-01 │ 6363114 │ ███▏ │ 277593 │ █▍ │ 5202 │ █▎ │ -│ 2011-03-01 │ 7556165 │ ███▊ │ 314748 │ █▌ │ 5445 │ █▎ │ -│ 2011-04-01 │ 7571398 │ ███▊ │ 329920 │ █▋ │ 6128 │ █▌ │ -│ 2011-05-01 │ 8803949 │ ████▍ │ 365013 │ █▊ │ 6834 │ █▋ │ -│ 2011-06-01 │ 9766511 │ ████▉ │ 393945 │ █▉ │ 7519 │ █▉ │ -│ 2011-07-01 │ 10557466 │ █████▎ │ 424235 │ ██ │ 8293 │ ██ │ -│ 2011-08-01 │ 12316144 │ ██████▏ │ 475326 │ ██▍ │ 9657 │ ██▍ │ -│ 2011-09-01 │ 12150412 │ ██████ │ 503142 │ ██▌ │ 10278 │ ██▌ │ -│ 2011-10-01 │ 13470278 │ ██████▋ │ 548801 │ ██▋ │ 10922 │ ██▋ │ -│ 2011-11-01 │ 13621533 │ ██████▊ │ 574435 │ ██▊ │ 11572 │ ██▉ │ -│ 2011-12-01 │ 14509469 │ ███████▎ │ 622849 │ ███ │ 12335 │ ███ │ -│ 2012-01-01 │ 16350205 │ ████████▏ │ 696110 │ ███▍ │ 14281 │ ███▌ │ -│ 2012-02-01 │ 16015695 │ ████████ │ 722892 │ ███▌ │ 14949 │ ███▋ │ -│ 2012-03-01 │ 17881943 │ ████████▉ │ 789664 │ ███▉ │ 15795 │ ███▉ │ -│ 2012-04-01 │ 19044534 │ █████████▌ │ 842491 │ ████▏ │ 16440 │ ████ │ -│ 2012-05-01 │ 20388260 │ ██████████▏ │ 886176 │ ████▍ │ 16974 │ ████▏ │ -│ 2012-06-01 │ 21897913 │ ██████████▉ │ 946798 │ ████▋ │ 17952 │ ████▍ │ -│ 2012-07-01 │ 24087517 │ ████████████ │ 1018636 │ █████ │ 19069 │ ████▊ │ -│ 2012-08-01 │ 25703326 │ ████████████▊ │ 1094445 │ █████▍ │ 20553 │ █████▏ │ -│ 2012-09-01 │ 23419524 │ ███████████▋ │ 1088491 │ █████▍ │ 20831 │ █████▏ │ -│ 2012-10-01 │ 24788236 │ ████████████▍ │ 1131885 │ █████▋ │ 21868 │ █████▍ │ -│ 2012-11-01 │ 24648302 │ ████████████▎ │ 1167608 │ █████▊ │ 21791 │ █████▍ │ -│ 2012-12-01 │ 26080276 │ █████████████ │ 1218402 │ ██████ │ 22622 │ █████▋ │ -│ 2013-01-01 │ 30365867 │ ███████████████▏ │ 1341703 │ ██████▋ │ 24696 │ ██████▏ │ -│ 2013-02-01 │ 27213960 │ █████████████▌ │ 1304756 │ ██████▌ │ 24514 │ ██████▏ │ -│ 2013-03-01 │ 30771274 │ ███████████████▍ │ 1391703 │ ██████▉ │ 25730 │ ██████▍ │ -│ 2013-04-01 │ 33259557 │ ████████████████▋ │ 1485971 │ ███████▍ │ 27294 │ ██████▊ │ -│ 2013-05-01 │ 33126225 │ ████████████████▌ │ 1506473 │ ███████▌ │ 27299 │ ██████▊ │ -│ 2013-06-01 │ 32648247 │ ████████████████▎ │ 1506650 │ ███████▌ │ 27450 │ ██████▊ │ -│ 2013-07-01 │ 34922133 │ █████████████████▍ │ 1561771 │ ███████▊ │ 28294 │ ███████ │ -│ 2013-08-01 │ 34766579 │ █████████████████▍ │ 1589781 │ ███████▉ │ 28943 │ ███████▏ │ -│ 2013-09-01 │ 31990369 │ ███████████████▉ │ 1570342 │ ███████▊ │ 29408 │ ███████▎ │ -│ 2013-10-01 │ 35940040 │ █████████████████▉ │ 1683770 │ ████████▍ │ 30273 │ ███████▌ │ -│ 2013-11-01 │ 37396497 │ ██████████████████▋ │ 1757467 │ ████████▊ │ 31173 │ ███████▊ │ -│ 2013-12-01 │ 39810216 │ ███████████████████▉ │ 1846204 │ █████████▏ │ 32326 │ ████████ │ -│ 2014-01-01 │ 42420655 │ █████████████████████▏ │ 1927229 │ █████████▋ │ 35603 │ ████████▉ │ -│ 2014-02-01 │ 38703362 │ ███████████████████▎ │ 1874067 │ █████████▎ │ 37007 │ █████████▎ │ -│ 2014-03-01 │ 42459956 │ █████████████████████▏ │ 1959888 │ █████████▊ │ 37948 │ █████████▍ │ -│ 2014-04-01 │ 42440735 │ █████████████████████▏ │ 1951369 │ █████████▊ │ 38362 │ █████████▌ │ -│ 2014-05-01 │ 42514094 │ █████████████████████▎ │ 1970197 │ █████████▊ │ 39078 │ █████████▊ │ -│ 2014-06-01 │ 41990650 │ ████████████████████▉ │ 1943850 │ █████████▋ │ 38268 │ █████████▌ │ -│ 2014-07-01 │ 46868899 │ ███████████████████████▍ │ 2059346 │ ██████████▎ │ 40634 │ ██████████▏ │ -│ 2014-08-01 │ 46990813 │ ███████████████████████▍ │ 2117335 │ ██████████▌ │ 41764 │ ██████████▍ │ -│ 2014-09-01 │ 44992201 │ ██████████████████████▍ │ 2124708 │ ██████████▌ │ 41890 │ ██████████▍ │ -│ 2014-10-01 │ 47497520 │ ███████████████████████▋ │ 2206535 │ ███████████ │ 43109 │ ██████████▊ │ -│ 2014-11-01 │ 46118074 │ ███████████████████████ │ 2239747 │ ███████████▏ │ 43718 │ ██████████▉ │ -│ 2014-12-01 │ 48807699 │ ████████████████████████▍ │ 2372945 │ ███████████▊ │ 43823 │ ██████████▉ │ -│ 2015-01-01 │ 53851542 │ █████████████████████████ │ 2499536 │ ████████████▍ │ 47172 │ ███████████▊ │ -│ 2015-02-01 │ 48342747 │ ████████████████████████▏ │ 2448496 │ ████████████▏ │ 47229 │ ███████████▊ │ -│ 2015-03-01 │ 54564441 │ █████████████████████████ │ 2550534 │ ████████████▊ │ 48156 │ ████████████ │ -│ 2015-04-01 │ 55005780 │ █████████████████████████ │ 2609443 │ █████████████ │ 49865 │ ████████████▍ │ -│ 2015-05-01 │ 54504410 │ █████████████████████████ │ 2585535 │ ████████████▉ │ 50137 │ ████████████▌ │ -│ 2015-06-01 │ 54258492 │ █████████████████████████ │ 2595129 │ ████████████▉ │ 49598 │ ████████████▍ │ -│ 2015-07-01 │ 58451788 │ █████████████████████████ │ 2720026 │ █████████████▌ │ 55022 │ █████████████▊ │ -│ 2015-08-01 │ 58075327 │ █████████████████████████ │ 2743994 │ █████████████▋ │ 55302 │ █████████████▊ │ -│ 2015-09-01 │ 55574825 │ █████████████████████████ │ 2672793 │ █████████████▎ │ 53960 │ █████████████▍ │ -│ 2015-10-01 │ 59494045 │ █████████████████████████ │ 2816426 │ ██████████████ │ 70210 │ █████████████████▌ │ -│ 2015-11-01 │ 57117500 │ █████████████████████████ │ 2847146 │ ██████████████▏ │ 71363 │ █████████████████▊ │ -│ 2015-12-01 │ 58523312 │ █████████████████████████ │ 2854840 │ ██████████████▎ │ 94559 │ ███████████████████████▋ │ -│ 2016-01-01 │ 61991732 │ █████████████████████████ │ 2920366 │ ██████████████▌ │ 108438 │ █████████████████████████ │ -│ 2016-02-01 │ 59189875 │ █████████████████████████ │ 2854683 │ ██████████████▎ │ 109916 │ █████████████████████████ │ -│ 2016-03-01 │ 63918864 │ █████████████████████████ │ 2969542 │ ██████████████▊ │ 84787 │ █████████████████████▏ │ -│ 2016-04-01 │ 64271256 │ █████████████████████████ │ 2999086 │ ██████████████▉ │ 61647 │ ███████████████▍ │ -│ 2016-05-01 │ 65212004 │ █████████████████████████ │ 3034674 │ ███████████████▏ │ 67465 │ ████████████████▊ │ -│ 2016-06-01 │ 65867743 │ █████████████████████████ │ 3057604 │ ███████████████▎ │ 75170 │ ██████████████████▊ │ -│ 2016-07-01 │ 66974735 │ █████████████████████████ │ 3199374 │ ███████████████▉ │ 77732 │ ███████████████████▍ │ -│ 2016-08-01 │ 69654819 │ █████████████████████████ │ 3239957 │ ████████████████▏ │ 63080 │ ███████████████▊ │ -│ 2016-09-01 │ 67024973 │ █████████████████████████ │ 3190864 │ ███████████████▉ │ 62324 │ ███████████████▌ │ -│ 2016-10-01 │ 71826553 │ █████████████████████████ │ 3284340 │ ████████████████▍ │ 62549 │ ███████████████▋ │ -│ 2016-11-01 │ 71022319 │ █████████████████████████ │ 3300822 │ ████████████████▌ │ 69718 │ █████████████████▍ │ -│ 2016-12-01 │ 72942967 │ █████████████████████████ │ 3430324 │ █████████████████▏ │ 71705 │ █████████████████▉ │ -│ 2017-01-01 │ 78946585 │ █████████████████████████ │ 3572093 │ █████████████████▊ │ 78198 │ ███████████████████▌ │ -│ 2017-02-01 │ 70609487 │ █████████████████████████ │ 3421115 │ █████████████████ │ 69823 │ █████████████████▍ │ -│ 2017-03-01 │ 79723106 │ █████████████████████████ │ 3638122 │ ██████████████████▏ │ 73865 │ ██████████████████▍ │ -│ 2017-04-01 │ 77478009 │ █████████████████████████ │ 3620591 │ ██████████████████ │ 74387 │ ██████████████████▌ │ -│ 2017-05-01 │ 79810360 │ █████████████████████████ │ 3650820 │ ██████████████████▎ │ 74356 │ ██████████████████▌ │ -│ 2017-06-01 │ 79901711 │ █████████████████████████ │ 3737614 │ ██████████████████▋ │ 72114 │ ██████████████████ │ -│ 2017-07-01 │ 81798725 │ █████████████████████████ │ 3872330 │ ███████████████████▎ │ 76052 │ ███████████████████ │ -│ 2017-08-01 │ 84658503 │ █████████████████████████ │ 3960093 │ ███████████████████▊ │ 77798 │ ███████████████████▍ │ -│ 2017-09-01 │ 83165192 │ █████████████████████████ │ 3880501 │ ███████████████████▍ │ 78402 │ ███████████████████▌ │ -│ 2017-10-01 │ 85828912 │ █████████████████████████ │ 3980335 │ ███████████████████▉ │ 80685 │ ████████████████████▏ │ -│ 2017-11-01 │ 84965681 │ █████████████████████████ │ 4026749 │ ████████████████████▏ │ 82659 │ ████████████████████▋ │ -│ 2017-12-01 │ 85973810 │ █████████████████████████ │ 4196354 │ ████████████████████▉ │ 91984 │ ██████████████████████▉ │ -│ 2018-01-01 │ 91558594 │ █████████████████████████ │ 4364443 │ █████████████████████▊ │ 102577 │ █████████████████████████ │ -│ 2018-02-01 │ 86467179 │ █████████████████████████ │ 4277899 │ █████████████████████▍ │ 104610 │ █████████████████████████ │ -│ 2018-03-01 │ 96490262 │ █████████████████████████ │ 4422470 │ ██████████████████████ │ 112559 │ █████████████████████████ │ -│ 2018-04-01 │ 98101232 │ █████████████████████████ │ 4572434 │ ██████████████████████▊ │ 105284 │ █████████████████████████ │ -│ 2018-05-01 │ 100109100 │ █████████████████████████ │ 4698908 │ ███████████████████████▍ │ 103910 │ █████████████████████████ │ -│ 2018-06-01 │ 100009462 │ █████████████████████████ │ 4697426 │ ███████████████████████▍ │ 101107 │ █████████████████████████ │ -│ 2018-07-01 │ 108151359 │ █████████████████████████ │ 5099492 │ █████████████████████████ │ 106184 │ █████████████████████████ │ -│ 2018-08-01 │ 107330940 │ █████████████████████████ │ 5084082 │ █████████████████████████ │ 109985 │ █████████████████████████ │ -│ 2018-09-01 │ 104473929 │ █████████████████████████ │ 5011953 │ █████████████████████████ │ 109710 │ █████████████████████████ │ -│ 2018-10-01 │ 112346556 │ █████████████████████████ │ 5320405 │ █████████████████████████ │ 112533 │ █████████████████████████ │ -│ 2018-11-01 │ 112573001 │ █████████████████████████ │ 5353282 │ █████████████████████████ │ 112211 │ █████████████████████████ │ -│ 2018-12-01 │ 121953600 │ █████████████████████████ │ 5611543 │ █████████████████████████ │ 118291 │ █████████████████████████ │ -│ 2019-01-01 │ 129386587 │ █████████████████████████ │ 6016687 │ █████████████████████████ │ 125725 │ █████████████████████████ │ -│ 2019-02-01 │ 120645639 │ █████████████████████████ │ 5974488 │ █████████████████████████ │ 125420 │ █████████████████████████ │ -│ 2019-03-01 │ 137650471 │ █████████████████████████ │ 6410197 │ █████████████████████████ │ 135924 │ █████████████████████████ │ -│ 2019-04-01 │ 138473643 │ █████████████████████████ │ 6416384 │ █████████████████████████ │ 139844 │ █████████████████████████ │ -│ 2019-05-01 │ 142463421 │ █████████████████████████ │ 6574836 │ █████████████████████████ │ 142012 │ █████████████████████████ │ -│ 2019-06-01 │ 134172939 │ █████████████████████████ │ 6601267 │ █████████████████████████ │ 140997 │ █████████████████████████ │ -│ 2019-07-01 │ 145965083 │ █████████████████████████ │ 6901822 │ █████████████████████████ │ 147802 │ █████████████████████████ │ -│ 2019-08-01 │ 146854393 │ █████████████████████████ │ 6993882 │ █████████████████████████ │ 151888 │ █████████████████████████ │ -│ 2019-09-01 │ 137540219 │ █████████████████████████ │ 7001362 │ █████████████████████████ │ 148839 │ █████████████████████████ │ -│ 2019-10-01 │ 129771456 │ █████████████████████████ │ 6825690 │ █████████████████████████ │ 144453 │ █████████████████████████ │ -│ 2019-11-01 │ 107990259 │ █████████████████████████ │ 6368286 │ █████████████████████████ │ 141768 │ █████████████████████████ │ -│ 2019-12-01 │ 112895934 │ █████████████████████████ │ 6640902 │ █████████████████████████ │ 148277 │ █████████████████████████ │ -│ 2020-01-01 │ 54354879 │ █████████████████████████ │ 4782339 │ ███████████████████████▉ │ 111658 │ █████████████████████████ │ -│ 2020-02-01 │ 22696923 │ ███████████▎ │ 3135175 │ ███████████████▋ │ 79521 │ ███████████████████▉ │ -│ 2020-03-01 │ 3466677 │ █▋ │ 987960 │ ████▉ │ 40901 │ ██████████▏ │ -└──────────────┴───────────┴───────────────────────────┴─────────┴───────────────────────────┴────────────┴───────────────────────────┘ +┌─firstOfMonth─┬─────────c─┬─bar_count─────────────────┬──authors─┬─bar_authors───────────────┬─subreddits─┬─bar_subreddits────────────┐ +│ 2005-12-01 │ 1075 │ │ 394 │ │ 1 │ │ +│ 2006-01-01 │ 3666 │ │ 791 │ │ 2 │ │ +│ 2006-02-01 │ 9095 │ │ 1464 │ │ 18 │ │ +│ 2006-03-01 │ 13859 │ │ 1958 │ │ 15 │ │ +│ 2006-04-01 │ 19090 │ │ 2334 │ │ 21 │ │ +│ 2006-05-01 │ 26859 │ │ 2698 │ │ 21 │ │ +│ 2006-06-01 │ 29163 │ │ 3043 │ │ 19 │ │ +│ 2006-07-01 │ 37031 │ │ 3532 │ │ 22 │ │ +│ 2006-08-01 │ 50559 │ │ 4750 │ │ 24 │ │ +│ 2006-09-01 │ 50675 │ │ 4908 │ │ 21 │ │ +│ 2006-10-01 │ 54148 │ │ 5654 │ │ 31 │ │ +│ 2006-11-01 │ 62021 │ │ 6490 │ │ 23 │ │ +│ 2006-12-01 │ 61018 │ │ 6707 │ │ 24 │ │ +│ 2007-01-01 │ 81341 │ │ 7931 │ │ 23 │ │ +│ 2007-02-01 │ 95634 │ │ 9020 │ │ 21 │ │ +│ 2007-03-01 │ 112444 │ │ 10842 │ │ 23 │ │ +│ 2007-04-01 │ 126773 │ │ 10701 │ │ 26 │ │ +│ 2007-05-01 │ 170097 │ │ 11365 │ │ 25 │ │ +│ 2007-06-01 │ 178800 │ │ 11267 │ │ 22 │ │ +│ 2007-07-01 │ 203319 │ │ 12482 │ │ 25 │ │ +│ 2007-08-01 │ 225111 │ │ 14124 │ │ 30 │ │ +│ 2007-09-01 │ 259497 │ ▏ │ 15416 │ │ 33 │ │ +│ 2007-10-01 │ 274170 │ ▏ │ 15302 │ │ 36 │ │ +│ 2007-11-01 │ 372983 │ ▏ │ 15134 │ │ 43 │ │ +│ 2007-12-01 │ 363390 │ ▏ │ 15915 │ │ 31 │ │ +│ 2008-01-01 │ 452990 │ ▏ │ 18857 │ │ 126 │ │ +│ 2008-02-01 │ 441768 │ ▏ │ 18266 │ │ 173 │ │ +│ 2008-03-01 │ 463728 │ ▏ │ 18947 │ │ 292 │ │ +│ 2008-04-01 │ 468317 │ ▏ │ 18590 │ │ 323 │ │ +│ 2008-05-01 │ 536380 │ ▎ │ 20861 │ │ 375 │ │ +│ 2008-06-01 │ 577684 │ ▎ │ 22557 │ │ 575 │ ▏ │ +│ 2008-07-01 │ 592610 │ ▎ │ 23123 │ │ 657 │ ▏ │ +│ 2008-08-01 │ 595959 │ ▎ │ 23729 │ │ 707 │ ▏ │ +│ 2008-09-01 │ 680892 │ ▎ │ 26374 │ ▏ │ 801 │ ▏ │ +│ 2008-10-01 │ 789874 │ ▍ │ 28970 │ ▏ │ 893 │ ▏ │ +│ 2008-11-01 │ 792310 │ ▍ │ 30272 │ ▏ │ 1024 │ ▎ │ +│ 2008-12-01 │ 850359 │ ▍ │ 34073 │ ▏ │ 1103 │ ▎ │ +│ 2009-01-01 │ 1051649 │ ▌ │ 38978 │ ▏ │ 1316 │ ▎ │ +│ 2009-02-01 │ 944711 │ ▍ │ 43390 │ ▏ │ 1132 │ ▎ │ +│ 2009-03-01 │ 1048643 │ ▌ │ 46516 │ ▏ │ 1203 │ ▎ │ +│ 2009-04-01 │ 1094599 │ ▌ │ 48284 │ ▏ │ 1334 │ ▎ │ +│ 2009-05-01 │ 1201257 │ ▌ │ 52512 │ ▎ │ 1395 │ ▎ │ +│ 2009-06-01 │ 1258750 │ ▋ │ 57728 │ ▎ │ 1473 │ ▎ │ +│ 2009-07-01 │ 1470290 │ ▋ │ 60098 │ ▎ │ 1686 │ ▍ │ +│ 2009-08-01 │ 1750688 │ ▉ │ 67347 │ ▎ │ 1777 │ ▍ │ +│ 2009-09-01 │ 2032276 │ █ │ 78051 │ ▍ │ 1784 │ ▍ │ +│ 2009-10-01 │ 2242017 │ █ │ 93409 │ ▍ │ 2071 │ ▌ │ +│ 2009-11-01 │ 2207444 │ █ │ 95940 │ ▍ │ 2141 │ ▌ │ +│ 2009-12-01 │ 2560510 │ █▎ │ 104239 │ ▌ │ 2141 │ ▌ │ +│ 2010-01-01 │ 2884096 │ █▍ │ 114314 │ ▌ │ 2313 │ ▌ │ +│ 2010-02-01 │ 2687779 │ █▎ │ 115683 │ ▌ │ 2522 │ ▋ │ +│ 2010-03-01 │ 3228254 │ █▌ │ 125775 │ ▋ │ 2890 │ ▋ │ +│ 2010-04-01 │ 3209898 │ █▌ │ 128936 │ ▋ │ 3170 │ ▊ │ +│ 2010-05-01 │ 3267363 │ █▋ │ 131851 │ ▋ │ 3166 │ ▊ │ +│ 2010-06-01 │ 3532867 │ █▊ │ 139522 │ ▋ │ 3301 │ ▊ │ +│ 2010-07-01 │ 806612 │ ▍ │ 76486 │ ▍ │ 1955 │ ▍ │ +│ 2010-08-01 │ 4247982 │ ██ │ 164071 │ ▊ │ 3653 │ ▉ │ +│ 2010-09-01 │ 4704069 │ ██▎ │ 186613 │ ▉ │ 4009 │ █ │ +│ 2010-10-01 │ 5032368 │ ██▌ │ 203800 │ █ │ 4154 │ █ │ +│ 2010-11-01 │ 5689002 │ ██▊ │ 226134 │ █▏ │ 4383 │ █ │ +│ 2010-12-01 │ 3642690 │ █▊ │ 196847 │ ▉ │ 3914 │ ▉ │ +│ 2011-01-01 │ 3924540 │ █▉ │ 215057 │ █ │ 4240 │ █ │ +│ 2011-02-01 │ 3859131 │ █▉ │ 223485 │ █ │ 4371 │ █ │ +│ 2011-03-01 │ 2877996 │ █▍ │ 208607 │ █ │ 3870 │ ▉ │ +│ 2011-04-01 │ 3859131 │ █▉ │ 248931 │ █▏ │ 4881 │ █▏ │ +│ 2011-06-01 │ 3859131 │ █▉ │ 267197 │ █▎ │ 5255 │ █▎ │ +│ 2011-08-01 │ 2943405 │ █▍ │ 259428 │ █▎ │ 5806 │ █▍ │ +│ 2011-10-01 │ 3859131 │ █▉ │ 327342 │ █▋ │ 6958 │ █▋ │ +│ 2011-12-01 │ 3728313 │ █▊ │ 354817 │ █▊ │ 7713 │ █▉ │ +│ 2012-01-01 │ 16350205 │ ████████▏ │ 696110 │ ███▍ │ 14281 │ ███▌ │ +│ 2012-02-01 │ 16015695 │ ████████ │ 722892 │ ███▌ │ 14949 │ ███▋ │ +│ 2012-03-01 │ 17881943 │ ████████▉ │ 789664 │ ███▉ │ 15795 │ ███▉ │ +│ 2012-04-01 │ 19044534 │ █████████▌ │ 842491 │ ████▏ │ 16440 │ ████ │ +│ 2012-05-01 │ 20388260 │ ██████████▏ │ 886176 │ ████▍ │ 16974 │ ████▏ │ +│ 2012-06-01 │ 21897913 │ ██████████▉ │ 946798 │ ████▋ │ 17952 │ ████▍ │ +│ 2012-07-01 │ 24087517 │ ████████████ │ 1018636 │ █████ │ 19069 │ ████▊ │ +│ 2012-08-01 │ 25703326 │ ████████████▊ │ 1094445 │ █████▍ │ 20553 │ █████▏ │ +│ 2012-09-01 │ 23419524 │ ███████████▋ │ 1088491 │ █████▍ │ 20831 │ █████▏ │ +│ 2012-10-01 │ 24788236 │ ████████████▍ │ 1131885 │ █████▋ │ 21868 │ █████▍ │ +│ 2012-11-01 │ 24648302 │ ████████████▎ │ 1167608 │ █████▊ │ 21791 │ █████▍ │ +│ 2012-12-01 │ 26080276 │ █████████████ │ 1218402 │ ██████ │ 22622 │ █████▋ │ +│ 2013-01-01 │ 30365867 │ ███████████████▏ │ 1341703 │ ██████▋ │ 24696 │ ██████▏ │ +│ 2013-02-01 │ 27213960 │ █████████████▌ │ 1304756 │ ██████▌ │ 24514 │ ██████▏ │ +│ 2013-03-01 │ 30771274 │ ███████████████▍ │ 1391703 │ ██████▉ │ 25730 │ ██████▍ │ +│ 2013-04-01 │ 33259557 │ ████████████████▋ │ 1485971 │ ███████▍ │ 27294 │ ██████▊ │ +│ 2013-05-01 │ 33126225 │ ████████████████▌ │ 1506473 │ ███████▌ │ 27299 │ ██████▊ │ +│ 2013-06-01 │ 32648247 │ ████████████████▎ │ 1506650 │ ███████▌ │ 27450 │ ██████▊ │ +│ 2013-07-01 │ 34922133 │ █████████████████▍ │ 1561771 │ ███████▊ │ 28294 │ ███████ │ +│ 2013-08-01 │ 34766579 │ █████████████████▍ │ 1589781 │ ███████▉ │ 28943 │ ███████▏ │ +│ 2013-09-01 │ 31990369 │ ███████████████▉ │ 1570342 │ ███████▊ │ 29408 │ ███████▎ │ +│ 2013-10-01 │ 35940040 │ █████████████████▉ │ 1683770 │ ████████▍ │ 30273 │ ███████▌ │ +│ 2013-11-01 │ 37396497 │ ██████████████████▋ │ 1757467 │ ████████▊ │ 31173 │ ███████▊ │ +│ 2013-12-01 │ 39810216 │ ███████████████████▉ │ 1846204 │ █████████▏ │ 32326 │ ████████ │ +│ 2014-01-01 │ 42420655 │ █████████████████████▏ │ 1927229 │ █████████▋ │ 35603 │ ████████▉ │ +│ 2014-02-01 │ 38703362 │ ███████████████████▎ │ 1874067 │ █████████▎ │ 37007 │ █████████▎ │ +│ 2014-03-01 │ 42459956 │ █████████████████████▏ │ 1959888 │ █████████▊ │ 37948 │ █████████▍ │ +│ 2014-04-01 │ 42440735 │ █████████████████████▏ │ 1951369 │ █████████▊ │ 38362 │ █████████▌ │ +│ 2014-05-01 │ 42514094 │ █████████████████████▎ │ 1970197 │ █████████▊ │ 39078 │ █████████▊ │ +│ 2014-06-01 │ 41990650 │ ████████████████████▉ │ 1943850 │ █████████▋ │ 38268 │ █████████▌ │ +│ 2014-07-01 │ 46868899 │ ███████████████████████▍ │ 2059346 │ ██████████▎ │ 40634 │ ██████████▏ │ +│ 2014-08-01 │ 46990813 │ ███████████████████████▍ │ 2117335 │ ██████████▌ │ 41764 │ ██████████▍ │ +│ 2014-09-01 │ 44992201 │ ██████████████████████▍ │ 2124708 │ ██████████▌ │ 41890 │ ██████████▍ │ +│ 2014-10-01 │ 47497520 │ ███████████████████████▋ │ 2206535 │ ███████████ │ 43109 │ ██████████▊ │ +│ 2014-11-01 │ 46118074 │ ███████████████████████ │ 2239747 │ ███████████▏ │ 43718 │ ██████████▉ │ +│ 2014-12-01 │ 48807699 │ ████████████████████████▍ │ 2372945 │ ███████████▊ │ 43823 │ ██████████▉ │ +│ 2015-01-01 │ 53851542 │ █████████████████████████ │ 2499536 │ ████████████▍ │ 47172 │ ███████████▊ │ +│ 2015-02-01 │ 48342747 │ ████████████████████████▏ │ 2448496 │ ████████████▏ │ 47229 │ ███████████▊ │ +│ 2015-03-01 │ 54564441 │ █████████████████████████ │ 2550534 │ ████████████▊ │ 48156 │ ████████████ │ +│ 2015-04-01 │ 55005780 │ █████████████████████████ │ 2609443 │ █████████████ │ 49865 │ ████████████▍ │ +│ 2015-05-01 │ 54504410 │ █████████████████████████ │ 2585535 │ ████████████▉ │ 50137 │ ████████████▌ │ +│ 2015-06-01 │ 54258492 │ █████████████████████████ │ 2595129 │ ████████████▉ │ 49598 │ ████████████▍ │ +│ 2015-07-01 │ 58451788 │ █████████████████████████ │ 2720026 │ █████████████▌ │ 55022 │ █████████████▊ │ +│ 2015-08-01 │ 58075327 │ █████████████████████████ │ 2743994 │ █████████████▋ │ 55302 │ █████████████▊ │ +│ 2015-09-01 │ 55574825 │ █████████████████████████ │ 2672793 │ █████████████▎ │ 53960 │ █████████████▍ │ +│ 2015-10-01 │ 59494045 │ █████████████████████████ │ 2816426 │ ██████████████ │ 70210 │ █████████████████▌ │ +│ 2015-11-01 │ 57117500 │ █████████████████████████ │ 2847146 │ ██████████████▏ │ 71363 │ █████████████████▊ │ +│ 2015-12-01 │ 58523312 │ █████████████████████████ │ 2854840 │ ██████████████▎ │ 94559 │ ███████████████████████▋ │ +│ 2016-01-01 │ 61991732 │ █████████████████████████ │ 2920366 │ ██████████████▌ │ 108438 │ █████████████████████████ │ +│ 2016-02-01 │ 59189875 │ █████████████████████████ │ 2854683 │ ██████████████▎ │ 109916 │ █████████████████████████ │ +│ 2016-03-01 │ 63918864 │ █████████████████████████ │ 2969542 │ ██████████████▊ │ 84787 │ █████████████████████▏ │ +│ 2016-04-01 │ 64271256 │ █████████████████████████ │ 2999086 │ ██████████████▉ │ 61647 │ ███████████████▍ │ +│ 2016-05-01 │ 65212004 │ █████████████████████████ │ 3034674 │ ███████████████▏ │ 67465 │ ████████████████▊ │ +│ 2016-06-01 │ 65867743 │ █████████████████████████ │ 3057604 │ ███████████████▎ │ 75170 │ ██████████████████▊ │ +│ 2016-07-01 │ 66974735 │ █████████████████████████ │ 3199374 │ ███████████████▉ │ 77732 │ ███████████████████▍ │ +│ 2016-08-01 │ 69654819 │ █████████████████████████ │ 3239957 │ ████████████████▏ │ 63080 │ ███████████████▊ │ +│ 2016-09-01 │ 67024973 │ █████████████████████████ │ 3190864 │ ███████████████▉ │ 62324 │ ███████████████▌ │ +│ 2016-10-01 │ 71826553 │ █████████████████████████ │ 3284340 │ ████████████████▍ │ 62549 │ ███████████████▋ │ +│ 2016-11-01 │ 71022319 │ █████████████████████████ │ 3300822 │ ████████████████▌ │ 69718 │ █████████████████▍ │ +│ 2016-12-01 │ 72942967 │ █████████████████████████ │ 3430324 │ █████████████████▏ │ 71705 │ █████████████████▉ │ +│ 2017-01-01 │ 78946585 │ █████████████████████████ │ 3572093 │ █████████████████▊ │ 78198 │ ███████████████████▌ │ +│ 2017-02-01 │ 70609487 │ █████████████████████████ │ 3421115 │ █████████████████ │ 69823 │ █████████████████▍ │ +│ 2017-03-01 │ 79723106 │ █████████████████████████ │ 3638122 │ ██████████████████▏ │ 73865 │ ██████████████████▍ │ +│ 2017-04-01 │ 77478009 │ █████████████████████████ │ 3620591 │ ██████████████████ │ 74387 │ ██████████████████▌ │ +│ 2017-05-01 │ 79810360 │ █████████████████████████ │ 3650820 │ ██████████████████▎ │ 74356 │ ██████████████████▌ │ +│ 2017-06-01 │ 79901711 │ █████████████████████████ │ 3737614 │ ██████████████████▋ │ 72114 │ ██████████████████ │ +│ 2017-07-01 │ 81798725 │ █████████████████████████ │ 3872330 │ ███████████████████▎ │ 76052 │ ███████████████████ │ +│ 2017-08-01 │ 84658503 │ █████████████████████████ │ 3960093 │ ███████████████████▊ │ 77798 │ ███████████████████▍ │ +│ 2017-09-01 │ 83165192 │ █████████████████████████ │ 3880501 │ ███████████████████▍ │ 78402 │ ███████████████████▌ │ +│ 2017-10-01 │ 85828912 │ █████████████████████████ │ 3980335 │ ███████████████████▉ │ 80685 │ ████████████████████▏ │ +│ 2017-11-01 │ 84965681 │ █████████████████████████ │ 4026749 │ ████████████████████▏ │ 82659 │ ████████████████████▋ │ +│ 2017-12-01 │ 85973810 │ █████████████████████████ │ 4196354 │ ████████████████████▉ │ 91984 │ ██████████████████████▉ │ +│ 2018-01-01 │ 91558594 │ █████████████████████████ │ 4364443 │ █████████████████████▊ │ 102577 │ █████████████████████████ │ +│ 2018-02-01 │ 86467179 │ █████████████████████████ │ 4277899 │ █████████████████████▍ │ 104610 │ █████████████████████████ │ +│ 2018-03-01 │ 96490262 │ █████████████████████████ │ 4422470 │ ██████████████████████ │ 112559 │ █████████████████████████ │ +│ 2018-04-01 │ 98101232 │ █████████████████████████ │ 4572434 │ ██████████████████████▊ │ 105284 │ █████████████████████████ │ +│ 2018-05-01 │ 100109100 │ █████████████████████████ │ 4698908 │ ███████████████████████▍ │ 103910 │ █████████████████████████ │ +│ 2018-06-01 │ 100009462 │ █████████████████████████ │ 4697426 │ ███████████████████████▍ │ 101107 │ █████████████████████████ │ +│ 2018-07-01 │ 108151359 │ █████████████████████████ │ 5099492 │ █████████████████████████ │ 106184 │ █████████████████████████ │ +│ 2018-08-01 │ 107330940 │ █████████████████████████ │ 5084082 │ █████████████████████████ │ 109985 │ █████████████████████████ │ +│ 2018-09-01 │ 104473929 │ █████████████████████████ │ 5011953 │ █████████████████████████ │ 109710 │ █████████████████████████ │ +│ 2018-10-01 │ 112346556 │ █████████████████████████ │ 5320405 │ █████████████████████████ │ 112533 │ █████████████████████████ │ +│ 2018-11-01 │ 112573001 │ █████████████████████████ │ 5353282 │ █████████████████████████ │ 112211 │ █████████████████████████ │ +│ 2018-12-01 │ 121953600 │ █████████████████████████ │ 5611543 │ █████████████████████████ │ 118291 │ █████████████████████████ │ +│ 2019-01-01 │ 129386587 │ █████████████████████████ │ 6016687 │ █████████████████████████ │ 125725 │ █████████████████████████ │ +│ 2019-02-01 │ 120645639 │ █████████████████████████ │ 5974488 │ █████████████████████████ │ 125420 │ █████████████████████████ │ +│ 2019-03-01 │ 137650471 │ █████████████████████████ │ 6410197 │ █████████████████████████ │ 135924 │ █████████████████████████ │ +│ 2019-04-01 │ 138473643 │ █████████████████████████ │ 6416384 │ █████████████████████████ │ 139844 │ █████████████████████████ │ +│ 2019-05-01 │ 142463421 │ █████████████████████████ │ 6574836 │ █████████████████████████ │ 142012 │ █████████████████████████ │ +│ 2019-06-01 │ 134172939 │ █████████████████████████ │ 6601267 │ █████████████████████████ │ 140997 │ █████████████████████████ │ +│ 2019-07-01 │ 145965083 │ █████████████████████████ │ 6901822 │ █████████████████████████ │ 147802 │ █████████████████████████ │ +│ 2019-08-01 │ 146854393 │ █████████████████████████ │ 6993882 │ █████████████████████████ │ 151888 │ █████████████████████████ │ +│ 2019-09-01 │ 137540219 │ █████████████████████████ │ 7001362 │ █████████████████████████ │ 148839 │ █████████████████████████ │ +│ 2019-10-01 │ 145909884 │ █████████████████████████ │ 7160126 │ █████████████████████████ │ 152075 │ █████████████████████████ │ +│ 2019-11-01 │ 138512489 │ █████████████████████████ │ 7098723 │ █████████████████████████ │ 164597 │ █████████████████████████ │ +│ 2019-12-01 │ 146012313 │ █████████████████████████ │ 7438261 │ █████████████████████████ │ 166966 │ █████████████████████████ │ +│ 2020-01-01 │ 153498208 │ █████████████████████████ │ 7703548 │ █████████████████████████ │ 174390 │ █████████████████████████ │ +│ 2020-02-01 │ 148386817 │ █████████████████████████ │ 7582031 │ █████████████████████████ │ 170257 │ █████████████████████████ │ +│ 2020-03-01 │ 166266315 │ █████████████████████████ │ 8339049 │ █████████████████████████ │ 192460 │ █████████████████████████ │ +│ 2020-04-01 │ 178511581 │ █████████████████████████ │ 8991649 │ █████████████████████████ │ 202334 │ █████████████████████████ │ +│ 2020-05-01 │ 189993779 │ █████████████████████████ │ 9331358 │ █████████████████████████ │ 217357 │ █████████████████████████ │ +│ 2020-06-01 │ 187914434 │ █████████████████████████ │ 9085003 │ █████████████████████████ │ 223362 │ █████████████████████████ │ +│ 2020-07-01 │ 194244994 │ █████████████████████████ │ 9321706 │ █████████████████████████ │ 228222 │ █████████████████████████ │ +│ 2020-08-01 │ 196099301 │ █████████████████████████ │ 9368408 │ █████████████████████████ │ 230251 │ █████████████████████████ │ +│ 2020-09-01 │ 182549761 │ █████████████████████████ │ 9271571 │ █████████████████████████ │ 227889 │ █████████████████████████ │ +│ 2020-10-01 │ 186583890 │ █████████████████████████ │ 9396112 │ █████████████████████████ │ 233715 │ █████████████████████████ │ +│ 2020-11-01 │ 186083723 │ █████████████████████████ │ 9623053 │ █████████████████████████ │ 234963 │ █████████████████████████ │ +│ 2020-12-01 │ 191317162 │ █████████████████████████ │ 9898168 │ █████████████████████████ │ 249115 │ █████████████████████████ │ +│ 2021-01-01 │ 210496207 │ █████████████████████████ │ 10503943 │ █████████████████████████ │ 259805 │ █████████████████████████ │ +│ 2021-02-01 │ 193510365 │ █████████████████████████ │ 10215033 │ █████████████████████████ │ 253656 │ █████████████████████████ │ +│ 2021-03-01 │ 207454415 │ █████████████████████████ │ 10365629 │ █████████████████████████ │ 267263 │ █████████████████████████ │ +│ 2021-04-01 │ 204573086 │ █████████████████████████ │ 10391984 │ █████████████████████████ │ 270543 │ █████████████████████████ │ +│ 2021-05-01 │ 217655366 │ █████████████████████████ │ 10648130 │ █████████████████████████ │ 288555 │ █████████████████████████ │ +│ 2021-06-01 │ 208027069 │ █████████████████████████ │ 10397311 │ █████████████████████████ │ 291520 │ █████████████████████████ │ +│ 2021-07-01 │ 210955954 │ █████████████████████████ │ 10063967 │ █████████████████████████ │ 252061 │ █████████████████████████ │ +│ 2021-08-01 │ 225681244 │ █████████████████████████ │ 10383556 │ █████████████████████████ │ 254569 │ █████████████████████████ │ +│ 2021-09-01 │ 220086513 │ █████████████████████████ │ 10298344 │ █████████████████████████ │ 256826 │ █████████████████████████ │ +│ 2021-10-01 │ 227527379 │ █████████████████████████ │ 10729882 │ █████████████████████████ │ 283328 │ █████████████████████████ │ +│ 2021-11-01 │ 228289963 │ █████████████████████████ │ 10995197 │ █████████████████████████ │ 302386 │ █████████████████████████ │ +│ 2021-12-01 │ 235807471 │ █████████████████████████ │ 11312798 │ █████████████████████████ │ 313876 │ █████████████████████████ │ +│ 2022-01-01 │ 256766679 │ █████████████████████████ │ 12074520 │ █████████████████████████ │ 340407 │ █████████████████████████ │ +│ 2022-02-01 │ 219927645 │ █████████████████████████ │ 10846045 │ █████████████████████████ │ 293236 │ █████████████████████████ │ +│ 2022-03-01 │ 236554668 │ █████████████████████████ │ 11330285 │ █████████████████████████ │ 302387 │ █████████████████████████ │ +│ 2022-04-01 │ 231188077 │ █████████████████████████ │ 11697995 │ █████████████████████████ │ 316303 │ █████████████████████████ │ +│ 2022-05-01 │ 230492108 │ █████████████████████████ │ 11448584 │ █████████████████████████ │ 323725 │ █████████████████████████ │ +│ 2022-06-01 │ 218842949 │ █████████████████████████ │ 11400399 │ █████████████████████████ │ 324846 │ █████████████████████████ │ +│ 2022-07-01 │ 242504279 │ █████████████████████████ │ 12049204 │ █████████████████████████ │ 335621 │ █████████████████████████ │ +│ 2022-08-01 │ 247215325 │ █████████████████████████ │ 12189276 │ █████████████████████████ │ 337873 │ █████████████████████████ │ +│ 2022-09-01 │ 234131223 │ █████████████████████████ │ 11674079 │ █████████████████████████ │ 326325 │ █████████████████████████ │ +│ 2022-10-01 │ 237365072 │ █████████████████████████ │ 11804508 │ █████████████████████████ │ 336063 │ █████████████████████████ │ +│ 2022-11-01 │ 229478878 │ █████████████████████████ │ 11543020 │ █████████████████████████ │ 323122 │ █████████████████████████ │ +│ 2022-12-01 │ 238862690 │ █████████████████████████ │ 11967451 │ █████████████████████████ │ 331668 │ █████████████████████████ │ +│ 2023-01-01 │ 253577512 │ █████████████████████████ │ 12264087 │ █████████████████████████ │ 332711 │ █████████████████████████ │ +│ 2023-02-01 │ 221285501 │ █████████████████████████ │ 11537091 │ █████████████████████████ │ 317879 │ █████████████████████████ │ +└──────────────┴───────────┴───────────────────────────┴──────────┴───────────────────────────┴────────────┴───────────────────────────┘ -172 rows in set. Elapsed: 184.809 sec. Processed 6.74 billion rows, 89.56 GB (36.47 million rows/s., 484.62 MB/s.) +203 rows in set. Elapsed: 48.492 sec. Processed 14.69 billion rows, 213.35 GB (302.91 million rows/s., 4.40 GB/s.) ``` 10. Here are the top 10 subreddits of 2022: @@ -451,23 +525,21 @@ ORDER BY count DESC LIMIT 10; ``` -The response is: - ```response -┌─subreddit────────┬───count─┐ -│ AskReddit │ 3858203 │ -│ politics │ 1356782 │ -│ memes │ 1249120 │ -│ nfl │ 883667 │ -│ worldnews │ 866065 │ -│ teenagers │ 777095 │ -│ AmItheAsshole │ 752720 │ -│ dankmemes │ 657932 │ -│ nba │ 514184 │ -│ unpopularopinion │ 473649 │ -└──────────────────┴─────────┘ +┌─subreddit──────┬────count─┐ +│ AskReddit │ 72312060 │ +│ AmItheAsshole │ 25323210 │ +│ teenagers │ 22355960 │ +│ worldnews │ 17797707 │ +│ FreeKarma4U │ 15652274 │ +│ FreeKarma4You │ 14929055 │ +│ wallstreetbets │ 14235271 │ +│ politics │ 12511136 │ +│ memes │ 11610792 │ +│ nba │ 11586571 │ +└────────────────┴──────────┘ -10 rows in set. Elapsed: 27.824 sec. Processed 6.74 billion rows, 53.26 GB (242.22 million rows/s., 1.91 GB/s.) +10 rows in set. Elapsed: 5.956 sec. Processed 14.69 billion rows, 126.19 GB (2.47 billion rows/s., 21.19 GB/s.) ``` 11. Let's see which subreddits had the biggest increase in commnents from 2018 to 2019: @@ -503,62 +575,62 @@ It looks like memes and teenagers were busy on Reddit in 2019: ```response ┌─subreddit────────────┬─────diff─┐ -│ memes │ 15368369 │ -│ AskReddit │ 14663662 │ -│ teenagers │ 12266991 │ -│ AmItheAsshole │ 11561538 │ -│ dankmemes │ 11305158 │ -│ unpopularopinion │ 6332772 │ -│ PewdiepieSubmissions │ 5930818 │ -│ Market76 │ 5014668 │ -│ relationship_advice │ 3776383 │ -│ freefolk │ 3169236 │ -│ Minecraft │ 3160241 │ -│ classicwow │ 2907056 │ -│ Animemes │ 2673398 │ -│ gameofthrones │ 2402835 │ -│ PublicFreakout │ 2267605 │ -│ ShitPostCrusaders │ 2207266 │ -│ RoastMe │ 2195715 │ -│ gonewild │ 2148649 │ -│ AnthemTheGame │ 1803818 │ -│ entitledparents │ 1706270 │ -│ MortalKombat │ 1679508 │ -│ Cringetopia │ 1620555 │ -│ pokemon │ 1615266 │ -│ HistoryMemes │ 1608289 │ -│ Brawlstars │ 1574977 │ -│ iamatotalpieceofshit │ 1558315 │ -│ trashy │ 1518549 │ -│ ChapoTrapHouse │ 1505748 │ -│ Pikabu │ 1501001 │ -│ Showerthoughts │ 1475101 │ -│ cursedcomments │ 1465607 │ -│ ukpolitics │ 1386043 │ -│ wallstreetbets │ 1384431 │ -│ interestingasfuck │ 1378900 │ -│ wholesomememes │ 1353333 │ -│ AskOuija │ 1233263 │ -│ borderlands3 │ 1197192 │ -│ aww │ 1168257 │ -│ insanepeoplefacebook │ 1155473 │ -│ FortniteCompetitive │ 1122778 │ -│ EpicSeven │ 1117380 │ -│ FreeKarma4U │ 1116423 │ -│ YangForPresidentHQ │ 1086700 │ -│ SquaredCircle │ 1044089 │ -│ MurderedByWords │ 1042511 │ -│ AskMen │ 1024434 │ -│ thedivision │ 1016634 │ -│ barstoolsports │ 985032 │ -│ nfl │ 978340 │ -│ BattlefieldV │ 971408 │ +│ AskReddit │ 18765909 │ +│ memes │ 16496996 │ +│ teenagers │ 13071715 │ +│ AmItheAsshole │ 12312663 │ +│ dankmemes │ 12016716 │ +│ unpopularopinion │ 6809935 │ +│ PewdiepieSubmissions │ 6330844 │ +│ Market76 │ 5213690 │ +│ relationship_advice │ 4060717 │ +│ Minecraft │ 3328659 │ +│ freefolk │ 3227970 │ +│ classicwow │ 3063133 │ +│ Animemes │ 2866876 │ +│ gonewild │ 2457680 │ +│ PublicFreakout │ 2452288 │ +│ gameofthrones │ 2411661 │ +│ RoastMe │ 2378781 │ +│ ShitPostCrusaders │ 2345414 │ +│ AnthemTheGame │ 1813152 │ +│ nfl │ 1804407 │ +│ Showerthoughts │ 1797968 │ +│ Cringetopia │ 1764034 │ +│ pokemon │ 1763269 │ +│ entitledparents │ 1744852 │ +│ HistoryMemes │ 1721645 │ +│ MortalKombat │ 1718184 │ +│ trashy │ 1684357 │ +│ ChapoTrapHouse │ 1675363 │ +│ Brawlstars │ 1663763 │ +│ iamatotalpieceofshit │ 1647381 │ +│ ukpolitics │ 1599204 │ +│ cursedcomments │ 1590781 │ +│ Pikabu │ 1578597 │ +│ wallstreetbets │ 1535225 │ +│ AskOuija │ 1533214 │ +│ interestingasfuck │ 1528910 │ +│ aww │ 1439008 │ +│ wholesomememes │ 1436566 │ +│ SquaredCircle │ 1432172 │ +│ insanepeoplefacebook │ 1290686 │ +│ borderlands3 │ 1274462 │ +│ FreeKarma4U │ 1217769 │ +│ YangForPresidentHQ │ 1186918 │ +│ FortniteCompetitive │ 1184508 │ +│ AskMen │ 1180820 │ +│ EpicSeven │ 1172061 │ +│ MurderedByWords │ 1112476 │ +│ politics │ 1084087 │ +│ barstoolsports │ 1068020 │ +│ BattlefieldV │ 1053878 │ └──────────────────────┴──────────┘ -50 rows in set. Elapsed: 65.954 sec. Processed 13.48 billion rows, 79.67 GB (204.37 million rows/s., 1.21 GB/s.) +50 rows in set. Elapsed: 10.680 sec. Processed 29.38 billion rows, 198.67 GB (2.75 billion rows/s., 18.60 GB/s.) ``` -12. One more query: let's compare ClickHouse mentions to other technologies like Snowflake and Postgres. This query is a big one because it has to search all the comments three times for a substring, and unfortunately ClickHouse user are obviously not very active on Reddit yet: +12. One more query: let's compare ClickHouse mentions to other technologies like Snowflake and Postgres. This query is a big one because it has to search all 14.69 billion comments three times for a substring, but the performance is actually quite impressive. (Unfortunately ClickHouse users are not very active on Reddit yet): ```sql SELECT @@ -572,7 +644,7 @@ ORDER BY quarter ASC; ``` ```response -┌────Quarter─┬─clickhouse─┬─snowflake─┬─postgres─┐ +┌────quarter─┬─clickhouse─┬─snowflake─┬─postgres─┐ │ 2005-10-01 │ 0 │ 0 │ 0 │ │ 2006-01-01 │ 0 │ 2 │ 23 │ │ 2006-04-01 │ 0 │ 2 │ 24 │ @@ -592,12 +664,12 @@ ORDER BY quarter ASC; │ 2009-10-01 │ 0 │ 633 │ 589 │ │ 2010-01-01 │ 0 │ 555 │ 501 │ │ 2010-04-01 │ 0 │ 587 │ 469 │ -│ 2010-07-01 │ 0 │ 770 │ 821 │ -│ 2010-10-01 │ 0 │ 1480 │ 550 │ -│ 2011-01-01 │ 0 │ 1482 │ 568 │ -│ 2011-04-01 │ 0 │ 1558 │ 406 │ -│ 2011-07-01 │ 0 │ 2163 │ 628 │ -│ 2011-10-01 │ 0 │ 4064 │ 566 │ +│ 2010-07-01 │ 0 │ 601 │ 696 │ +│ 2010-10-01 │ 0 │ 1246 │ 505 │ +│ 2011-01-01 │ 0 │ 758 │ 247 │ +│ 2011-04-01 │ 0 │ 537 │ 113 │ +│ 2011-07-01 │ 0 │ 173 │ 64 │ +│ 2011-10-01 │ 0 │ 649 │ 96 │ │ 2012-01-01 │ 0 │ 4621 │ 662 │ │ 2012-04-01 │ 0 │ 5737 │ 785 │ │ 2012-07-01 │ 0 │ 6097 │ 1127 │ @@ -629,9 +701,21 @@ ORDER BY quarter ASC; │ 2019-01-01 │ 14 │ 80250 │ 4305 │ │ 2019-04-01 │ 30 │ 70307 │ 3872 │ │ 2019-07-01 │ 33 │ 77149 │ 4164 │ -│ 2019-10-01 │ 13 │ 76746 │ 3541 │ -│ 2020-01-01 │ 16 │ 54475 │ 846 │ +│ 2019-10-01 │ 22 │ 113011 │ 4369 │ +│ 2020-01-01 │ 34 │ 238273 │ 5133 │ +│ 2020-04-01 │ 52 │ 454467 │ 6100 │ +│ 2020-07-01 │ 37 │ 406623 │ 5507 │ +│ 2020-10-01 │ 49 │ 212143 │ 5385 │ +│ 2021-01-01 │ 56 │ 151262 │ 5749 │ +│ 2021-04-01 │ 71 │ 119928 │ 6039 │ +│ 2021-07-01 │ 53 │ 110342 │ 5765 │ +│ 2021-10-01 │ 92 │ 121144 │ 6401 │ +│ 2022-01-01 │ 93 │ 107512 │ 6772 │ +│ 2022-04-01 │ 120 │ 91560 │ 6687 │ +│ 2022-07-01 │ 183 │ 99764 │ 7377 │ +│ 2022-10-01 │ 123 │ 99447 │ 7052 │ +│ 2023-01-01 │ 126 │ 58733 │ 4891 │ └────────────┴────────────┴───────────┴──────────┘ -58 rows in set. Elapsed: 2663.751 sec. Processed 6.74 billion rows, 1.21 TB (2.53 million rows/s., 454.37 MB/s.) +70 rows in set. Elapsed: 325.835 sec. Processed 14.69 billion rows, 2.57 TB (45.08 million rows/s., 7.87 GB/s.) ``` \ No newline at end of file From 6a5ae6558a1340d2d42f350879ccc88498dbb78e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Jun 2023 12:58:03 +0200 Subject: [PATCH 211/308] Add comment for usage of prepare-ci-ami.sh --- tests/ci/worker/prepare-ci-ami.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index 57ab149237a..20e7e3fd53e 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -1,6 +1,10 @@ #!/usr/bin/env bash # The script is downloaded the AWS image builder Task Orchestrator and Executor (AWSTOE) # We can't use `user data script` because cloud-init does not check the exit code +# The script is downloaded in the component named ci-infrastructure-prepare in us-east-1 +# The link there must be adjusted to a particular RAW link, e.g. +# https://github.com/ClickHouse/ClickHouse/raw/653da5f00219c088af66d97a8f1ea3e35e798268/tests/ci/worker/prepare-ci-ami.sh + set -xeuo pipefail echo "Running prepare script" From 9a2b3afb7e35382eb7f5755cdf8e778c75bf8f88 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Jun 2023 15:02:32 +0300 Subject: [PATCH 212/308] Revert "Fix msan issue in keyed siphash" --- docs/en/sql-reference/functions/hash-functions.md | 8 ++++---- src/Functions/FunctionsHashing.h | 13 ++----------- .../0_stateless/02534_keyed_siphash.reference | 4 ---- tests/queries/0_stateless/02534_keyed_siphash.sql | 6 ------ 4 files changed, 6 insertions(+), 25 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 89afcca3799..663b8468a94 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -51,7 +51,7 @@ Calculates the MD5 from a string and returns the resulting set of bytes as Fixed If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead. If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))). -## sipHash64 {#hash_functions-siphash64} +## sipHash64 (#hash_functions-siphash64) Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value. @@ -63,9 +63,9 @@ This is a cryptographic hash function. It works at least three times faster than The function [interprets](/docs/en/sql-reference/functions/type-conversion-functions.md/#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. It then combines the hashes by the following algorithm: -1. The first and the second hash value are concatenated to an array which is hashed. -2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. -3. This calculation is repeated for all remaining hash values of the original input. +1. The first and the second hash value are concatenated to an array which is hashed. +2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. +3. This calculation is repeated for all remaining hash values of the original input. **Arguments** diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 14fad62bd85..3de757bfa3f 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -91,18 +91,12 @@ namespace impl throw Exception(ErrorCodes::NOT_IMPLEMENTED, "wrong tuple size: key must be a tuple of 2 UInt64"); if (const auto * key0col = checkAndGetColumn(&(tuple->getColumn(0)))) - { - const auto & key0col_data = key0col->getData(); - ret.key0 = key0col_data[0]; - } + ret.key0 = key0col->get64(0); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); if (const auto * key1col = checkAndGetColumn(&(tuple->getColumn(1)))) - { - const auto & key1col_data = key1col->getData(); - ret.key1 = key1col_data[0]; - } + ret.key1 = key1col->get64(0); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); @@ -1426,9 +1420,6 @@ public: { auto col_to = ColumnVector::create(input_rows_count); - if (input_rows_count == 0) - return col_to; - typename ColumnVector::Container & vec_to = col_to->getData(); /// If using a "keyed" algorithm, the first argument is the key and diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index a891d01e99d..3606b9a41db 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -194,7 +194,3 @@ E28DBDE7FE22E41C 1 E28DBDE7FE22E41C 1 -1CE422FEE7BD8DE20000000000000000 -7766709361750702608 -20AF99D3A87829E0 -12489502208762728797 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 7ce50583cdb..9c914f586f0 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -272,9 +272,3 @@ select hex(sipHash64()); SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; select hex(sipHash64Keyed()); SELECT hex(sipHash128Keyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; - --- Crashed with memory sanitizer -SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); -SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); -SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646)); -SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806); From 68d46c81b99726a1e0467fc21d02a5311b1e49ca Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 11 Apr 2023 09:53:48 +0000 Subject: [PATCH 213/308] Do not store blocks in hash join if nothing is inseted --- src/Interpreters/HashJoin.cpp | 55 +++++++++++++++++++++++------------ 1 file changed, 37 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 146b57049a6..92597f87f4b 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -543,16 +543,20 @@ namespace template struct Inserter { - static ALWAYS_INLINE void insertOne(const HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, + static ALWAYS_INLINE bool insertOne(const HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); if (emplace_result.isInserted() || join.anyTakeLastRow()) + { new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i); + return true; + } + return false; } - static ALWAYS_INLINE void insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) + static ALWAYS_INLINE bool insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -563,9 +567,10 @@ namespace /// The first element of the list is stored in the value of the hash table, the rest in the pool. emplace_result.getMapped().insert({stored_block, i}, pool); } + return true; } - static ALWAYS_INLINE void insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, + static ALWAYS_INLINE bool insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, const IColumn & asof_column) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -575,6 +580,7 @@ namespace if (emplace_result.isInserted()) time_series_map = new (time_series_map) typename Map::mapped_type(createAsofRowRef(asof_type, join.getAsofInequality())); (*time_series_map)->insert(asof_column, stored_block, i); + return true; } }; @@ -582,7 +588,7 @@ namespace template size_t NO_INLINE insertFromBlockImplTypeCase( HashJoin & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool) + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted) { [[maybe_unused]] constexpr bool mapped_one = std::is_same_v; constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; @@ -593,6 +599,7 @@ namespace auto key_getter = createKeyGetter(key_columns, key_sizes); + is_inserted = false; for (size_t i = 0; i < rows; ++i) { if (has_null_map && (*null_map)[i]) @@ -603,11 +610,11 @@ namespace continue; if constexpr (is_asof_join) - Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); + is_inserted |= Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); else if constexpr (mapped_one) - Inserter::insertOne(join, map, key_getter, stored_block, i, pool); + is_inserted |= Inserter::insertOne(join, map, key_getter, stored_block, i, pool); else - Inserter::insertAll(join, map, key_getter, stored_block, i, pool); + is_inserted |= Inserter::insertAll(join, map, key_getter, stored_block, i, pool); } return map.getBufferSizeInCells(); } @@ -616,32 +623,37 @@ namespace template size_t insertFromBlockImplType( HashJoin & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool) + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted) { if (null_map) return insertFromBlockImplTypeCase( - join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool); + join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); else return insertFromBlockImplTypeCase( - join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool); + join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); } template size_t insertFromBlockImpl( HashJoin & join, HashJoin::Type type, Maps & maps, size_t rows, const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool) + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted) { switch (type) { - case HashJoin::Type::EMPTY: return 0; - case HashJoin::Type::CROSS: return 0; /// Do nothing. We have already saved block, and it is enough. + case HashJoin::Type::EMPTY: + [[fallthrough]]; + case HashJoin::Type::CROSS: + /// Do nothing. We will only save block, and it is enough + is_inserted = true; + return 0; #define M(TYPE) \ case HashJoin::Type::TYPE: \ return insertFromBlockImplType>::Type>(\ - join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool); \ + join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \ break; + APPLY_FOR_JOIN_VARIANTS(M) #undef M } @@ -816,6 +828,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block_, bool check_limits) } } + bool is_inserted = false; if (kind != JoinKind::Cross) { joinDispatch(kind, strictness, data->maps[onexpr_idx], [&](auto kind_, auto strictness_, auto & map) @@ -824,28 +837,34 @@ bool HashJoin::addJoinedBlock(const Block & source_block_, bool check_limits) *this, data->type, map, rows, key_columns, key_sizes[onexpr_idx], stored_block, null_map, /// If mask is false constant, rows are added to hashmap anyway. It's not a happy-flow, so this case is not optimized join_mask_col.getData(), - data->pool); + data->pool, is_inserted); if (multiple_disjuncts) used_flags.reinit(stored_block); - else + else if (is_inserted) /// Number of buckets + 1 value from zero storage used_flags.reinit(size + 1); }); } - if (!multiple_disjuncts && save_nullmap) + if (!multiple_disjuncts && save_nullmap && is_inserted) { data->blocks_nullmaps_allocated_size += null_map_holder->allocatedBytes(); data->blocks_nullmaps.emplace_back(stored_block, null_map_holder); } - if (!multiple_disjuncts && not_joined_map) + if (!multiple_disjuncts && not_joined_map && is_inserted) { data->blocks_nullmaps_allocated_size += not_joined_map->allocatedBytes(); data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); } + if (!multiple_disjuncts && !is_inserted) + { + data->blocks_allocated_size -= stored_block->allocatedBytes(); + data->blocks.pop_back(); + } + if (!check_limits) return true; From 62f950ddaff7178fc479e2ccab236be39567e0a7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 May 2023 12:15:28 +0000 Subject: [PATCH 214/308] Keep blocks with nulls for right and full join --- src/Interpreters/HashJoin.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 92597f87f4b..436ecd382cd 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -603,7 +603,12 @@ namespace for (size_t i = 0; i < rows; ++i) { if (has_null_map && (*null_map)[i]) + { + /// nulls are not inserted into hash table, + /// keep them for RIGHT and FULL joins + is_inserted = true; continue; + } /// Check condition for right table from ON section if (join_mask && !(*join_mask)[i]) @@ -861,6 +866,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block_, bool check_limits) if (!multiple_disjuncts && !is_inserted) { + LOG_TRACE(log, "Skipping inserting block with {} rows", rows); data->blocks_allocated_size -= stored_block->allocatedBytes(); data->blocks.pop_back(); } From 0f0958f82d5048c0aa3b28bdf9c5ce655e853219 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 4 May 2023 14:08:03 +0000 Subject: [PATCH 215/308] Add test 02725_any_join_single_row --- .../02725_any_join_single_row.reference | 3 +++ .../0_stateless/02725_any_join_single_row.sql | 26 +++++++++++++++++++ 2 files changed, 29 insertions(+) create mode 100644 tests/queries/0_stateless/02725_any_join_single_row.reference create mode 100644 tests/queries/0_stateless/02725_any_join_single_row.sql diff --git a/tests/queries/0_stateless/02725_any_join_single_row.reference b/tests/queries/0_stateless/02725_any_join_single_row.reference new file mode 100644 index 00000000000..5d748fc6dbb --- /dev/null +++ b/tests/queries/0_stateless/02725_any_join_single_row.reference @@ -0,0 +1,3 @@ +Join(ANY, LEFT, key) 0 1 +Join(ANY, LEFT, key) 1 1 +Join(ANY, LEFT, key) 1 1 diff --git a/tests/queries/0_stateless/02725_any_join_single_row.sql b/tests/queries/0_stateless/02725_any_join_single_row.sql new file mode 100644 index 00000000000..5e5c959c278 --- /dev/null +++ b/tests/queries/0_stateless/02725_any_join_single_row.sql @@ -0,0 +1,26 @@ +CREATE TABLE join_test +( + `key` UInt64, + `value` UInt64 +) +ENGINE = Join(ANY, LEFT, key); + +CREATE TEMPORARY TABLE initial_table_size AS + SELECT engine_full, total_rows, total_bytes FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); + +SELECT engine_full, total_rows, total_bytes < 100_000 FROM initial_table_size; + +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); + + +CREATE TEMPORARY TABLE one_row_table_size AS + SELECT engine_full, total_rows, total_bytes FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); + +SELECT engine_full, total_rows, total_bytes < 2 * (SELECT total_bytes FROM initial_table_size) FROM one_row_table_size; + +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(10_000); + +SELECT engine_full, total_rows, total_bytes == (SELECT total_bytes FROM one_row_table_size) FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); From 56fa98cb77c30cba3f93dfcd5b04c33dd5b1ec68 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 09:23:39 +0000 Subject: [PATCH 216/308] Rename variables for better readability --- src/Functions/FunctionsHashing.h | 71 ++++++++++++++++---------------- 1 file changed, 36 insertions(+), 35 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 3de757bfa3f..70adc7eba0f 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1073,42 +1073,43 @@ private: size_t size = vec_from.size(); for (size_t i = 0; i < size; ++i) { - ToType h; + ToType hash; if constexpr (Impl::use_int_hash_for_pods) { if constexpr (std::is_same_v) - h = IntHash64Impl::apply(bit_cast(vec_from[i])); + hash = IntHash64Impl::apply(bit_cast(vec_from[i])); else - h = IntHash32Impl::apply(bit_cast(vec_from[i])); + hash = IntHash32Impl::apply(bit_cast(vec_from[i])); } else { if constexpr (std::is_same_v) - h = JavaHashImpl::apply(vec_from[i]); + hash = JavaHashImpl::apply(vec_from[i]); else { - FromType v = vec_from[i]; + FromType value = vec_from[i]; if constexpr (std::endian::native == std::endian::big) { - FromType tmp_v; - reverseMemcpy(&tmp_v, &v, sizeof(v)); - v = tmp_v; + FromType value_reversed; + reverseMemcpy(&value_reversed, &value, sizeof(value)); + value = value_reversed; } - h = apply(key, reinterpret_cast(&v), sizeof(v)); + hash = apply(key, reinterpret_cast(&value), sizeof(value)); } } if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } else if (auto col_from_const = checkAndGetColumnConst(column)) { auto value = col_from_const->template getValue(); ToType hash; + if constexpr (std::is_same_v) hash = IntHash64Impl::apply(bit_cast(value)); else @@ -1139,45 +1140,45 @@ private: size_t size = vec_from.size(); for (size_t i = 0; i < size; ++i) { - ToType h; + ToType hash; if constexpr (std::endian::native == std::endian::little) { - h = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); + hash = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); } else { char tmp_buffer[sizeof(vec_from[i])]; reverseMemcpy(tmp_buffer, &vec_from[i], sizeof(vec_from[i])); - h = apply(key, reinterpret_cast(tmp_buffer), sizeof(vec_from[i])); + hash = apply(key, reinterpret_cast(tmp_buffer), sizeof(vec_from[i])); } if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } else if (auto col_from_const = checkAndGetColumnConst(column)) { auto value = col_from_const->template getValue(); - ToType h; + ToType hash; if constexpr (std::endian::native == std::endian::little) { - h = apply(key, reinterpret_cast(&value), sizeof(value)); + hash = apply(key, reinterpret_cast(&value), sizeof(value)); } else { char tmp_buffer[sizeof(value)]; reverseMemcpy(tmp_buffer, &value, sizeof(value)); - h = apply(key, reinterpret_cast(tmp_buffer), sizeof(value)); + hash = apply(key, reinterpret_cast(tmp_buffer), sizeof(value)); } size_t size = vec_to.size(); if constexpr (first) - vec_to.assign(size, h); + vec_to.assign(size, hash); else { for (size_t i = 0; i < size; ++i) - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } else @@ -1191,11 +1192,11 @@ private: for (size_t i = 0, size = column->size(); i < size; ++i) { StringRef bytes = column->getDataAt(i); - const ToType h = apply(key, bytes.data, bytes.size); + const ToType hash = apply(key, bytes.data, bytes.size); if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } @@ -1211,14 +1212,14 @@ private: ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - const ToType h = apply(key, + const ToType hash = apply(key, reinterpret_cast(&data[current_offset]), offsets[i] - current_offset - 1); if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); current_offset = offsets[i]; } @@ -1231,11 +1232,11 @@ private: for (size_t i = 0; i < size; ++i) { - const ToType h = apply(key, reinterpret_cast(&data[i * n]), n); + const ToType hash = apply(key, reinterpret_cast(&data[i * n]), n); if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column)) @@ -1283,16 +1284,16 @@ private: { ColumnArray::Offset next_offset = offsets[i]; - ToType h; + ToType hash; if constexpr (std::is_same_v) - h = IntHash64Impl::apply(next_offset - current_offset); + hash = IntHash64Impl::apply(next_offset - current_offset); else - h = IntHash32Impl::apply(next_offset - current_offset); + hash = IntHash32Impl::apply(next_offset - current_offset); if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); for (size_t j = current_offset; j < next_offset; ++j) vec_to[i] = combineHashes(key, vec_to[i], vec_temp[j]); From 93ba75b370c6ece34fe8480440193b4775b3d105 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 12:23:04 +0000 Subject: [PATCH 217/308] Remove parentheses from single-line if/for --- src/Functions/FunctionsHashing.h | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 70adc7eba0f..9896adca7f3 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1119,10 +1119,8 @@ private: if constexpr (first) vec_to.assign(size, hash); else - { for (size_t i = 0; i < size; ++i) vec_to[i] = combineHashes(key, vec_to[i], hash); - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1142,9 +1140,7 @@ private: { ToType hash; if constexpr (std::endian::native == std::endian::little) - { hash = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); - } else { char tmp_buffer[sizeof(vec_from[i])]; @@ -1163,9 +1159,7 @@ private: ToType hash; if constexpr (std::endian::native == std::endian::little) - { hash = apply(key, reinterpret_cast(&value), sizeof(value)); - } else { char tmp_buffer[sizeof(value)]; @@ -1176,10 +1170,8 @@ private: if constexpr (first) vec_to.assign(size, hash); else - { for (size_t i = 0; i < size; ++i) vec_to[i] = combineHashes(key, vec_to[i], hash); - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1246,16 +1238,10 @@ private: const size_t size = vec_to.size(); if constexpr (first) - { vec_to.assign(size, hash); - } else - { for (size_t i = 0; i < size; ++i) - { vec_to[i] = combineHashes(key, vec_to[i], hash); - } - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", From 3f08e3e03f7f61a7e70a5ed89ed66f09f3002f52 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 1 Jun 2023 14:53:07 +0200 Subject: [PATCH 218/308] make filter push down through cross join --- .../Optimizations/filterPushDown.cpp | 6 ++--- .../01763_filter_push_down_bugs.reference | 22 +++++++++++++++++++ .../01763_filter_push_down_bugs.sql | 16 ++++++++++++++ 3 files changed, 41 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 37bc894339f..db29038999b 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -272,7 +272,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes { /// If totals step has HAVING expression, skip it for now. /// TODO: - /// We can merge HAVING expression with current filer. + /// We can merge HAVING expression with current filter. /// Also, we can push down part of HAVING which depend only on aggregation keys. if (totals_having->getActions()) return 0; @@ -323,9 +323,9 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes { const auto & table_join = join ? join->getJoin()->getTableJoin() : filled_join->getJoin()->getTableJoin(); - /// Only inner and left(/right) join are supported. Other types may generate default values for left table keys. + /// Only inner, cross and left(/right) join are supported. Other types may generate default values for left table keys. /// So, if we push down a condition like `key != 0`, not all rows may be filtered. - if (table_join.kind() != JoinKind::Inner && table_join.kind() != kind) + if (table_join.kind() != JoinKind::Inner && table_join.kind() != JoinKind::Cross && table_join.kind() != kind) return 0; bool is_left = kind == JoinKind::Left; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index 5aa2e645509..eb4e88a1f81 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -6,3 +6,25 @@ String1_0 String2_0 String3_0 String4_0 1 String1_0 String2_0 String3_0 String4_0 1 1 [0,1,2] 1 +Expression ((Projection + Before ORDER BY)) + Filter (WHERE) + Join (JOIN FillRightFirst) + Filter (( + Before JOIN)) + ReadFromMergeTree (default.t1) + Indexes: + PrimaryKey + Keys: + id + Condition: (id in 1-element set) + Parts: 0/19 + Granules: 0/1204 + Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) + Filter (WHERE) + ReadFromMergeTree (default.t2) + Indexes: + PrimaryKey + Keys: + delete_time + Condition: (delete_time in [1685397601, +Inf)) + Parts: 0/1 + Granules: 0/1 diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 1058bf75144..917c350dadb 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -38,6 +38,22 @@ DROP TABLE IF EXISTS Test; select x, y from (select [0, 1, 2] as y, 1 as a, 2 as b) array join y as x where a = 1 and b = 2 and (x = 1 or x != 1) and x = 1; +DROP TABLE IF EXISTS t; create table t(a UInt8) engine=MergeTree order by a; insert into t select * from numbers(2); select a from t t1 join t t2 on t1.a = t2.a where t1.a; +DROP TABLE IF EXISTS t; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +CREATE TABLE t1 (id Int64, create_time DateTime) ENGINE = MergeTree ORDER BY id AS SELECT number, toDateTime(number + 1600000000) from numbers(10000000) settings min_insert_block_size_rows=100000; +CREATE TABLE t2 (delete_time DateTime) ENGINE = MergeTree ORDER BY delete_time AS SELECT toDateTime(number + 1610000000) from numbers(100); + +EXPLAIN indexes=1 SELECT id, delete_time FROM t1 + CROSS JOIN ( + SELECT delete_time + FROM t2 WHERE delete_time > '2023-05-30 00:00:00' +) AS d WHERE create_time < delete_time AND id IN (10000001); + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; From 03628bde422823922d1fdd52531d34212270edae Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 12:54:34 +0000 Subject: [PATCH 219/308] Fix hashing of const integer values --- src/Functions/FunctionsHashing.h | 26 ++++++++++++++++--- .../0_stateless/02534_keyed_siphash.reference | 3 +++ .../0_stateless/02534_keyed_siphash.sql | 9 +++++++ 3 files changed, 34 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 9896adca7f3..32e3fbbd4ea 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1096,7 +1096,7 @@ private: value = value_reversed; } hash = apply(key, reinterpret_cast(&value), sizeof(value)); - } + } } if constexpr (first) @@ -1110,10 +1110,28 @@ private: auto value = col_from_const->template getValue(); ToType hash; - if constexpr (std::is_same_v) - hash = IntHash64Impl::apply(bit_cast(value)); + if constexpr (Impl::use_int_hash_for_pods) + { + if constexpr (std::is_same_v) + hash = IntHash64Impl::apply(bit_cast(value)); + else + hash = IntHash32Impl::apply(bit_cast(value)); + } else - hash = IntHash32Impl::apply(bit_cast(value)); + { + if constexpr (std::is_same_v) + hash = JavaHashImpl::apply(value); + else + { + if constexpr (std::endian::native == std::endian::big) + { + FromType value_reversed; + reverseMemcpy(&value_reversed, &value, sizeof(value)); + value = value_reversed; + } + hash = apply(key, reinterpret_cast(&value), sizeof(value)); + } + } size_t size = vec_to.size(); if constexpr (first) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 3606b9a41db..ccc514e7ea2 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -194,3 +194,6 @@ E28DBDE7FE22E41C 1 E28DBDE7FE22E41C 1 +Check bug with hashing of const integer values +11862823756610506724 +11862823756610506724 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 9c914f586f0..900b99f548a 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -272,3 +272,12 @@ select hex(sipHash64()); SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; select hex(sipHash64Keyed()); SELECT hex(sipHash128Keyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; + +SELECT 'Check bug with hashing of const integer values'; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; +INSERT INTO tab VALUES ((2, 2), 4); +-- these two statements must produce the same result +SELECT sipHash64Keyed(key, val) FROM tab; +SELECT sipHash64Keyed(key, 4::UInt64) FROM tab; +DROP TABLE tab; From 53eb360ac21c23a7de58e2e483452846619086d7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 31 May 2023 15:30:41 +0200 Subject: [PATCH 220/308] Fix merge_tree_min_rows_for_seek/merge_tree_min_bytes_for_seek for data skipping indexes Signed-off-by: Azat Khuzhin --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++-- ...ng_index_merge_tree_min_for_seek.reference | 0 ...skipping_index_merge_tree_min_for_seek.sql | 22 +++++++++++++++++++ 3 files changed, 24 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.reference create mode 100644 tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.sql diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 2b16ea43179..16b27c2c820 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1729,7 +1729,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( std::max(ranges[i].begin, index_mark * index_granularity), std::min(ranges[i].end, (index_mark + 1) * index_granularity)); - if (res.empty() || res.back().end - data_range.begin > min_marks_for_seek) + if (res.empty() || data_range.begin - res.back().end > min_marks_for_seek) res.push_back(data_range); else res.back().end = data_range.end; @@ -1829,7 +1829,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex( std::max(range.begin, index_mark * index_granularity), std::min(range.end, (index_mark + 1) * index_granularity)); - if (res.empty() || res.back().end - data_range.begin > min_marks_for_seek) + if (res.empty() || data_range.begin - res.back().end > min_marks_for_seek) res.push_back(data_range); else res.back().end = data_range.end; diff --git a/tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.reference b/tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.sql b/tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.sql new file mode 100644 index 00000000000..4cebdde3dfe --- /dev/null +++ b/tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.sql @@ -0,0 +1,22 @@ +-- Tags: no-random-merge-tree-settings, no-random-settings + +DROP TABLE IF EXISTS data; + +CREATE TABLE data +( + key Int, + v1 DateTime, + INDEX v1_index v1 TYPE minmax GRANULARITY 1 +) ENGINE=AggregatingMergeTree() +ORDER BY key +SETTINGS index_granularity=8192; + +SYSTEM STOP MERGES data; + +-- generate 50% of marks that cannot be skipped with v1_index +-- this will create a gap in marks +INSERT INTO data SELECT number, if(number/8192 % 2 == 0, now(), now() - INTERVAL 200 DAY) FROM numbers(1e6); +INSERT INTO data SELECT number+1e6, if(number/8192 % 2 == 0, now(), now() - INTERVAL 200 DAY) FROM numbers(1e6); + +SELECT * FROM data WHERE v1 >= now() - INTERVAL 180 DAY FORMAT Null SETTINGS max_threads=1, max_final_threads=1, force_data_skipping_indices='v1_index', merge_tree_min_rows_for_seek=0, max_rows_to_read=1999999; +SELECT * FROM data WHERE v1 >= now() - INTERVAL 180 DAY FORMAT Null SETTINGS max_threads=1, max_final_threads=1, force_data_skipping_indices='v1_index', merge_tree_min_rows_for_seek=1, max_rows_to_read=1999999; -- { serverError TOO_MANY_ROWS } From 9490cd44acf7bf8a1db59e8e0ed453b1eb85a872 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 13:16:33 +0000 Subject: [PATCH 221/308] Include msan fix in protobuf --- contrib/google-protobuf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/google-protobuf b/contrib/google-protobuf index 3b3d8fe1913..c47efe2d8f6 160000 --- a/contrib/google-protobuf +++ b/contrib/google-protobuf @@ -1 +1 @@ -Subproject commit 3b3d8fe191314ea903ea6b072f0e73ef18e15faa +Subproject commit c47efe2d8f6a60022b49ecd6cc23660687c8598f From e5c21e588afc6defa9b5b4bc58aaa8526a9013ce Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 1 Jun 2023 15:19:47 +0200 Subject: [PATCH 222/308] Update app.py (#50407) --- tests/ci/workflow_approve_rerun_lambda/app.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 2cdbdecca6e..32cba5d466b 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -126,6 +126,7 @@ TRUSTED_CONTRIBUTORS = { "aalexfvk", "MikhailBurdukov", "tsolodov", # ClickHouse Employee + "kitaisreal", ] } From 97e120ee3c4bd83664eb1e31117103fde6a3464c Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 1 Jun 2023 09:31:05 -0400 Subject: [PATCH 223/308] add explain video to docs --- docs/en/sql-reference/statements/explain.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/en/sql-reference/statements/explain.md b/docs/en/sql-reference/statements/explain.md index 1c93707402f..befe85ab4a4 100644 --- a/docs/en/sql-reference/statements/explain.md +++ b/docs/en/sql-reference/statements/explain.md @@ -7,6 +7,18 @@ title: "EXPLAIN Statement" Shows the execution plan of a statement. +
+ +
+ Syntax: ```sql From 238b0927a918b8a36db60a81dc322b204bb387c5 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 1 Jun 2023 16:15:42 +0200 Subject: [PATCH 224/308] make test easier --- .../01763_filter_push_down_bugs.reference | 21 ++++++++----------- .../01763_filter_push_down_bugs.sql | 11 ++++++---- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index eb4e88a1f81..7df35e2948d 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -15,16 +15,13 @@ Expression ((Projection + Before ORDER BY)) PrimaryKey Keys: id - Condition: (id in 1-element set) - Parts: 0/19 - Granules: 0/1204 + Condition: (id in [101, 101]) + Parts: 1/1 + Granules: 1/1 Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) - Filter (WHERE) - ReadFromMergeTree (default.t2) - Indexes: - PrimaryKey - Keys: - delete_time - Condition: (delete_time in [1685397601, +Inf)) - Parts: 0/1 - Granules: 0/1 + ReadFromMergeTree (default.t2) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 917c350dadb..2ee249b5ce7 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -46,14 +46,17 @@ DROP TABLE IF EXISTS t; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (id Int64, create_time DateTime) ENGINE = MergeTree ORDER BY id AS SELECT number, toDateTime(number + 1600000000) from numbers(10000000) settings min_insert_block_size_rows=100000; -CREATE TABLE t2 (delete_time DateTime) ENGINE = MergeTree ORDER BY delete_time AS SELECT toDateTime(number + 1610000000) from numbers(100); +CREATE TABLE t1 (id Int64, create_time DateTime) ENGINE = MergeTree ORDER BY id; +CREATE TABLE t2 (delete_time DateTime) ENGINE = MergeTree ORDER BY delete_time; + +insert into t1 values (101, '2023-05-28 00:00:00'), (102, '2023-05-28 00:00:00'); +insert into t2 values ('2023-05-31 00:00:00'); EXPLAIN indexes=1 SELECT id, delete_time FROM t1 CROSS JOIN ( SELECT delete_time - FROM t2 WHERE delete_time > '2023-05-30 00:00:00' -) AS d WHERE create_time < delete_time AND id IN (10000001); + FROM t2 +) AS d WHERE create_time < delete_time AND id = 101; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; From a75598ea655e0340ede8a6d46368147378a4072e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 1 Jun 2023 16:16:39 +0200 Subject: [PATCH 225/308] fix test --- .../test_memory_bound_aggregation.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py index 94c788f8f91..d76c4eba409 100644 --- a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py @@ -74,7 +74,7 @@ def test_backward_compatability(start_cluster): from remote('node{1,2,3}', default, t) group by a limit 1 offset 12345 - settings optimize_aggregation_in_order = 1 + settings optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 0 """ ) == "30\n" From d9113a3b757841b9b956b1d38b479af61be37b72 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 1 Jun 2023 16:57:43 +0200 Subject: [PATCH 226/308] Style --- src/Storages/StorageS3.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index f3cad4de31a..29f0a747372 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -424,7 +424,6 @@ public: , bucket(bucket_) , query(query_) , virtual_header(virtual_header_) - { Strings all_keys = keys_; From bdb192cf2742d6f3059f621068bbc59d78124229 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 1 Jun 2023 15:43:37 +0000 Subject: [PATCH 227/308] Test right join in 02725_any_join_single_row, style code changes --- src/Interpreters/HashJoin.cpp | 14 ++++---- .../02725_any_join_single_row.reference | 3 ++ .../0_stateless/02725_any_join_single_row.sql | 33 ++++++++++++++----- 3 files changed, 34 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 436ecd382cd..9306c9b99eb 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -556,7 +556,7 @@ namespace return false; } - static ALWAYS_INLINE bool insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) + static ALWAYS_INLINE void insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -567,10 +567,9 @@ namespace /// The first element of the list is stored in the value of the hash table, the rest in the pool. emplace_result.getMapped().insert({stored_block, i}, pool); } - return true; } - static ALWAYS_INLINE bool insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, + static ALWAYS_INLINE void insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, const IColumn & asof_column) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -580,7 +579,6 @@ namespace if (emplace_result.isInserted()) time_series_map = new (time_series_map) typename Map::mapped_type(createAsofRowRef(asof_type, join.getAsofInequality())); (*time_series_map)->insert(asof_column, stored_block, i); - return true; } }; @@ -599,7 +597,9 @@ namespace auto key_getter = createKeyGetter(key_columns, key_sizes); - is_inserted = false; + /// For ALL and ASOF join always insert values + is_inserted = !mapped_one || is_asof_join; + for (size_t i = 0; i < rows; ++i) { if (has_null_map && (*null_map)[i]) @@ -615,11 +615,11 @@ namespace continue; if constexpr (is_asof_join) - is_inserted |= Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); + Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); else if constexpr (mapped_one) is_inserted |= Inserter::insertOne(join, map, key_getter, stored_block, i, pool); else - is_inserted |= Inserter::insertAll(join, map, key_getter, stored_block, i, pool); + Inserter::insertAll(join, map, key_getter, stored_block, i, pool); } return map.getBufferSizeInCells(); } diff --git a/tests/queries/0_stateless/02725_any_join_single_row.reference b/tests/queries/0_stateless/02725_any_join_single_row.reference index 5d748fc6dbb..1e940bdc71e 100644 --- a/tests/queries/0_stateless/02725_any_join_single_row.reference +++ b/tests/queries/0_stateless/02725_any_join_single_row.reference @@ -1,3 +1,6 @@ Join(ANY, LEFT, key) 0 1 Join(ANY, LEFT, key) 1 1 Join(ANY, LEFT, key) 1 1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02725_any_join_single_row.sql b/tests/queries/0_stateless/02725_any_join_single_row.sql index 5e5c959c278..f7ddd2f402b 100644 --- a/tests/queries/0_stateless/02725_any_join_single_row.sql +++ b/tests/queries/0_stateless/02725_any_join_single_row.sql @@ -1,26 +1,41 @@ -CREATE TABLE join_test -( - `key` UInt64, - `value` UInt64 -) -ENGINE = Join(ANY, LEFT, key); +DROP TABLE IF EXISTS join_test; +DROP TABLE IF EXISTS join_test_right; +CREATE TABLE join_test ( `key` UInt64, `value` UInt64 ) ENGINE = Join(ANY, LEFT, key); + +-- Save table size before inserting any rows CREATE TEMPORARY TABLE initial_table_size AS SELECT engine_full, total_rows, total_bytes FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); +-- Check that table size is less than 100K SELECT engine_full, total_rows, total_bytes < 100_000 FROM initial_table_size; INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); - +-- Save table size after inserting one row CREATE TEMPORARY TABLE one_row_table_size AS SELECT engine_full, total_rows, total_bytes FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); +-- Check that table size is less than 2x after inserting one row SELECT engine_full, total_rows, total_bytes < 2 * (SELECT total_bytes FROM initial_table_size) FROM one_row_table_size; -INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); -INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); +-- Insert some more rows with the same key INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(10_000); +-- Check that rows with the same key are not duplicated SELECT engine_full, total_rows, total_bytes == (SELECT total_bytes FROM one_row_table_size) FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); + +-- For RIGHT join we save all rows from the right table +CREATE TABLE join_test_right ( `key` UInt64, `value` UInt64 ) ENGINE = Join(ANY, RIGHT, key); + +INSERT INTO join_test_right (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test_right (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test_right (key, value) SELECT 1, number FROM numbers(1); +SELECT count() == 3 FROM (SELECT 1 as key) t1 ANY RIGHT JOIN join_test_right ON t1.key = join_test_right.key; +INSERT INTO join_test_right (key, value) SELECT 1, number FROM numbers(7); +SELECT count() == 10 FROM (SELECT 1 as key) t1 ANY RIGHT JOIN join_test_right ON t1.key = join_test_right.key; +SELECT count() == 10 FROM (SELECT 2 as key) t1 ANY RIGHT JOIN join_test_right ON t1.key = join_test_right.key; + +DROP TABLE IF EXISTS join_test; +DROP TABLE IF EXISTS join_test_right; From 4d65be4dbc30aaa764ddd8c888dca79fdc60bf07 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Jun 2023 18:34:35 +0200 Subject: [PATCH 228/308] Ignore QEMU logging regarding IFA --- .../0_stateless/01103_check_cpu_instructions_at_startup.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh index 9b6e1e05f2d..01047aeb9ab 100755 --- a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh +++ b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh @@ -19,7 +19,9 @@ fi function run_with_cpu() { - qemu-x86_64-static -cpu "$@" "$command" --query "SELECT 1" 2>&1 | grep -v -F "warning: TCG doesn't support requested feature" ||: + qemu-x86_64-static -cpu "$@" "$command" --query "SELECT 1" 2>&1 | \ + grep -v -F "warning: TCG doesn't support requested feature" | \ + grep -v -F 'Unknown host IFA type' ||: } run_with_cpu qemu64 From 02e986a9e7c1c33bf8818411de538c58af8a5198 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 1 Jun 2023 19:06:11 +0200 Subject: [PATCH 229/308] Revert "Revert "less logs in WriteBufferFromS3" (#50390)" This reverts commit 4d4112ff536f819514973dfd0cb8274cf044bb3e. --- src/IO/WriteBufferFromS3.cpp | 8 -------- src/IO/WriteBufferFromS3TaskTracker.cpp | 11 ----------- 2 files changed, 19 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 462cf2674c3..6992c3ea4ac 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -195,18 +195,14 @@ void WriteBufferFromS3::finalizeImpl() if (request_settings.check_objects_after_upload) { - LOG_TRACE(log, "Checking object {} exists after upload", key); S3::checkObjectExists(*client_ptr, bucket, key, {}, request_settings, /* for_disk_s3= */ write_settings.for_object_storage, "Immediately after upload"); - LOG_TRACE(log, "Checking object {} has size as expected {}", key, total_size); size_t actual_size = S3::getObjectSize(*client_ptr, bucket, key, {}, request_settings, /* for_disk_s3= */ write_settings.for_object_storage); if (actual_size != total_size) throw Exception( ErrorCodes::S3_ERROR, "Object {} from bucket {} has unexpected size {} after upload, expected size {}, it's a bug in S3 or S3 API.", key, bucket, actual_size, total_size); - - LOG_TRACE(log, "Object {} exists after upload", key); } } @@ -292,8 +288,6 @@ void WriteBufferFromS3::reallocateFirstBuffer() WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size); chassert(offset() == 0); - - LOG_TRACE(log, "Reallocated first buffer with size {}. {}", memory.size(), getLogDetails()); } void WriteBufferFromS3::detachBuffer() @@ -316,8 +310,6 @@ void WriteBufferFromS3::allocateFirstBuffer() const auto size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), max_first_buffer); memory = Memory(size); WriteBuffer::set(memory.data(), memory.size()); - - LOG_TRACE(log, "Allocated first buffer with size {}. {}", memory.size(), getLogDetails()); } void WriteBufferFromS3::allocateBuffer() diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 7ae31044012..c10af5d0672 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -36,8 +36,6 @@ ThreadPoolCallbackRunner WriteBufferFromS3::TaskTracker::syncRunner() void WriteBufferFromS3::TaskTracker::waitAll() { - LOG_TEST(log, "waitAll, in queue {}", futures.size()); - /// Exceptions are propagated for (auto & future : futures) { @@ -51,8 +49,6 @@ void WriteBufferFromS3::TaskTracker::waitAll() void WriteBufferFromS3::TaskTracker::safeWaitAll() { - LOG_TEST(log, "safeWaitAll, wait in queue {}", futures.size()); - for (auto & future : futures) { if (future.valid()) @@ -76,7 +72,6 @@ void WriteBufferFromS3::TaskTracker::safeWaitAll() void WriteBufferFromS3::TaskTracker::waitIfAny() { - LOG_TEST(log, "waitIfAny, in queue {}", futures.size()); if (futures.empty()) return; @@ -101,8 +96,6 @@ void WriteBufferFromS3::TaskTracker::waitIfAny() watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); - - LOG_TEST(log, "waitIfAny ended, in queue {}", futures.size()); } void WriteBufferFromS3::TaskTracker::add(Callback && func) @@ -147,8 +140,6 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() if (!max_tasks_inflight) return; - LOG_TEST(log, "waitTilInflightShrink, in queue {}", futures.size()); - Stopwatch watch; /// Alternative approach is to wait until at least futures.size() - max_tasks_inflight element are finished @@ -171,8 +162,6 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); - - LOG_TEST(log, "waitTilInflightShrink ended, in queue {}", futures.size()); } bool WriteBufferFromS3::TaskTracker::isAsync() const From bd047ed9e0b580fc417c56239fa93340f6089388 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 1 Jun 2023 14:16:49 -0400 Subject: [PATCH 230/308] Update order-by.md add a note that ORDER BY sorting is case sensitive. closes https://github.com/ClickHouse/clickhouse-docs/issues/81 --- docs/en/sql-reference/statements/select/order-by.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 712395a0357..1da6c1d8468 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -5,7 +5,7 @@ sidebar_label: ORDER BY # ORDER BY Clause -The `ORDER BY` clause contains a list of expressions, which can each be attributed with `DESC` (descending) or `ASC` (ascending) modifier which determine the sorting direction. If the direction is not specified, `ASC` is assumed, so it’s usually omitted. The sorting direction applies to a single expression, not to the entire list. Example: `ORDER BY Visits DESC, SearchPhrase`. +The `ORDER BY` clause contains a list of expressions, which can each be attributed with `DESC` (descending) or `ASC` (ascending) modifier which determine the sorting direction. If the direction is not specified, `ASC` is assumed, so it’s usually omitted. The sorting direction applies to a single expression, not to the entire list. Example: `ORDER BY Visits DESC, SearchPhrase`. Sorting is case-sensitive. If you want to sort by column numbers instead of column names, enable the setting [enable_positional_arguments](../../../operations/settings/settings.md#enable-positional-arguments). From f62faaedc3c837b009479cb971e868526a6464fa Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Jun 2023 21:20:39 +0200 Subject: [PATCH 231/308] paranoid fix for removing parts from zk --- .../MergeTree/ReplicatedMergeTreeSink.cpp | 19 +- .../MergeTree/ReplicatedMergeTreeSink.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 171 +++++++----------- src/Storages/StorageReplicatedMergeTree.h | 1 + ...tem_parts_race_condition_zookeeper_long.sh | 2 + .../0_stateless/01154_move_partition_long.sh | 2 + 6 files changed, 86 insertions(+), 111 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index a38e9eba844..28dad454afe 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -532,12 +532,12 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF try { - commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num, false); + bool deduplicated = commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num, false).second; - last_block_is_duplicate = last_block_is_duplicate || part->is_duplicate; + last_block_is_duplicate = last_block_is_duplicate || deduplicated; /// Set a special error code if the block is duplicate - int error = (deduplicate && part->is_duplicate) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; + int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot), ExecutionStatus(error)); storage.incrementInsertedPartsProfileEvent(part->getType()); @@ -575,7 +575,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFa while (true) { partition.temp_part.finalize(); - auto conflict_block_ids = commitPart(zookeeper, partition.temp_part.part, partition.block_id, delayed_chunk->replicas_num, false); + auto conflict_block_ids = commitPart(zookeeper, partition.temp_part.part, partition.block_id, delayed_chunk->replicas_num, false).first; if (conflict_block_ids.empty()) break; ++retry_times; @@ -620,7 +620,7 @@ void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData: } template -std::vector ReplicatedMergeTreeSinkImpl::commitPart( +std::pair, bool> ReplicatedMergeTreeSinkImpl::commitPart( const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const BlockIDsType & block_id, @@ -644,6 +644,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( /// for retries due to keeper error bool part_committed_locally_but_zookeeper = false; + bool part_was_deduplicated = false; Coordination::Error write_part_info_keeper_error = Coordination::Error::ZOK; std::vector conflict_block_ids; @@ -844,7 +845,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( /// If it exists on our replica, ignore it. if (storage.getActiveContainingPart(existing_part_name)) { - part->is_duplicate = true; + part_was_deduplicated = true; ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); if (isQuorumEnabled()) { @@ -1040,7 +1041,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( ++loop_counter; if (loop_counter == max_iterations) { - part->is_duplicate = true; /// Part is duplicate, just remove it from local FS + part_was_deduplicated = true; /// Part is duplicate, just remove it from local FS throw Exception(ErrorCodes::DUPLICATE_DATA_PART, "Too many transaction retries - it may indicate an error"); } retries_ctl.requestUnconditionalRetry(); /// we want one more iteration w/o counting it as a try and timeout @@ -1093,7 +1094,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( [&zookeeper]() { zookeeper->cleanupEphemeralNodes(); }); if (!conflict_block_ids.empty()) - return conflict_block_ids; + return {conflict_block_ids, part_was_deduplicated}; if (isQuorumEnabled()) { @@ -1129,7 +1130,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( return; }); } - return {}; + return {conflict_block_ids, part_was_deduplicated}; } template diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 3777a9f7285..3efd364fc9c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -87,7 +87,7 @@ private: size_t checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper); /// Rename temporary part and commit to ZooKeeper. - std::vector commitPart( + std::pair, bool> commitPart( const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const BlockIDsType & block_id, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0f79e9f8f19..44403fc708b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -1992,6 +1993,16 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) /// Forcibly remove parts from ZooKeeper removePartsFromZooKeeperWithRetries(parts_to_remove); +#ifdef ABORT_ON_LOGICAL_ERROR + Strings parts_remain = getZooKeeper()->getChildren(replica_path + "/parts"); + for (const auto & part_name : parts_remain) + { + auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); + if (drop_range_info.contains(part_info)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} remains in ZooKeeper after DROP_RANGE {}", part_name, entry.new_part_name); + } +#endif + if (entry.detach) LOG_DEBUG(log, "Detached {} parts inside {}.", parts_to_remove.size(), entry.new_part_name); else @@ -6634,8 +6645,7 @@ bool StorageReplicatedMergeTree::hasLightweightDeletedMask() const void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() { - auto table_lock = lockForShare( - RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto table_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto zookeeper = getZooKeeper(); /// Now these parts are in Deleting state. If we fail to remove some of them we must roll them back to Outdated state. @@ -6644,6 +6654,12 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() if (parts.empty()) return; + NOEXCEPT_SCOPE({ clearOldPartsAndRemoveFromZKImpl(zookeeper, std::move(parts)); }); +} + +void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZKImpl(zkutil::ZooKeeperPtr zookeeper, DataPartsVector && parts) +{ + DataPartsVector parts_to_delete_only_from_filesystem; // Only duplicates DataPartsVector parts_to_delete_completely; // All parts except duplicates DataPartsVector parts_to_retry_deletion; // Parts that should be retried due to network problems @@ -6654,7 +6670,11 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() /// Broken part can be removed from zk by removePartAndEnqueueFetch(...) only. /// Removal without enqueueing a fetch leads to intersecting parts. if (part->is_duplicate || part->outdated_because_broken) + { + LOG_WARNING(log, "Will not remove part {} from ZooKeeper (is_duplicate: {}, outdated_because_broken: {})", + part->name, part->is_duplicate, part->outdated_because_broken); parts_to_delete_only_from_filesystem.emplace_back(part); + } else parts_to_delete_completely.emplace_back(part); } @@ -6680,7 +6700,7 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() if (!rollback_parts.empty()) rollbackDeletingParts(rollback_parts); } - else /// all parts was successfully removed + else /// all parts were successfully removed { finally_remove_parts = parts_to_delete; } @@ -6764,114 +6784,57 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeperWithRetries(PartsToRemo void StorageReplicatedMergeTree::removePartsFromZooKeeperWithRetries(const Strings & part_names, size_t max_retries) { + auto zookeeper = getZooKeeper(); + NameSet parts_to_retry_set; + removePartsFromZooKeeper(zookeeper, part_names, &parts_to_retry_set); + size_t num_tries = 0; - bool success = false; - - while (!success && (max_retries == 0 || num_tries < max_retries)) + while (!parts_to_retry_set.empty() && (max_retries == 0 || num_tries < max_retries)) { - try - { - ++num_tries; - success = true; - - auto zookeeper = getZooKeeper(); - - Strings exists_paths; - exists_paths.reserve(part_names.size()); - for (const String & part_name : part_names) - { - exists_paths.emplace_back(fs::path(replica_path) / "parts" / part_name); - } - - auto exists_results = zookeeper->exists(exists_paths); - - std::vector> remove_futures; - remove_futures.reserve(part_names.size()); - for (size_t i = 0; i < part_names.size(); ++i) - { - Coordination::ExistsResponse exists_resp = exists_results[i]; - if (exists_resp.error == Coordination::Error::ZOK) - { - Coordination::Requests ops; - getRemovePartFromZooKeeperOps(part_names[i], ops, exists_resp.stat.numChildren > 0); - remove_futures.emplace_back(zookeeper->asyncTryMultiNoThrow(ops)); - } - } - - for (auto & future : remove_futures) - { - auto response = future.get(); - - if (response.error == Coordination::Error::ZOK || response.error == Coordination::Error::ZNONODE) - continue; - - if (Coordination::isHardwareError(response.error)) - { - success = false; - continue; - } - - throw Coordination::Exception(response.error); - } - } - catch (Coordination::Exception & e) - { - success = false; - - if (Coordination::isHardwareError(e.code)) - tryLogCurrentException(log, __PRETTY_FUNCTION__); - else - throw; - } - - if (!success && num_tries < max_retries) - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + zookeeper = getZooKeeper(); + Strings parts_to_retry; + std::move(parts_to_retry_set.begin(), parts_to_retry_set.end(), std::back_inserter(parts_to_retry)); + parts_to_retry_set.clear(); + removePartsFromZooKeeper(zookeeper, parts_to_retry, &parts_to_retry_set); + ++num_tries; } - if (!success) - throw Exception(ErrorCodes::UNFINISHED, "Failed to remove parts from ZooKeeper after {} retries", num_tries); + if (!parts_to_retry_set.empty()) + throw Exception(ErrorCodes::UNFINISHED, "Failed to remove {} parts from ZooKeeper after {} retries", parts_to_retry_set.size(), num_tries); } void StorageReplicatedMergeTree::removePartsFromZooKeeper( zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names, NameSet * parts_should_be_retried) +try { Strings exists_paths; std::vector> remove_futures; exists_paths.reserve(part_names.size()); remove_futures.reserve(part_names.size()); - try + /// Exception can be thrown from loop + /// if zk session will be dropped + for (const String & part_name : part_names) { - /// Exception can be thrown from loop - /// if zk session will be dropped - for (const String & part_name : part_names) - { - exists_paths.emplace_back(fs::path(replica_path) / "parts" / part_name); - } - - auto exists_results = zookeeper->exists(exists_paths); - - for (size_t i = 0; i < part_names.size(); ++i) - { - auto exists_resp = exists_results[i]; - if (exists_resp.error == Coordination::Error::ZOK) - { - Coordination::Requests ops; - getRemovePartFromZooKeeperOps(part_names[i], ops, exists_resp.stat.numChildren > 0); - remove_futures.emplace_back(zookeeper->asyncTryMultiNoThrow(ops)); - } - else - { - LOG_DEBUG(log, "There is no part {} in ZooKeeper, it was only in filesystem", part_names[i]); - // emplace invalid future so that the total number of futures is the same as part_names.size(); - remove_futures.emplace_back(); - } - } + exists_paths.emplace_back(fs::path(replica_path) / "parts" / part_name); } - catch (const Coordination::Exception & e) + + auto exists_results = zookeeper->exists(exists_paths); + + for (size_t i = 0; i < part_names.size(); ++i) { - if (parts_should_be_retried && Coordination::isHardwareError(e.code)) - parts_should_be_retried->insert(part_names.begin(), part_names.end()); - throw; + auto exists_resp = exists_results[i]; + if (exists_resp.error == Coordination::Error::ZOK) + { + Coordination::Requests ops; + getRemovePartFromZooKeeperOps(part_names[i], ops, exists_resp.stat.numChildren > 0); + remove_futures.emplace_back(zookeeper->asyncTryMultiNoThrow(ops)); + } + else + { + LOG_DEBUG(log, "There is no part {} in ZooKeeper, it was only in filesystem", part_names[i]); + // emplace invalid future so that the total number of futures is the same as part_names.size(); + remove_futures.emplace_back(); + } } for (size_t i = 0; i < remove_futures.size(); ++i) @@ -6884,21 +6847,27 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper( auto response = future.get(); if (response.error == Coordination::Error::ZOK) continue; - else if (response.error == Coordination::Error::ZNONODE) + + if (response.error == Coordination::Error::ZNONODE) { LOG_DEBUG(log, "There is no part {} in ZooKeeper, it was only in filesystem", part_names[i]); - continue; } - else if (Coordination::isHardwareError(response.error)) + else { if (parts_should_be_retried) parts_should_be_retried->insert(part_names[i]); - continue; + + if (!Coordination::isHardwareError(response.error)) + LOG_WARNING(log, "Cannot remove part {} from ZooKeeper: {}", part_names[i], Coordination::errorMessage(response.error)); } - else - LOG_WARNING(log, "Cannot remove part {} from ZooKeeper: {}", part_names[i], Coordination::errorMessage(response.error)); } } +catch (...) +{ + if (parts_should_be_retried) + parts_should_be_retried->insert(part_names.begin(), part_names.end()); + throw; +} void StorageReplicatedMergeTree::clearLockedBlockNumbersInPartition( zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index cb93dd0b5e3..dd7ea84f76b 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -344,6 +344,7 @@ private: /// Delete old parts from disk and from ZooKeeper. void clearOldPartsAndRemoveFromZK(); + void clearOldPartsAndRemoveFromZKImpl(zkutil::ZooKeeperPtr zookeeper, DataPartsVector && parts); template friend class ReplicatedMergeTreeSinkImpl; diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh index 5b1c50262bf..862cc90fb1c 100755 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh @@ -79,6 +79,8 @@ timeout $TIMEOUT bash -c thread5 2> /dev/null & wait check_replication_consistency "alter_table" "count(), sum(a), sum(b), round(sum(c))" +$CLICKHOUSE_CLIENT -q "SELECT table, lost_part_count FROM system.replicas WHERE database=currentDatabase() AND lost_part_count!=0"; + $CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table0;" 2> >(grep -F -v 'is already started to be removing by another replica right now') & $CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table1;" 2> >(grep -F -v 'is already started to be removing by another replica right now') & wait diff --git a/tests/queries/0_stateless/01154_move_partition_long.sh b/tests/queries/0_stateless/01154_move_partition_long.sh index c68b0944407..3e068fa0e2d 100755 --- a/tests/queries/0_stateless/01154_move_partition_long.sh +++ b/tests/queries/0_stateless/01154_move_partition_long.sh @@ -125,6 +125,8 @@ wait check_replication_consistency "dst_" "count(), sum(p), sum(k), sum(v)" try_sync_replicas "src_" 300 +$CLICKHOUSE_CLIENT -q "SELECT table, lost_part_count FROM system.replicas WHERE database=currentDatabase() AND lost_part_count!=0"; + for ((i=0; i<16; i++)) do $CLICKHOUSE_CLIENT -q "DROP TABLE dst_$i" 2>&1| grep -Fv "is already started to be removing" & $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS src_$i" 2>&1| grep -Fv "is already started to be removing" & From 1544067fb37b53b5ba0e1101db9ab068e9903217 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Jun 2023 23:28:19 +0300 Subject: [PATCH 232/308] Update run.sh --- docker/test/unit/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/unit/run.sh b/docker/test/unit/run.sh index abc35fa40d2..a4784466e27 100644 --- a/docker/test/unit/run.sh +++ b/docker/test/unit/run.sh @@ -3,5 +3,5 @@ set -x service zookeeper start && sleep 7 && /usr/share/zookeeper/bin/zkCli.sh -server localhost:2181 -create create /clickhouse_test ''; -gdb -q -ex 'set print inferior-events off' -ex 'set confirm off' -ex 'set print thread-events off' -ex run -ex bt -ex quit --args ./unit_tests_dbms | tee test_output/test_result.txt +timeout 40m gdb -q -ex 'set print inferior-events off' -ex 'set confirm off' -ex 'set print thread-events off' -ex run -ex bt -ex quit --args ./unit_tests_dbms | tee test_output/test_result.txt ./process_unit_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv From 6b2c906dfd57a895b93605a4bfb07fda7a72945d Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 1 Jun 2023 19:03:35 -0400 Subject: [PATCH 233/308] add docs for boundingRatio --- .../aggregate-functions/reference/boundrat.md | 44 ++++++ .../aggregate-functions/reference/index.md | 131 +++++++++--------- 2 files changed, 110 insertions(+), 65 deletions(-) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/boundrat.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/boundrat.md b/docs/en/sql-reference/aggregate-functions/reference/boundrat.md new file mode 100644 index 00000000000..f3907af8030 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/boundrat.md @@ -0,0 +1,44 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/boundingRatio +sidebar_position: 2 +title: boundingRatio +--- + +Aggregate function that calculates the slope between the leftmost and rightmost points across a group of values. + +Example: + +Sample data: +```sql +SELECT + number, + number * 1.5 +FROM numbers(10) +``` +```response +┌─number─┬─multiply(number, 1.5)─┐ +│ 0 │ 0 │ +│ 1 │ 1.5 │ +│ 2 │ 3 │ +│ 3 │ 4.5 │ +│ 4 │ 6 │ +│ 5 │ 7.5 │ +│ 6 │ 9 │ +│ 7 │ 10.5 │ +│ 8 │ 12 │ +│ 9 │ 13.5 │ +└────────┴───────────────────────┘ +``` + +The boundingRatio() function returns the slope of the line between the leftmost and rightmost points, in the above data these points are `(0,0)` and `(9,13.5)`. + +```sql +SELECT boundingRatio(number, number * 1.5) +FROM numbers(10) +``` +```response +┌─boundingRatio(number, multiply(number, 1.5))─┐ +│ 1.5 │ +└──────────────────────────────────────────────┘ +``` + diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index 50208352f38..17ef494e9ad 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -9,74 +9,75 @@ toc_hidden: true Standard aggregate functions: -- [count](../../../sql-reference/aggregate-functions/reference/count.md) -- [min](../../../sql-reference/aggregate-functions/reference/min.md) -- [max](../../../sql-reference/aggregate-functions/reference/max.md) -- [sum](../../../sql-reference/aggregate-functions/reference/sum.md) -- [avg](../../../sql-reference/aggregate-functions/reference/avg.md) -- [any](../../../sql-reference/aggregate-functions/reference/any.md) -- [stddevPop](../../../sql-reference/aggregate-functions/reference/stddevpop.md) -- [stddevSamp](../../../sql-reference/aggregate-functions/reference/stddevsamp.md) -- [varPop](../../../sql-reference/aggregate-functions/reference/varpop.md) -- [varSamp](../../../sql-reference/aggregate-functions/reference/varsamp.md) -- [covarPop](../../../sql-reference/aggregate-functions/reference/covarpop.md) -- [covarSamp](../../../sql-reference/aggregate-functions/reference/covarsamp.md) +- [count](/docs/en/sql-reference/aggregate-functions/reference/count.md) +- [min](/docs/en/sql-reference/aggregate-functions/reference/min.md) +- [max](/docs/en/sql-reference/aggregate-functions/reference/max.md) +- [sum](/docs/en/sql-reference/aggregate-functions/reference/sum.md) +- [avg](/docs/en/sql-reference/aggregate-functions/reference/avg.md) +- [any](/docs/en/sql-reference/aggregate-functions/reference/any.md) +- [stddevPop](/docs/en/sql-reference/aggregate-functions/reference/stddevpop.md) +- [stddevSamp](/docs/en/sql-reference/aggregate-functions/reference/stddevsamp.md) +- [varPop](/docs/en/sql-reference/aggregate-functions/reference/varpop.md) +- [varSamp](/docs/en/sql-reference/aggregate-functions/reference/varsamp.md) +- [covarPop](/docs/en/sql-reference/aggregate-functions/reference/covarpop.md) +- [covarSamp](/docs/en/sql-reference/aggregate-functions/reference/covarsamp.md) ClickHouse-specific aggregate functions: -- [anyHeavy](../../../sql-reference/aggregate-functions/reference/anyheavy.md) -- [anyLast](../../../sql-reference/aggregate-functions/reference/anylast.md) -- [first_value](../../../sql-reference/aggregate-functions/reference/first_value.md) -- [last_value](../../../sql-reference/aggregate-functions/reference/last_value.md) -- [argMin](../../../sql-reference/aggregate-functions/reference/argmin.md) -- [argMax](../../../sql-reference/aggregate-functions/reference/argmax.md) -- [avgWeighted](../../../sql-reference/aggregate-functions/reference/avgweighted.md) -- [topK](../../../sql-reference/aggregate-functions/reference/topk.md) -- [topKWeighted](../../../sql-reference/aggregate-functions/reference/topkweighted.md) -- [groupArray](../../../sql-reference/aggregate-functions/reference/grouparray.md) -- [groupArrayLast](../../../sql-reference/aggregate-functions/reference/grouparraylast.md) -- [groupUniqArray](../../../sql-reference/aggregate-functions/reference/groupuniqarray.md) -- [groupArrayInsertAt](../../../sql-reference/aggregate-functions/reference/grouparrayinsertat.md) -- [groupArrayMovingAvg](../../../sql-reference/aggregate-functions/reference/grouparraymovingavg.md) -- [groupArrayMovingSum](../../../sql-reference/aggregate-functions/reference/grouparraymovingsum.md) -- [groupBitAnd](../../../sql-reference/aggregate-functions/reference/groupbitand.md) -- [groupBitOr](../../../sql-reference/aggregate-functions/reference/groupbitor.md) -- [groupBitXor](../../../sql-reference/aggregate-functions/reference/groupbitxor.md) -- [groupBitmap](../../../sql-reference/aggregate-functions/reference/groupbitmap.md) -- [groupBitmapAnd](../../../sql-reference/aggregate-functions/reference/groupbitmapand.md) -- [groupBitmapOr](../../../sql-reference/aggregate-functions/reference/groupbitmapor.md) -- [groupBitmapXor](../../../sql-reference/aggregate-functions/reference/groupbitmapxor.md) -- [sumWithOverflow](../../../sql-reference/aggregate-functions/reference/sumwithoverflow.md) -- [sumMap](../../../sql-reference/aggregate-functions/reference/summap.md) -- [minMap](../../../sql-reference/aggregate-functions/reference/minmap.md) -- [maxMap](../../../sql-reference/aggregate-functions/reference/maxmap.md) -- [skewSamp](../../../sql-reference/aggregate-functions/reference/skewsamp.md) -- [skewPop](../../../sql-reference/aggregate-functions/reference/skewpop.md) -- [kurtSamp](../../../sql-reference/aggregate-functions/reference/kurtsamp.md) -- [kurtPop](../../../sql-reference/aggregate-functions/reference/kurtpop.md) -- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md) -- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md) -- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md) -- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md) -- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md) -- [uniqTheta](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md) -- [quantile](../../../sql-reference/aggregate-functions/reference/quantile.md) -- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md) -- [quantileExact](../../../sql-reference/aggregate-functions/reference/quantileexact.md) -- [quantileExactLow](../../../sql-reference/aggregate-functions/reference/quantileexact.md#quantileexactlow) -- [quantileExactHigh](../../../sql-reference/aggregate-functions/reference/quantileexact.md#quantileexacthigh) -- [quantileExactWeighted](../../../sql-reference/aggregate-functions/reference/quantileexactweighted.md) -- [quantileTiming](../../../sql-reference/aggregate-functions/reference/quantiletiming.md) -- [quantileTimingWeighted](../../../sql-reference/aggregate-functions/reference/quantiletimingweighted.md) -- [quantileDeterministic](../../../sql-reference/aggregate-functions/reference/quantiledeterministic.md) -- [quantileTDigest](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md) -- [quantileTDigestWeighted](../../../sql-reference/aggregate-functions/reference/quantiletdigestweighted.md) -- [quantileBFloat16](../../../sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16) -- [quantileBFloat16Weighted](../../../sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16weighted) -- [simpleLinearRegression](../../../sql-reference/aggregate-functions/reference/simplelinearregression.md) -- [stochasticLinearRegression](../../../sql-reference/aggregate-functions/reference/stochasticlinearregression.md) -- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md) -- [categoricalInformationValue](../../../sql-reference/aggregate-functions/reference/categoricalinformationvalue.md) +- [anyHeavy](/docs/en/sql-reference/aggregate-functions/reference/anyheavy.md) +- [anyLast](/docs/en/sql-reference/aggregate-functions/reference/anylast.md) +- [boundingRatio](/docs/en/sql-reference/aggregate-functions/reference/boundrat.md) +- [first_value](/docs/en/sql-reference/aggregate-functions/reference/first_value.md) +- [last_value](/docs/en/sql-reference/aggregate-functions/reference/last_value.md) +- [argMin](/docs/en/sql-reference/aggregate-functions/reference/argmin.md) +- [argMax](/docs/en/sql-reference/aggregate-functions/reference/argmax.md) +- [avgWeighted](/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md) +- [topK](/docs/en/sql-reference/aggregate-functions/reference/topk.md) +- [topKWeighted](/docs/en/sql-reference/aggregate-functions/reference/topkweighted.md) +- [groupArray](/docs/en/sql-reference/aggregate-functions/reference/grouparray.md) +- [groupArrayLast](/docs/en/sql-reference/aggregate-functions/reference/grouparraylast.md) +- [groupUniqArray](/docs/en/sql-reference/aggregate-functions/reference/groupuniqarray.md) +- [groupArrayInsertAt](/docs/en/sql-reference/aggregate-functions/reference/grouparrayinsertat.md) +- [groupArrayMovingAvg](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingavg.md) +- [groupArrayMovingSum](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingsum.md) +- [groupBitAnd](/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md) +- [groupBitOr](/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md) +- [groupBitXor](/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md) +- [groupBitmap](/docs/en/sql-reference/aggregate-functions/reference/groupbitmap.md) +- [groupBitmapAnd](/docs/en/sql-reference/aggregate-functions/reference/groupbitmapand.md) +- [groupBitmapOr](/docs/en/sql-reference/aggregate-functions/reference/groupbitmapor.md) +- [groupBitmapXor](/docs/en/sql-reference/aggregate-functions/reference/groupbitmapxor.md) +- [sumWithOverflow](/docs/en/sql-reference/aggregate-functions/reference/sumwithoverflow.md) +- [sumMap](/docs/en/sql-reference/aggregate-functions/reference/summap.md) +- [minMap](/docs/en/sql-reference/aggregate-functions/reference/minmap.md) +- [maxMap](/docs/en/sql-reference/aggregate-functions/reference/maxmap.md) +- [skewSamp](/docs/en/sql-reference/aggregate-functions/reference/skewsamp.md) +- [skewPop](/docs/en/sql-reference/aggregate-functions/reference/skewpop.md) +- [kurtSamp](/docs/en/sql-reference/aggregate-functions/reference/kurtsamp.md) +- [kurtPop](/docs/en/sql-reference/aggregate-functions/reference/kurtpop.md) +- [uniq](/docs/en/sql-reference/aggregate-functions/reference/uniq.md) +- [uniqExact](/docs/en/sql-reference/aggregate-functions/reference/uniqexact.md) +- [uniqCombined](/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md) +- [uniqCombined64](/docs/en/sql-reference/aggregate-functions/reference/uniqcombined64.md) +- [uniqHLL12](/docs/en/sql-reference/aggregate-functions/reference/uniqhll12.md) +- [uniqTheta](/docs/en/sql-reference/aggregate-functions/reference/uniqthetasketch.md) +- [quantile](/docs/en/sql-reference/aggregate-functions/reference/quantile.md) +- [quantiles](/docs/en/sql-reference/aggregate-functions/reference/quantiles.md) +- [quantileExact](/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md) +- [quantileExactLow](/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md#quantileexactlow) +- [quantileExactHigh](/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md#quantileexacthigh) +- [quantileExactWeighted](/docs/en/sql-reference/aggregate-functions/reference/quantileexactweighted.md) +- [quantileTiming](/docs/en/sql-reference/aggregate-functions/reference/quantiletiming.md) +- [quantileTimingWeighted](/docs/en/sql-reference/aggregate-functions/reference/quantiletimingweighted.md) +- [quantileDeterministic](/docs/en/sql-reference/aggregate-functions/reference/quantiledeterministic.md) +- [quantileTDigest](/docs/en/sql-reference/aggregate-functions/reference/quantiletdigest.md) +- [quantileTDigestWeighted](/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md) +- [quantileBFloat16](/docs/en/sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16) +- [quantileBFloat16Weighted](/docs/en/sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16weighted) +- [simpleLinearRegression](/docs/en/sql-reference/aggregate-functions/reference/simplelinearregression.md) +- [stochasticLinearRegression](/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md) +- [stochasticLogisticRegression](/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md) +- [categoricalInformationValue](/docs/en/sql-reference/aggregate-functions/reference/categoricalinformationvalue.md) - [contingency](./contingency.md) - [cramersV](./cramersv.md) - [cramersVBiasCorrected](./cramersvbiascorrected.md) From e7868e576c71de07d7dd4d921382d4cd549d6493 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 2 Jun 2023 00:50:14 +0000 Subject: [PATCH 234/308] Don't crash if config doesn't have logger section --- src/Loggers/Loggers.cpp | 2 +- src/Loggers/Loggers.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 645ae5dcc7a..0c3a7bd615d 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -51,7 +51,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log #endif auto current_logger = config.getString("logger", ""); - if (config_logger == current_logger) + if (config_logger.has_value() && *config_logger == current_logger) return; config_logger = current_logger; diff --git a/src/Loggers/Loggers.h b/src/Loggers/Loggers.h index 31a215aa9ce..ebc10954b94 100644 --- a/src/Loggers/Loggers.h +++ b/src/Loggers/Loggers.h @@ -39,7 +39,7 @@ private: Poco::AutoPtr syslog_channel; /// Previous value of logger element in config. It is used to reinitialize loggers whenever the value changed. - std::string config_logger; + std::optional config_logger; #ifndef WITHOUT_TEXT_LOG std::weak_ptr text_log; From c5165c2236bfceed4ec9246d6964fd2f77f68a07 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Jun 2023 03:25:12 +0200 Subject: [PATCH 235/308] More thorough check for the availability of TaskStats --- src/Common/TaskStatsInfoGetter.cpp | 5 ++++- src/Common/ThreadStatus.h | 1 - 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/TaskStatsInfoGetter.cpp index 25030ee9670..d21229609dd 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/TaskStatsInfoGetter.cpp @@ -9,6 +9,7 @@ #include "hasLinuxCapability.h" #include +#include #include #include @@ -202,10 +203,12 @@ bool checkPermissionsImpl() /// Check that we can successfully initialize TaskStatsInfoGetter. /// It will ask about family id through Netlink. /// On some LXC containers we have capability but we still cannot use Netlink. + /// There is an evidence that Linux fedora-riscv 6.1.22 gives something strange instead of the expected result. try { - TaskStatsInfoGetter(); + ::taskstats stats{}; + TaskStatsInfoGetter().getStat(stats, static_cast(getThreadId())); } catch (const Exception & e) { diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 600dfc56d2b..061959d9f1f 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -37,7 +37,6 @@ class QueryThreadLog; class TasksStatsCounters; struct RUsageCounters; struct PerfEventsCounters; -class TaskStatsInfoGetter; class InternalTextLogsQueue; struct ViewRuntimeData; class QueryViewsLog; From b51064a5081ed9339e30650ccd6466262b258522 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 5 May 2023 03:11:51 +0000 Subject: [PATCH 236/308] Get rid of SeekableReadBufferFactory, add SeekableReadBuffer::readBigAt() instead --- src/Core/Settings.h | 1 - src/Formats/FormatFactory.cpp | 102 ++----- src/Formats/FormatFactory.h | 56 +--- src/Formats/FormatSettings.h | 2 - src/IO/MMapReadBufferFromFileDescriptor.cpp | 11 + src/IO/MMapReadBufferFromFileDescriptor.h | 3 + src/IO/ParallelReadBuffer.cpp | 258 ++++++++---------- src/IO/ParallelReadBuffer.h | 45 +-- src/IO/ReadBufferFromFileDescriptor.cpp | 48 +++- src/IO/ReadBufferFromFileDescriptor.h | 11 + src/IO/ReadBufferFromS3.cpp | 187 +++++++------ src/IO/ReadBufferFromS3.h | 49 +--- src/IO/ReadWriteBufferFromHTTP.cpp | 209 +++++++------- src/IO/ReadWriteBufferFromHTTP.h | 61 +---- src/IO/SeekableReadBuffer.cpp | 46 ++++ src/IO/SeekableReadBuffer.h | 52 ++-- src/IO/WithFileName.cpp | 2 +- src/IO/WithFileSize.cpp | 8 - .../Formats/Impl/ArrowBufferedStreams.cpp | 98 ++----- .../Formats/Impl/ArrowBufferedStreams.h | 19 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 24 +- .../Formats/Impl/ParquetBlockInputFormat.h | 6 +- src/Storages/StorageS3.cpp | 43 +-- src/Storages/StorageS3.h | 8 +- src/Storages/StorageURL.cpp | 78 +++--- src/Storages/StorageURL.h | 3 +- .../test_redirect_url_storage/test.py | 4 +- 27 files changed, 658 insertions(+), 776 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 566c9f32484..2747094451d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -832,7 +832,6 @@ class IColumn; M(Bool, input_format_orc_case_insensitive_column_matching, false, "Ignore case when matching ORC columns with CH columns.", 0) \ M(Bool, input_format_parquet_import_nested, false, "Allow to insert array of structs into Nested table in Parquet input format.", 0) \ M(Bool, input_format_parquet_case_insensitive_column_matching, false, "Ignore case when matching Parquet columns with CH columns.", 0) \ - /* TODO: Consider unifying this with https://github.com/ClickHouse/ClickHouse/issues/38755 */ \ M(Bool, input_format_parquet_preserve_order, false, "Avoid reordering rows when reading from Parquet files. Usually makes it much slower.", 0) \ M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \ M(Bool, input_format_orc_allow_missing_columns, false, "Allow missing columns while reading ORC input formats", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index dd6252b96f1..586e1bb7251 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -221,68 +221,16 @@ template FormatSettings getFormatSettings(ContextPtr context, const Se InputFormatPtr FormatFactory::getInput( const String & name, - ReadBuffer & buf, + ReadBuffer & _buf, const Block & sample, ContextPtr context, UInt64 max_block_size, - const std::optional & format_settings, - std::optional max_parsing_threads) const -{ - return getInputImpl( - name, - nullptr, - &buf, - sample, - context, - max_block_size, - /* is_remote_fs */ false, - CompressionMethod::None, - format_settings, - /* max_download_threads */ 1, - max_parsing_threads); -} - -InputFormatPtr FormatFactory::getInputRandomAccess( - const String & name, - SeekableReadBufferFactoryPtr buf_factory, - const Block & sample, - ContextPtr context, - UInt64 max_block_size, - bool is_remote_fs, - CompressionMethod compression, - const std::optional & format_settings, - std::optional max_download_threads, - std::optional max_parsing_threads) const -{ - return getInputImpl( - name, - std::move(buf_factory), - nullptr, - sample, - context, - max_block_size, - is_remote_fs, - compression, - format_settings, - max_download_threads, - max_parsing_threads); -} - -InputFormatPtr FormatFactory::getInputImpl( - const String & name, - // exactly one of the following two is nullptr - SeekableReadBufferFactoryPtr buf_factory, - ReadBuffer * _buf, - const Block & sample, - ContextPtr context, - UInt64 max_block_size, - bool is_remote_fs, - CompressionMethod compression, const std::optional & _format_settings, + std::optional _max_parsing_threads, std::optional _max_download_threads, - std::optional _max_parsing_threads) const + bool is_remote_fs, + CompressionMethod compression) const { - chassert((!_buf) != (!buf_factory)); const auto& creators = getCreators(name); if (!creators.input_creator && !creators.random_access_input_creator) throw Exception(ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT, "Format {} is not suitable for input", name); @@ -302,14 +250,12 @@ InputFormatPtr FormatFactory::getInputImpl( if (context->hasQueryContext() && settings.log_queries) context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Format, name); - // Prepare a read buffer. + // Add ParallelReadBuffer and decompression if needed. - std::unique_ptr owned_buf; - if (buf_factory) - owned_buf = prepareReadBuffer(buf_factory, compression, creators, format_settings, settings, max_download_threads); - auto * buf = owned_buf ? owned_buf.get() : _buf; + auto owned_buf = wrapReadBufferIfNeeded(_buf, compression, creators, format_settings, settings, is_remote_fs, max_download_threads); + auto & buf = owned_buf ? *owned_buf : _buf; - // Decide whether to use parallel ParallelParsingInputFormat. + // Decide whether to use ParallelParsingInputFormat. bool parallel_parsing = max_parsing_threads > 1 && settings.input_format_parallel_parsing && creators.file_segmentation_engine && !creators.random_access_input_creator; @@ -322,7 +268,7 @@ InputFormatPtr FormatFactory::getInputImpl( { const auto & non_trivial_prefix_and_suffix_checker = creators.non_trivial_prefix_and_suffix_checker; /// Disable parallel parsing for input formats with non-trivial readPrefix() and readSuffix(). - if (non_trivial_prefix_and_suffix_checker && non_trivial_prefix_and_suffix_checker(*buf)) + if (non_trivial_prefix_and_suffix_checker && non_trivial_prefix_and_suffix_checker(buf)) parallel_parsing = false; } @@ -340,7 +286,7 @@ InputFormatPtr FormatFactory::getInputImpl( { return input_getter(input, sample, row_input_format_params, format_settings); }; ParallelParsingInputFormat::Params params{ - *buf, sample, parser_creator, creators.file_segmentation_engine, name, max_parsing_threads, + buf, sample, parser_creator, creators.file_segmentation_engine, name, max_parsing_threads, settings.min_chunk_bytes_for_parallel_parsing, max_block_size, context->getApplicationType() == Context::ApplicationType::SERVER}; format = std::make_shared(params); @@ -349,7 +295,6 @@ InputFormatPtr FormatFactory::getInputImpl( { format = creators.random_access_input_creator( buf, - std::move(buf_factory), sample, format_settings, context->getReadSettings(), @@ -359,7 +304,7 @@ InputFormatPtr FormatFactory::getInputImpl( } else { - format = creators.input_creator(*buf, sample, row_input_format_params, format_settings); + format = creators.input_creator(buf, sample, row_input_format_params, format_settings); } if (owned_buf) @@ -375,26 +320,28 @@ InputFormatPtr FormatFactory::getInputImpl( return format; } -std::unique_ptr FormatFactory::prepareReadBuffer( - SeekableReadBufferFactoryPtr & buf_factory, +std::unique_ptr FormatFactory::wrapReadBufferIfNeeded( + ReadBuffer & buf, CompressionMethod compression, const Creators & creators, const FormatSettings & format_settings, const Settings & settings, + bool is_remote_fs, size_t max_download_threads) const { std::unique_ptr res; - bool parallel_read = max_download_threads > 1 && buf_factory && format_settings.seekable_read; + bool parallel_read = is_remote_fs && max_download_threads > 1 && format_settings.seekable_read && isBufferWithFileSize(buf); if (creators.random_access_input_creator) parallel_read &= compression != CompressionMethod::None; + size_t file_size = 0; if (parallel_read) { try { - parallel_read = buf_factory->checkIfActuallySeekable() - && buf_factory->getFileSize() >= 2 * settings.max_download_buffer_size; + file_size = getFileSizeFromReadBuffer(buf); + parallel_read = file_size >= 2 * settings.max_download_buffer_size; } catch (const Poco::Exception & e) { @@ -415,23 +362,18 @@ std::unique_ptr FormatFactory::prepareReadBuffer( max_download_threads, settings.max_download_buffer_size); - res = std::make_unique( - std::move(buf_factory), - threadPoolCallbackRunner(IOThreadPool::get(), "ParallelRead"), - max_download_threads, - settings.max_download_buffer_size); + res = wrapInParallelReadBufferIfSupported( + buf, threadPoolCallbackRunner(IOThreadPool::get(), "ParallelRead"), + max_download_threads, settings.max_download_buffer_size, file_size); } if (compression != CompressionMethod::None) { if (!res) - res = buf_factory->getReader(); // NOLINT + res = wrapReadBufferReference(buf); res = wrapReadBufferWithCompressionMethod(std::move(res), compression, static_cast(settings.zstd_window_log_max)); } - if (!creators.random_access_input_creator && !res) - res = buf_factory->getReader(); - return res; } diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 677e34845d8..1d258beca8d 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -90,15 +90,11 @@ private: const FormatSettings & settings)>; // Incompatible with FileSegmentationEngine. - // When created using SeekableReadBufferFactoryPtr, the IInputFormat doesn't support - // resetParser() and setReadBuffer(). // // In future we may also want to pass some information about WHERE conditions (SelectQueryInfo?) // and get some information about projections (min/max/count per column per row group). using RandomAccessInputCreator = std::function & format_settings = std::nullopt, - std::optional max_parsing_threads = std::nullopt) const; - - // Format parser from a random-access source (factory of seekable read buffers). - // Parallelizes both parsing and reading when possible. - // Prefer this over getInput() when reading from random-access source like file or HTTP. - InputFormatPtr getInputRandomAccess( - const String & name, - SeekableReadBufferFactoryPtr buf_factory, - const Block & sample, - ContextPtr context, - UInt64 max_block_size, - bool is_remote_fs, - CompressionMethod compression, - // if nullopt, getFormatSettings(context) is used - const std::optional & format_settings = std::nullopt, + std::optional max_parsing_threads = std::nullopt, std::optional max_download_threads = std::nullopt, - std::optional max_parsing_threads = std::nullopt) const; + // affects things like buffer sizes and parallel reading + bool is_remote_fs = false, + // allows to do: buf -> parallel read -> decompression, + // because parallel read after decompression is not possible + CompressionMethod compression = CompressionMethod::None) const; /// Checks all preconditions. Returns ordinary format if parallel formatting cannot be done. OutputFormatPtr getOutputFormatParallelIfPossible( @@ -272,28 +260,14 @@ private: const Creators & getCreators(const String & name) const; - InputFormatPtr getInputImpl( - const String & name, - // exactly one of the following two is nullptr - SeekableReadBufferFactoryPtr buf_factory, - ReadBuffer * buf, - const Block & sample, - ContextPtr context, - UInt64 max_block_size, - bool is_remote_fs, - CompressionMethod compression, - const std::optional & format_settings, - std::optional max_download_threads, - std::optional max_parsing_threads) const; - - // Creates a ReadBuffer to give to an input format. - // Returns nullptr if we should give it the whole factory. - std::unique_ptr prepareReadBuffer( - SeekableReadBufferFactoryPtr & buf_factory, + // Creates a ReadBuffer to give to an input format. Returns nullptr if we should use `buf` directly. + std::unique_ptr wrapReadBufferIfNeeded( + ReadBuffer & buf, CompressionMethod compression, const Creators & creators, const FormatSettings & format_settings, const Settings & settings, + bool is_remote_fs, size_t max_download_threads) const; }; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index c88af650671..e332bd749a1 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -214,8 +214,6 @@ struct FormatSettings std::unordered_set skip_row_groups = {}; bool output_string_as_string = false; bool output_fixed_string_as_fixed_byte_array = true; - // TODO: This should probably be shared among all formats and with - // https://github.com/ClickHouse/ClickHouse/issues/38755 bool preserve_order = false; UInt64 max_block_size = 8192; ParquetVersion output_version; diff --git a/src/IO/MMapReadBufferFromFileDescriptor.cpp b/src/IO/MMapReadBufferFromFileDescriptor.cpp index c0eb73f8638..9b1c132cc01 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -91,4 +91,15 @@ size_t MMapReadBufferFromFileDescriptor::getFileSize() { return getSizeFromFileDescriptor(getFD(), getFileName()); } + +size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) +{ + if (offset >= mapped.getLength()) + return 0; + + n = std::min(n, mapped.getLength() - offset); + memcpy(to, mapped.getData() + offset, n); + return n; +} + } diff --git a/src/IO/MMapReadBufferFromFileDescriptor.h b/src/IO/MMapReadBufferFromFileDescriptor.h index 1a4bcd4f3ed..2a039e04971 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.h +++ b/src/IO/MMapReadBufferFromFileDescriptor.h @@ -39,6 +39,9 @@ public: int getFD() const; size_t getFileSize() override; + + size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) override; + bool supportsReadAt() override { return true; } }; } diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index fff02db1bd6..07240ab3a4f 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -13,51 +14,44 @@ namespace ErrorCodes } -// A subrange of the input, read by one SeekableReadBuffer. +// A subrange of the input, read by one thread. struct ParallelReadBuffer::ReadWorker { - ReadWorker(std::unique_ptr reader_, size_t offset_, size_t size) - : reader(std::move(reader_)), offset(offset_), bytes_left(size), range_end(offset + bytes_left) + ReadWorker(SeekableReadBuffer & input_, size_t offset, size_t size) + : input(input_), start_offset(offset), segment(size) { - assert(bytes_left); + chassert(size); + chassert(segment.size() == size); } - auto hasSegment() const { return current_segment_index < segments.size(); } + bool hasBytesToConsume() const { return bytes_produced > bytes_consumed; } + bool hasBytesToProduce() const { return bytes_produced < segment.size(); } - auto nextSegment() - { - assert(hasSegment()); - auto next_segment = std::move(segments[current_segment_index]); - ++current_segment_index; - offset += next_segment.size(); - return next_segment; - } + SeekableReadBuffer & input; + const size_t start_offset; // start of the segment - std::unique_ptr reader; - // Reader thread produces segments, nextImpl() consumes them. - std::vector> segments; // segments that were produced - size_t current_segment_index = 0; // first segment that's not consumed - bool finished{false}; // no more segments will be produced - size_t offset; // start of segments[current_segment_idx] - size_t bytes_left; // bytes left to produce above segments end - size_t range_end; // segments end + bytes_left, i.e. how far this worker will read - - // segments[current_segment_idx..end] range_end - // |-------------|--------------------------------------|------------| - // offset bytes_left + Memory<> segment; + /// Reader thread produces data, nextImpl() consumes it. + /// segment[bytes_consumed..bytes_produced-1] is data waiting to be picked up by nextImpl() + /// segment[bytes_produced..] needs to be read from the input ReadBuffer + size_t bytes_produced = 0; + size_t bytes_consumed = 0; std::atomic_bool cancel{false}; std::mutex worker_mutex; }; ParallelReadBuffer::ParallelReadBuffer( - std::unique_ptr reader_factory_, ThreadPoolCallbackRunner schedule_, size_t max_working_readers_, size_t range_step_) + SeekableReadBuffer & input_, ThreadPoolCallbackRunner schedule_, size_t max_working_readers_, size_t range_step_, size_t file_size_) : SeekableReadBuffer(nullptr, 0) , max_working_readers(max_working_readers_) , schedule(std::move(schedule_)) - , reader_factory(std::move(reader_factory_)) + , input(input_) + , file_size(file_size_) , range_step(std::max(1ul, range_step_)) { + LOG_TRACE(&Poco::Logger::get("ParallelReadBuffer"), "Parallel reading is used"); + try { addReaders(); @@ -71,22 +65,15 @@ ParallelReadBuffer::ParallelReadBuffer( bool ParallelReadBuffer::addReaderToPool() { - size_t file_size = reader_factory->getFileSize(); if (next_range_start >= file_size) return false; size_t range_start = next_range_start; size_t size = std::min(range_step, file_size - range_start); next_range_start += size; - auto reader = reader_factory->getReader(); - if (!reader) - { - return false; - } + auto worker = read_workers.emplace_back(std::make_shared(input, range_start, size)); - auto worker = read_workers.emplace_back(std::make_shared(std::move(reader), range_start, size)); - - ++active_working_reader; + ++active_working_readers; schedule([this, my_worker = std::move(worker)]() mutable { readerThreadFunction(std::move(my_worker)); }, Priority{}); return true; @@ -116,9 +103,9 @@ off_t ParallelReadBuffer::seek(off_t offset, int whence) } const auto offset_is_in_range - = [&](const auto & worker) { return static_cast(offset) >= worker->offset && static_cast(offset) < worker->range_end; }; + = [&](const auto & worker) { return static_cast(offset) >= worker->start_offset && static_cast(offset) < worker->start_offset + worker->segment.size(); }; - while (!read_workers.empty() && (offset < current_position || !offset_is_in_range(read_workers.front()))) + while (!read_workers.empty() && !offset_is_in_range(read_workers.front())) { read_workers.front()->cancel = true; read_workers.pop_front(); @@ -126,32 +113,31 @@ off_t ParallelReadBuffer::seek(off_t offset, int whence) if (!read_workers.empty()) { - auto & front_worker = read_workers.front(); - current_position = front_worker->offset; + auto & w = read_workers.front(); + size_t diff = static_cast(offset) - w->start_offset; while (true) { - std::unique_lock lock{front_worker->worker_mutex}; - next_condvar.wait(lock, [&] { return emergency_stop || front_worker->hasSegment(); }); + std::unique_lock lock{w->worker_mutex}; if (emergency_stop) handleEmergencyStop(); - auto next_segment = front_worker->nextSegment(); - current_position += next_segment.size(); - if (offset < current_position) + if (w->bytes_produced > diff) { - current_segment = std::move(next_segment); - working_buffer = internal_buffer = Buffer(current_segment.data(), current_segment.data() + current_segment.size()); - pos = working_buffer.end() - (current_position - offset); + working_buffer = internal_buffer = Buffer( + w->segment.data() + diff, w->segment.data() + w->bytes_produced); + w->bytes_consumed = w->bytes_produced; + current_position += w->start_offset + w->bytes_consumed; addReaders(); return offset; } + + next_condvar.wait_for(lock, std::chrono::seconds(10)); } } finishAndWait(); - all_completed = false; read_workers.clear(); next_range_start = offset; @@ -166,7 +152,7 @@ off_t ParallelReadBuffer::seek(off_t offset, int whence) size_t ParallelReadBuffer::getFileSize() { - return reader_factory->getFileSize(); + return file_size; } off_t ParallelReadBuffer::getPosition() @@ -174,17 +160,6 @@ off_t ParallelReadBuffer::getPosition() return current_position - available(); } -bool ParallelReadBuffer::currentWorkerReady() const -{ - assert(!read_workers.empty()); - return read_workers.front()->finished || read_workers.front()->hasSegment(); -} - -bool ParallelReadBuffer::currentWorkerCompleted() const -{ - return read_workers.front()->finished && !read_workers.front()->hasSegment(); -} - void ParallelReadBuffer::handleEmergencyStop() { // this can only be called from the main thread when there is an exception @@ -194,106 +169,99 @@ void ParallelReadBuffer::handleEmergencyStop() bool ParallelReadBuffer::nextImpl() { - if (all_completed) - return false; - while (true) { - std::unique_lock lock{read_workers.front()->worker_mutex}; - next_condvar.wait( - lock, - [this]() - { - /// Check if no more readers left or current reader can be processed - return emergency_stop || currentWorkerReady(); - }); - - bool worker_removed = false; - /// Remove completed units - while (currentWorkerCompleted() && !emergency_stop) - { - lock.unlock(); - read_workers.pop_front(); - worker_removed = true; - - if (read_workers.empty()) - break; - - lock = std::unique_lock{read_workers.front()->worker_mutex}; - } - - if (emergency_stop) - handleEmergencyStop(); - - if (worker_removed) - addReaders(); - /// All readers processed, stop if (read_workers.empty()) { - all_completed = true; + chassert(next_range_start >= file_size); return false; } - auto & front_worker = read_workers.front(); - /// Read data from first segment of the first reader - if (front_worker->hasSegment()) + auto * w = read_workers.front().get(); + + std::unique_lock lock{w->worker_mutex}; + + if (emergency_stop) + handleEmergencyStop(); // throws + + /// Read data from front reader + if (w->bytes_produced > w->bytes_consumed) { - current_segment = front_worker->nextSegment(); - if (currentWorkerCompleted()) - { - lock.unlock(); - read_workers.pop_front(); - all_completed = !addReaderToPool() && read_workers.empty(); - } - break; + chassert(w->start_offset + w->bytes_consumed == static_cast(current_position)); + + working_buffer = internal_buffer = Buffer( + w->segment.data() + w->bytes_consumed, w->segment.data() + w->bytes_produced); + current_position += working_buffer.size(); + w->bytes_consumed = w->bytes_produced; + + return true; } + + /// Front reader is done, remove it and add another + if (!w->hasBytesToProduce()) + { + lock.unlock(); + read_workers.pop_front(); + addReaders(); + + continue; + } + + /// Nothing to do right now, wait for something to change. + /// + /// The timeout is a workaround for a race condition. + /// emergency_stop is assigned while holding a *different* mutex from the one we're holding + /// (exception_mutex vs worker_mutex). So it's possible that our emergency_stop check (above) + /// happens before a onBackgroundException() call, but our wait(lock) happens after it. + /// Then the wait may get stuck forever. + /// + /// Note that using wait(lock, [&]{ return emergency_stop || ...; }) wouldn't help because + /// it does effectively the same "check, then wait" sequence. + /// + /// One possible proper fix would be to make onBackgroundException() lock all read_workers + /// mutexes too (not necessarily simultaneously - just locking+unlocking them one by one + /// between the emergency_stop change and the notify_all() would be enough), but then we + /// need another mutex to protect read_workers itself... + next_condvar.wait_for(lock, std::chrono::seconds(10)); } - working_buffer = internal_buffer = Buffer(current_segment.data(), current_segment.data() + current_segment.size()); - current_position += working_buffer.size(); - return true; + chassert(false); + return false; } void ParallelReadBuffer::readerThreadFunction(ReadWorkerPtr read_worker) { SCOPE_EXIT({ - if (active_working_reader.fetch_sub(1) == 1) - active_working_reader.notify_all(); + if (active_working_readers.fetch_sub(1) == 1) + active_working_readers.notify_all(); }); try { - read_worker->reader->setReadUntilPosition(read_worker->range_end); - read_worker->reader->seek(read_worker->offset, SEEK_SET); - - while (!emergency_stop && !read_worker->cancel) + auto on_progress = [&](size_t bytes_read) -> bool { - if (!read_worker->reader->next()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Failed to read all the data from the reader, missing {} bytes", read_worker->bytes_left); - if (emergency_stop || read_worker->cancel) - break; + return true; - Buffer buffer = read_worker->reader->buffer(); - size_t bytes_to_copy = std::min(buffer.size(), read_worker->bytes_left); - Memory<> new_segment(bytes_to_copy); - memcpy(new_segment.data(), buffer.begin(), bytes_to_copy); - read_worker->reader->ignore(bytes_to_copy); - read_worker->bytes_left -= bytes_to_copy; - { - /// New data ready to be read - std::lock_guard lock(read_worker->worker_mutex); - read_worker->segments.emplace_back(std::move(new_segment)); - read_worker->finished = read_worker->bytes_left == 0; + std::lock_guard lock(read_worker->worker_mutex); + if (bytes_read <= read_worker->bytes_produced) + return false; + + bool need_notify = read_worker->bytes_produced == read_worker->bytes_consumed; + read_worker->bytes_produced = bytes_read; + if (need_notify) next_condvar.notify_all(); - } - if (read_worker->finished) - { - break; - } - } + return false; + }; + + size_t r = input.readBigAt(read_worker->segment.data(), read_worker->segment.size(), read_worker->start_offset); + + if (!on_progress(r) && r < read_worker->segment.size()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Failed to read all the data from the reader at offset {}, got {}/{} bytes", + read_worker->start_offset, r, read_worker->segment.size()); } catch (...) { @@ -315,12 +283,24 @@ void ParallelReadBuffer::finishAndWait() { emergency_stop = true; - size_t active_readers = active_working_reader.load(); + size_t active_readers = active_working_readers.load(); while (active_readers != 0) { - active_working_reader.wait(active_readers); - active_readers = active_working_reader.load(); + active_working_readers.wait(active_readers); + active_readers = active_working_readers.load(); } } +std::unique_ptr wrapInParallelReadBufferIfSupported( + ReadBuffer & buf, ThreadPoolCallbackRunner schedule, size_t max_working_readers, + size_t range_step, size_t file_size) +{ + auto * seekable = dynamic_cast(&buf); + if (!seekable || !seekable->supportsReadAt()) + return nullptr; + + return std::make_unique( + *seekable, schedule, max_working_readers, range_step, file_size); +} + } diff --git a/src/IO/ParallelReadBuffer.h b/src/IO/ParallelReadBuffer.h index 70f925f9735..e76b40f77b7 100644 --- a/src/IO/ParallelReadBuffer.h +++ b/src/IO/ParallelReadBuffer.h @@ -10,18 +10,17 @@ namespace DB { /** - * Reads from multiple ReadBuffers in parallel. - * Preserves order of readers obtained from SeekableReadBufferFactory. + * Reads from multiple positions in a ReadBuffer in parallel. + * Then reassembles the data into one stream in the original order. * - * It consumes multiple readers and yields data from them in order as it passed. - * Each working reader save segments of data to internal queue. + * Each working reader reads its segment of data into a buffer. * - * ParallelReadBuffer in nextImpl method take first available segment from first reader in deque and fed it to user. - * When first reader finish reading, they will be removed from worker deque and data from next reader consumed. + * ParallelReadBuffer in nextImpl method take first available segment from first reader in deque and reports it it to user. + * When first reader finishes reading, they will be removed from worker deque and data from next reader consumed. * * Number of working readers limited by max_working_readers. */ -class ParallelReadBuffer : public SeekableReadBuffer +class ParallelReadBuffer : public SeekableReadBuffer, public WithFileSize { private: /// Blocks until data occurred in the first reader or this reader indicate finishing @@ -29,19 +28,19 @@ private: bool nextImpl() override; public: - ParallelReadBuffer(SeekableReadBufferFactoryPtr reader_factory_, ThreadPoolCallbackRunner schedule_, size_t max_working_readers, size_t range_step_); + ParallelReadBuffer(SeekableReadBuffer & input, ThreadPoolCallbackRunner schedule_, size_t max_working_readers, size_t range_step_, size_t file_size); ~ParallelReadBuffer() override { finishAndWait(); } off_t seek(off_t off, int whence) override; - size_t getFileSize(); + size_t getFileSize() override; off_t getPosition() override; - const SeekableReadBufferFactory & getReadBufferFactory() const { return *reader_factory; } - SeekableReadBufferFactory & getReadBufferFactory() { return *reader_factory; } + const SeekableReadBuffer & getReadBuffer() const { return input; } + SeekableReadBuffer & getReadBuffer() { return input; } private: - /// Reader in progress with a list of read segments + /// Reader in progress with a buffer for the segment struct ReadWorker; using ReadWorkerPtr = std::shared_ptr; @@ -55,28 +54,28 @@ private: void addReaders(); bool addReaderToPool(); - /// Process read_worker, read data and save into internal segments queue + /// Process read_worker, read data and save into the buffer void readerThreadFunction(ReadWorkerPtr read_worker); void onBackgroundException(); void finishAndWait(); - Memory<> current_segment; - size_t max_working_readers; - std::atomic_size_t active_working_reader{0}; + std::atomic_size_t active_working_readers{0}; ThreadPoolCallbackRunner schedule; - std::unique_ptr reader_factory; + SeekableReadBuffer & input; + size_t file_size; size_t range_step; size_t next_range_start{0}; /** * FIFO queue of readers. - * Each worker contains reader itself and downloaded segments. - * When reader read all available data it will be removed from - * deque and data from next reader will be consumed to user. + * Each worker contains a buffer for the downloaded segment. + * After all data for the segment is read and delivered to the user, the reader will be removed + * from deque and data from next reader will be delivered. + * After removing from deque, call addReaders(). */ std::deque read_workers; @@ -92,4 +91,10 @@ private: bool all_completed{false}; }; +/// If `buf` is a SeekableReadBuffer with supportsReadAt() == true, creates a ParallelReadBuffer +/// from it. Otherwise returns nullptr; +std::unique_ptr wrapInParallelReadBufferIfSupported( + ReadBuffer & buf, ThreadPoolCallbackRunner schedule, size_t max_working_readers, + size_t range_step, size_t file_size); + } diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index bf44d9d10da..67bc01279c3 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -50,30 +50,30 @@ std::string ReadBufferFromFileDescriptor::getFileName() const } -bool ReadBufferFromFileDescriptor::nextImpl() +size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset) { - /// If internal_buffer size is empty, then read() cannot be distinguished from EOF - assert(!internal_buffer.empty()); + chassert(min_bytes <= max_bytes); - /// This is a workaround of a read pass EOF bug in linux kernel with pread() - if (file_size.has_value() && file_offset_of_buffer_end >= *file_size) - return false; + /// This is a workaround of a read past EOF bug in linux kernel with pread() + if (file_size.has_value() && offset >= *file_size) + return 0; size_t bytes_read = 0; - while (!bytes_read) + while (bytes_read < min_bytes) { ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorRead); Stopwatch watch(profile_callback ? clock_type : CLOCK_MONOTONIC); ssize_t res = 0; + size_t to_read = max_bytes - bytes_read; { CurrentMetrics::Increment metric_increment{CurrentMetrics::Read}; if (use_pread) - res = ::pread(fd, internal_buffer.begin(), internal_buffer.size(), file_offset_of_buffer_end); + res = ::pread(fd, to + bytes_read, to_read, offset + bytes_read); else - res = ::read(fd, internal_buffer.begin(), internal_buffer.size()); + res = ::read(fd, to + bytes_read, to_read); } if (!res) break; @@ -102,18 +102,31 @@ bool ReadBufferFromFileDescriptor::nextImpl() if (profile_callback) { ProfileInfo info; - info.bytes_requested = internal_buffer.size(); + info.bytes_requested = to_read; info.bytes_read = res; info.nanoseconds = watch.elapsed(); profile_callback(info); } } + if (bytes_read) + ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadBytes, bytes_read); + + return bytes_read; +} + + +bool ReadBufferFromFileDescriptor::nextImpl() +{ + /// If internal_buffer size is empty, then read() cannot be distinguished from EOF + assert(!internal_buffer.empty()); + + size_t bytes_read = readImpl(internal_buffer.begin(), 1, internal_buffer.size(), file_offset_of_buffer_end); + file_offset_of_buffer_end += bytes_read; if (bytes_read) { - ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadBytes, bytes_read); working_buffer = internal_buffer; working_buffer.resize(bytes_read); } @@ -259,4 +272,17 @@ size_t ReadBufferFromFileDescriptor::getFileSize() return getSizeFromFileDescriptor(fd, getFileName()); } +bool ReadBufferFromFileDescriptor::checkIfActuallySeekable() +{ + struct stat stat; + auto res = ::fstat(fd, &stat); + return res == 0 && S_ISREG(stat.st_mode); +} + +size_t ReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) +{ + chassert(use_pread); + return readImpl(to, n, n, offset); +} + } diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index 10f140275bb..64340770cf2 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -30,6 +30,12 @@ protected: /// Name or some description of file. std::string getFileName() const override; + /// Does the read()/pread(), with all the metric increments, error handling, throttling, etc. + /// Doesn't seek (`offset` must match fd's position if !use_pread). + /// Stops after min_bytes or eof. Returns 0 if eof. + /// Thread safe. + size_t readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset); + public: explicit ReadBufferFromFileDescriptor( int fd_, @@ -65,6 +71,11 @@ public: size_t getFileSize() override; + bool checkIfActuallySeekable() override; + + size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) override; + bool supportsReadAt() override { return use_pread; } + private: /// Assuming file descriptor supports 'select', check that we have data to read or wait until timeout. bool poll(size_t timeout_microseconds) const; diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 93e2c46b080..d1cb1ec9ab0 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -109,9 +109,12 @@ bool ReadBufferFromS3::nextImpl() } size_t sleep_time_with_backoff_milliseconds = 100; - for (size_t attempt = 0; attempt < request_settings.max_single_read_retries && !next_result; ++attempt) + for (size_t attempt = 0; !next_result; ++attempt) { - Stopwatch watch; + bool last_attempt = attempt + 1 >= request_settings.max_single_read_retries; + + ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); + try { if (!impl) @@ -133,44 +136,11 @@ bool ReadBufferFromS3::nextImpl() /// Try to read a next portion of data. next_result = impl->next(); - watch.stop(); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Microseconds, watch.elapsedMicroseconds()); break; } catch (Exception & e) { - watch.stop(); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Microseconds, watch.elapsedMicroseconds()); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3RequestsErrors, 1); - - if (auto * s3_exception = dynamic_cast(&e)) - { - /// It doesn't make sense to retry Access Denied or No Such Key - if (!s3_exception->isRetryableError()) - { - s3_exception->addMessage("while reading key: {}, from bucket: {}", key, bucket); - throw; - } - } - - /// It doesn't make sense to retry allocator errors - if (e.code() == ErrorCodes::CANNOT_ALLOCATE_MEMORY) - { - tryLogCurrentException(log); - throw; - } - - LOG_DEBUG( - log, - "Caught exception while reading S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}, Attempt: {}, Message: {}", - bucket, - key, - version_id.empty() ? "Latest" : version_id, - getPosition(), - attempt, - e.message()); - - if (attempt + 1 == request_settings.max_single_read_retries) + if (!processException(e, getPosition(), attempt) || last_attempt) throw; /// Pause before next attempt. @@ -197,6 +167,74 @@ bool ReadBufferFromS3::nextImpl() } +size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) +{ + if (n == 0) + return 0; + + size_t sleep_time_with_backoff_milliseconds = 100; + for (size_t attempt = 0;; ++attempt) + { + bool last_attempt = attempt + 1 >= request_settings.max_single_read_retries; + + ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); + + try + { + auto result = sendRequest(range_begin, range_begin + n - 1); + std::istream & istr = result.GetBody(); + + size_t bytes = copyFromIStreamWithProgressCallback(istr, to, n, progress_callback); + + ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Bytes, bytes); + + if (read_settings.remote_throttler) + read_settings.remote_throttler->add(bytes, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); + + return bytes; + } + catch (Poco::Exception & e) + { + if (!processException(e, range_begin, attempt) || last_attempt) + throw; + + sleepForMilliseconds(sleep_time_with_backoff_milliseconds); + sleep_time_with_backoff_milliseconds *= 2; + } + } +} + +bool ReadBufferFromS3::processException(Poco::Exception & e, size_t read_offset, size_t attempt) const +{ + ProfileEvents::increment(ProfileEvents::ReadBufferFromS3RequestsErrors, 1); + + LOG_DEBUG( + log, + "Caught exception while reading S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}, " + "Attempt: {}, Message: {}", + bucket, key, version_id.empty() ? "Latest" : version_id, read_offset, attempt, e.message()); + + if (auto * s3_exception = dynamic_cast(&e)) + { + /// It doesn't make sense to retry Access Denied or No Such Key + if (!s3_exception->isRetryableError()) + { + s3_exception->addMessage("while reading key: {}, from bucket: {}", key, bucket); + return false; + } + } + + /// It doesn't make sense to retry allocator errors + if (e.code() == ErrorCodes::CANNOT_ALLOCATE_MEMORY) + { + tryLogCurrentException(log); + return false; + } + + return true; +} + + off_t ReadBufferFromS3::seek(off_t offset_, int whence) { if (offset_ == getPosition() && whence == SEEK_SET) @@ -315,44 +353,40 @@ bool ReadBufferFromS3::atEndOfRequestedRangeGuess() std::unique_ptr ReadBufferFromS3::initialize() { - S3::GetObjectRequest req; - req.SetBucket(bucket); - req.SetKey(key); - if (!version_id.empty()) - { - req.SetVersionId(version_id); - } - /** * If remote_filesystem_read_method = 'threadpool', then for MergeTree family tables * exact byte ranges to read are always passed here. */ - if (read_until_position) - { - if (offset >= read_until_position) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); + if (read_until_position && offset >= read_until_position) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); - req.SetRange(fmt::format("bytes={}-{}", offset, read_until_position - 1)); - LOG_TEST( - log, - "Read S3 object. Bucket: {}, Key: {}, Version: {}, Range: {}-{}", - bucket, - key, - version_id.empty() ? "Latest" : version_id, - offset, - read_until_position - 1); - } - else + read_result = sendRequest(offset, read_until_position ? std::make_optional(read_until_position - 1) : std::nullopt); + + size_t buffer_size = use_external_buffer ? 0 : read_settings.remote_fs_buffer_size; + return std::make_unique(read_result.GetBody(), buffer_size); +} + +Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t range_begin, std::optional range_end_incl) const +{ + S3::GetObjectRequest req; + req.SetBucket(bucket); + req.SetKey(key); + if (!version_id.empty()) + req.SetVersionId(version_id); + + if (range_end_incl) { - if (offset) - req.SetRange(fmt::format("bytes={}-", offset)); + req.SetRange(fmt::format("bytes={}-{}", range_begin, *range_end_incl)); LOG_TEST( - log, - "Read S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}", - bucket, - key, - version_id.empty() ? "Latest" : version_id, - offset); + log, "Read S3 object. Bucket: {}, Key: {}, Version: {}, Range: {}-{}", + bucket, key, version_id.empty() ? "Latest" : version_id, range_begin, *range_end_incl); + } + else if (range_begin) + { + req.SetRange(fmt::format("bytes={}-", range_begin)); + LOG_TEST( + log, "Read S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}", + bucket, key, version_id.empty() ? "Latest" : version_id, range_begin); } ProfileEvents::increment(ProfileEvents::S3GetObject); @@ -371,9 +405,7 @@ std::unique_ptr ReadBufferFromS3::initialize() { ResourceCost bytes_read = outcome.GetResult().GetContentLength(); read_settings.resource_link.adjust(estimated_cost, bytes_read); - size_t buffer_size = use_external_buffer ? 0 : read_settings.remote_fs_buffer_size; - read_result = outcome.GetResultWithOwnership(); - return std::make_unique(read_result.GetBody(), buffer_size); + return outcome.GetResultWithOwnership(); } else { @@ -383,21 +415,6 @@ std::unique_ptr ReadBufferFromS3::initialize() } } -std::unique_ptr ReadBufferS3Factory::getReader() -{ - return std::make_unique( - client_ptr, - bucket, - key, - version_id, - request_settings, - read_settings.adjustBufferSize(object_size)); -} - -size_t ReadBufferS3Factory::getFileSize() -{ - return object_size; -} } #endif diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 52dd74bdd14..0f665861a1e 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -77,12 +77,22 @@ public: String getFileName() const override { return bucket + "/" + key; } + size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) override; + + bool supportsReadAt() override { return true; } + private: std::unique_ptr initialize(); - // If true, if we destroy impl now, no work was wasted. Just for metrics. + /// If true, if we destroy impl now, no work was wasted. Just for metrics. bool atEndOfRequestedRangeGuess(); + /// Call inside catch() block if GetObject fails. Bumps metrics, logs the error. + /// Returns true if the error looks retriable. + bool processException(Poco::Exception & e, size_t read_offset, size_t attempt) const; + + Aws::S3::Model::GetObjectResult sendRequest(size_t range_begin, std::optional range_end_incl) const; + ReadSettings read_settings; bool use_external_buffer; @@ -92,43 +102,6 @@ private: bool restricted_seek; }; -/// Creates separate ReadBufferFromS3 for sequence of ranges of particular object -class ReadBufferS3Factory : public SeekableReadBufferFactory, public WithFileName -{ -public: - explicit ReadBufferS3Factory( - std::shared_ptr client_ptr_, - const String & bucket_, - const String & key_, - const String & version_id_, - size_t object_size_, - const S3Settings::RequestSettings & request_settings_, - const ReadSettings & read_settings_) - : client_ptr(client_ptr_) - , bucket(bucket_) - , key(key_) - , version_id(version_id_) - , read_settings(read_settings_) - , object_size(object_size_) - , request_settings(request_settings_) - {} - - std::unique_ptr getReader() override; - - size_t getFileSize() override; - - String getFileName() const override { return bucket + "/" + key; } - -private: - std::shared_ptr client_ptr; - const String bucket; - const String key; - const String version_id; - ReadSettings read_settings; - size_t object_size; - const S3Settings::RequestSettings request_settings; -}; - } #endif diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 8c3ab704d2b..cf1159bfb4b 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -40,6 +40,12 @@ void UpdatableSession::updateSession(const Poco::URI & uri) throw Exception(ErrorCodes::TOO_MANY_REDIRECTS, "Too many redirects while trying to access {}", initial_uri.toString()); } +template +typename UpdatableSession::SessionPtr UpdatableSession::createDetachedSession(const Poco::URI & uri) +{ + return session_factory->buildNewSession(uri); +} + template std::shared_ptr> UpdatableSession::clone(const Poco::URI & uri) { @@ -89,21 +95,11 @@ bool ReadWriteBufferFromHTTPBase::withPartialContent(const } template -size_t ReadWriteBufferFromHTTPBase::getRangeBegin() const { return read_range.begin.value_or(0); } +size_t ReadWriteBufferFromHTTPBase::getOffset() const { return read_range.begin.value_or(0) + offset_from_begin_pos; } template -size_t ReadWriteBufferFromHTTPBase::getOffset() const { return getRangeBegin() + offset_from_begin_pos; } - -template -std::istream * ReadWriteBufferFromHTTPBase::callImpl( - UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, - const std::string & method_, bool for_object_info) +void ReadWriteBufferFromHTTPBase::prepareRequest(Poco::Net::HTTPRequest & request, Poco::URI uri_, std::optional range) const { - // With empty path poco will send "POST HTTP/1.1" its bug. - if (uri_.getPath().empty()) - uri_.setPath("/"); - - Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(uri_.getHost()); // use original, not resolved host name in header if (out_stream_callback) @@ -111,16 +107,9 @@ std::istream * ReadWriteBufferFromHTTPBase::callImpl( else if (method == Poco::Net::HTTPRequest::HTTP_POST) request.setContentLength(0); /// No callback - no body - for (auto & [header, value] : http_header_entries) + for (const auto & [header, value] : http_header_entries) request.set(header, value); - std::optional range; - if (!for_object_info) - { - if (withPartialContent(read_range)) - range = HTTPRange{getOffset(), read_range.end}; - } - if (range) { String range_header_value; @@ -134,6 +123,25 @@ std::istream * ReadWriteBufferFromHTTPBase::callImpl( if (!credentials.getUsername().empty()) credentials.authenticate(request); +} + +template +std::istream * ReadWriteBufferFromHTTPBase::callImpl( + UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_, bool for_object_info) +{ + // With empty path poco will send "POST HTTP/1.1" its bug. + if (uri_.getPath().empty()) + uri_.setPath("/"); + + std::optional range; + if (!for_object_info) + { + if (withPartialContent(read_range)) + range = HTTPRange{getOffset(), read_range.end}; + } + + Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + prepareRequest(request, uri_, range); LOG_TRACE(log, "Sending request to {}", uri_.toString()); @@ -176,6 +184,14 @@ size_t ReadWriteBufferFromHTTPBase::getFileSize() throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for: {}", uri.toString()); } +template +bool ReadWriteBufferFromHTTPBase::supportsReadAt() +{ + if (!file_info) + file_info = getFileInfo(); + return method == Poco::Net::HTTPRequest::HTTP_GET && file_info->seekable; +} + template bool ReadWriteBufferFromHTTPBase::checkIfActuallySeekable() { @@ -405,7 +421,7 @@ void ReadWriteBufferFromHTTPBase::initialize() { /// We could have range.begin == 0 and range.end != 0 in case of DiskWeb and failing to read with partial content /// will affect only performance, so a warning is enough. - LOG_WARNING(log, "Unable to read with range header: [{}, {}]", getRangeBegin(), *read_range.end); + LOG_WARNING(log, "Unable to read with range header: [{}, {}]", read_range.begin.value_or(0), *read_range.end); } } @@ -538,8 +554,8 @@ bool ReadWriteBufferFromHTTPBase::nextImpl() throw; /** Retry request unconditionally if nothing has been read yet. - * Otherwise if it is GET method retry with range header. - */ + * Otherwise if it is GET method retry with range header. + */ bool can_retry_request = !offset_from_begin_pos || method == Poco::Net::HTTPRequest::HTTP_GET; if (!can_retry_request) throw; @@ -574,6 +590,83 @@ bool ReadWriteBufferFromHTTPBase::nextImpl() return true; } +template +size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) +{ + /// Caller must have checked supportsReadAt(). + /// This ensures we've sent at least one HTTP request and populated saved_uri_redirect. + chassert(file_info && file_info->seekable); + + if (n == 0) + return 0; + + Poco::URI uri_ = saved_uri_redirect.value_or(uri); + if (uri_.getPath().empty()) + uri_.setPath("/"); + + size_t milliseconds_to_wait = settings.http_retry_initial_backoff_ms; + + for (size_t attempt = 0;; ++attempt) + { + bool last_attempt = attempt + 1 >= settings.http_max_tries; + + Poco::Net::HTTPRequest request(method, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + prepareRequest(request, uri_, HTTPRange { .begin = offset, .end = offset + n - 1}); + + LOG_TRACE(log, "Sending request to {} for range [{}, {})", uri_.toString(), offset, offset + n); + + auto sess = session->createDetachedSession(uri_); + + Poco::Net::HTTPResponse response; + std::istream * result_istr; + + try + { + sess->sendRequest(request); + result_istr = receiveResponse(*sess, request, response, /*allow_redirects*/ false); + + if (response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_PARTIAL_CONTENT && + (offset != 0 || offset + n < *file_info->file_size)) + throw Exception( + ErrorCodes::HTTP_RANGE_NOT_SATISFIABLE, + "Expected 206 Partial Content, got {} when reading {} range [{}, {})", + toString(response.getStatus()), uri_.toString(), offset, offset + n); + + bool cancelled; + size_t r = copyFromIStreamWithProgressCallback(*result_istr, to, n, progress_callback, &cancelled); + + return r; + } + catch (const Poco::Exception & e) + { + sess->attachSessionData(e.message()); + + LOG_ERROR( + log, + "HTTP request (positioned) to `{}` with range [{}, {}) failed at try {}/{}: {}", + uri_.toString(), offset, offset + n, attempt + 1, settings.http_max_tries, + e.what()); + + /// Decide whether to retry. + + if (last_attempt) + throw; + + /// Too many open files - non-retryable. + if (e.code() == POCO_EMFILE) + throw; + + if (const auto * h = dynamic_cast(&e); + h && !isRetriableError(static_cast(h->getHTTPStatus()))) + throw; + + sleepForMilliseconds(milliseconds_to_wait); + milliseconds_to_wait = std::min(milliseconds_to_wait * 2, settings.http_retry_max_backoff_ms); + continue; + } + } +} + template off_t ReadWriteBufferFromHTTPBase::getPosition() { return getOffset() - available(); } @@ -793,75 +886,6 @@ ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP( skip_not_found_url_, file_info_) {} -RangedReadWriteBufferFromHTTPFactory::RangedReadWriteBufferFromHTTPFactory( - Poco::URI uri_, - std::string method_, - OutStreamCallback out_stream_callback_, - ConnectionTimeouts timeouts_, - const Poco::Net::HTTPBasicCredentials & credentials_, - UInt64 max_redirects_, - size_t buffer_size_, - ReadSettings settings_, - HTTPHeaderEntries http_header_entries_, - const RemoteHostFilter * remote_host_filter_, - bool delay_initialization_, - bool use_external_buffer_, - bool skip_not_found_url_) - : uri(uri_) - , method(std::move(method_)) - , out_stream_callback(out_stream_callback_) - , timeouts(std::move(timeouts_)) - , credentials(credentials_) - , max_redirects(max_redirects_) - , buffer_size(buffer_size_) - , settings(std::move(settings_)) - , http_header_entries(std::move(http_header_entries_)) - , remote_host_filter(remote_host_filter_) - , delay_initialization(delay_initialization_) - , use_external_buffer(use_external_buffer_) - , skip_not_found_url(skip_not_found_url_) {} - -std::unique_ptr RangedReadWriteBufferFromHTTPFactory::getReader() -{ - return std::make_unique( - uri, - method, - out_stream_callback, - timeouts, - credentials, - max_redirects, - buffer_size, - settings, - http_header_entries, - remote_host_filter, - delay_initialization, - use_external_buffer, - skip_not_found_url, - file_info); -} - -size_t RangedReadWriteBufferFromHTTPFactory::getFileSize() -{ - auto s = getFileInfo().file_size; - if (!s) - throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for: {}", uri.toString()); - return *s; -} - -bool RangedReadWriteBufferFromHTTPFactory::checkIfActuallySeekable() -{ - return getFileInfo().seekable; -} - -HTTPFileInfo RangedReadWriteBufferFromHTTPFactory::getFileInfo() -{ - if (!file_info) - file_info = static_cast(getReader().get())->getFileInfo(); - return *file_info; -} - -String RangedReadWriteBufferFromHTTPFactory::getFileName() const { return uri.toString(); } - PooledSessionFactory::PooledSessionFactory( const ConnectionTimeouts & timeouts_, size_t per_endpoint_pool_size_) @@ -891,6 +915,7 @@ PooledReadWriteBufferFromHTTP::PooledReadWriteBufferFromHTTP( out_stream_callback_, buffer_size_) {} + template class UpdatableSession; template class UpdatableSession; template class detail::ReadWriteBufferFromHTTPBase>>; diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index de1946ced48..2d2ae5fe724 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -42,6 +42,9 @@ public: void updateSession(const Poco::URI & uri); + /// Thread safe. + SessionPtr createDetachedSession(const Poco::URI & uri); + std::shared_ptr> clone(const Poco::URI & uri); private: @@ -110,14 +113,16 @@ namespace detail bool withPartialContent(const HTTPRange & range) const; - size_t getRangeBegin() const; - size_t getOffset() const; + void prepareRequest(Poco::Net::HTTPRequest & request, Poco::URI uri_, std::optional range) const; + std::istream * callImpl(UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_, bool for_object_info = false); size_t getFileSize() override; + bool supportsReadAt() override; + bool checkIfActuallySeekable() override; String getFileName() const override; @@ -171,6 +176,8 @@ namespace detail bool nextImpl() override; + size_t readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) override; + off_t getPosition() override; off_t seek(off_t offset_, int whence) override; @@ -237,53 +244,6 @@ public: std::optional file_info_ = std::nullopt); }; -class RangedReadWriteBufferFromHTTPFactory : public SeekableReadBufferFactory, public WithFileName -{ - using OutStreamCallback = ReadWriteBufferFromHTTP::OutStreamCallback; - -public: - RangedReadWriteBufferFromHTTPFactory( - Poco::URI uri_, - std::string method_, - OutStreamCallback out_stream_callback_, - ConnectionTimeouts timeouts_, - const Poco::Net::HTTPBasicCredentials & credentials_, - UInt64 max_redirects_ = 0, - size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, - ReadSettings settings_ = {}, - HTTPHeaderEntries http_header_entries_ = {}, - const RemoteHostFilter * remote_host_filter_ = nullptr, - bool delay_initialization_ = true, - bool use_external_buffer_ = false, - bool skip_not_found_url_ = false); - - std::unique_ptr getReader() override; - - size_t getFileSize() override; - - bool checkIfActuallySeekable() override; - - HTTPFileInfo getFileInfo(); - - String getFileName() const override; - -private: - Poco::URI uri; - std::string method; - OutStreamCallback out_stream_callback; - ConnectionTimeouts timeouts; - const Poco::Net::HTTPBasicCredentials & credentials; - UInt64 max_redirects; - size_t buffer_size; - ReadSettings settings; - HTTPHeaderEntries http_header_entries; - const RemoteHostFilter * remote_host_filter; - std::optional file_info; - bool delay_initialization; - bool use_external_buffer; - bool skip_not_found_url; -}; - class PooledSessionFactory { public: @@ -292,7 +252,9 @@ public: using SessionType = PooledHTTPSessionPtr; + /// Thread safe. SessionType buildNewSession(const Poco::URI & uri); + private: ConnectionTimeouts timeouts; size_t per_endpoint_pool_size; @@ -315,6 +277,7 @@ public: size_t max_connections_per_endpoint = DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT); }; + extern template class UpdatableSession; extern template class UpdatableSession; extern template class detail::ReadWriteBufferFromHTTPBase>>; diff --git a/src/IO/SeekableReadBuffer.cpp b/src/IO/SeekableReadBuffer.cpp index 99c43d6671b..b83e382db01 100644 --- a/src/IO/SeekableReadBuffer.cpp +++ b/src/IO/SeekableReadBuffer.cpp @@ -3,6 +3,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_READ_FROM_ISTREAM; +} namespace { @@ -60,4 +64,46 @@ std::unique_ptr wrapSeekableReadBufferPointer(SeekableReadBu return std::make_unique>(*ptr, SeekableReadBufferPtr{ptr}); } +size_t copyFromIStreamWithProgressCallback(std::istream & istr, char * to, size_t n, const std::function & progress_callback, bool * out_cancelled) +{ + const size_t chunk = DBMS_DEFAULT_BUFFER_SIZE; + if (out_cancelled) + *out_cancelled = false; + + size_t copied = 0; + while (copied < n) + { + size_t to_copy = std::min(chunk, n - copied); + istr.read(to + copied, to_copy); + size_t gcount = istr.gcount(); + + copied += gcount; + + bool cancelled = false; + if (gcount && progress_callback) + cancelled = progress_callback(copied); + + if (gcount != to_copy) + { + if (!istr.eof()) + throw Exception( + ErrorCodes::CANNOT_READ_FROM_ISTREAM, + "{} at offset {}", + istr.fail() ? "Cannot read from istream" : "Unexpected state of istream", + copied); + + break; + } + + if (cancelled) + { + if (out_cancelled != nullptr) + *out_cancelled = true; + break; + } + } + + return copied; +} + } diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index 736ab5bbc71..8ced9d752de 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -59,39 +59,41 @@ public: /// * Sometimes when we create such buffer we don't know in advance whether we'll need it to be /// seekable or not. So we don't want to pay the price for this check in advance. virtual bool checkIfActuallySeekable() { return true; } + + /// Unbuffered positional read. + /// Doesn't affect the buffer state (position, working_buffer, etc). + /// + /// `progress_callback` may be called periodically during the read, reporting that to[0..m-1] + /// has been filled. If it returns true, reading is stopped, and readBigAt() returns bytes read + /// so far. Called only from inside readBigAt(), from the same thread, with increasing m. + /// + /// Stops either after n bytes, or at end of file, or on exception. Returns number of bytes read. + /// If offset is past the end of file, may return 0 or throw exception. + /// + /// Caller needs to be careful: + /// * supportsReadAt() must be checked (called and return true) before calling readBigAt(). + /// Otherwise readBigAt() may crash. + /// * Thread safety: multiple readBigAt() calls may be performed in parallel. + /// But readBigAt() may not be called in parallel with any other methods + /// (e.g. next() or supportsReadAt()). + /// * Performance: there's no buffering. Each readBigAt() call typically translates into actual + /// IO operation (e.g. HTTP request). Don't use it for small adjacent reads. + virtual size_t readBigAt(char * /*to*/, size_t /*n*/, size_t /*offset*/, const std::function & /*progress_callback*/ = nullptr) + { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method readBigAt() not implemented"); } + + /// Checks if readBigAt() is allowed. May be slow, may throw (e.g. it may do an HTTP request or an fstat). + virtual bool supportsReadAt() { return false; } }; -/// Useful for reading in parallel. -/// The created read buffers may outlive the factory. -/// -/// There are 2 ways to use this: -/// (1) Never call seek() or getFileSize(), read the file sequentially. -/// For HTTP, this usually translates to just one HTTP request. -/// (2) Call checkIfActuallySeekable(), then: -/// a. If it returned false, go to (1). seek() and getFileSize() are not available (throw if called). -/// b. If it returned true, seek() and getFileSize() are available, knock yourself out. -/// For HTTP, checkIfActuallySeekable() sends a HEAD request and returns false if the web server -/// doesn't support ranges (or doesn't support HEAD requests). -class SeekableReadBufferFactory : public WithFileSize -{ -public: - ~SeekableReadBufferFactory() override = default; - - // We usually call setReadUntilPosition() and seek() on the returned buffer before reading. - // So it's recommended that the returned implementation be lazy, i.e. don't start reading - // before the first call to nextImpl(). - virtual std::unique_ptr getReader() = 0; - - virtual bool checkIfActuallySeekable() { return true; } -}; using SeekableReadBufferPtr = std::shared_ptr; -using SeekableReadBufferFactoryPtr = std::unique_ptr; - /// Wraps a reference to a SeekableReadBuffer into an unique pointer to SeekableReadBuffer. /// This function is like wrapReadBufferReference() but for SeekableReadBuffer. std::unique_ptr wrapSeekableReadBufferReference(SeekableReadBuffer & ref); std::unique_ptr wrapSeekableReadBufferPointer(SeekableReadBufferPtr ptr); +/// Helper for implementing readBigAt(). +size_t copyFromIStreamWithProgressCallback(std::istream & istr, char * to, size_t n, const std::function & progress_callback, bool * out_cancelled = nullptr); + } diff --git a/src/IO/WithFileName.cpp b/src/IO/WithFileName.cpp index 9d9f264c861..2383182f7e7 100644 --- a/src/IO/WithFileName.cpp +++ b/src/IO/WithFileName.cpp @@ -19,7 +19,7 @@ String getFileNameFromReadBuffer(const ReadBuffer & in) if (const auto * compressed = dynamic_cast(&in)) return getFileName(compressed->getWrappedReadBuffer()); else if (const auto * parallel = dynamic_cast(&in)) - return getFileName(parallel->getReadBufferFactory()); + return getFileName(parallel->getReadBuffer()); else if (const auto * peekable = dynamic_cast(&in)) return getFileNameFromReadBuffer(peekable->getSubBuffer()); else diff --git a/src/IO/WithFileSize.cpp b/src/IO/WithFileSize.cpp index f71690fcdee..28542db7a73 100644 --- a/src/IO/WithFileSize.cpp +++ b/src/IO/WithFileSize.cpp @@ -33,10 +33,6 @@ size_t getFileSizeFromReadBuffer(ReadBuffer & in) { return getFileSize(compressed->getWrappedReadBuffer()); } - else if (auto * parallel = dynamic_cast(&in)) - { - return getFileSize(parallel->getReadBufferFactory()); - } return getFileSize(in); } @@ -51,10 +47,6 @@ bool isBufferWithFileSize(const ReadBuffer & in) { return isBufferWithFileSize(compressed->getWrappedReadBuffer()); } - else if (const auto * parallel = dynamic_cast(&in)) - { - return dynamic_cast(¶llel->getReadBufferFactory()) != nullptr; - } return dynamic_cast(&in) != nullptr; } diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp index 54f3b76ff60..37505f94e98 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp @@ -146,45 +146,19 @@ arrow::Status ArrowInputStreamFromReadBuffer::Close() return arrow::Status(); } -RandomAccessFileFromManyReadBuffers::RandomAccessFileFromManyReadBuffers(SeekableReadBufferFactory & factory) : buf_factory(factory) {} +RandomAccessFileFromRandomAccessReadBuffer::RandomAccessFileFromRandomAccessReadBuffer(SeekableReadBuffer & in_, size_t file_size_) : in(in_), file_size(file_size_) {} -arrow::Result RandomAccessFileFromManyReadBuffers::GetSize() +arrow::Result RandomAccessFileFromRandomAccessReadBuffer::GetSize() { - return buf_factory.getFileSize(); + return file_size; } -arrow::Result RandomAccessFileFromManyReadBuffers::ReadAt(int64_t position, int64_t nbytes, void* out) +arrow::Result RandomAccessFileFromRandomAccessReadBuffer::ReadAt(int64_t position, int64_t nbytes, void* out) { - std::unique_lock lock(mutex); - if (free_bufs.empty()) - free_bufs.push_back(buf_factory.getReader()); - auto buf = std::move(free_bufs.back()); - free_bufs.pop_back(); - lock.unlock(); - - // To work well with this, ReadBuffer implementations need to respect setReadUntilPosition() and - // not read above it. We often do very small reads here. - // Also nice if they: - // * Make readBig() read directly into the provided memory, instead of copying from internal - // buffer. - // * Allocate the internal buffer (if any) lazily in first nextImpl() call. If all reads are - // tiny readBig() calls (as is typical here), it won't allocate an unnecessary 1 MB buffer. - - buf->seek(position, SEEK_SET); - buf->setReadUntilPosition(position + nbytes); - size_t bytes_read = buf->readBig(reinterpret_cast(out), nbytes); - - // Seeking to a position above a previous setReadUntilPosition() confuses some of the - // ReadBuffer implementations. So we reset it before next seek. - buf->setReadUntilEnd(); - - lock.lock(); - free_bufs.push_back(std::move(buf)); - - return static_cast(bytes_read); + return in.readBigAt(reinterpret_cast(out), nbytes, position); } -arrow::Result> RandomAccessFileFromManyReadBuffers::ReadAt(int64_t position, int64_t nbytes) +arrow::Result> RandomAccessFileFromRandomAccessReadBuffer::ReadAt(int64_t position, int64_t nbytes) { ARROW_ASSIGN_OR_RAISE(auto buffer, arrow::AllocateResizableBuffer(nbytes)) ARROW_ASSIGN_OR_RAISE(int64_t bytes_read, ReadAt(position, nbytes, buffer->mutable_data())) @@ -195,22 +169,23 @@ arrow::Result> RandomAccessFileFromManyReadBuffer return buffer; } -arrow::Future> RandomAccessFileFromManyReadBuffers::ReadAsync(const arrow::io::IOContext&, int64_t position, int64_t nbytes) +arrow::Future> RandomAccessFileFromRandomAccessReadBuffer::ReadAsync(const arrow::io::IOContext&, int64_t position, int64_t nbytes) { return arrow::Future>::MakeFinished(ReadAt(position, nbytes)); } -arrow::Status RandomAccessFileFromManyReadBuffers::Close() +arrow::Status RandomAccessFileFromRandomAccessReadBuffer::Close() { chassert(is_open); is_open = false; return arrow::Status::OK(); } -arrow::Status RandomAccessFileFromManyReadBuffers::Seek(int64_t) { return arrow::Status::NotImplemented(""); } -arrow::Result RandomAccessFileFromManyReadBuffers::Tell() const { return arrow::Status::NotImplemented(""); } -arrow::Result RandomAccessFileFromManyReadBuffers::Read(int64_t, void*) { return arrow::Status::NotImplemented(""); } -arrow::Result> RandomAccessFileFromManyReadBuffers::Read(int64_t) { return arrow::Status::NotImplemented(""); } +arrow::Status RandomAccessFileFromRandomAccessReadBuffer::Seek(int64_t) { return arrow::Status::NotImplemented(""); } +arrow::Result RandomAccessFileFromRandomAccessReadBuffer::Tell() const { return arrow::Status::NotImplemented(""); } +arrow::Result RandomAccessFileFromRandomAccessReadBuffer::Read(int64_t, void*) { return arrow::Status::NotImplemented(""); } +arrow::Result> RandomAccessFileFromRandomAccessReadBuffer::Read(int64_t) { return arrow::Status::NotImplemented(""); } + std::shared_ptr asArrowFile( ReadBuffer & in, @@ -220,19 +195,16 @@ std::shared_ptr asArrowFile( const std::string & magic_bytes, bool avoid_buffering) { - if (auto * fd_in = dynamic_cast(&in)) + bool has_file_size = isBufferWithFileSize(in); + auto * seekable_in = dynamic_cast(&in); + + if (has_file_size && seekable_in && settings.seekable_read) { - struct stat stat; - auto res = ::fstat(fd_in->getFD(), &stat); - // if fd is a regular file i.e. not stdin - if (res == 0 && S_ISREG(stat.st_mode)) - return std::make_shared(*fd_in, stat.st_size, avoid_buffering); - } - else if (auto * seekable_in = dynamic_cast(&in); - seekable_in && settings.seekable_read && isBufferWithFileSize(in) && - seekable_in->checkIfActuallySeekable()) - { - return std::make_shared(in, std::nullopt, avoid_buffering); + if (avoid_buffering && seekable_in->supportsReadAt()) + return std::make_shared(*seekable_in, getFileSizeFromReadBuffer(in)); + + if (seekable_in->checkIfActuallySeekable()) + return std::make_shared(*seekable_in, std::nullopt, avoid_buffering); } // fallback to loading the entire file in memory @@ -245,26 +217,16 @@ std::shared_ptr asArrowFileLoadIntoMemory( const std::string & format_name, const std::string & magic_bytes) { - std::string file_data; - { - PeekableReadBuffer buf(in); - std::string magic_bytes_from_data; - magic_bytes_from_data.resize(magic_bytes.size()); - bool read_magic_bytes = false; - try - { - PeekableReadBufferCheckpoint checkpoint(buf, true); - buf.readStrict(magic_bytes_from_data.data(), magic_bytes_from_data.size()); - read_magic_bytes = true; - } - catch (const Exception &) {} + std::string file_data(magic_bytes.size(), '\0'); - if (!read_magic_bytes || magic_bytes_from_data != magic_bytes) - throw Exception(ErrorCodes::INCORRECT_DATA, "Not a {} file", format_name); + /// Avoid loading the whole file if it doesn't seem to even be in the correct format. + size_t bytes_read = in.read(file_data.data(), magic_bytes.size()); + if (bytes_read < magic_bytes.size() || file_data != magic_bytes) + throw Exception(ErrorCodes::INCORRECT_DATA, "Not a {} file", format_name); - WriteBufferFromString file_buffer(file_data); - copyData(buf, file_buffer, is_cancelled); - } + WriteBufferFromString file_buffer(file_data, AppendModeTag{}); + copyData(in, file_buffer, is_cancelled); + file_buffer.finalize(); return std::make_shared(arrow::Buffer::FromString(std::move(file_data))); } diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.h b/src/Processors/Formats/Impl/ArrowBufferedStreams.h index 9307172cb11..f455bcdfb1a 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.h +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.h @@ -18,7 +18,6 @@ class ReadBuffer; class WriteBuffer; class SeekableReadBuffer; -class SeekableReadBufferFactory; struct FormatSettings; class ArrowBufferedOutputStream : public arrow::io::OutputStream @@ -78,19 +77,17 @@ private: ARROW_DISALLOW_COPY_AND_ASSIGN(RandomAccessFileFromSeekableReadBuffer); }; -// Thread-safe. -// Maintains a pool of SeekableReadBuffer-s. For each ReadAt(), takes a buffer, seeks it, and reads. -class RandomAccessFileFromManyReadBuffers : public arrow::io::RandomAccessFile +class RandomAccessFileFromRandomAccessReadBuffer : public arrow::io::RandomAccessFile { public: - explicit RandomAccessFileFromManyReadBuffers(SeekableReadBufferFactory & factory); + explicit RandomAccessFileFromRandomAccessReadBuffer(SeekableReadBuffer & in_, size_t file_size_); // These are thread safe. arrow::Result GetSize() override; arrow::Result ReadAt(int64_t position, int64_t nbytes, void* out) override; arrow::Result> ReadAt(int64_t position, int64_t nbytes) override; - arrow::Future> ReadAsync(const arrow::io::IOContext&, int64_t position, - int64_t nbytes) override; + arrow::Future> ReadAsync( + const arrow::io::IOContext&, int64_t position, int64_t nbytes) override; // These are not thread safe, and arrow shouldn't call them. Return NotImplemented error. arrow::Status Seek(int64_t) override; @@ -102,13 +99,11 @@ public: bool closed() const override { return !is_open; } private: - SeekableReadBufferFactory & buf_factory; + SeekableReadBuffer & in; + size_t file_size; bool is_open = true; - std::mutex mutex; - std::vector> free_bufs; - - ARROW_DISALLOW_COPY_AND_ASSIGN(RandomAccessFileFromManyReadBuffers); + ARROW_DISALLOW_COPY_AND_ASSIGN(RandomAccessFileFromRandomAccessReadBuffer); }; class ArrowInputStreamFromReadBuffer : public arrow::io::InputStream diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 7f90c1197ce..2f3c68aa481 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -43,14 +43,12 @@ namespace ErrorCodes } while (false) ParquetBlockInputFormat::ParquetBlockInputFormat( - ReadBuffer * buf, - SeekableReadBufferFactoryPtr buf_factory_, + ReadBuffer & buf, const Block & header_, const FormatSettings & format_settings_, size_t max_decoding_threads_, size_t min_bytes_for_seek_) - : IInputFormat(header_, buf) - , buf_factory(std::move(buf_factory_)) + : IInputFormat(header_, &buf) , format_settings(format_settings_) , skip_row_groups(format_settings.parquet.skip_row_groups) , max_decoding_threads(max_decoding_threads_) @@ -71,17 +69,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() // Create arrow file adapter. // TODO: Make the adapter do prefetching on IO threads, based on the full set of ranges that // we'll need to read (which we know in advance). Use max_download_threads for that. - if (buf_factory) - { - if (format_settings.seekable_read && buf_factory->checkIfActuallySeekable()) - arrow_file = std::make_shared(*buf_factory); - else - arrow_file = asArrowFileLoadIntoMemory(*buf_factory->getReader(), is_stopped, "Parquet", PARQUET_MAGIC_BYTES); - } - else - { - arrow_file = asArrowFile(*in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); - } + arrow_file = asArrowFile(*in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); if (is_stopped) return; @@ -388,7 +376,7 @@ ParquetSchemaReader::ParquetSchemaReader(ReadBuffer & in_, const FormatSettings NamesAndTypesList ParquetSchemaReader::readSchema() { std::atomic is_stopped{0}; - auto file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES); + auto file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); auto metadata = parquet::ReadMetaData(file); @@ -406,8 +394,7 @@ void registerInputFormatParquet(FormatFactory & factory) { factory.registerRandomAccessInputFormat( "Parquet", - [](ReadBuffer * buf, - SeekableReadBufferFactoryPtr buf_factory, + [](ReadBuffer & buf, const Block & sample, const FormatSettings & settings, const ReadSettings& read_settings, @@ -418,7 +405,6 @@ void registerInputFormatParquet(FormatFactory & factory) size_t min_bytes_for_seek = is_remote_fs ? read_settings.remote_read_min_bytes_for_seek : 8 * 1024; return std::make_shared( buf, - std::move(buf_factory), sample, settings, max_parsing_threads, diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index f17eee59414..ad7074547fc 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -15,7 +15,6 @@ namespace DB { class ArrowColumnToCHColumn; -class SeekableReadBufferFactory; // Parquet files contain a metadata block with the following information: // * list of columns, @@ -48,9 +47,7 @@ class ParquetBlockInputFormat : public IInputFormat { public: ParquetBlockInputFormat( - // exactly one of these two is nullptr - ReadBuffer * buf, - std::unique_ptr buf_factory, + ReadBuffer & buf, const Block & header, const FormatSettings & format_settings, size_t max_decoding_threads, @@ -234,7 +231,6 @@ private: }; }; - std::unique_ptr buf_factory; const FormatSettings format_settings; const std::unordered_set & skip_row_groups; size_t max_decoding_threads; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 8380fa26a39..2d8aaec0f07 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -582,31 +582,11 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() size_t object_size = info ? info->size : S3::getObjectSize(*client, bucket, current_key, version_id, request_settings); auto compression_method = chooseCompressionMethod(current_key, compression_hint); - InputFormatPtr input_format; - std::unique_ptr owned_read_buf; - - auto read_buf_or_factory = createS3ReadBuffer(current_key, object_size); - if (read_buf_or_factory.buf_factory) - { - input_format = FormatFactory::instance().getInputRandomAccess( - format, - std::move(read_buf_or_factory.buf_factory), - sample_block, - getContext(), - max_block_size, - /* is_remote_fs */ true, - compression_method, - format_settings); - } - else - { - owned_read_buf = wrapReadBufferWithCompressionMethod( - std::move(read_buf_or_factory.buf), - compression_method, - static_cast(getContext()->getSettingsRef().zstd_window_log_max)); - input_format = FormatFactory::instance().getInput( - format, *owned_read_buf, sample_block, getContext(), max_block_size, format_settings); - } + auto read_buf = createS3ReadBuffer(current_key, object_size); + auto input_format = FormatFactory::instance().getInput( + format, *read_buf, sample_block, getContext(), max_block_size, + format_settings, std::nullopt, std::nullopt, + /* is_remote_fs */ true, compression_method); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); @@ -621,7 +601,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); - return ReaderHolder{fs::path(bucket) / current_key, std::move(owned_read_buf), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{fs::path(bucket) / current_key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3Source::createReaderAsync() @@ -629,7 +609,7 @@ std::future StorageS3Source::createReaderAsync() return create_reader_scheduler([this] { return createReader(); }, Priority{}); } -StorageS3Source::ReadBufferOrFactory StorageS3Source::createS3ReadBuffer(const String & key, size_t object_size) +std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & key, size_t object_size) { auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); read_settings.enable_filesystem_cache = false; @@ -642,12 +622,13 @@ StorageS3Source::ReadBufferOrFactory StorageS3Source::createS3ReadBuffer(const S if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { LOG_TRACE(log, "Downloading object of size {} from S3 with initial prefetch", object_size); - return {.buf = createAsyncS3ReadBuffer(key, read_settings, object_size)}; + return createAsyncS3ReadBuffer(key, read_settings, object_size); } - auto factory = std::make_unique( - client, bucket, key, version_id, object_size, request_settings, read_settings); - return {.buf_factory = std::move(factory)}; + return std::make_unique( + client, bucket, key, version_id, request_settings, read_settings, + /*use_external_buffer*/ false, /*offset_*/ 0, /*read_until_position_*/ 0, + /*restricted_seek_*/ false, object_size); } std::unique_ptr StorageS3Source::createAsyncS3ReadBuffer( diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 66652a45e3a..a4c120b99a6 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -204,12 +204,6 @@ private: std::unique_ptr reader; }; - struct ReadBufferOrFactory - { - std::unique_ptr buf; - SeekableReadBufferFactoryPtr buf_factory; - }; - ReaderHolder reader; std::vector requested_virtual_columns; @@ -230,7 +224,7 @@ private: ReaderHolder createReader(); std::future createReaderAsync(); - ReadBufferOrFactory createS3ReadBuffer(const String & key, size_t object_size); + std::unique_ptr createS3ReadBuffer(const String & key, size_t object_size); std::unique_ptr createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size); }; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 4c5ed08e26e..efc44a069dd 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -248,7 +248,7 @@ StorageURLSource::StorageURLSource( throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list"); auto first_option = uri_options.begin(); - auto [actual_uri, buf_factory] = getFirstAvailableURIAndReadBuffer( + auto [actual_uri, buf] = getFirstAvailableURIAndReadBuffer( first_option, uri_options.end(), context, @@ -262,10 +262,11 @@ StorageURLSource::StorageURLSource( uri_options.size() == 1); curr_uri = actual_uri; + read_buf = std::move(buf); try { - total_size += buf_factory->getFileSize(); + total_size += getFileSizeFromReadBuffer(*read_buf); } catch (...) { @@ -273,16 +274,17 @@ StorageURLSource::StorageURLSource( } // TODO: Pass max_parsing_threads and max_download_threads adjusted for num_streams. - auto input_format = FormatFactory::instance().getInputRandomAccess( + auto input_format = FormatFactory::instance().getInput( format, - std::move(buf_factory), + *read_buf, sample_block, context, max_block_size, - /* is_remote_fs */ true, - compression_method, format_settings, - download_threads); + download_threads, + /*max_download_threads*/ std::nullopt, + /* is_remote_fs */ true, + compression_method); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); @@ -348,7 +350,7 @@ Chunk StorageURLSource::generate() return {}; } -std::tuple StorageURLSource::getFirstAvailableURIAndReadBuffer( +std::tuple> StorageURLSource::getFirstAvailableURIAndReadBuffer( std::vector::const_iterator & option, const std::vector::const_iterator & end, ContextPtr context, @@ -376,40 +378,38 @@ std::tuple StorageURLSource::getFirstAv setCredentials(credentials, request_uri); const auto settings = context->getSettings(); - auto res = std::make_unique( - request_uri, - http_method, - callback, - timeouts, - credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - read_settings, - headers, - &context->getRemoteHostFilter(), - delay_initialization, - /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error); - if (options > 1) + try { - // Send a HEAD request to check availability. - try - { - res->getFileInfo(); - } - catch (...) - { - if (first_exception_message.empty()) - first_exception_message = getCurrentExceptionMessage(false); + auto res = std::make_unique( + request_uri, + http_method, + callback, + timeouts, + credentials, + settings.max_http_get_redirects, + settings.max_read_buffer_size, + read_settings, + headers, + &context->getRemoteHostFilter(), + delay_initialization, + /* use_external_buffer */ false, + /* skip_url_not_found_error */ skip_url_not_found_error); - tryLogCurrentException(__PRETTY_FUNCTION__); - - continue; - } + return std::make_tuple(request_uri, std::move(res)); } + catch (...) + { + if (options == 1) + throw; - return std::make_tuple(request_uri, std::move(res)); + if (first_exception_message.empty()) + first_exception_message = getCurrentExceptionMessage(false); + + tryLogCurrentException(__PRETTY_FUNCTION__); + + continue; + } } throw Exception(ErrorCodes::NETWORK_ERROR, "All uri ({}) options are unreachable: {}", options, first_exception_message); @@ -598,7 +598,7 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( if (it == urls_to_check.cend()) return nullptr; - auto [_, buf_factory] = StorageURLSource::getFirstAvailableURIAndReadBuffer( + auto [_, buf] = StorageURLSource::getFirstAvailableURIAndReadBuffer( it, urls_to_check.cend(), context, @@ -612,7 +612,7 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( false); ++it; return wrapReadBufferWithCompressionMethod( - buf_factory->getReader(), + std::move(buf), compression_method, static_cast(context->getSettingsRef().zstd_window_log_max)); }; diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index d53b72105e4..316b142aec0 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -183,7 +183,7 @@ public: static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); - static std::tuple getFirstAvailableURIAndReadBuffer( + static std::tuple> getFirstAvailableURIAndReadBuffer( std::vector::const_iterator & option, const std::vector::const_iterator & end, ContextPtr context, @@ -205,6 +205,7 @@ private: std::shared_ptr uri_iterator; Poco::URI curr_uri; + std::unique_ptr read_buf; std::unique_ptr pipeline; std::unique_ptr reader; diff --git a/tests/integration/test_redirect_url_storage/test.py b/tests/integration/test_redirect_url_storage/test.py index 06ff78707d7..b2178655444 100644 --- a/tests/integration/test_redirect_url_storage/test.py +++ b/tests/integration/test_redirect_url_storage/test.py @@ -151,7 +151,7 @@ def test_url_reconnect(started_cluster): result = node1.query( "select sum(cityHash64(id)) from url('http://hdfs1:50075/webhdfs/v1/storage_big?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'id Int32') settings http_max_tries = 10, http_retry_max_backoff_ms=1000" ) - assert (int(result), 6581218782194912115) + assert int(result) == 6581218782194912115 thread = threading.Thread(target=select) thread.start() @@ -161,5 +161,5 @@ def test_url_reconnect(started_cluster): thread.join() - assert (int(result), 6581218782194912115) + assert int(result) == 6581218782194912115 assert node1.contains_in_log("Timeout: connect timed out") From 16be379fa55c6dc8172004460799be93b7a52b88 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Jun 2023 11:18:46 +0200 Subject: [PATCH 237/308] Update src/Common/AsynchronousMetrics.cpp --- src/Common/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index a4cb18249b6..7064559800a 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -950,7 +950,7 @@ void AsynchronousMetrics::update(TimePoint update_time) auto space = line.find_first_of(" "); - if (line.rfind("max", 0) == std::string::npos) + if (line.rfind("max", space) == std::string::npos) { auto field1 = line.substr(0, space); quota = std::stoull(field1); From 57a2bfd0ff59f3ad4c5f6966c08ae7ddb6f0dda2 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Jun 2023 11:19:11 +0200 Subject: [PATCH 238/308] Update src/Common/AsynchronousMetrics.cpp --- src/Common/AsynchronousMetrics.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 7064559800a..cf8d451385b 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -956,8 +956,11 @@ void AsynchronousMetrics::update(TimePoint update_time) quota = std::stoull(field1); } - auto field2 = line.substr(space + 1); - period = std::stoull(field2); + if (space != std::string::npos) + { + auto field2 = line.substr(space + 1); + period = std::stoull(field2); + } new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."}; new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."}; From f6da66cbb87d0c7c54c9b885cb5efeb0c330f801 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 18:27:34 +0000 Subject: [PATCH 239/308] Fix some typos --- docs/en/sql-reference/functions/files.md | 8 +- .../functions/functions-for-nulls.md | 117 ++-- .../functions/logical-functions.md | 24 +- .../functions/other-functions.md | 549 +++++++----------- .../functions/random-functions.md | 181 ++++-- 5 files changed, 442 insertions(+), 437 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 5cd2d8e0a74..73d72aa50e5 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -6,7 +6,7 @@ sidebar_label: Files ## file -Reads file as string and loads the data into the specified column. The actual file content is not interpreted. +Reads a file as string and loads the data into the specified column. The file content is not interpreted. Also see table function [file](../table-functions/file.md). @@ -18,15 +18,13 @@ file(path[, default]) **Arguments** -- `path` — The path of the file relative to [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports the following wildcards: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` are numbers and `'abc', 'def'` are strings. -- `default` — The value that will be returned in the case the file does not exist or cannot be accessed. Supported data types: [String](../../sql-reference/data-types/string.md) and [NULL](../../sql-reference/syntax.md#null-literal). +- `path` — The path of the file relative to [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports wildcards `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` are numbers and `'abc', 'def'` are strings. +- `default` — The value returned if the file does not exist or cannot be accessed. Supported data types: [String](../../sql-reference/data-types/string.md) and [NULL](../../sql-reference/syntax.md#null-literal). **Example** Inserting data from files a.txt and b.txt into a table as strings: -Query: - ``` sql INSERT INTO table SELECT file('a.txt'), file('b.txt'); ``` diff --git a/docs/en/sql-reference/functions/functions-for-nulls.md b/docs/en/sql-reference/functions/functions-for-nulls.md index 0b7df54b776..f4ca27e9b16 100644 --- a/docs/en/sql-reference/functions/functions-for-nulls.md +++ b/docs/en/sql-reference/functions/functions-for-nulls.md @@ -8,7 +8,7 @@ sidebar_label: Nullable ## isNull -Checks whether the argument is [NULL](../../sql-reference/syntax.md#null-literal). +Returns whether the argument is [NULL](../../sql-reference/syntax.md#null-literal). ``` sql isNull(x) @@ -18,7 +18,7 @@ Alias: `ISNULL`. **Arguments** -- `x` — A value with a non-compound data type. +- `x` — A value of non-compound data type. **Returned value** @@ -27,7 +27,7 @@ Alias: `ISNULL`. **Example** -Input table +Table: ``` text ┌─x─┬────y─┐ @@ -36,12 +36,14 @@ Input table └───┴──────┘ ``` -Query +Query: ``` sql SELECT x FROM t_null WHERE isNull(y); ``` +Result: + ``` text ┌─x─┐ │ 1 │ @@ -50,7 +52,7 @@ SELECT x FROM t_null WHERE isNull(y); ## isNotNull -Checks whether the argument is [NULL](../../sql-reference/syntax.md#null-literal). +Returns whether the argument is not [NULL](../../sql-reference/syntax.md#null-literal). ``` sql isNotNull(x) @@ -58,16 +60,16 @@ isNotNull(x) **Arguments:** -- `x` — A value with a non-compound data type. +- `x` — A value of non-compound data type. **Returned value** -- `0` if `x` is `NULL`. - `1` if `x` is not `NULL`. +- `0` if `x` is `NULL`. **Example** -Input table +Table: ``` text ┌─x─┬────y─┐ @@ -76,12 +78,14 @@ Input table └───┴──────┘ ``` -Query +Query: ``` sql SELECT x FROM t_null WHERE isNotNull(y); ``` +Result: + ``` text ┌─x─┐ │ 2 │ @@ -90,7 +94,7 @@ SELECT x FROM t_null WHERE isNotNull(y); ## coalesce -Checks from left to right whether `NULL` arguments were passed and returns the first non-`NULL` argument. +Returns the leftmost non-`NULL` argument. ``` sql coalesce(x,...) @@ -98,11 +102,11 @@ coalesce(x,...) **Arguments:** -- Any number of parameters of a non-compound type. All parameters must be compatible by data type. +- Any number of parameters of non-compound type. All parameters must be of mutually compatible data types. **Returned values** -- The first non-`NULL` argument. +- The first non-`NULL` argument - `NULL`, if all arguments are `NULL`. **Example** @@ -110,10 +114,10 @@ coalesce(x,...) Consider a list of contacts that may specify multiple ways to contact a customer. ``` text -┌─name─────┬─mail─┬─phone─────┬──icq─┐ -│ client 1 │ ᴺᵁᴸᴸ │ 123-45-67 │ 123 │ -│ client 2 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -└──────────┴──────┴───────────┴──────┘ +┌─name─────┬─mail─┬─phone─────┬──telegram─┐ +│ client 1 │ ᴺᵁᴸᴸ │ 123-45-67 │ 123 │ +│ client 2 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +└──────────┴──────┴───────────┴───────────┘ ``` The `mail` and `phone` fields are of type String, but the `icq` field is `UInt32`, so it needs to be converted to `String`. @@ -121,22 +125,22 @@ The `mail` and `phone` fields are of type String, but the `icq` field is `UInt32 Get the first available contact method for the customer from the contact list: ``` sql -SELECT name, coalesce(mail, phone, CAST(icq,'Nullable(String)')) FROM aBook; +SELECT name, coalesce(mail, phone, CAST(telegram,'Nullable(String)')) FROM aBook; ``` ``` text -┌─name─────┬─coalesce(mail, phone, CAST(icq, 'Nullable(String)'))─┐ -│ client 1 │ 123-45-67 │ -│ client 2 │ ᴺᵁᴸᴸ │ -└──────────┴──────────────────────────────────────────────────────┘ +┌─name─────┬─coalesce(mail, phone, CAST(telegram, 'Nullable(String)'))─┐ +│ client 1 │ 123-45-67 │ +│ client 2 │ ᴺᵁᴸᴸ │ +└──────────┴───────────────────────────────────────────────────────────┘ ``` ## ifNull -Returns an alternative value if the main argument is `NULL`. +Returns an alternative value if the argument is `NULL`. ``` sql -ifNull(x,alt) +ifNull(x, alt) ``` **Arguments:** @@ -146,25 +150,33 @@ ifNull(x,alt) **Returned values** -- The value `x`, if `x` is not `NULL`. -- The value `alt`, if `x` is `NULL`. +- `x` if `x` is not `NULL`. +- `alt` if `x` is `NULL`. **Example** +Query: + ``` sql SELECT ifNull('a', 'b'); ``` +Result: + ``` text ┌─ifNull('a', 'b')─┐ │ a │ └──────────────────┘ ``` +Query: + ``` sql SELECT ifNull(NULL, 'b'); ``` +Result: + ``` text ┌─ifNull(NULL, 'b')─┐ │ b │ @@ -173,7 +185,7 @@ SELECT ifNull(NULL, 'b'); ## nullIf -Returns `NULL` if the arguments are equal. +Returns `NULL` if both arguments are equal. ``` sql nullIf(x, y) @@ -181,29 +193,37 @@ nullIf(x, y) **Arguments:** -`x`, `y` — Values for comparison. They must be compatible types, or ClickHouse will generate an exception. +`x`, `y` — Values to compare. Must be of compatible types. **Returned values** -- `NULL`, if the arguments are equal. -- The `x` value, if the arguments are not equal. +- `NULL` if the arguments are equal. +- `x` if the arguments are not equal. **Example** +Query: + ``` sql SELECT nullIf(1, 1); ``` +Result: + ``` text ┌─nullIf(1, 1)─┐ │ ᴺᵁᴸᴸ │ └──────────────┘ ``` +Query: + ``` sql SELECT nullIf(1, 2); ``` +Result: + ``` text ┌─nullIf(1, 2)─┐ │ 1 │ @@ -212,7 +232,7 @@ SELECT nullIf(1, 2); ## assumeNotNull -Results in an equivalent non-`Nullable` value for a [Nullable](../../sql-reference/data-types/nullable.md) type. In case the original value is `NULL` the result is undetermined. See also `ifNull` and `coalesce` functions. +Returns the corresponding non-`Nullable` value for a value of [Nullable](../../sql-reference/data-types/nullable.md) type. If the original value is `NULL`, an arbitrary result can be returned. See also functions `ifNull` and `coalesce`. ``` sql assumeNotNull(x) @@ -224,36 +244,29 @@ assumeNotNull(x) **Returned values** -- The original value from the non-`Nullable` type, if it is not `NULL`. -- Implementation specific result if the original value was `NULL`. +- The input value as non-`Nullable` type, if it is not `NULL`. +- An arbirary value, if the input value is `NULL`. **Example** -Consider the `t_null` table. - -``` sql -SHOW CREATE TABLE t_null; -``` +Table: ``` text -┌─statement─────────────────────────────────────────────────────────────────┐ -│ CREATE TABLE default.t_null ( x Int8, y Nullable(Int8)) ENGINE = TinyLog │ -└───────────────────────────────────────────────────────────────────────────┘ -``` -``` text ┌─x─┬────y─┐ │ 1 │ ᴺᵁᴸᴸ │ │ 2 │ 3 │ └───┴──────┘ ``` -Apply the `assumeNotNull` function to the `y` column. +Query: ``` sql -SELECT assumeNotNull(y) FROM t_null; +SELECT assumeNotNull(y) FROM table; ``` +Result: + ``` text ┌─assumeNotNull(y)─┐ │ 0 │ @@ -261,10 +274,14 @@ SELECT assumeNotNull(y) FROM t_null; └──────────────────┘ ``` +Query: + ``` sql SELECT toTypeName(assumeNotNull(y)) FROM t_null; ``` +Result: + ``` text ┌─toTypeName(assumeNotNull(y))─┐ │ Int8 │ @@ -282,28 +299,36 @@ toNullable(x) **Arguments:** -- `x` — The value of any non-compound type. +- `x` — A value of non-compound type. **Returned value** -- The input value with a `Nullable` type. +- The input value but of `Nullable` type. **Example** +Query: + ``` sql SELECT toTypeName(10); ``` +Result: + ``` text ┌─toTypeName(10)─┐ │ UInt8 │ └────────────────┘ ``` +Query: + ``` sql SELECT toTypeName(toNullable(10)); ``` +Result: + ``` text ┌─toTypeName(toNullable(10))─┐ │ Nullable(UInt8) │ diff --git a/docs/en/sql-reference/functions/logical-functions.md b/docs/en/sql-reference/functions/logical-functions.md index f5a1a6aac12..2e1a8f28227 100644 --- a/docs/en/sql-reference/functions/logical-functions.md +++ b/docs/en/sql-reference/functions/logical-functions.md @@ -12,7 +12,9 @@ Zero as an argument is considered `false`, non-zero values are considered `true` ## and -Calculates the logical conjunction between two or more values. +Calculates the logical conjunction of two or more values. + +Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `(val_1 AND val_2 AND ... AND val_{i-1})` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT and(number = 2, intDiv(1, number)) FROM numbers(5)`. **Syntax** @@ -20,9 +22,7 @@ Calculates the logical conjunction between two or more values. and(val1, val2...) ``` -Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `(val_1 AND val_2 AND ... AND val_{i-1})` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT and(number = 2, intDiv(1, number)) FROM numbers(5)`. - -Alias: The [AND Operator](../../sql-reference/operators/index.md#logical-and-operator). +Alias: The [AND operator](../../sql-reference/operators/index.md#logical-and-operator). **Arguments** @@ -30,7 +30,7 @@ Alias: The [AND Operator](../../sql-reference/operators/index.md#logical-and-ope **Returned value** -- `0`, if there at least one argument evaluates to `false`, +- `0`, if at least one argument evaluates to `false`, - `NULL`, if no argumetn evaluates to `false` and at least one argument is `NULL`, - `1`, otherwise. @@ -66,7 +66,9 @@ Result: ## or -Calculates the logical disjunction between two or more values. +Calculates the logical disjunction of two or more values. + +Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `((NOT val_1) AND (NOT val_2) AND ... AND (NOT val_{i-1}))` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT or(number = 0, intDiv(1, number) != 0) FROM numbers(5)`. **Syntax** @@ -74,9 +76,7 @@ Calculates the logical disjunction between two or more values. or(val1, val2...) ``` -Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `((NOT val_1) AND (NOT val_2) AND ... AND (NOT val_{i-1}))` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT or(number = 0, intDiv(1, number) != 0) FROM numbers(5)`. - -Alias: The [OR Operator](../../sql-reference/operators/index.md#logical-or-operator). +Alias: The [OR operator](../../sql-reference/operators/index.md#logical-or-operator). **Arguments** @@ -120,7 +120,7 @@ Result: ## not -Calculates logical negation of a value. +Calculates the logical negation of a value. **Syntax** @@ -128,7 +128,7 @@ Calculates logical negation of a value. not(val); ``` -Alias: The [Negation Operator](../../sql-reference/operators/index.md#logical-negation-operator). +Alias: The [Negation operator](../../sql-reference/operators/index.md#logical-negation-operator). **Arguments** @@ -158,7 +158,7 @@ Result: ## xor -Calculates the logical exclusive disjunction between two or more values. For more than two values the function first xor-s the first two values, then xor-s the result with the third value etc. +Calculates the logical exclusive disjunction of two or more values. For more than two input values, the function first xor-s the first two values, then xor-s the result with the third value etc. **Syntax** diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index efe1a77c285..037eb9e63c5 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -8,12 +8,12 @@ sidebar_label: Other ## hostName() -Returns a string with the name of the host that this function was performed on. For distributed processing, this is the name of the remote server host, if the function is performed on a remote server. -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +Returns the name of the host on which this function was executed. If the function executes on a remote server (distributed processing), the remote server name is returned. +If the function executes in the context of a distributed table, it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. ## getMacro -Gets a named value from the [macros](../../operations/server-configuration-parameters/settings.md#macros) section of the server configuration. +Returns a named value from the [macros](../../operations/server-configuration-parameters/settings.md#macros) section of the server configuration. **Syntax** @@ -23,7 +23,7 @@ getMacro(name); **Arguments** -- `name` — Name to retrieve from the `macros` section. [String](../../sql-reference/data-types/string.md#string). +- `name` — Macro name to retrieve from the `` section. [String](../../sql-reference/data-types/string.md#string). **Returned value** @@ -33,7 +33,7 @@ Type: [String](../../sql-reference/data-types/string.md). **Example** -The example `macros` section in the server configuration file: +Example `` section in the server configuration file: ``` xml @@ -55,7 +55,7 @@ Result: └──────────────────┘ ``` -An alternative way to get the same value: +The same value can be retrieved as follows: ``` sql SELECT * FROM system.macros @@ -70,7 +70,7 @@ WHERE macro = 'test'; ## FQDN -Returns the fully qualified domain name. +Returns the fully qualified domain name of the ClickHouse server. **Syntax** @@ -88,8 +88,6 @@ Type: `String`. **Example** -Query: - ``` sql SELECT FQDN(); ``` @@ -104,52 +102,61 @@ Result: ## basename -Extracts the trailing part of a string after the last slash or backslash. This function if often used to extract the filename from a path. +Extracts the tail of a string following its last slash or backslash. This function if often used to extract the filename from a path. ``` sql -basename( expr ) +basename(expr) ``` **Arguments** -- `expr` — Expression resulting in a [String](../../sql-reference/data-types/string.md) type value. All the backslashes must be escaped in the resulting value. +- `expr` — A value of type [String](../../sql-reference/data-types/string.md). Backslashes must be escaped. **Returned Value** A string that contains: -- The trailing part of a string after the last slash or backslash. - - If the input string contains a path ending with slash or backslash, for example, `/` or `c:\`, the function returns an empty string. - +- The tail of the input string after its last slash or backslash. If the input string ends with a slash or backslash (e.g. `/` or `c:\`), the function returns an empty string. - The original string if there are no slashes or backslashes. **Example** +Query: + ``` sql SELECT 'some/long/path/to/file' AS a, basename(a) ``` +Result: + ``` text ┌─a──────────────────────┬─basename('some\\long\\path\\to\\file')─┐ │ some\long\path\to\file │ file │ └────────────────────────┴────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT 'some\\long\\path\\to\\file' AS a, basename(a) ``` +Result: + ``` text ┌─a──────────────────────┬─basename('some\\long\\path\\to\\file')─┐ │ some\long\path\to\file │ file │ └────────────────────────┴────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT 'some-file-name' AS a, basename(a) ``` +Result: + ``` text ┌─a──────────────┬─basename('some-file-name')─┐ │ some-file-name │ some-file-name │ @@ -159,7 +166,7 @@ SELECT 'some-file-name' AS a, basename(a) ## visibleWidth(x) Calculates the approximate width when outputting values to the console in text format (tab-separated). -This function is used by the system for implementing Pretty formats. +This function is used by the system to implement Pretty formats. `NULL` is represented as a string corresponding to `NULL` in `Pretty` formats. @@ -175,18 +182,18 @@ SELECT visibleWidth(NULL) ## toTypeName(x) -Returns a string containing the type name of the passed argument. +Returns the type name of the passed argument. -If `NULL` is passed to the function as input, then it returns the `Nullable(Nothing)` type, which corresponds to an internal `NULL` representation in ClickHouse. +If `NULL` is passed, then the function returns type `Nullable(Nothing)`, which corresponds to ClickHouse's internal `NULL` representation. ## blockSize() -Gets the size of the block. -In ClickHouse, queries are always run on blocks (sets of column parts). This function allows getting the size of the block that you called it for. +In ClickHouse, queries are processed in blocks (chunks). +This function returns the size (row count) of the block the function is called on. ## byteSize -Returns estimation of uncompressed byte size of its arguments in memory. +Returns an estimation of uncompressed byte size of its arguments in memory. **Syntax** @@ -206,7 +213,7 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). **Examples** -For [String](../../sql-reference/data-types/string.md) arguments the funtion returns the string length + 9 (terminating zero + length). +For [String](../../sql-reference/data-types/string.md) arguments, the funtion returns the string length + 9 (terminating zero + length). Query: @@ -265,7 +272,7 @@ byteSize(Float32): 4 byteSize(Float64): 8 ``` -If the function takes multiple arguments, it returns their combined byte size. +If the function has multiple arguments, the function accumulates their byte sizes. Query: @@ -283,30 +290,30 @@ Result: ## materialize(x) -Turns a constant into a full column containing just one value. -In ClickHouse, full columns and constants are represented differently in memory. Functions work differently for constant arguments and normal arguments (different code is executed), although the result is almost always the same. This function is for debugging this behavior. +Turns a constant into a full column containing a single value. +Full columns and constants are represented differently in memory. Functions usually execute different code for normal and constant arguments, although the result should typically be the same. This function can be used to debug this behavior. ## ignore(…) -Accepts any arguments, including `NULL`. Always returns 0. -However, the argument is still evaluated. This can be used for benchmarks. +Accepts any arguments, including `NULL` and does nothing. Always returns 0. +The argument is internally still evaluated. Useful e.g. for benchmarks. ## sleep(seconds) -Sleeps ‘seconds’ seconds on each data block. You can specify an integer or a floating-point number. +Sleeps ‘seconds’ seconds for each data block. The sleep time can be specified as integer or as floating-point number. ## sleepEachRow(seconds) -Sleeps ‘seconds’ seconds on each row. You can specify an integer or a floating-point number. +Sleeps ‘seconds’ seconds for each row. The sleep time can be specified as integer or as floating-point number. ## currentDatabase() Returns the name of the current database. -You can use this function in table engine parameters in a CREATE TABLE query where you need to specify the database. +Useful in table engine parameters of `CREATE TABLE` queries where you need to specify the database. ## currentUser() -Returns the login of current user. Login of user, that initiated query, will be returned in case distributed query. +Returns the name of the current user. In case of a distributed query, the name of the user who initiated the query is returned. ``` sql SELECT currentUser(); @@ -316,15 +323,13 @@ Alias: `user()`, `USER()`. **Returned values** -- Login of current user. -- Login of user that initiated query in case of distributed query. +- The name of the current user. +- In distributed queries, the login of the user who initiated the query. Type: `String`. **Example** -Query: - ``` sql SELECT currentUser(); ``` @@ -339,11 +344,11 @@ Result: ## isConstant -Checks whether the argument is a constant expression. +Returns whether the argument is a constant expression. -A constant expression means an expression whose resulting value is known at the query analysis (i.e. before execution). For example, expressions over [literals](../../sql-reference/syntax.md#literals) are constant expressions. +A constant expression is an expression whose result is known during query analysis, i.e. before execution. For example, expressions over [literals](../../sql-reference/syntax.md#literals) are constant expressions. -The function is intended for development, debugging and demonstration. +This function is mostly intended for development, debugging and demonstration. **Syntax** @@ -357,8 +362,8 @@ isConstant(x) **Returned values** -- `1` — `x` is constant. -- `0` — `x` is non-constant. +- `1` if `x` is constant. +- `0` if `x` is non-constant. Type: [UInt8](../../sql-reference/data-types/int-uint.md). @@ -408,23 +413,25 @@ Result: ## isFinite(x) -Accepts Float32 and Float64 and returns UInt8 equal to 1 if the argument is not infinite and not a NaN, otherwise 0. +Returns 1 if the Float32 or Float64 argument not infinite and not a NaN, otherwise this function returns 0. ## isInfinite(x) -Accepts Float32 and Float64 and returns UInt8 equal to 1 if the argument is infinite, otherwise 0. Note that 0 is returned for a NaN. +Returns 1 if the Float32 or Float64 argument is infinite, otherwise this function returns 0. Note that 0 is returned for a NaN. ## ifNotFinite -Checks whether floating point value is finite. +Checks whether a floating point value is finite. **Syntax** - ifNotFinite(x,y) +``` sql +ifNotFinite(x,y) +``` **Arguments** -- `x` — Value to be checked for infinity. Type: [Float\*](../../sql-reference/data-types/float.md). +- `x` — Value to check for infinity. Type: [Float\*](../../sql-reference/data-types/float.md). - `y` — Fallback value. Type: [Float\*](../../sql-reference/data-types/float.md). **Returned value** @@ -444,23 +451,23 @@ Result: │ inf │ 42 │ └─────────┴───────────────────────────────┘ -You can get similar result by using [ternary operator](../../sql-reference/functions/conditional-functions.md#ternary-operator): `isFinite(x) ? x : y`. +You can get similar result by using the [ternary operator](../../sql-reference/functions/conditional-functions.md#ternary-operator): `isFinite(x) ? x : y`. ## isNaN(x) -Accepts Float32 and Float64 and returns UInt8 equal to 1 if the argument is a NaN, otherwise 0. +Returns 1 if the Float32 and Float64 argument is NaN, otherwise this function 0. ## hasColumnInTable(\[‘hostname’\[, ‘username’\[, ‘password’\]\],\] ‘database’, ‘table’, ‘column’) -Accepts constant strings: database name, table name, and column name. Returns a UInt8 constant expression equal to 1 if there is a column, otherwise 0. If the hostname parameter is set, the test will run on a remote server. -The function throws an exception if the table does not exist. +Given the database name, the table name, and the column name as constant strings, returns 1 if the given column exists, otherwise 0. If parameter `hostname` is given, the check is performed on a remote server. +If the table does not exist, an exception is thrown. For elements in a nested data structure, the function checks for the existence of a column. For the nested data structure itself, the function returns 0. ## bar -Allows building a unicode-art diagram. +Builds a bar chart. -`bar(x, min, max, width)` draws a band with a width proportional to `(x - min)` and equal to `width` characters when `x = max`. +`bar(x, min, max, width)` draws a band with width proportional to `(x - min)` and equal to `width` characters when `x = max`. **Arguments** @@ -520,23 +527,23 @@ There are two variations of this function: `x` – What to transform. -`array_from` – Constant array of values for converting. +`array_from` – Constant array of values to convert. `array_to` – Constant array of values to convert the values in ‘from’ to. `default` – Which value to use if ‘x’ is not equal to any of the values in ‘from’. -`array_from` and `array_to` – Arrays of the same size. +`array_from` and `array_to` must have equally many elements. -Types: +Signature: + +For `x` equal to one of the elements in `array_from`, the function returns the corresponding element in `array_to`, i.e. the one at the same array index. Otherwise, it returns `default`. If multiple matching elements exist `array_from`, an arbitrary corresponding element from `array_to` is returned. `transform(T, Array(T), Array(U), U) -> U` `T` and `U` can be numeric, string, or Date or DateTime types. -Where the same letter is indicated (T or U), for numeric types these might not be matching types, but types that have a common type. -For example, the first argument can have the Int64 type, while the second has the Array(UInt16) type. - -If the ‘x’ value is equal to one of the elements in the ‘array_from’ array, it returns the existing element (that is numbered the same) from the ‘array_to’ array. Otherwise, it returns ‘default’. If there are multiple matching elements in ‘array_from’, it returns one of the matches. +The same letter (T or U) means that types must be mutually compatible and not necessarily equal. +For example, the first argument could have type `Int64`, while the second argument could have type `Array(UInt16)`. Example: @@ -560,12 +567,7 @@ ORDER BY c DESC ### transform(x, array_from, array_to) -Differs from the first variation in that the ‘default’ argument is omitted. -If the ‘x’ value is equal to one of the elements in the ‘array_from’ array, it returns the matching element (that is numbered the same) from the ‘array_to’ array. Otherwise, it returns ‘x’. - -Types: - -`transform(T, Array(T), Array(T)) -> T` +Similar to the other variation but has no ‘default’ argument. In case no match can be found, `x` is returned. Example: @@ -595,7 +597,7 @@ LIMIT 10 ## formatReadableDecimalSize(x) -Accepts the size (number of bytes). Returns a rounded size with a suffix (KB, MB, etc.) as a string. +Given a size (number of bytes), this function returns a readable, rounded size with suffix (KB, MB, etc.) as string. Example: @@ -616,7 +618,7 @@ SELECT ## formatReadableSize(x) -Accepts the size (number of bytes). Returns a rounded size with a suffix (KiB, MiB, etc.) as a string. +Given a size (number of bytes), this function returns a readable, rounded size with suffix (KiB, MiB, etc.) as string. Example: @@ -637,9 +639,7 @@ SELECT ## formatReadableQuantity(x) -Accepts the number. Returns a rounded number with a suffix (thousand, million, billion, etc.) as a string. - -It is useful for reading big numbers by human. +Given a number, this function returns a rounded number with suffix (thousand, million, billion, etc.) as string. Example: @@ -660,7 +660,7 @@ SELECT ## formatReadableTimeDelta -Accepts the time delta in seconds. Returns a time delta with (year, month, day, hour, minute, second) as a string. +Given a time interval (delta) in seconds, this function returns a time delta with year/month/day/hour/minute/second as string. **Syntax** @@ -670,8 +670,8 @@ formatReadableTimeDelta(column[, maximum_unit]) **Arguments** -- `column` — A column with numeric time delta. -- `maximum_unit` — Optional. Maximum unit to show. Acceptable values seconds, minutes, hours, days, months, years. +- `column` — A column with a numeric time delta. +- `maximum_unit` — Optional. Maximum unit to show. Acceptable values `seconds`, `minutes`, `hours`, `days`, `months`, `years`. Example: @@ -746,33 +746,32 @@ SELECT parseTimeDelta('1yr2mo') ## least(a, b) -Returns the smallest value from a and b. +Returns the smaller value of a and b. ## greatest(a, b) -Returns the largest value of a and b. +Returns the larger value of a and b. ## uptime() Returns the server’s uptime in seconds. -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. ## version() -Returns the version of the server as a string. -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +Returns the server version as a string. +If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. ## buildId() Returns the build ID generated by a compiler for the running ClickHouse server binary. -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. - -## blockNumber +## blockNumber() Returns the sequence number of the data block where the row is located. -## rowNumberInBlock +## rowNumberInBlock() Returns the ordinal number of the row in the data block. Different data blocks are always recalculated. @@ -782,7 +781,7 @@ Returns the ordinal number of the row in the data block. This function only cons ## neighbor -The window function that provides access to a row at a specified offset which comes before or after the current row of a given column. +The window function that provides access to a row at a specified offset before or after the current row of a given column. **Syntax** @@ -792,23 +791,23 @@ neighbor(column, offset[, default_value]) The result of the function depends on the affected data blocks and the order of data in the block. -:::tip -It can reach the neighbor rows only inside the currently processed data block. +:::note +Only returns neighbor inside the currently processed data block. ::: -The rows order used during the calculation of `neighbor` can differ from the order of rows returned to the user. -To prevent that you can make a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. +The order of rows during calculation of `neighbor()` can differ from the order of rows returned to the user. +To prevent that you can create a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. **Arguments** - `column` — A column name or scalar expression. -- `offset` — The number of rows forwards or backwards from the current row of `column`. [Int64](../../sql-reference/data-types/int-uint.md). -- `default_value` — Optional. The value to be returned if offset goes beyond the scope of the block. Type of data blocks affected. +- `offset` — The number of rows to look before or ahead of the current row in `column`. [Int64](../../sql-reference/data-types/int-uint.md). +- `default_value` — Optional. The returned value if offset is beyond the block boundaries. Type of data blocks affected. **Returned values** -- Value for `column` in `offset` distance from current row if `offset` value is not outside block bounds. -- Default value for `column` if `offset` value is outside block bounds. If `default_value` is given, then it will be used. +- Value of `column` with `offset` distance from current row, if `offset` is not outside the block boundaries. +- The default value of `column` or `default_value` (if given), if `offset` is outside the block boundaries. Type: type of data blocks affected or default value type. @@ -899,17 +898,17 @@ Result: ## runningDifference(x) -Calculates the difference between successive row values ​​in the data block. -Returns 0 for the first row and the difference from the previous row for each subsequent row. +Calculates the difference between two consecutive row values in the data block. +Returns 0 for the first row, and for subsequent rows the difference to the previous row. -:::tip -It can reach the previous row only inside the currently processed data block. +:::note +Only returns differences inside the currently processed data block. ::: The result of the function depends on the affected data blocks and the order of data in the block. -The rows order used during the calculation of `runningDifference` can differ from the order of rows returned to the user. -To prevent that you can make a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. +The order of rows during calculation of `runningDifference()` can differ from the order of rows returned to the user. +To prevent that you can create a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. Example: @@ -940,7 +939,7 @@ FROM └─────────┴─────────────────────┴───────┘ ``` -Please note - block size affects the result. With each new block, the `runningDifference` state is reset. +Please note that the block size affects the result. The internal state of `runningDifference` state is reset for each new block. ``` sql SELECT @@ -977,7 +976,7 @@ WHERE diff != 1 ## runningDifferenceStartingWithFirstValue -Same as for [runningDifference](./other-functions.md#other_functions-runningdifference), the difference is the value of the first row, returned the value of the first row, and each subsequent row returns the difference from the previous row. +Same as [runningDifference](./other-functions.md#other_functions-runningdifference), but returns the value of the first row as the value on the first row. ## runningConcurrency @@ -1039,7 +1038,7 @@ Result: ## MACNumToString(num) -Accepts a UInt64 number. Interprets it as a MAC address in big endian. Returns a string containing the corresponding MAC address in the format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form). +Interprets a UInt64 number as a MAC address in big endian format. Returns the corresponding MAC address in format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form) as string. ## MACStringToNum(s) @@ -1047,11 +1046,12 @@ The inverse function of MACNumToString. If the MAC address has an invalid format ## MACStringToOUI(s) -Accepts a MAC address in the format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form). Returns the first three octets as a UInt64 number. If the MAC address has an invalid format, it returns 0. +Given a MAC address in format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form), returns the first three octets as a UInt64 number. If the MAC address has an invalid format, it returns 0. ## getSizeOfEnumType Returns the number of fields in [Enum](../../sql-reference/data-types/enum.md). +An exception is thrown if the type is not `Enum`. ``` sql getSizeOfEnumType(value) @@ -1064,7 +1064,6 @@ getSizeOfEnumType(value) **Returned values** - The number of fields with `Enum` input values. -- An exception is thrown if the type is not `Enum`. **Example** @@ -1080,7 +1079,7 @@ SELECT getSizeOfEnumType( CAST('a' AS Enum8('a' = 1, 'b' = 2) ) ) AS x ## blockSerializedSize -Returns size on disk (without taking into account compression). +Returns the size on disk without considering compression. ``` sql blockSerializedSize(value[, value[, ...]]) @@ -1092,7 +1091,7 @@ blockSerializedSize(value[, value[, ...]]) **Returned values** -- The number of bytes that will be written to disk for block of values (without compression). +- The number of bytes that will be written to disk for block of values without compression. **Example** @@ -1112,7 +1111,7 @@ Result: ## toColumnTypeName -Returns the name of the class that represents the data type of the column in RAM. +Returns the internal name of the data type that represents the value. ``` sql toColumnTypeName(value) @@ -1124,31 +1123,39 @@ toColumnTypeName(value) **Returned values** -- A string with the name of the class that is used for representing the `value` data type in RAM. +- The internal data type name used to represent `value`. -**Example of the difference between`toTypeName ' and ' toColumnTypeName`** +**Example** + +Difference between `toTypeName ' and ' toColumnTypeName`: ``` sql SELECT toTypeName(CAST('2018-01-01 01:02:03' AS DateTime)) ``` +Result: + ``` text ┌─toTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ │ DateTime │ └─────────────────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT toColumnTypeName(CAST('2018-01-01 01:02:03' AS DateTime)) ``` +Result: + ``` text ┌─toColumnTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ │ Const(UInt32) │ └───────────────────────────────────────────────────────────┘ ``` -The example shows that the `DateTime` data type is stored in memory as `Const(UInt32)`. +The example shows that the `DateTime` data type is internally stored as `Const(UInt32)`. ## dumpColumnStructure @@ -1164,7 +1171,7 @@ dumpColumnStructure(value) **Returned values** -- A string describing the structure that is used for representing the `value` data type in RAM. +- A description of the column structure used for representing `value`. **Example** @@ -1180,7 +1187,7 @@ SELECT dumpColumnStructure(CAST('2018-01-01 01:02:03', 'DateTime')) ## defaultValueOfArgumentType -Outputs the default value for the data type. +Returns the default value for the given data type. Does not include default values for custom columns set by the user. @@ -1200,20 +1207,28 @@ defaultValueOfArgumentType(expression) **Example** +Query: + ``` sql SELECT defaultValueOfArgumentType( CAST(1 AS Int8) ) ``` +Result: + ``` text ┌─defaultValueOfArgumentType(CAST(1, 'Int8'))─┐ │ 0 │ └─────────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT defaultValueOfArgumentType( CAST(1 AS Nullable(Int8) ) ) ``` +Result: + ``` text ┌─defaultValueOfArgumentType(CAST(1, 'Nullable(Int8)'))─┐ │ ᴺᵁᴸᴸ │ @@ -1222,7 +1237,7 @@ SELECT defaultValueOfArgumentType( CAST(1 AS Nullable(Int8) ) ) ## defaultValueOfTypeName -Outputs the default value for given type name. +Returns the default value for the given type name. Does not include default values for custom columns set by the user. @@ -1242,20 +1257,28 @@ defaultValueOfTypeName(type) **Example** +Query: + ``` sql SELECT defaultValueOfTypeName('Int8') ``` +Result: + ``` text ┌─defaultValueOfTypeName('Int8')─┐ │ 0 │ └────────────────────────────────┘ ``` +Query: + ``` sql SELECT defaultValueOfTypeName('Nullable(Int8)') ``` +Result: + ``` text ┌─defaultValueOfTypeName('Nullable(Int8)')─┐ │ ᴺᵁᴸᴸ │ @@ -1263,9 +1286,10 @@ SELECT defaultValueOfTypeName('Nullable(Int8)') ``` ## indexHint -The function is intended for debugging and introspection purposes. The function ignores it's argument and always returns 1. Arguments are not even evaluated. -But for the purpose of index analysis, the argument of this function is analyzed as if it was present directly without being wrapped inside `indexHint` function. This allows to select data in index ranges by the corresponding condition but without further filtering by this condition. The index in ClickHouse is sparse and using `indexHint` will yield more data than specifying the same condition directly. +This function is intended for debugging and introspection. It ignores its argument and always returns 1. The arguments are not evaluated. + +But during index analysis, the argument of this function is assumed to be not wrapped in `indexHint`. This allows to select data in index ranges by the corresponding condition but without further filtering by this condition. The index in ClickHouse is sparse and using `indexHint` will yield more data than specifying the same condition directly. **Syntax** @@ -1275,13 +1299,13 @@ SELECT * FROM table WHERE indexHint() **Returned value** -1. Type: [Uint8](https://clickhouse.com/docs/en/data_types/int_uint/#diapazony-uint). +Type: [Uint8](https://clickhouse.com/docs/en/data_types/int_uint/#diapazony-uint). **Example** Here is the example of test data from the table [ontime](../../getting-started/example-datasets/ontime.md). -Input table: +Table: ```sql SELECT count() FROM ontime @@ -1295,9 +1319,7 @@ SELECT count() FROM ontime The table has indexes on the fields `(FlightDate, (Year, FlightDate))`. -Create a query, where the index is not used. - -Query: +Create a query which does not use the index: ```sql SELECT FlightDate AS k, count() FROM ontime GROUP BY k ORDER BY k @@ -1318,15 +1340,13 @@ Result: └────────────┴─────────┘ ``` -To apply the index, select a specific date. - -Query: +To apply the index, select a specific date: ```sql SELECT FlightDate AS k, count() FROM ontime WHERE k = '2017-09-15' GROUP BY k ORDER BY k ``` -By using the index, ClickHouse processed a significantly smaller number of rows (`Processed 32.74 thousand rows`). +ClickHouse now uses the index to process a significantly smaller number of rows (`Processed 32.74 thousand rows`). Result: @@ -1336,7 +1356,7 @@ Result: └────────────┴─────────┘ ``` -Now wrap the expression `k = '2017-09-15'` into `indexHint` function. +Now wrap the expression `k = '2017-09-15'` in function `indexHint`: Query: @@ -1350,9 +1370,9 @@ GROUP BY k ORDER BY k ASC ``` -ClickHouse used the index in the same way as the previous time (`Processed 32.74 thousand rows`). +ClickHouse used the index the same way as previously (`Processed 32.74 thousand rows`). The expression `k = '2017-09-15'` was not used when generating the result. -In examle the `indexHint` function allows to see adjacent dates. +In example, the `indexHint` function allows to see adjacent dates. Result: @@ -1369,7 +1389,7 @@ Result: Creates an array with a single value. -Used for internal implementation of [arrayJoin](../../sql-reference/functions/array-join.md#functions_arrayjoin). +Used for the internal implementation of [arrayJoin](../../sql-reference/functions/array-join.md#functions_arrayjoin). ``` sql SELECT replicate(x, arr); @@ -1377,12 +1397,12 @@ SELECT replicate(x, arr); **Arguments:** -- `arr` — Original array. ClickHouse creates a new array of the same length as the original and fills it with the value `x`. -- `x` — The value that the resulting array will be filled with. +- `arr` — An array. +- `x` — The value to fill the result array with. **Returned value** -An array filled with the value `x`. +An array of the lame length as `arr` filled with value `x`. Type: `Array`. @@ -1404,7 +1424,7 @@ Result: ## filesystemAvailable -Returns amount of remaining space on the filesystem where the files of the databases located. It is always smaller than total free space ([filesystemFree](#filesystemfree)) because some space is reserved for OS. +Returns the amount of free space in the filesystem hosting the database persistence. The returned value is always smaller than total free space ([filesystemFree](#filesystemfree)) because some space is reserved for the operating system. **Syntax** @@ -1423,20 +1443,20 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT formatReadableSize(filesystemAvailable()) AS "Available space", toTypeName(filesystemAvailable()) AS "Type"; +SELECT formatReadableSize(filesystemAvailable()) AS "Available space"; ``` Result: ``` text -┌─Available space─┬─Type───┐ -│ 30.75 GiB │ UInt64 │ -└─────────────────┴────────┘ +┌─Available space─┐ +│ 30.75 GiB │ +└─────────────────┘ ``` ## filesystemFree -Returns total amount of the free space on the filesystem where the files of the databases located. See also `filesystemAvailable` +Returns the total amount of the free space on the filesystem hosting the database persistence. See also `filesystemAvailable` **Syntax** @@ -1446,7 +1466,7 @@ filesystemFree() **Returned value** -- Amount of free space in bytes. +- The amount of free space in bytes. Type: [UInt64](../../sql-reference/data-types/int-uint.md). @@ -1455,20 +1475,20 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT formatReadableSize(filesystemFree()) AS "Free space", toTypeName(filesystemFree()) AS "Type"; +SELECT formatReadableSize(filesystemFree()) AS "Free space"; ``` Result: ``` text -┌─Free space─┬─Type───┐ -│ 32.39 GiB │ UInt64 │ -└────────────┴────────┘ +┌─Free space─┐ +│ 32.39 GiB │ +└────────────┘ ``` ## filesystemCapacity -Returns the capacity of the filesystem in bytes. For evaluation, the [path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) to the data directory must be configured. +Returns the capacity of the filesystem in bytes. Needs the [path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) to the data directory to be configured. **Syntax** @@ -1478,7 +1498,7 @@ filesystemCapacity() **Returned value** -- Capacity information of the filesystem in bytes. +- Capacity of the filesystem in bytes. Type: [UInt64](../../sql-reference/data-types/int-uint.md). @@ -1487,20 +1507,20 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT formatReadableSize(filesystemCapacity()) AS "Capacity", toTypeName(filesystemCapacity()) AS "Type" +SELECT formatReadableSize(filesystemCapacity()) AS "Capacity"; ``` Result: ``` text -┌─Capacity──┬─Type───┐ -│ 39.32 GiB │ UInt64 │ -└───────────┴────────┘ +┌─Capacity──┐ +│ 39.32 GiB │ +└───────────┘ ``` ## initializeAggregation -Calculates result of aggregate function based on single value. It is intended to use this function to initialize aggregate functions with combinator [-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state). You can create states of aggregate functions and insert them to columns of type [AggregateFunction](../../sql-reference/data-types/aggregatefunction.md#data-type-aggregatefunction) or use initialized aggregates as default values. +Calculates the result of an aggregate function based on a single value. This function can be used to initialize aggregate functions with combinator [-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state). You can create states of aggregate functions and insert them to columns of type [AggregateFunction](../../sql-reference/data-types/aggregatefunction.md#data-type-aggregatefunction) or use initialized aggregates as default values. **Syntax** @@ -1539,6 +1559,7 @@ Query: ```sql SELECT finalizeAggregation(state), toTypeName(state) FROM (SELECT initializeAggregation('sumState', number % 3) AS state FROM numbers(5)); ``` + Result: ```text @@ -1568,11 +1589,12 @@ INSERT INTO metrics VALUES (0, initializeAggregation('sumState', toUInt64(42))) ``` **See Also** + - [arrayReduce](../../sql-reference/functions/array-functions.md#arrayreduce) ## finalizeAggregation -Takes state of aggregate function. Returns result of aggregation (or finalized state when using[-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state) combinator). +Given a state of aggregate function, this function returns the result of aggregation (or finalized state when using a [-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state) combinator). **Syntax** @@ -1667,15 +1689,16 @@ Result: ``` **See Also** + - [arrayReduce](../../sql-reference/functions/array-functions.md#arrayreduce) - [initializeAggregation](#initializeaggregation) ## runningAccumulate -Accumulates states of an aggregate function for each row of a data block. +Accumulates the states of an aggregate function for each row of a data block. -:::tip -The state is reset for each new data block. +:::note +The state is reset for each new block of data. ::: **Syntax** @@ -1726,10 +1749,10 @@ The subquery generates `sumState` for every number from `0` to `9`. `sumState` r The whole query does the following: -1. For the first row, `runningAccumulate` takes `sumState(0)` and returns `0`. -2. For the second row, the function merges `sumState(0)` and `sumState(1)` resulting in `sumState(0 + 1)`, and returns `1` as a result. -3. For the third row, the function merges `sumState(0 + 1)` and `sumState(2)` resulting in `sumState(0 + 1 + 2)`, and returns `3` as a result. -4. The actions are repeated until the block ends. +1. For the first row, `runningAccumulate` takes `sumState(0)` and returns `0`. +2. For the second row, the function merges `sumState(0)` and `sumState(1)` resulting in `sumState(0 + 1)`, and returns `1` as a result. +3. For the third row, the function merges `sumState(0 + 1)` and `sumState(2)` resulting in `sumState(0 + 1 + 2)`, and returns `3` as a result. +4. The actions are repeated until the block ends. The following example shows the `groupping` parameter usage: @@ -1780,7 +1803,7 @@ As you can see, `runningAccumulate` merges states for each group of rows separat The function lets you extract data from the table the same way as from a [dictionary](../../sql-reference/dictionaries/index.md). -Gets data from [Join](../../engines/table-engines/special/join.md#creating-a-table) tables using the specified join key. +Gets the data from [Join](../../engines/table-engines/special/join.md#creating-a-table) tables using the specified join key. Only supports tables created with the `ENGINE = Join(ANY, LEFT, )` statement. @@ -1792,13 +1815,13 @@ joinGet(join_storage_table_name, `value_column`, join_keys) **Arguments** -- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicates where search is performed. The identifier is searched in the default database (see parameter `default_database` in the config file). To override the default database, use the `USE db_name` or specify the database and the table through the separator `db_name.db_table`, see the example. +- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicating where the search is performed. The identifier is searched in the default database (see setting `default_database` in the config file). To override the default database, use `USE db_name` or specify the database and the table through the separator `db_name.db_table` as in the example. - `value_column` — name of the column of the table that contains required data. - `join_keys` — list of keys. **Returned value** -Returns list of values corresponded to list of keys. +Returns a list of values corresponded to list of keys. If certain does not exist in source table then `0` or `null` will be returned based on [join_use_nulls](../../operations/settings/settings.md#join_use_nulls) setting. @@ -1825,7 +1848,7 @@ INSERT INTO db_test.id_val VALUES (1,11)(2,12)(4,13) Query: ``` sql -SELECT joinGet(db_test.id_val,'val',toUInt32(number)) from numbers(4) SETTINGS join_use_nulls = 1 +SELECT joinGet(db_test.id_val, 'val', toUInt32(number)) from numbers(4) SETTINGS join_use_nulls = 1 ``` Result: @@ -1845,7 +1868,7 @@ Result: This function is not available in ClickHouse Cloud. ::: -Evaluate external catboost model. [CatBoost](https://catboost.ai) is an open-source gradient boosting library developed by Yandex for machine learing. +Evaluate an external catboost model. [CatBoost](https://catboost.ai) is an open-source gradient boosting library developed by Yandex for machine learing. Accepts a path to a catboost model and model arguments (features). Returns Float64. ``` sql @@ -1886,16 +1909,24 @@ See [Training and applying models](https://catboost.ai/docs/features/training.ht ## throwIf(x\[, message\[, error_code\]\]) -Throw an exception if the argument is non zero. -`message` - is an optional parameter: a constant string providing a custom error message -`error_code` - is an optional parameter: a constant integer providing a custom error code +Throw an exception if argument `x` is true. + +**Arguments** + +- `x` - the condition to check. +- `message` - a constant string providing a custom error message. Optional. +- `error_code` - A constant integer providing a custom error code. Optional. To use the `error_code` argument, configuration parameter `allow_custom_error_code_in_throwif` must be enabled. +**Example** + ``` sql SELECT throwIf(number = 3, 'Too many') FROM numbers(10); ``` +Result: + ``` text ↙ Progress: 0.00 rows, 0.00 B (0.00 rows/s., 0.00 B/s.) Received exception from server (version 19.14.1): Code: 395. DB::Exception: Received from localhost:9000. DB::Exception: Too many. @@ -1903,7 +1934,7 @@ Code: 395. DB::Exception: Received from localhost:9000. DB::Exception: Too many. ## identity -Returns the same value that was used as its argument. Used for debugging and testing, allows to cancel using index, and get the query performance of a full scan. When query is analyzed for possible use of index, the analyzer does not look inside `identity` functions. Also constant folding is not applied too. +Returns its argument. Intended for debugging and testing. Allows to cancel using index, and get the query performance of a full scan. When the query is analyzed for possible use of an index, the analyzer ignores everything in `identity` functions. Also disables constant folding. **Syntax** @@ -1916,7 +1947,7 @@ identity(x) Query: ``` sql -SELECT identity(42) +SELECT identity(42); ``` Result: @@ -1927,164 +1958,6 @@ Result: └──────────────┘ ``` -## randomPrintableASCII - -Generates a string with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) printable characters. - -**Syntax** - -``` sql -randomPrintableASCII(length) -``` - -**Arguments** - -- `length` — Resulting string length. Positive integer. - - If you pass `length < 0`, behavior of the function is undefined. - -**Returned value** - -- String with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) printable characters. - -Type: [String](../../sql-reference/data-types/string.md) - -**Example** - -``` sql -SELECT number, randomPrintableASCII(30) as str, length(str) FROM system.numbers LIMIT 3 -``` - -``` text -┌─number─┬─str────────────────────────────┬─length(randomPrintableASCII(30))─┐ -│ 0 │ SuiCOSTvC0csfABSw=UcSzp2.`rv8x │ 30 │ -│ 1 │ 1Ag NlJ &RCN:*>HVPG;PE-nO"SUFD │ 30 │ -│ 2 │ /"+<"wUTh:=LjJ Vm!c&hI*m#XTfzz │ 30 │ -└────────┴────────────────────────────────┴──────────────────────────────────┘ -``` - -## randomString - -Generates a binary string of the specified length filled with random bytes (including zero bytes). - -**Syntax** - -``` sql -randomString(length) -``` - -**Arguments** - -- `length` — String length. Positive integer. - -**Returned value** - -- String filled with random bytes. - -Type: [String](../../sql-reference/data-types/string.md). - -**Example** - -Query: - -``` sql -SELECT randomString(30) AS str, length(str) AS len FROM numbers(2) FORMAT Vertical; -``` - -Result: - -``` text -Row 1: -────── -str: 3 G : pT ?w тi k aV f6 -len: 30 - -Row 2: -────── -str: 9 ,] ^ ) ]?? 8 -len: 30 -``` - -**See Also** - -- [generateRandom](../../sql-reference/table-functions/generate.md#generaterandom) -- [randomPrintableASCII](../../sql-reference/functions/other-functions.md#randomascii) - - -## randomFixedString - -Generates a binary string of the specified length filled with random bytes (including zero bytes). - -**Syntax** - -``` sql -randomFixedString(length); -``` - -**Arguments** - -- `length` — String length in bytes. [UInt64](../../sql-reference/data-types/int-uint.md). - -**Returned value(s)** - -- String filled with random bytes. - -Type: [FixedString](../../sql-reference/data-types/fixedstring.md). - -**Example** - -Query: - -```sql -SELECT randomFixedString(13) as rnd, toTypeName(rnd) -``` - -Result: - -```text -┌─rnd──────┬─toTypeName(randomFixedString(13))─┐ -│ j▒h㋖HɨZ'▒ │ FixedString(13) │ -└──────────┴───────────────────────────────────┘ - -``` - -## randomStringUTF8 - -Generates a random string of a specified length. Result string contains valid UTF-8 code points. The value of code points may be outside of the range of assigned Unicode. - -**Syntax** - -``` sql -randomStringUTF8(length); -``` - -**Arguments** - -- `length` — Required length of the resulting string in code points. [UInt64](../../sql-reference/data-types/int-uint.md). - -**Returned value(s)** - -- UTF-8 random string. - -Type: [String](../../sql-reference/data-types/string.md). - -**Example** - -Query: - -```sql -SELECT randomStringUTF8(13) -``` - -Result: - -```text -┌─randomStringUTF8(13)─┐ -│ 𘤗𙉝д兠庇󡅴󱱎󦐪􂕌𔊹𓰛 │ -└──────────────────────┘ - -``` - ## getSetting Returns the current value of a [custom setting](../../operations/settings/index.md#custom_settings). @@ -2101,7 +1974,7 @@ getSetting('custom_setting'); **Returned value** -- The setting current value. +- The setting's current value. **Example** @@ -2110,7 +1983,7 @@ SET custom_a = 123; SELECT getSetting('custom_a'); ``` -**Result** +Result: ``` 123 @@ -2122,7 +1995,7 @@ SELECT getSetting('custom_a'); ## isDecimalOverflow -Checks whether the [Decimal](../../sql-reference/data-types/decimal.md) value is out of its (or specified) precision. +Checks whether the [Decimal](../../sql-reference/data-types/decimal.md) value is outside its precision or outside the specified precision. **Syntax** @@ -2133,11 +2006,11 @@ isDecimalOverflow(d, [p]) **Arguments** - `d` — value. [Decimal](../../sql-reference/data-types/decimal.md). -- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. Using of this paratemer could be helpful for data extraction to another DBMS or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). +- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. This paratemer can be helpful to migrate data from/to another database or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). **Returned values** -- `1` — Decimal value has more digits then it's precision allow, +- `1` — Decimal value has more digits then allowed by its precision, - `0` — Decimal value satisfies the specified precision. **Example** @@ -2159,7 +2032,7 @@ Result: ## countDigits -Returns number of decimal digits you need to represent the value. +Returns number of decimal digits need to represent a value. **Syntax** @@ -2199,9 +2072,7 @@ Result: ## errorCodeToName -**Returned value** - -- Variable name for the error code. +Returns the textual name of an error code. Type: [LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md). @@ -2220,7 +2091,7 @@ UNSUPPORTED_METHOD ## tcpPort Returns [native interface](../../interfaces/tcp.md) TCP port number listened by this server. -If it is executed in the context of a distributed table, then it generates a normal column, otherwise it produces a constant value. +If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. **Syntax** @@ -2310,7 +2181,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## currentRoles -Returns the names of the roles which are current for the current user. The current roles can be changed by the [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement) statement. If the `SET ROLE` statement was not used, the function `currentRoles` returns the same as `defaultRoles`. +Returns the roles assigned to the current user. The roles can be changed by the [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement) statement. If no `SET ROLE` statement was not, the function `currentRoles` returns the same as `defaultRoles`. **Syntax** @@ -2320,7 +2191,7 @@ currentRoles() **Returned value** -- List of the current roles for the current user. +- A list of the current roles for the current user. Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). @@ -2342,7 +2213,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## defaultRoles -Returns the names of the roles which are enabled by default for the current user when he logins. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant.md#grant-select)), but that can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. +Returns the roles which are enabled by default for the current user when he logs in. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant.md#grant-select)), but that can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. **Syntax** @@ -2358,7 +2229,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## getServerPort -Returns the number of the server port. When the port is not used by the server, throws an exception. +Returns the server port number. When the port is not used by the server, throws an exception. **Syntax** @@ -2407,7 +2278,7 @@ Result: Returns the ID of the current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. -In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see example). +In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see the example). **Syntax** @@ -2477,7 +2348,7 @@ Result: ## shardNum -Returns the index of a shard which processes a part of data for a distributed query. Indices are started from `1`. +Returns the index of a shard which processes a part of data in a distributed query. Indices are started from `1`. If a query is not distributed then constant value `0` is returned. **Syntax** diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index e90d537fb74..63d5174b494 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -31,7 +31,7 @@ Uses a linear congruential generator. ## randCanonical -Returns a Float64 value, evenly distributed in [0, 1). +Returns a random Float64 value, evenly distributed in interval [0, 1). ## randConstant @@ -54,11 +54,9 @@ Result: └────────────┴────────────┴──────────────┴────────────────┴─────────────────┴──────────────────────┘ ``` -# Functions for Generating Random Numbers based on a Distribution - ## randUniform -Returns a Float64 drawn uniformly from the interval between `min` and `max` ([continuous uniform distribution](https://en.wikipedia.org/wiki/Continuous_uniform_distribution)). +Returns a random Float64 drawn uniformly from interval [`min`, `max`) ([continuous uniform distribution](https://en.wikipedia.org/wiki/Continuous_uniform_distribution)). **Syntax** @@ -68,8 +66,8 @@ randUniform(min, max) **Arguments** -- `min` - `Float64` - min value of the range, -- `max` - `Float64` - max value of the range. +- `min` - `Float64` - left boundary of the range, +- `max` - `Float64` - reight boundary of the range. **Returned value** @@ -97,7 +95,7 @@ Result: ## randNormal -Returns a Float64 drawn from a [normal distribution](https://en.wikipedia.org/wiki/Normal_distribution). +Returns a random Float64 drawn from a [normal distribution](https://en.wikipedia.org/wiki/Normal_distribution). **Syntax** @@ -108,7 +106,7 @@ randNormal(mean, variance) **Arguments** - `mean` - `Float64` - mean value of distribution, -- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance). +- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance) of the distribution. **Returned value** @@ -136,7 +134,7 @@ Result: ## randLogNormal -Returns a Float64 drawn from a [log-normal distribution](https://en.wikipedia.org/wiki/Log-normal_distribution). +Returns a random Float64 drawn from a [log-normal distribution](https://en.wikipedia.org/wiki/Log-normal_distribution). **Syntax** @@ -147,7 +145,7 @@ randLogNormal(mean, variance) **Arguments** - `mean` - `Float64` - mean value of distribution, -- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance). +- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance) of the distribution. **Returned value** @@ -175,7 +173,7 @@ Result: ## randBinomial -Returns a UInt64 drawn from a [binomial distribution](https://en.wikipedia.org/wiki/Binomial_distribution). +Returns a random UInt64 drawn from a [binomial distribution](https://en.wikipedia.org/wiki/Binomial_distribution). **Syntax** @@ -186,7 +184,7 @@ randBinomial(experiments, probability) **Arguments** - `experiments` - `UInt64` - number of experiments, -- `probability` - `Float64` - probability of success in each experiment (values in `0...1` range only). +- `probability` - `Float64` - probability of success in each experiment, a value between 0 and 1. **Returned value** @@ -214,7 +212,7 @@ Result: ## randNegativeBinomial -Returns a UInt64 drawn from a [negative binomial distribution](https://en.wikipedia.org/wiki/Negative_binomial_distribution). +Returns a random UInt64 drawn from a [negative binomial distribution](https://en.wikipedia.org/wiki/Negative_binomial_distribution). **Syntax** @@ -225,7 +223,7 @@ randNegativeBinomial(experiments, probability) **Arguments** - `experiments` - `UInt64` - number of experiments, -- `probability` - `Float64` - probability of failure in each experiment (values in `0...1` range only). +- `probability` - `Float64` - probability of failure in each experiment, a value between 0 and 1. **Returned value** @@ -253,7 +251,7 @@ Result: ## randPoisson -Returns a UInt64 drawn from a [Poisson distribution](https://en.wikipedia.org/wiki/Poisson_distribution). +Returns a random UInt64 drawn from a [Poisson distribution](https://en.wikipedia.org/wiki/Poisson_distribution). **Syntax** @@ -291,7 +289,7 @@ Result: ## randBernoulli -Returns a UInt64 drawn from a [Bernoulli distribution](https://en.wikipedia.org/wiki/Bernoulli_distribution). +Returns a random UInt64 drawn from a [Bernoulli distribution](https://en.wikipedia.org/wiki/Bernoulli_distribution). **Syntax** @@ -301,7 +299,7 @@ randBernoulli(probability) **Arguments** -- `probability` - `Float64` - probability of success (values in `0...1` range only). +- `probability` - `Float64` - probability of success, a value between 0 and 1. **Returned value** @@ -329,7 +327,7 @@ Result: ## randExponential -Returns a Float64 drawn from a [exponential distribution](https://en.wikipedia.org/wiki/Exponential_distribution). +Returns a random Float64 drawn from a [exponential distribution](https://en.wikipedia.org/wiki/Exponential_distribution). **Syntax** @@ -367,7 +365,7 @@ Result: ## randChiSquared -Returns a Float64 drawn from a [Chi-square distribution](https://en.wikipedia.org/wiki/Chi-squared_distribution) - a distribution of a sum of the squares of k independent standard normal random variables. +Returns a random Float64 drawn from a [Chi-square distribution](https://en.wikipedia.org/wiki/Chi-squared_distribution) - a distribution of a sum of the squares of k independent standard normal random variables. **Syntax** @@ -405,7 +403,7 @@ Result: ## randStudentT -Returns a Float64 drawn from a [Student's t-distribution](https://en.wikipedia.org/wiki/Student%27s_t-distribution). +Returns a random Float64 drawn from a [Student's t-distribution](https://en.wikipedia.org/wiki/Student%27s_t-distribution). **Syntax** @@ -443,7 +441,7 @@ Result: ## randFisherF -Returns a Float64 drawn from a [F-distribution](https://en.wikipedia.org/wiki/F-distribution). +Returns a random Float64 drawn from a [F-distribution](https://en.wikipedia.org/wiki/F-distribution). **Syntax** @@ -480,47 +478,160 @@ Result: └─────────────────────┘ ``` -# Functions for Generating Random Strings - ## randomString -Returns a random String of specified `length`. Not all characters may be printable. +Generates a string of the specified length filled with random bytes (including zero bytes). Not all characters may be printable. **Syntax** -```sql +``` sql randomString(length) ``` +**Arguments** + +- `length` — String length in bytes. Positive integer. + +**Returned value** + +- String filled with random bytes. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT randomString(30) AS str, length(str) AS len FROM numbers(2) FORMAT Vertical; +``` + +Result: + +``` text +Row 1: +────── +str: 3 G : pT ?w тi k aV f6 +len: 30 + +Row 2: +────── +str: 9 ,] ^ ) ]?? 8 +len: 30 +``` + ## randomFixedString -Like `randomString` but returns a FixedString. - -## randomPrintableASCII - -Returns a random String of specified `length`. All characters are printable. +Generates a binary string of the specified length filled with random bytes (including zero bytes). Not all characters may be printable. **Syntax** +``` sql +randomFixedString(length); +``` + +**Arguments** + +- `length` — String length in bytes. [UInt64](../../sql-reference/data-types/int-uint.md). + +**Returned value(s)** + +- String filled with random bytes. + +Type: [FixedString](../../sql-reference/data-types/fixedstring.md). + +**Example** + +Query: + ```sql +SELECT randomFixedString(13) as rnd, toTypeName(rnd) +``` + +Result: + +```text +┌─rnd──────┬─toTypeName(randomFixedString(13))─┐ +│ j▒h㋖HɨZ'▒ │ FixedString(13) │ +└──────────┴───────────────────────────────────┘ +``` + +## randomPrintableASCII + +Generates a string with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) characters. All characters are printable. +If you pass `length < 0`, the behavior of the function is undefined. + +**Syntax** + +``` sql randomPrintableASCII(length) ``` +**Arguments** + +- `length` — String length in bytes. Positive integer. + +**Returned value** + +- String with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) printable characters. + +Type: [String](../../sql-reference/data-types/string.md) + +**Example** + +``` sql +SELECT number, randomPrintableASCII(30) as str, length(str) FROM system.numbers LIMIT 3 +``` + +``` text +┌─number─┬─str────────────────────────────┬─length(randomPrintableASCII(30))─┐ +│ 0 │ SuiCOSTvC0csfABSw=UcSzp2.`rv8x │ 30 │ +│ 1 │ 1Ag NlJ &RCN:*>HVPG;PE-nO"SUFD │ 30 │ +│ 2 │ /"+<"wUTh:=LjJ Vm!c&hI*m#XTfzz │ 30 │ +└────────┴────────────────────────────────┴──────────────────────────────────┘ +``` + ## randomStringUTF8 -Returns a random String containing `length` many UTF8 codepoints. Not all characters may be printable +Generates a random string of a specified length. Result string contains valid UTF-8 code points. The value of code points may be outside of the range of assigned Unicode. **Syntax** +``` sql +randomStringUTF8(length); +``` + +**Arguments** + +- `length` — Length of the string in code points. [UInt64](../../sql-reference/data-types/int-uint.md). + +**Returned value(s)** + +- UTF-8 random string. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: + ```sql -randomStringUTF8(length) +SELECT randomStringUTF8(13) +``` + +Result: + +```text +┌─randomStringUTF8(13)─┐ +│ 𘤗𙉝д兠庇󡅴󱱎󦐪􂕌𔊹𓰛 │ +└──────────────────────┘ ``` ## fuzzBits **Syntax** -Inverts the bits of String or FixedString `s`, each with probability `prob`. +Flips the bits of String or FixedString `s`, each with probability `prob`. **Syntax** @@ -529,8 +640,8 @@ fuzzBits(s, prob) ``` **Arguments** -- `s` - `String` or `FixedString` -- `prob` - constant `Float32/64` +- `s` - `String` or `FixedString`, +- `prob` - constant `Float32/64` between 0.0 and 1.0. **Returned value** From 0181ea63993bc6dd9c924c092bc7294a1bf71e2f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 12:55:46 +0300 Subject: [PATCH 240/308] Revert "make filter push down through cross join" --- .../Optimizations/filterPushDown.cpp | 6 +++--- .../01763_filter_push_down_bugs.reference | 19 ------------------- .../01763_filter_push_down_bugs.sql | 19 ------------------- 3 files changed, 3 insertions(+), 41 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index db29038999b..37bc894339f 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -272,7 +272,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes { /// If totals step has HAVING expression, skip it for now. /// TODO: - /// We can merge HAVING expression with current filter. + /// We can merge HAVING expression with current filer. /// Also, we can push down part of HAVING which depend only on aggregation keys. if (totals_having->getActions()) return 0; @@ -323,9 +323,9 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes { const auto & table_join = join ? join->getJoin()->getTableJoin() : filled_join->getJoin()->getTableJoin(); - /// Only inner, cross and left(/right) join are supported. Other types may generate default values for left table keys. + /// Only inner and left(/right) join are supported. Other types may generate default values for left table keys. /// So, if we push down a condition like `key != 0`, not all rows may be filtered. - if (table_join.kind() != JoinKind::Inner && table_join.kind() != JoinKind::Cross && table_join.kind() != kind) + if (table_join.kind() != JoinKind::Inner && table_join.kind() != kind) return 0; bool is_left = kind == JoinKind::Left; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index 7df35e2948d..5aa2e645509 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -6,22 +6,3 @@ String1_0 String2_0 String3_0 String4_0 1 String1_0 String2_0 String3_0 String4_0 1 1 [0,1,2] 1 -Expression ((Projection + Before ORDER BY)) - Filter (WHERE) - Join (JOIN FillRightFirst) - Filter (( + Before JOIN)) - ReadFromMergeTree (default.t1) - Indexes: - PrimaryKey - Keys: - id - Condition: (id in [101, 101]) - Parts: 1/1 - Granules: 1/1 - Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) - ReadFromMergeTree (default.t2) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 1/1 diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 2ee249b5ce7..1058bf75144 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -38,25 +38,6 @@ DROP TABLE IF EXISTS Test; select x, y from (select [0, 1, 2] as y, 1 as a, 2 as b) array join y as x where a = 1 and b = 2 and (x = 1 or x != 1) and x = 1; -DROP TABLE IF EXISTS t; create table t(a UInt8) engine=MergeTree order by a; insert into t select * from numbers(2); select a from t t1 join t t2 on t1.a = t2.a where t1.a; -DROP TABLE IF EXISTS t; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (id Int64, create_time DateTime) ENGINE = MergeTree ORDER BY id; -CREATE TABLE t2 (delete_time DateTime) ENGINE = MergeTree ORDER BY delete_time; - -insert into t1 values (101, '2023-05-28 00:00:00'), (102, '2023-05-28 00:00:00'); -insert into t2 values ('2023-05-31 00:00:00'); - -EXPLAIN indexes=1 SELECT id, delete_time FROM t1 - CROSS JOIN ( - SELECT delete_time - FROM t2 -) AS d WHERE create_time < delete_time AND id = 101; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; From 8cadd89ebedc2fee73c3081992d35bbf8ad3280c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 2 Jun 2023 12:34:22 +0200 Subject: [PATCH 241/308] Update src/Common/TaskStatsInfoGetter.cpp --- src/Common/TaskStatsInfoGetter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/TaskStatsInfoGetter.cpp index d21229609dd..867a50c8cce 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/TaskStatsInfoGetter.cpp @@ -208,7 +208,7 @@ bool checkPermissionsImpl() try { ::taskstats stats{}; - TaskStatsInfoGetter().getStat(stats, static_cast(getThreadId())); + TaskStatsInfoGetter().getStat(stats, static_cast(getThreadId())); } catch (const Exception & e) { From 572f15b2cd70fa7b4293c7ec9682e361c9989d77 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 11:14:52 +0000 Subject: [PATCH 242/308] Fix typo --- docs/en/sql-reference/functions/functions-for-nulls.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/functions-for-nulls.md b/docs/en/sql-reference/functions/functions-for-nulls.md index f4ca27e9b16..6f82fedaab7 100644 --- a/docs/en/sql-reference/functions/functions-for-nulls.md +++ b/docs/en/sql-reference/functions/functions-for-nulls.md @@ -245,7 +245,7 @@ assumeNotNull(x) **Returned values** - The input value as non-`Nullable` type, if it is not `NULL`. -- An arbirary value, if the input value is `NULL`. +- An arbitrary value, if the input value is `NULL`. **Example** From 73db383727550e040a939d367e6e59bb037780bd Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 2 Jun 2023 14:10:26 +0200 Subject: [PATCH 243/308] Limit the number of in-flight tasks for loading outdated parts (#50450) * Done * Update programs/local/LocalServer.cpp Co-authored-by: Alexander Tokmakov * Bump --------- Co-authored-by: Alexander Tokmakov --- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 2 +- src/Core/ServerSettings.h | 1 - src/Interpreters/threadPoolCallbackRunner.h | 4 ++-- 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 944a60d4e4c..96c1ca261b5 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -138,7 +138,7 @@ void LocalServer::initialize(Poco::Util::Application & self) OutdatedPartsLoadingThreadPool::initialize( config().getUInt("max_outdated_parts_loading_thread_pool_size", 16), 0, // We don't need any threads one all the parts will be loaded - config().getUInt("outdated_part_loading_thread_pool_queue_size", 10000)); + config().getUInt("max_outdated_parts_loading_thread_pool_size", 16)); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 03ada89e86f..9eb3e6c9ebc 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -696,7 +696,7 @@ try OutdatedPartsLoadingThreadPool::initialize( server_settings.max_outdated_parts_loading_thread_pool_size, 0, // We don't need any threads one all the parts will be loaded - server_settings.outdated_part_loading_thread_pool_queue_size); + server_settings.max_outdated_parts_loading_thread_pool_size); /// Initialize global local cache for remote filesystem. if (config().has("local_cache_for_remote_fs")) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 511b8d68f6d..cb43d62ecd1 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -22,7 +22,6 @@ namespace DB M(UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0) \ M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \ M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The maximum number of threads that would be used for loading outdated data parts on startup", 0) \ - M(UInt64, outdated_part_loading_thread_pool_queue_size, 10000, "Queue size for parts loading thread pool.", 0) \ M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \ diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index 55c6a848b77..f7324bfafe6 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -13,7 +13,7 @@ namespace DB template > using ThreadPoolCallbackRunner = std::function(Callback &&, Priority)>; -/// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()'. +/// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrowOnError()'. template > ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name) { @@ -44,7 +44,7 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & auto future = task->get_future(); - my_pool->scheduleOrThrow([my_task = std::move(task)]{ (*my_task)(); }, priority); + my_pool->scheduleOrThrowOnError([my_task = std::move(task)]{ (*my_task)(); }, priority); return future; }; From 65cc92a78d89f088c6c160dd4cb1748f48ed726d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 11:30:05 +0000 Subject: [PATCH 244/308] CI: Fix aspell on nested docs --- .../database-engines/materialized-mysql.md | 2 +- .../materialized-postgresql.md | 4 +- docs/en/engines/database-engines/sqlite.md | 2 +- .../table-engines/integrations/hdfs.md | 4 +- .../table-engines/integrations/hive.md | 2 +- .../table-engines/integrations/nats.md | 4 +- .../table-engines/integrations/postgresql.md | 2 +- .../engines/table-engines/integrations/s3.md | 6 +- .../mergetree-family/annindexes.md | 4 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/mergetree.md | 4 +- .../table-engines/special/executable.md | 4 +- .../table-engines/special/keepermap.md | 2 +- .../example-datasets/amazon-reviews.md | 4 +- .../example-datasets/cell-towers.md | 4 +- .../example-datasets/github.md | 4 +- .../example-datasets/opensky.md | 2 +- .../example-datasets/reddit-comments.md | 4 +- .../example-datasets/youtube-dislikes.md | 6 +- docs/en/operations/settings/settings.md | 2 +- docs/en/operations/system-tables/columns.md | 2 +- .../system-tables/dropped_tables.md | 2 +- .../system-tables/information_schema.md | 2 +- docs/en/operations/system-tables/licenses.md | 2 +- docs/en/operations/system-tables/parts.md | 2 +- docs/en/operations/system-tables/tables.md | 2 +- docs/en/operations/system-tables/trace_log.md | 2 +- .../operations/system-tables/zookeeper_log.md | 4 +- .../utilities/clickhouse-obfuscator.md | 2 +- .../aggregate-functions/reference/cramersv.md | 4 +- .../reference/cramersvbiascorrected.md | 2 +- .../reference/groupbitand.md | 2 +- .../reference/groupbitor.md | 2 +- .../reference/groupbitxor.md | 2 +- .../reference/kolmogorovsmirnovtest.md | 4 +- .../reference/quantiletdigestweighted.md | 2 +- .../reference/stochasticlinearregression.md | 2 +- docs/en/sql-reference/data-types/datetime.md | 8 +- docs/en/sql-reference/data-types/index.md | 2 +- docs/en/sql-reference/dictionaries/index.md | 2 +- .../sql-reference/functions/bit-functions.md | 4 +- .../functions/encryption-functions.md | 8 +- docs/en/sql-reference/functions/geo/h3.md | 2 +- docs/en/sql-reference/functions/geo/s2.md | 2 +- .../sql-reference/functions/hash-functions.md | 2 +- .../functions/logical-functions.md | 2 +- .../sql-reference/functions/math-functions.md | 2 +- .../functions/other-functions.md | 4 +- .../functions/string-functions.md | 2 +- .../functions/string-replace-functions.md | 2 +- docs/en/sql-reference/functions/udf.md | 2 +- docs/en/sql-reference/operators/in.md | 2 +- .../sql-reference/statements/alter/quota.md | 2 +- .../sql-reference/statements/create/quota.md | 2 +- docs/en/sql-reference/statements/explain.md | 4 +- .../sql-reference/statements/select/from.md | 2 +- .../statements/select/order-by.md | 2 +- .../sql-reference/window-functions/index.md | 4 +- .../aspell-ignore/en/aspell-dict.txt | 2429 +++++++++++++++-- utils/check-style/check-doc-aspell | 3 + 60 files changed, 2300 insertions(+), 302 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 1f1c996d4bf..f7cc52e622e 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -119,7 +119,7 @@ When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree]( The data of TIME type in MySQL is converted to microseconds in ClickHouse. -Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws exception "Unhandled data type" and stops replication. +Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws an exception and stops replication. ## Specifics and Recommendations {#specifics-and-recommendations} diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 08e9f998626..33d75dc9582 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -55,7 +55,7 @@ ATTACH TABLE postgres_database.new_table; ``` :::warning -Before version 22.1, adding a table to replication left an unremoved temporary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in ClickHouse version before 22.1, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. This issue is fixed in 22.1. +Before version 22.1, adding a table to replication left a non-removed temporary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in ClickHouse version before 22.1, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. This issue is fixed in 22.1. ::: ## Dynamically removing tables from replication {#dynamically-removing-table-from-replication} @@ -257,7 +257,7 @@ Please note that this should be used only if it is actually needed. If there is 1. [CREATE PUBLICATION](https://postgrespro.ru/docs/postgresql/14/sql-createpublication) -- create query privilege. -2. [CREATE_REPLICATION_SLOT](https://postgrespro.ru/docs/postgrespro/10/protocol-replication#PROTOCOL-REPLICATION-CREATE-SLOT) -- replication privelege. +2. [CREATE_REPLICATION_SLOT](https://postgrespro.ru/docs/postgrespro/10/protocol-replication#PROTOCOL-REPLICATION-CREATE-SLOT) -- replication privilege. 3. [pg_drop_replication_slot](https://postgrespro.ru/docs/postgrespro/9.5/functions-admin#functions-replication) -- replication privilege or superuser. diff --git a/docs/en/engines/database-engines/sqlite.md b/docs/en/engines/database-engines/sqlite.md index fc2a6525a68..0fa3c0fff58 100644 --- a/docs/en/engines/database-engines/sqlite.md +++ b/docs/en/engines/database-engines/sqlite.md @@ -30,7 +30,7 @@ Allows to connect to [SQLite](https://www.sqlite.org/index.html) database and pe ## Specifics and Recommendations {#specifics-and-recommendations} -SQLite stores the entire database (definitions, tables, indices, and the data itself) as a single cross-platform file on a host machine. During writing SQLite locks the entire database file, therefore write operations are performed sequentially. Read operations can be multitasked. +SQLite stores the entire database (definitions, tables, indices, and the data itself) as a single cross-platform file on a host machine. During writing SQLite locks the entire database file, therefore write operations are performed sequentially. Read operations can be multi-tasked. SQLite does not require service management (such as startup scripts) or access control based on `GRANT` and passwords. Access control is handled by means of file-system permissions given to the database file itself. ## Usage Example {#usage-example} diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index b9db0fae68f..08cd88826e5 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -156,7 +156,7 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us | rpc\_client\_connect\_timeout | 600 * 1000 | | rpc\_client\_read\_timeout | 3600 * 1000 | | rpc\_client\_write\_timeout | 3600 * 1000 | -| rpc\_client\_socekt\_linger\_timeout | -1 | +| rpc\_client\_socket\_linger\_timeout | -1 | | rpc\_client\_connect\_retry | 10 | | rpc\_client\_timeout | 3600 * 1000 | | dfs\_default\_replica | 3 | @@ -176,7 +176,7 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us | output\_write\_timeout | 3600 * 1000 | | output\_close\_timeout | 3600 * 1000 | | output\_packetpool\_size | 1024 | -| output\_heeartbeat\_interval | 10 * 1000 | +| output\_heartbeat\_interval | 10 * 1000 | | dfs\_client\_failover\_max\_attempts | 15 | | dfs\_client\_read\_shortcircuit\_streams\_cache\_size | 256 | | dfs\_client\_socketcache\_expiryMsec | 3000 | diff --git a/docs/en/engines/table-engines/integrations/hive.md b/docs/en/engines/table-engines/integrations/hive.md index adcb73605bb..5d10e417ae3 100644 --- a/docs/en/engines/table-engines/integrations/hive.md +++ b/docs/en/engines/table-engines/integrations/hive.md @@ -6,7 +6,7 @@ sidebar_label: Hive # Hive -The Hive engine allows you to perform `SELECT` quries on HDFS Hive table. Currently it supports input formats as below: +The Hive engine allows you to perform `SELECT` queries on HDFS Hive table. Currently it supports input formats as below: - Text: only supports simple scalar column types except `binary` diff --git a/docs/en/engines/table-engines/integrations/nats.md b/docs/en/engines/table-engines/integrations/nats.md index 7f09c516d6f..a82d74e0d95 100644 --- a/docs/en/engines/table-engines/integrations/nats.md +++ b/docs/en/engines/table-engines/integrations/nats.md @@ -10,7 +10,7 @@ This engine allows integrating ClickHouse with [NATS](https://nats.io/). `NATS` lets you: -- Publish or subcribe to message subjects. +- Publish or subscribe to message subjects. - Process new messages as they become available. ## Creating a Table {#table_engine-redisstreams-creating-a-table} @@ -46,7 +46,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Required parameters: - `nats_url` – host:port (for example, `localhost:5672`).. -- `nats_subjects` – List of subject for NATS table to subscribe/publsh to. Supports wildcard subjects like `foo.*.bar` or `baz.>` +- `nats_subjects` – List of subject for NATS table to subscribe/publish to. Supports wildcard subjects like `foo.*.bar` or `baz.>` - `nats_format` – Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section. Optional parameters: diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index f27d4d48f75..51b3048706f 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -57,7 +57,7 @@ or via config (since version 21.11): ``` -Some parameters can be overriden by key value arguments: +Some parameters can be overridden by key value arguments: ``` sql SELECT * FROM postgresql(postgres1, schema='schema1', table='table1'); ``` diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 595bc0c344f..f2eaacd92a5 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -23,7 +23,7 @@ CREATE TABLE s3_engine_table (name String, value UInt32) - `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed. - `format` — The [format](../../../interfaces/formats.md#formats) of the file. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). -- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension. +- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will auto-detect compression by file extension. ### PARTITION BY @@ -140,8 +140,8 @@ The following settings can be set before query execution or placed into configur - `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited). - `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`. - `s3_upload_part_size_multiply_factor` - 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. Default values is `2`. -- `s3_upload_part_size_multiply_parts_count_threshold` - Each time this number of parts was uploaded to S3 `s3_min_upload_part_size multiplied` by `s3_upload_part_size_multiply_factor`. DEfault value us `500`. -- `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurenly for one object. Its number should be limited. The value `0` means unlimited. Default value is `20`. Each inflight part has a buffer with size `s3_min_upload_part_size` for the first `s3_upload_part_size_multiply_factor` parts and more when file is big enought, see `upload_part_size_multiply_factor`. With default settings one uploaded file consumes not more than `320Mb` for a file which is less than `8G`. The consumption is greater for a larger file. +- `s3_upload_part_size_multiply_parts_count_threshold` - Each time this number of parts was uploaded to S3 `s3_min_upload_part_size multiplied` by `s3_upload_part_size_multiply_factor`. Default value us `500`. +- `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurrently for one object. Its number should be limited. The value `0` means unlimited. Default value is `20`. Each in-flight part has a buffer with size `s3_min_upload_part_size` for the first `s3_upload_part_size_multiply_factor` parts and more when file is big enough, see `upload_part_size_multiply_factor`. With default settings one uploaded file consumes not more than `320Mb` for a file which is less than `8G`. The consumption is greater for a larger file. Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration. diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index f841f157376..03617a1a709 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -78,7 +78,7 @@ ENGINE = MergeTree ORDER BY id; ``` -With greater `GRANULARITY` indexes remember the data structure better. The `GRANULARITY` indicates how many granules will be used to construct the index. The more data is provided for the index, the more of it can be handled by one index and the more chances that with the right hyperparameters the index will remember the data structure better. But some indexes can't be built if they don't have enough data, so this granule will always participate in the query. For more information, see the description of indexes. +With greater `GRANULARITY` indexes remember the data structure better. The `GRANULARITY` indicates how many granules will be used to construct the index. The more data is provided for the index, the more of it can be handled by one index and the more chances that with the right hyper parameters the index will remember the data structure better. But some indexes can't be built if they don't have enough data, so this granule will always participate in the query. For more information, see the description of indexes. As the indexes are built only during insertions into table, `INSERT` and `OPTIMIZE` queries are slower than for ordinary table. At this stage indexes remember all the information about the given data. ANNIndexes should be used if you have immutable or rarely changed data and many read requests. @@ -135,7 +135,7 @@ ORDER BY id; Annoy supports `L2Distance` and `cosineDistance`. -In the `SELECT` in the settings (`ann_index_select_query_params`) you can specify the size of the internal buffer (more details in the description above or in the [original repository](https://github.com/spotify/annoy)). During the query it will inspect up to `search_k` nodes which defaults to `n_trees * n` if not provided. `search_k` gives you a run-time tradeoff between better accuracy and speed. +In the `SELECT` in the settings (`ann_index_select_query_params`) you can specify the size of the internal buffer (more details in the description above or in the [original repository](https://github.com/spotify/annoy)). During the query it will inspect up to `search_k` nodes which defaults to `n_trees * n` if not provided. `search_k` gives you a run-time trade-off between better accuracy and speed. __Example__: ``` sql diff --git a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md index edb320a2507..7e564b23676 100644 --- a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -165,7 +165,7 @@ Performance of such a query heavily depends on the table layout. Because of that The key factors for a good performance: -- number of partitions involved in the query should be sufficiently large (more than `max_threads / 2`), otherwise query will underutilize the machine +- number of partitions involved in the query should be sufficiently large (more than `max_threads / 2`), otherwise query will under-utilize the machine - partitions shouldn't be too small, so batch processing won't degenerate into row-by-row processing - partitions should be comparable in size, so all threads will do roughly the same amount of work diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 79ced0b6ce5..1ab0f4057ff 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -856,7 +856,7 @@ Tags: - `perform_ttl_move_on_insert` — Disables TTL move on data part INSERT. By default if we insert a data part that already expired by the TTL move rule it immediately goes to a volume/disk declared in move rule. This can significantly slowdown insert in case if destination volume/disk is slow (e.g. S3). - `load_balancing` - Policy for disk balancing, `round_robin` or `least_used`. -Cofiguration examples: +Configuration examples: ``` xml @@ -1224,7 +1224,7 @@ Limit parameters (mainly for internal usage): * `max_single_read_retries` - Limits the number of attempts to read a chunk of data from Blob Storage. * `max_single_download_retries` - Limits the number of attempts to download a readable buffer from Blob Storage. * `thread_pool_size` - Limits the number of threads with which `IDiskRemote` is instantiated. -* `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurenly for one object. +* `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurrently for one object. Other parameters: * `metadata_path` - Path on local FS to store metadata files for Blob Storage. Default value is `/var/lib/clickhouse/disks//`. diff --git a/docs/en/engines/table-engines/special/executable.md b/docs/en/engines/table-engines/special/executable.md index 25049d7b46e..d976beeab6c 100644 --- a/docs/en/engines/table-engines/special/executable.md +++ b/docs/en/engines/table-engines/special/executable.md @@ -65,7 +65,7 @@ if __name__ == "__main__": main() ``` -The following `my_executable_table` is built from the output of `my_script.py`, which will generate 10 random strings everytime you run a `SELECT` from `my_executable_table`: +The following `my_executable_table` is built from the output of `my_script.py`, which will generate 10 random strings every time you run a `SELECT` from `my_executable_table`: ```sql CREATE TABLE my_executable_table ( @@ -223,4 +223,4 @@ SETTINGS pool_size = 4; ``` -ClickHouse will maintain 4 processes on-demand when your client queries the `sentiment_pooled` table. \ No newline at end of file +ClickHouse will maintain 4 processes on-demand when your client queries the `sentiment_pooled` table. diff --git a/docs/en/engines/table-engines/special/keepermap.md b/docs/en/engines/table-engines/special/keepermap.md index a1c7009b712..6ce56adbae9 100644 --- a/docs/en/engines/table-engines/special/keepermap.md +++ b/docs/en/engines/table-engines/special/keepermap.md @@ -72,7 +72,7 @@ Additionally, number of keys will have a soft limit of 4 for the number of keys. If multiple tables are created on the same ZooKeeper path, the values are persisted until there exists at least 1 table using it. As a result, it is possible to use `ON CLUSTER` clause when creating the table and sharing the data from multiple ClickHouse instances. -Of course, it's possible to manually run `CREATE TABLE` with same path on nonrelated ClickHouse instances to have same data sharing effect. +Of course, it's possible to manually run `CREATE TABLE` with same path on unrelated ClickHouse instances to have same data sharing effect. ## Supported operations {#table_engine-KeeperMap-supported-operations} diff --git a/docs/en/getting-started/example-datasets/amazon-reviews.md b/docs/en/getting-started/example-datasets/amazon-reviews.md index f35806aa66f..75e4549cb78 100644 --- a/docs/en/getting-started/example-datasets/amazon-reviews.md +++ b/docs/en/getting-started/example-datasets/amazon-reviews.md @@ -87,7 +87,7 @@ ORDER BY (marketplace, review_date, product_category); 3. We are now ready to insert the data into ClickHouse. Before we do, check out the [list of files in the dataset](https://s3.amazonaws.com/amazon-reviews-pds/tsv/index.txt) and decide which ones you want to include. -4. We will insert all of the US reviews - which is about 151M rows. The following `INSERT` command uses the `s3Cluster` table function, which allows the processing of mulitple S3 files in parallel using all the nodes of your cluster. We also use a wildcard to insert any file that starts with the name `https://s3.amazonaws.com/amazon-reviews-pds/tsv/amazon_reviews_us_`: +4. We will insert all of the US reviews - which is about 151M rows. The following `INSERT` command uses the `s3Cluster` table function, which allows the processing of multiple S3 files in parallel using all the nodes of your cluster. We also use a wildcard to insert any file that starts with the name `https://s3.amazonaws.com/amazon-reviews-pds/tsv/amazon_reviews_us_`: ```sql INSERT INTO amazon_reviews @@ -473,4 +473,4 @@ It runs quite a bit faster - which means the cache is helping us out here: └────────────┴───────────────────────────────────────────────────────────────────────┴────────────────────┴───────┘ 50 rows in set. Elapsed: 33.954 sec. Processed 150.96 million rows, 68.95 GB (4.45 million rows/s., 2.03 GB/s.) -``` \ No newline at end of file +``` diff --git a/docs/en/getting-started/example-datasets/cell-towers.md b/docs/en/getting-started/example-datasets/cell-towers.md index 048eecb285b..a84eb5d561f 100644 --- a/docs/en/getting-started/example-datasets/cell-towers.md +++ b/docs/en/getting-started/example-datasets/cell-towers.md @@ -317,7 +317,7 @@ To build a Superset dashboard using the OpenCelliD dataset you should: Make sure that you set **SSL** on when connecting to ClickHouse Cloud or other ClickHouse systems that enforce the use of SSL. ::: - ![Add ClickHouse as a Superset datasource](@site/docs/en/getting-started/example-datasets/images/superset-connect-a-database.png) + ![Add ClickHouse as a Superset data source](@site/docs/en/getting-started/example-datasets/images/superset-connect-a-database.png) ### Add the table **cell_towers** as a Superset **dataset** @@ -364,5 +364,5 @@ The data is also available for interactive queries in the [Playground](https://p This [example](https://play.clickhouse.com/play?user=play#U0VMRUNUIG1jYywgY291bnQoKSBGUk9NIGNlbGxfdG93ZXJzIEdST1VQIEJZIG1jYyBPUkRFUiBCWSBjb3VudCgpIERFU0M=) will populate the username and even the query for you. -Although you cannot create tables in the Playground, you can run all of the queries and even use Superset (adjust the hostname and port number). +Although you cannot create tables in the Playground, you can run all of the queries and even use Superset (adjust the host name and port number). ::: diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index 02965ed5e33..9ed8782e512 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -806,7 +806,7 @@ FROM 31 rows in set. Elapsed: 0.043 sec. Processed 7.54 million rows, 40.53 MB (176.71 million rows/s., 950.40 MB/s.) ``` -Maybe a little more near the end of the month, but overall we keep a good even distribution. Again this is unrealiable due to the filtering of the docs filter during data insertion. +Maybe a little more near the end of the month, but overall we keep a good even distribution. Again this is unreliable due to the filtering of the docs filter during data insertion. ## Authors with the most diverse impact @@ -940,7 +940,7 @@ LIMIT 10 10 rows in set. Elapsed: 0.106 sec. Processed 798.15 thousand rows, 13.97 MB (7.51 million rows/s., 131.41 MB/s.) ``` -This makes sense because Alexey has been responsible for maintaining the Change log. But what if we use the basename of the file to identify his popular files - this allows for renames and should focus on code contributions. +This makes sense because Alexey has been responsible for maintaining the Change log. But what if we use the base name of the file to identify his popular files - this allows for renames and should focus on code contributions. [play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBiYXNlLAogICAgY291bnQoKSBBUyBjCkZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCldIRVJFIChhdXRob3IgPSAnQWxleGV5IE1pbG92aWRvdicpIEFORCAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKSkKR1JPVVAgQlkgYmFzZW5hbWUocGF0aCkgQVMgYmFzZQpPUkRFUiBCWSBjIERFU0MKTElNSVQgMTA=) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index 7093a2df04f..df28809495c 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -9,7 +9,7 @@ The data in this dataset is derived and cleaned from the full OpenSky dataset to Source: https://zenodo.org/record/5092942#.YRBCyTpRXYd -Martin Strohmeier, Xavier Olive, Jannis Lübbe, Matthias Schäfer, and Vincent Lenders +Martin Strohmeier, Xavier Olive, Jannis Luebbe, Matthias Schaefer, and Vincent Lenders "Crowdsourced air traffic data from the OpenSky Network 2019–2020" Earth System Science Data 13(2), 2021 https://doi.org/10.5194/essd-13-357-2021 diff --git a/docs/en/getting-started/example-datasets/reddit-comments.md b/docs/en/getting-started/example-datasets/reddit-comments.md index e1e372746c9..49c7bd25f9f 100644 --- a/docs/en/getting-started/example-datasets/reddit-comments.md +++ b/docs/en/getting-started/example-datasets/reddit-comments.md @@ -469,7 +469,7 @@ The response is: 10 rows in set. Elapsed: 27.824 sec. Processed 6.74 billion rows, 53.26 GB (242.22 million rows/s., 1.91 GB/s.) ``` -11. Let's see which subreddits had the biggest increase in commnents from 2018 to 2019: +11. Let's see which subreddits had the biggest increase in comments from 2018 to 2019: ```sql SELECT @@ -633,4 +633,4 @@ ORDER BY quarter ASC; └────────────┴────────────┴───────────┴──────────┘ 58 rows in set. Elapsed: 2663.751 sec. Processed 6.74 billion rows, 1.21 TB (2.53 million rows/s., 454.37 MB/s.) -``` \ No newline at end of file +``` diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md index 5f4ef696b8b..e24c6e5a6dc 100644 --- a/docs/en/getting-started/example-datasets/youtube-dislikes.md +++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md @@ -22,7 +22,7 @@ The steps below will easily work on a local install of ClickHouse too. The only ## Step-by-step instructions -1. Let's see what the data looks like. The `s3cluster` table function returns a table, so we can `DESCRIBE` the reult: +1. Let's see what the data looks like. The `s3cluster` table function returns a table, so we can `DESCRIBE` the result: ```sql DESCRIBE s3Cluster( @@ -322,7 +322,7 @@ ORDER BY month ASC; A spike of uploaders [around covid is noticeable](https://www.theverge.com/2020/3/27/21197642/youtube-with-me-style-videos-views-coronavirus-cook-workout-study-home-beauty). -### More subtitiles over time and when +### More subtitles over time and when With advances in speech recognition, it’s easier than ever to create subtitles for video with youtube adding auto-captioning in late 2009 - was the jump then? @@ -484,4 +484,4 @@ ARRAY JOIN │ 20th │ 16 │ │ 10th │ 6 │ └────────────┴─────────┘ -``` \ No newline at end of file +``` diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4f3b4e43358..ad113c58bce 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3492,7 +3492,7 @@ Default value: `0`. ## database_replicated_initial_query_timeout_sec {#database_replicated_initial_query_timeout_sec} -Sets how long initial DDL query should wait for Replicated database to precess previous DDL queue entries in seconds. +Sets how long initial DDL query should wait for Replicated database to process previous DDL queue entries in seconds. Possible values: diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index ccdc2d8c742..2915b053458 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -28,7 +28,7 @@ The `system.columns` table contains the following columns (the column type is sh - `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sampling key expression. - `compression_codec` ([String](../../sql-reference/data-types/string.md)) — Compression codec name. - `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bit width for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for `Decimal` types. Otherwise, the `NULL` value is returned. - `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Decimal precision of `DateTime64` data type. For other data types, the `NULL` value is returned. diff --git a/docs/en/operations/system-tables/dropped_tables.md b/docs/en/operations/system-tables/dropped_tables.md index 144c03109ac..e2a09094c87 100644 --- a/docs/en/operations/system-tables/dropped_tables.md +++ b/docs/en/operations/system-tables/dropped_tables.md @@ -12,7 +12,7 @@ Columns: - `table` ([String](../../sql-reference/data-types/string.md)) — Table name. - `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Table uuid. - `engine` ([String](../../sql-reference/data-types/string.md)) — Table engine name. -- `metadata_dropped_path` ([String](../../sql-reference/data-types/string.md)) — Path of table's metadata file in metadate_dropped directory. +- `metadata_dropped_path` ([String](../../sql-reference/data-types/string.md)) — Path of table's metadata file in metadata_dropped directory. - `table_dropped_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The time when the next attempt to remove table's data is scheduled on. Usually it's the table when the table was dropped plus `database_atomic_delay_before_drop_table_sec` **Example** diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index 07e9a9e2f58..35fd3a753b5 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -43,7 +43,7 @@ Columns: - `data_type` ([String](../../sql-reference/data-types/string.md)) — Column type. - `character_maximum_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. - `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bit width for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for `Decimal` types. Otherwise, the `NULL` value is returned. - `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Decimal precision of `DateTime64` data type. For other data types, the `NULL` value is returned. diff --git a/docs/en/operations/system-tables/licenses.md b/docs/en/operations/system-tables/licenses.md index 9296e78c797..0f09d559d8b 100644 --- a/docs/en/operations/system-tables/licenses.md +++ b/docs/en/operations/system-tables/licenses.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/licenses --- # licenses -Сontains licenses of third-party libraries that are located in the [contrib](https://github.com/ClickHouse/ClickHouse/tree/master/contrib) directory of ClickHouse sources. +Contains licenses of third-party libraries that are located in the [contrib](https://github.com/ClickHouse/ClickHouse/tree/master/contrib) directory of ClickHouse sources. Columns: diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index e61c6ed2ba4..9159d1e9284 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -100,7 +100,7 @@ Columns: - `move_ttl_info.expression` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of expressions. Each expression defines a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). :::note -The `move_ttl_info.expression` array is kept mostly for backward compatibility, now the simpliest way to check `TTL MOVE` rule is to use the `move_ttl_info.min` and `move_ttl_info.max` fields. +The `move_ttl_info.expression` array is kept mostly for backward compatibility, now the simplest way to check `TTL MOVE` rule is to use the `move_ttl_info.min` and `move_ttl_info.max` fields. ::: - `move_ttl_info.min` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — Array of date and time values. Each element describes the minimum key value for a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). diff --git a/docs/en/operations/system-tables/tables.md b/docs/en/operations/system-tables/tables.md index 82e9fa206ea..e4461e14236 100644 --- a/docs/en/operations/system-tables/tables.md +++ b/docs/en/operations/system-tables/tables.md @@ -50,7 +50,7 @@ Columns: - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) - [Distributed](../../engines/table-engines/special/distributed.md#distributed) -- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise `NULL` (including underying `Buffer` table). +- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise `NULL` (including underlying `Buffer` table). - `total_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of bytes, if it is possible to quickly determine exact number of bytes for the table on storage, otherwise `NULL` (does not includes any underlying storage). diff --git a/docs/en/operations/system-tables/trace_log.md b/docs/en/operations/system-tables/trace_log.md index a5aae422be7..89d54adc30d 100644 --- a/docs/en/operations/system-tables/trace_log.md +++ b/docs/en/operations/system-tables/trace_log.md @@ -43,7 +43,7 @@ Columns: - `event` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) - For trace type `ProfileEvent` is the name of updated profile event, for other trace types is an empty string. -- `increment` ([UInt64](../../sql-reference/data-types/int-uint.md)) - For trace type `ProfileEvent` is the amount of incremnt of profile event, for other trace types is 0. +- `increment` ([UInt64](../../sql-reference/data-types/int-uint.md)) - For trace type `ProfileEvent` is the amount of increment of profile event, for other trace types is 0. **Example** diff --git a/docs/en/operations/system-tables/zookeeper_log.md b/docs/en/operations/system-tables/zookeeper_log.md index 970ed192a48..b7cc4e22cd6 100644 --- a/docs/en/operations/system-tables/zookeeper_log.md +++ b/docs/en/operations/system-tables/zookeeper_log.md @@ -33,7 +33,7 @@ Columns with request response parameters: - `zxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — ZooKeeper transaction ID. The serial number issued by the ZooKeeper server in response to a successfully executed request (`0` if the request was not executed/returned an error/the client does not know whether the request was executed). - `error` ([Nullable(Enum)](../../sql-reference/data-types/nullable.md)) — Error code. Can have many values, here are just some of them: - - `ZOK` — The request was executed seccessfully. + - `ZOK` — The request was executed successfully. - `ZCONNECTIONLOSS` — The connection was lost. - `ZOPERATIONTIMEOUT` — The request execution timeout has expired. - `ZSESSIONEXPIRED` — The session has expired. @@ -43,7 +43,7 @@ Columns with request response parameters: - `path_created` ([String](../../sql-reference/data-types/string.md)) — The path to the created ZooKeeper node (for responses to the `CREATE` request), may differ from the `path` if the node is created as a `sequential`. - `stat_czxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The `zxid` of the change that caused this ZooKeeper node to be created. - `stat_mzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The `zxid` of the change that last modified this ZooKeeper node. -- `stat_pzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The transaction ID of the change that last modified childern of this ZooKeeper node. +- `stat_pzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The transaction ID of the change that last modified children of this ZooKeeper node. - `stat_version` ([Int32](../../sql-reference/data-types/int-uint.md)) — The number of changes to the data of this ZooKeeper node. - `stat_cversion` ([Int32](../../sql-reference/data-types/int-uint.md)) — The number of changes to the children of this ZooKeeper node. - `stat_dataLength` ([Int32](../../sql-reference/data-types/int-uint.md)) — The length of the data field of this ZooKeeper node. diff --git a/docs/en/operations/utilities/clickhouse-obfuscator.md b/docs/en/operations/utilities/clickhouse-obfuscator.md index 077489ba76e..ad51e9c7776 100644 --- a/docs/en/operations/utilities/clickhouse-obfuscator.md +++ b/docs/en/operations/utilities/clickhouse-obfuscator.md @@ -24,7 +24,7 @@ It is designed to retain the following properties of data: Most of the properties above are viable for performance testing: -reading data, filtering, aggregatio, and sorting will work at almost the same speed +reading data, filtering, aggregation, and sorting will work at almost the same speed as on original data due to saved cardinalities, magnitudes, compression ratios, etc. It works in a deterministic fashion: you define a seed value and the transformation is determined by input data and by seed. diff --git a/docs/en/sql-reference/aggregate-functions/reference/cramersv.md b/docs/en/sql-reference/aggregate-functions/reference/cramersv.md index f412724ea08..e9e2c367610 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/cramersv.md +++ b/docs/en/sql-reference/aggregate-functions/reference/cramersv.md @@ -5,7 +5,7 @@ sidebar_position: 351 # cramersV -[Cramér's V](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V) (sometimes referred to as Cramér's phi) is a measure of association between two columns in a table. The result of the `cramersV` function ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. It may be viewed as the association between two variables as a percentage of their maximum possible variation. +[Cramer's V](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V) (sometimes referred to as Cramer's phi) is a measure of association between two columns in a table. The result of the `cramersV` function ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. It may be viewed as the association between two variables as a percentage of their maximum possible variation. **Syntax** @@ -69,4 +69,4 @@ Result: ┌─────cramersV(a, b)─┐ │ 0.8944271909999159 │ └────────────────────┘ -``` \ No newline at end of file +``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md b/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md index 8e577efbc4d..f5ad3a8a937 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md +++ b/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md @@ -6,7 +6,7 @@ sidebar_position: 352 # cramersVBiasCorrected -Cramér's V is a measure of association between two columns in a table. The result of the [`cramersV` function](./cramersv.md) ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. The function can be heavily biased, so this version of Cramér's V uses the [bias correction](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V#Bias_correction). +Cramer's V is a measure of association between two columns in a table. The result of the [`cramersV` function](./cramersv.md) ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. The function can be heavily biased, so this version of Cramer's V uses the [bias correction](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V#Bias_correction). diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md b/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md index 5f57407a419..3d833555a43 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md @@ -5,7 +5,7 @@ sidebar_position: 125 # groupBitAnd -Applies bitwise `AND` for series of numbers. +Applies bit-wise `AND` for series of numbers. ``` sql groupBitAnd(expr) diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md b/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md index 59be69540b0..138ee998405 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md @@ -5,7 +5,7 @@ sidebar_position: 126 # groupBitOr -Applies bitwise `OR` for series of numbers. +Applies bit-wise `OR` for series of numbers. ``` sql groupBitOr(expr) diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md b/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md index b00876a2fdf..168335a010c 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md @@ -5,7 +5,7 @@ sidebar_position: 127 # groupBitXor -Applies bitwise `XOR` for series of numbers. +Applies bit-wise `XOR` for series of numbers. ``` sql groupBitXor(expr) diff --git a/docs/en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest.md b/docs/en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest.md index 3da9645181e..d159eec7ce6 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest.md +++ b/docs/en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest.md @@ -30,11 +30,11 @@ Samples must belong to continuous, one-dimensional probability distributions. The null hypothesis is that samples come from the same distribution, e.g. F(x) = G(x) for all x. And the alternative is that the distributions are not identical. - `'greater'` - The null hypothesis is that values in the first sample are *stohastically smaller* than those in the second one, + The null hypothesis is that values in the first sample are *stochastically smaller* than those in the second one, e.g. the CDF of first distribution lies above and hence to the left of that for the second one. Which in fact means that F(x) >= G(x) for all x. And the alternative in this case is that F(x) < G(x) for at least one x. - `'less'`. - The null hypothesis is that values in the first sample are *stohastically greater* than those in the second one, + The null hypothesis is that values in the first sample are *stochastically greater* than those in the second one, e.g. the CDF of first distribution lies below and hence to the right of that for the second one. Which in fact means that F(x) <= G(x) for all x. And the alternative in this case is that F(x) > G(x) for at least one x. - `computation_method` — the method used to compute p-value. (Optional, default: `'auto'`.) [String](../../../sql-reference/data-types/string.md). diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md index 7b9addbbdde..b3e21e0e69e 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md @@ -14,7 +14,7 @@ The result depends on the order of running the query, and is nondeterministic. When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function. :::note -Using `quantileTDigestWeighted` [is not recommended for tiny data sets](https://github.com/tdunning/t-digest/issues/167#issuecomment-828650275) and can lead to significat error. In this case, consider possibility of using [`quantileTDigest`](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md) instead. +Using `quantileTDigestWeighted` [is not recommended for tiny data sets](https://github.com/tdunning/t-digest/issues/167#issuecomment-828650275) and can lead to significant error. In this case, consider possibility of using [`quantileTDigest`](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md) instead. ::: **Syntax** diff --git a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md index 9481172c25b..f7615d90790 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md @@ -18,7 +18,7 @@ stochasticLinearRegression(1.0, 1.0, 10, 'SGD') 1. `learning rate` is the coefficient on step length, when gradient descent step is performed. Too big learning rate may cause infinite weights of the model. Default is `0.00001`. 2. `l2 regularization coefficient` which may help to prevent overfitting. Default is `0.1`. 3. `mini-batch size` sets the number of elements, which gradients will be computed and summed to perform one step of gradient descent. Pure stochastic descent uses one element, however having small batches(about 10 elements) make gradient steps more stable. Default is `15`. -4. `method for updating weights`, they are: `Adam` (by default), `SGD`, `Momentum`, `Nesterov`. `Momentum` and `Nesterov` require little bit more computations and memory, however they happen to be useful in terms of speed of convergance and stability of stochastic gradient methods. +4. `method for updating weights`, they are: `Adam` (by default), `SGD`, `Momentum`, `Nesterov`. `Momentum` and `Nesterov` require little bit more computations and memory, however they happen to be useful in terms of speed of convergence and stability of stochastic gradient methods. ### Usage diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index 059c6acdb9e..0da273e01ad 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -22,7 +22,7 @@ Resolution: 1 second. The point in time is saved as a [Unix timestamp](https://en.wikipedia.org/wiki/Unix_time), regardless of the time zone or daylight saving time. The time zone affects how the values of the `DateTime` type values are displayed in text format and how the values specified as strings are parsed (‘2020-01-01 05:00:01’). -Timezone agnostic unix timestamp is stored in tables, and the timezone is used to transform it to text format or back during data import/export or to make calendar calculations on the values (example: `toDate`, `toHour` functions et cetera). The time zone is not stored in the rows of the table (or in resultset), but is stored in the column metadata. +Timezone agnostic Unix timestamp is stored in tables, and the timezone is used to transform it to text format or back during data import/export or to make calendar calculations on the values (example: `toDate`, `toHour` functions etc.). The time zone is not stored in the rows of the table (or in resultset), but is stored in the column metadata. A list of supported time zones can be found in the [IANA Time Zone Database](https://www.iana.org/time-zones) and also can be queried by `SELECT * FROM system.time_zones`. [The list](https://en.wikipedia.org/wiki/List_of_tz_database_time_zones) is also available at Wikipedia. @@ -30,7 +30,7 @@ You can explicitly set a time zone for `DateTime`-type columns when creating a t The [clickhouse-client](../../interfaces/cli.md) applies the server time zone by default if a time zone isn’t explicitly set when initializing the data type. To use the client time zone, run `clickhouse-client` with the `--use_client_time_zone` parameter. -ClickHouse outputs values depending on the value of the [date_time_output_format](../../operations/settings/settings.md#settings-date_time_output_format) setting. `YYYY-MM-DD hh:mm:ss` text format by default. Additionaly you can change the output with the [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) function. +ClickHouse outputs values depending on the value of the [date_time_output_format](../../operations/settings/settings.md#settings-date_time_output_format) setting. `YYYY-MM-DD hh:mm:ss` text format by default. Additionally, you can change the output with the [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) function. When inserting data into ClickHouse, you can use different formats of date and time strings, depending on the value of the [date_time_input_format](../../operations/settings/settings.md#settings-date_time_input_format) setting. @@ -120,9 +120,9 @@ FROM dt As timezone conversion only changes the metadata, the operation has no computation cost. -## Limitations on timezones support +## Limitations on time zones support -Some timezones may not be supported completely. There are a few cases: +Some time zones may not be supported completely. There are a few cases: If the offset from UTC is not a multiple of 15 minutes, the calculation of hours and minutes can be incorrect. For example, the time zone in Monrovia, Liberia has offset UTC -0:44:30 before 7 Jan 1972. If you are doing calculations on the historical time in Monrovia timezone, the time processing functions may give incorrect results. The results after 7 Jan 1972 will be correct nevertheless. diff --git a/docs/en/sql-reference/data-types/index.md b/docs/en/sql-reference/data-types/index.md index 88663968e50..508307a0543 100644 --- a/docs/en/sql-reference/data-types/index.md +++ b/docs/en/sql-reference/data-types/index.md @@ -27,7 +27,7 @@ ClickHouse data types include: - **Aggregation function types**: use [`SimpleAggregateFunction`](./simpleaggregatefunction.md) and [`AggregateFunction`](./aggregatefunction.md) for storing the intermediate status of aggregate function results - **Nested data structures**: A [`Nested` data structure](./nested-data-structures/index.md) is like a table inside a cell - **Tuples**: A [`Tuple` of elements](./tuple.md), each having an individual type. -- **Nullable**: [`Nullable`](./nullable.md) allows you to store a value as `NULL` when a value is "missing" (instead of the column gettings its default value for the data type) +- **Nullable**: [`Nullable`](./nullable.md) allows you to store a value as `NULL` when a value is "missing" (instead of the column settings its default value for the data type) - **IP addresses**: use [`IPv4`](./domains/ipv4.md) and [`IPv6`](./domains/ipv6.md) to efficiently store IP addresses - **Geo types**: for [geographical data](./geo.md), including `Point`, `Ring`, `Polygon` and `MultiPolygon` - **Special data types**: including [`Expression`](./special-data-types/expression.md), [`Set`](./special-data-types/set.md), [`Nothing`](./special-data-types/nothing.md) and [`Interval`](./special-data-types/interval.md) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index f7b4be64851..3a968992c13 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -984,7 +984,7 @@ SOURCE(ODBC(... invalidate_query 'SELECT update_time FROM dictionary_source wher ... ``` -For `Cache`, `ComplexKeyCache`, `SSDCache`, and `SSDComplexKeyCache` dictionaries both synchronious and asynchronous updates are supported. +For `Cache`, `ComplexKeyCache`, `SSDCache`, and `SSDComplexKeyCache` dictionaries both synchronous and asynchronous updates are supported. It is also possible for `Flat`, `Hashed`, `ComplexKeyHashed` dictionaries to only request data that was changed after the previous update. If `update_field` is specified as part of the dictionary source configuration, value of the previous update time in seconds will be added to the data request. Depends on source type (Executable, HTTP, MySQL, PostgreSQL, ClickHouse, or ODBC) different logic will be applied to `update_field` before request data from an external source. diff --git a/docs/en/sql-reference/functions/bit-functions.md b/docs/en/sql-reference/functions/bit-functions.md index 5b342fe4f24..3c07fe8bcbe 100644 --- a/docs/en/sql-reference/functions/bit-functions.md +++ b/docs/en/sql-reference/functions/bit-functions.md @@ -226,7 +226,7 @@ Result: Returns result of [logical conjuction](https://en.wikipedia.org/wiki/Logical_conjunction) (AND operator) of all bits at given positions. The countdown starts from 0 from the right to the left. -The conjuction for bitwise operations: +The conjuction for bit-wise operations: 0 AND 0 = 0 @@ -291,7 +291,7 @@ Result: Returns result of [logical disjunction](https://en.wikipedia.org/wiki/Logical_disjunction) (OR operator) of all bits at given positions. The countdown starts from 0 from the right to the left. -The disjunction for bitwise operations: +The disjunction for bit-wise operations: 0 OR 0 = 0 diff --git a/docs/en/sql-reference/functions/encryption-functions.md b/docs/en/sql-reference/functions/encryption-functions.md index 1224b7bc92b..b11bee83582 100644 --- a/docs/en/sql-reference/functions/encryption-functions.md +++ b/docs/en/sql-reference/functions/encryption-functions.md @@ -31,9 +31,9 @@ encrypt('mode', 'plaintext', 'key' [, iv, aad]) **Arguments** - `mode` — Encryption mode. [String](../../sql-reference/data-types/string.md#string). -- `plaintext` — Text thats need to be encrypted. [String](../../sql-reference/data-types/string.md#string). +- `plaintext` — Text that need to be encrypted. [String](../../sql-reference/data-types/string.md#string). - `key` — Encryption key. [String](../../sql-reference/data-types/string.md#string). -- `iv` — Initialization vector. Required for `-gcm` modes, optinal for others. [String](../../sql-reference/data-types/string.md#string). +- `iv` — Initialization vector. Required for `-gcm` modes, optional for others. [String](../../sql-reference/data-types/string.md#string). - `aad` — Additional authenticated data. It isn't encrypted, but it affects decryption. Works only in `-gcm` modes, for others would throw an exception. [String](../../sql-reference/data-types/string.md#string). **Returned value** @@ -233,7 +233,7 @@ decrypt('mode', 'ciphertext', 'key' [, iv, aad]) - `mode` — Decryption mode. [String](../../sql-reference/data-types/string.md#string). - `ciphertext` — Encrypted text that needs to be decrypted. [String](../../sql-reference/data-types/string.md#string). - `key` — Decryption key. [String](../../sql-reference/data-types/string.md#string). -- `iv` — Initialization vector. Required for `-gcm` modes, optinal for others. [String](../../sql-reference/data-types/string.md#string). +- `iv` — Initialization vector. Required for `-gcm` modes, Optional for others. [String](../../sql-reference/data-types/string.md#string). - `aad` — Additional authenticated data. Won't decrypt if this value is incorrect. Works only in `-gcm` modes, for others would throw an exception. [String](../../sql-reference/data-types/string.md#string). **Returned value** @@ -364,7 +364,7 @@ aes_decrypt_mysql('mode', 'ciphertext', 'key' [, iv]) - `mode` — Decryption mode. [String](../../sql-reference/data-types/string.md#string). - `ciphertext` — Encrypted text that needs to be decrypted. [String](../../sql-reference/data-types/string.md#string). - `key` — Decryption key. [String](../../sql-reference/data-types/string.md#string). -- `iv` — Initialization vector. Optinal. [String](../../sql-reference/data-types/string.md#string). +- `iv` — Initialization vector. Optional. [String](../../sql-reference/data-types/string.md#string). **Returned value** diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md index 1f695a13598..29486c58e6a 100644 --- a/docs/en/sql-reference/functions/geo/h3.md +++ b/docs/en/sql-reference/functions/geo/h3.md @@ -12,7 +12,7 @@ A latitude and longitude pair can be transformed to a 64-bit H3 index, identifyi The H3 index is used primarily for bucketing locations and other geospatial manipulations. -The full description of the H3 system is available at [the Uber Engeneering site](https://eng.uber.com/h3/). +The full description of the H3 system is available at [the Uber Engineering site](https://eng.uber.com/h3/). ## h3IsValid diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md index 63fe5ca8530..f4702eff44b 100644 --- a/docs/en/sql-reference/functions/geo/s2.md +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -249,7 +249,7 @@ s2RectAdd(s2pointLow, s2pointHigh, s2Point) **Returned values** - `s2PointLow` — Low S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). -- `s2PointHigh` — Hight S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/float.md). +- `s2PointHigh` — Height S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/float.md). **Example** diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 89afcca3799..2cf3408534f 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -1161,7 +1161,7 @@ wordShingleSimHashUTF8(string[, shinglesize]) **Arguments** - `string` — String. [String](/docs/en/sql-reference/data-types/string.md). -- `shinglesize` — The size of a word shingle. Optinal. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](/docs/en/sql-reference/data-types/int-uint.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](/docs/en/sql-reference/data-types/int-uint.md). **Returned value** diff --git a/docs/en/sql-reference/functions/logical-functions.md b/docs/en/sql-reference/functions/logical-functions.md index f5a1a6aac12..17a9fbb19fe 100644 --- a/docs/en/sql-reference/functions/logical-functions.md +++ b/docs/en/sql-reference/functions/logical-functions.md @@ -31,7 +31,7 @@ Alias: The [AND Operator](../../sql-reference/operators/index.md#logical-and-ope **Returned value** - `0`, if there at least one argument evaluates to `false`, -- `NULL`, if no argumetn evaluates to `false` and at least one argument is `NULL`, +- `NULL`, if no argument evaluates to `false` and at least one argument is `NULL`, - `1`, otherwise. Type: [UInt8](../../sql-reference/data-types/int-uint.md) or [Nullable](../../sql-reference/data-types/nullable.md)([UInt8](../../sql-reference/data-types/int-uint.md)). diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 9851378d4fd..22492f2830b 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -52,7 +52,7 @@ Alias: `ln(x)` ## exp2 -Returns 2 to the power of the given argumetn +Returns 2 to the power of the given argument **Syntax** diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index efe1a77c285..41ccfe121a4 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -206,7 +206,7 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). **Examples** -For [String](../../sql-reference/data-types/string.md) arguments the funtion returns the string length + 9 (terminating zero + length). +For [String](../../sql-reference/data-types/string.md) arguments the function returns the string length + 9 (terminating zero + length). Query: @@ -1352,7 +1352,7 @@ ORDER BY k ASC ClickHouse used the index in the same way as the previous time (`Processed 32.74 thousand rows`). The expression `k = '2017-09-15'` was not used when generating the result. -In examle the `indexHint` function allows to see adjacent dates. +In example the `indexHint` function allows to see adjacent dates. Result: diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 8662d08431c..5175bbf0615 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -393,7 +393,7 @@ Reverses a sequence of Unicode code points in a string. Assumes that the string ## format -Format the `pattern` string with the strings listed in the arguments, similar to formatting in Python. The pattern string can contain replacement fields surrounded by curly braces `{}`. Anything not contained in braces is considered literal text and copied verbatim into the output. Literal brace character can be escaped by two braces: `{{ '{{' }}` and `{{ '}}' }}`. Field names can be numbers (starting from zero) or empty (then they are implicitely given monotonically increasing numbers). +Format the `pattern` string with the strings listed in the arguments, similar to formatting in Python. The pattern string can contain replacement fields surrounded by curly braces `{}`. Anything not contained in braces is considered literal text and copied verbatim into the output. Literal brace character can be escaped by two braces: `{{ '{{' }}` and `{{ '}}' }}`. Field names can be numbers (starting from zero) or empty (then they are implicitly given monotonically increasing numbers). **Syntax** diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 56c527d734e..74d5d747193 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -6,7 +6,7 @@ sidebar_label: Replacing in Strings # Functions for Replacing in Strings -[General strings functions](string-functions.md) and [functions for searchin in strings](string-search-functions.md) are described separately. +[General strings functions](string-functions.md) and [functions for searching in strings](string-search-functions.md) are described separately. ## replaceOne diff --git a/docs/en/sql-reference/functions/udf.md b/docs/en/sql-reference/functions/udf.md index a58c1364780..9c6b1b0c66b 100644 --- a/docs/en/sql-reference/functions/udf.md +++ b/docs/en/sql-reference/functions/udf.md @@ -19,7 +19,7 @@ A function configuration contains the following settings: - `argument` - argument description with the `type`, and optional `name` of an argument. Each argument is described in a separate setting. Specifying name is necessary if argument names are part of serialization for user defined function format like [Native](../../interfaces/formats.md#native) or [JSONEachRow](../../interfaces/formats.md#jsoneachrow). Default argument name value is `c` + argument_number. - `format` - a [format](../../interfaces/formats.md) in which arguments are passed to the command. - `return_type` - the type of a returned value. -- `return_name` - name of retuned value. Specifying return name is necessary if return name is part of serialization for user defined function format like [Native](../../interfaces/formats.md#native) or [JSONEachRow](../../interfaces/formats.md#jsoneachrow). Optional. Default value is `result`. +- `return_name` - name of returned value. Specifying return name is necessary if return name is part of serialization for user defined function format like [Native](../../interfaces/formats.md#native) or [JSONEachRow](../../interfaces/formats.md#jsoneachrow). Optional. Default value is `result`. - `type` - an executable type. If `type` is set to `executable` then single command is started. If it is set to `executable_pool` then a pool of commands is created. - `max_command_execution_time` - maximum execution time in seconds for processing block of data. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`. - `command_termination_timeout` - time in seconds during which a command should finish after its pipe is closed. After that time `SIGTERM` is sent to the process executing the command. Optional. Default value is `10`. diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 8a8c86624d2..bfad16f8365 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -222,7 +222,7 @@ It also makes sense to specify a local table in the `GLOBAL IN` clause, in case ### Distributed Subqueries and max_rows_in_set -You can use [`max_rows_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) and [`max_bytes_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) to control how much data is tranferred during distributed queries. +You can use [`max_rows_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) and [`max_bytes_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) to control how much data is transferred during distributed queries. This is specially important if the `global in` query returns a large amount of data. Consider the following sql - ```sql diff --git a/docs/en/sql-reference/statements/alter/quota.md b/docs/en/sql-reference/statements/alter/quota.md index 74a184c1479..d41e2ff0f61 100644 --- a/docs/en/sql-reference/statements/alter/quota.md +++ b/docs/en/sql-reference/statements/alter/quota.md @@ -32,7 +32,7 @@ Limit the maximum number of queries for the current user with 123 queries in 15 ALTER QUOTA IF EXISTS qA FOR INTERVAL 15 month MAX queries = 123 TO CURRENT_USER; ``` -For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quaters: +For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quarters: ``` sql ALTER QUOTA IF EXISTS qB FOR INTERVAL 30 minute MAX execution_time = 0.5, FOR INTERVAL 5 quarter MAX queries = 321, errors = 10 TO default; diff --git a/docs/en/sql-reference/statements/create/quota.md b/docs/en/sql-reference/statements/create/quota.md index 7c31f93fff7..c69285171ab 100644 --- a/docs/en/sql-reference/statements/create/quota.md +++ b/docs/en/sql-reference/statements/create/quota.md @@ -32,7 +32,7 @@ Limit the maximum number of queries for the current user with 123 queries in 15 CREATE QUOTA qA FOR INTERVAL 15 month MAX queries = 123 TO CURRENT_USER; ``` -For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quaters: +For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quarters: ``` sql CREATE QUOTA qB FOR INTERVAL 30 minute MAX execution_time = 0.5, FOR INTERVAL 5 quarter MAX queries = 321, errors = 10 TO default; diff --git a/docs/en/sql-reference/statements/explain.md b/docs/en/sql-reference/statements/explain.md index 1c93707402f..2d7204c2796 100644 --- a/docs/en/sql-reference/statements/explain.md +++ b/docs/en/sql-reference/statements/explain.md @@ -115,7 +115,7 @@ CROSS JOIN system.numbers AS c Settings: -- `run_passes` — Run all query tree passes before dumping the query tree. Defaul: `1`. +- `run_passes` — Run all query tree passes before dumping the query tree. Default: `1`. - `dump_passes` — Dump information about used passes before dumping the query tree. Default: `0`. - `passes` — Specifies how many passes to run. If set to `-1`, runs all the passes. Default: `-1`. @@ -463,5 +463,5 @@ Result: ``` :::note -The validation is not complete, so a successfull query does not guarantee that the override would not cause issues. +The validation is not complete, so a successful query does not guarantee that the override would not cause issues. ::: diff --git a/docs/en/sql-reference/statements/select/from.md b/docs/en/sql-reference/statements/select/from.md index 4ca8e8287c0..a4f449ad321 100644 --- a/docs/en/sql-reference/statements/select/from.md +++ b/docs/en/sql-reference/statements/select/from.md @@ -34,7 +34,7 @@ Queries that use `FINAL` are executed slightly slower than similar queries that - Data is merged during query execution. - Queries with `FINAL` read primary key columns in addition to the columns specified in the query. -**In most cases, avoid using `FINAL`.** The common approach is to use different queries that assume the background processes of the `MergeTree` engine have’t happened yet and deal with it by applying aggregation (for example, to discard duplicates). +**In most cases, avoid using `FINAL`.** The common approach is to use different queries that assume the background processes of the `MergeTree` engine haven’t happened yet and deal with it by applying aggregation (for example, to discard duplicates). `FINAL` can be applied automatically using [FINAL](../../../operations/settings/settings.md#final) setting to all tables in a query using a session or a user profile. diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 712395a0357..3dfbd133364 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -289,7 +289,7 @@ When `FROM const_expr` not defined sequence of filling use minimal `expr` field When `TO const_expr` not defined sequence of filling use maximum `expr` field value from `ORDER BY`. When `STEP const_numeric_expr` defined then `const_numeric_expr` interprets `as is` for numeric types, as `days` for Date type, as `seconds` for DateTime type. It also supports [INTERVAL](https://clickhouse.com/docs/en/sql-reference/data-types/special-data-types/interval/) data type representing time and date intervals. When `STEP const_numeric_expr` omitted then sequence of filling use `1.0` for numeric type, `1 day` for Date type and `1 second` for DateTime type. -`INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous fields values by applying `expr`. If `expr` is not present will repeate previous value. Omitted list will result in including all allowed columns. +`INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous fields values by applying `expr`. If `expr` is not present will repeat previous value. Omitted list will result in including all allowed columns. Example of a query without `WITH FILL`: diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 7ee2102c14d..a8f494a5afc 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -21,7 +21,7 @@ ClickHouse supports the standard grammar for defining windows and window functio | `lag/lead(value, offset)` | Not supported. Workarounds: | | | 1) replace with `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead` | | | 2) use `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) | Supported. Specify window like, (partition by x order by y rows between unbounded preceding and unounded following). | +| ntile(buckets) | Supported. Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | ## ClickHouse-specific Window Functions @@ -39,7 +39,7 @@ The computed value is the following for each row: The roadmap for the initial support of window functions is [in this issue](https://github.com/ClickHouse/ClickHouse/issues/18097). -All GitHub issues related to window funtions have the [comp-window-functions](https://github.com/ClickHouse/ClickHouse/labels/comp-window-functions) tag. +All GitHub issues related to window functions have the [comp-window-functions](https://github.com/ClickHouse/ClickHouse/labels/comp-window-functions) tag. ### Tests diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ded7a4643a9..583a49631a3 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,376 +1,1159 @@ personal_ws-1.1 en 543 -AArch -ACLs -AMQP -ARMv -ASLR -ASan -Actian -AddressSanitizer -AppleClang -ArrowStream -AvroConfluent -BSON -BSONEachRow -Bool -BuilderBinAarch -BuilderBinAmd -CCTOOLS -CLion -CMake -CMakeLists -CPUs -CSVWithNames -CSVWithNamesAndTypes -CamelCase -CapnProto -CentOS -ClickHouse -ClickHouse's -ClickableSquare -CodeBlock -CodeLLDB -Compat -Config -ConnectionDetails -Contrib -Ctrl -CustomSeparated -CustomSeparatedWithNames -CustomSeparatedWithNamesAndTypes -DBMSs -DateTime -DateTimes -DockerHub -Doxygen -Encodings -Enum -Eoan -FixedString -FreeBSD -Fuzzer -Fuzzers -GTest -Gb -Gcc -GoogleTest -HDDs -Heredoc -Homebrew -Homebrew's -HorizontalDivide -Hostname -INSERTed -IPv -IntN -Integrations -JSONAsObject -JSONAsString -JSONColumns -JSONColumnsWithMetadata -JSONCompact -JSONCompactColumns -JSONCompactEachRow -JSONCompactEachRowWithNames -JSONCompactEachRowWithNamesAndTypes -JSONCompactStrings -JSONCompactStringsEachRow -JSONCompactStringsEachRowWithNames -JSONCompactStringsEachRowWithNamesAndTypes -JSONEachRow -JSONEachRowWithProgress -JSONObjectEachRow -JSONStrings -JSONStringsEachRow -JSONStringsEachRowWithProgress -JSONs -Jaeger -Jemalloc -Jepsen -KDevelop -LGPL -LLDB -LLVM's -LOCALTIME -LOCALTIMESTAMP -LibFuzzer -LineAsString -LinksDeployment -LowCardinality -MEMTABLE -MSan -MVCC -MacOS -Memcheck -MemorySanitizer -MergeTree -MessagePack -MiB -MsgPack -Multiline -Multithreading -MySQLDump -NEKUDOTAYIM -NULLIF -NVME -NYPD -NuRaft -OLAP -OLTP -ObjectId -Observability -Ok -OpenSSL -OpenSUSE -OpenStack -OpenTelemetry -PAAMAYIM -ParquetMetadata -Parsers -Postgres -Precompiled -PrettyCompact -PrettyCompactMonoBlock -PrettyCompactNoEscapes -PrettyCompactNoEscapesMonoBlock -PrettyJSONEachRow -PrettyMonoBlock -PrettyNoEscapes -PrettyNoEscapesMonoBlock -PrettySpace -PrettySpaceMonoBlock -PrettySpaceNoEscapes -PrettySpaceNoEscapesMonoBlock -Protobuf -ProtobufSingle -QEMU -QTCreator -QueryCacheHits -QueryCacheMisses -RBAC -RawBLOB -RedHat -ReplicatedMergeTree -RowBinary -RowBinaryWithNames -RowBinaryWithNamesAndTypes -Runtime -SATA -SELECTs -SERIALIZABLE -SIMD -SLES -SMALLINT -SQLInsert -SQLSTATE -SSSE -Schemas -SelfManaged -Stateful -Submodules -Subqueries -TSVRaw -TSan -TabItem -TabSeparated -TabSeparatedRaw -TabSeparatedRawWithNames -TabSeparatedRawWithNamesAndTypes -TabSeparatedWithNames -TabSeparatedWithNamesAndTypes -TargetSpecific -TemplateIgnoreSpaces -Testflows -Tgz -Toolset -Tradeoff -Transactional -TwoColumnList -UBSan -UInt -UIntN -UPDATEs -Uint -Updatable -Util -Valgrind -Vectorized -VideoContainer -ViewAllLink -VirtualBox -Werror -WithNamesAndTypes -Woboq -WriteBuffer -WriteBuffers -XCode -YAML -YYYY -Yasm -Zipkin -ZooKeeper -ZooKeeper's aarch +AArch +accurateCast +accurateCastOrDefault +accurateCastOrNull +ACLs +acos +acosh +Actian +ActionsMenu +activecube +activerecord +ActiveRecord +addDays +addHours +Additionaly +addMinutes +addMonths +addQuarters +addr +AddressSanitizer +addressToLine +addressToLineWithInlines +addressToSymbol +addSeconds +addWeeks +addYears +adhoc +adviced +aggregatefunction +Aggregatefunction +AggregateFunction +aggregatingmergetree +AggregatingMergeTree +aggregatio +AggregatorThreads +AggregatorThreadsActive +aggretate +aiochclient +Akka +alculates +AlertManager +Alexey allocator +alphaTokens +ALTERs +amplab +AMPLab +AMQP analytics +ANNIndex +ANNIndexes +anonymize anonymized ansi +AnyEvent +anyheavy +anyHeavy +anyIf +anylast +anyLast +AORM +APIs +appendTrailingCharIfAbsent +AppleClang +approximatly +argmax +argMax +argmin +argMin +arguments's +argumetn +arithmetics +ARMv +arrayAll +arrayAUC +arrayAvg +arrayCompact +arrayConcat +arrayCount +arrayCumSum +arrayCumSumNonNegative +arrayDifference +arrayDistinct +arrayElement +arrayEnumerate +arrayEnumerateDense +arrayEnumerateUniq +arrayExists +arrayFill +arrayFilter +arrayFirst +arrayFirstIndex +arrayFlatten +arrayIntersect +arrayJoin +ArrayJoin +arrayLast +arrayLastIndex +arrayMap +arrayMax +arrayMin +arrayPartialReverseSort +arrayPartialSort +arrayPopBack +arrayPopFront +arrayProduct +arrayPushBack +arrayPushFront +arrayReduce +arrayReduceInRanges +arrayResize +arrayReverse +arrayReverseFill +arrayReverseSort +arrayReverseSplit +arraySlice +arraySort +arraySplit +arrayStringConcat +arraySum +arrayUniq +arrayWithConstant +arrayZip +ArrowStream +ASan +ascii +asin +asinh +ASLR +ASOF +assumeNotNull +asterics async +asynch +AsynchronousHeavyMetricsCalculationTimeSpent +AsynchronousHeavyMetricsUpdateInterval +AsynchronousInsert +AsynchronousInsertThreads +AsynchronousInsertThreadsActive +AsynchronousMetricsCalculationTimeSpent +AsynchronousMetricsUpdateInterval +AsynchronousReadWait +AsyncInsertCacheSize +atan +atanh atomicity +auth +authenticator +Authenticator +authenticators +Authenticators +autocompletion +Autocompletion +autodetect +autodetected autogenerated autogeneration +AutoML autostart +avgweighted +avgWeighted avro +AvroConfluent avx aws backend +BackgroundBufferFlushSchedulePool +BackgroundBufferFlushSchedulePoolSize +BackgroundBufferFlushSchedulePoolTask +BackgroundCommonPoolSize +BackgroundCommonPoolTask +BackgroundDistributedSchedulePool +BackgroundDistributedSchedulePoolSize +BackgroundDistributedSchedulePoolTask +BackgroundFetchesPoolSize +BackgroundFetchesPoolTask +BackgroundMergesAndMutationsPoolSize +BackgroundMergesAndMutationsPoolTask +BackgroundMessageBrokerSchedulePoolSize +BackgroundMessageBrokerSchedulePoolTask +BackgroundMovePoolSize +BackgroundMovePoolTask +BackgroundProcessingPool +BackgroundSchedulePool +BackgroundSchedulePoolSize +BackgroundSchedulePoolTask backoff backticks +BackupsIO +BackupsIOThreads +BackupsIOThreadsActive +BackupsThreads +BackupsThreadsActive +balancer +basename +bcrypt benchmarking +BestEffort +BestEffortOrNull +BestEffortOrZero +BestEffortUS +BestEffortUSOrNull +BestEffortUSOrZero +bfloat +BIGINT +BIGSERIAL +binlog +bitAnd +bitCount +bitHammingDistance +bitmapAnd +bitmapAndCardinality +bitmapAndnot +bitmapAndnotCardinality +bitmapBuild +bitmapCardinality +bitmapContains +bitmapHasAll +bitmapHasAny +bitmapMax +bitmapMin +bitmapOr +bitmapOrCardinality +bitmapSubsetInRange +bitmapSubsetLimit +bitmapToArray +bitmapTransform +bitmapXor +bitmapXorCardinality +bitmask +bitmaskToArray +bitmaskToList +bitNot +bitOr +bitov +bitPositionsToArray +bitRotateLeft +bitRotateRight +bitShiftLeft +bitShiftRight +bitSlice +bitTest +bitTestAll +bitTestAny +bitXor blake +Blazingly +BlockActiveTime +BlockDiscardBytes +BlockDiscardMerges +BlockDiscardOps +BlockDiscardTime +BlockInFlightOps +blockinfo +blockNumber +BlockQueueTime +BlockReadBytes +blockreader +BlockReadMerges +BlockReadOps +BlockReadTime +blockSerializedSize +blocksize blockSize +BlockWriteBytes +BlockWriteMerges +BlockWriteOps +BlockWriteTime +bool +Bool boolean bools boringssl +BORO +bozerkins +broadcasted +BrokenDistributedFilesToInsert brotli bson +BSON bsoneachrow +BSONEachRow +buffersize buildable +BuilderBinAarch +BuilderBinAmd +buildId +BuildID +builtins +bytebase +Bytebase +byteSize +bytesToCutForIPv +CacheDetachedFileSegments +CacheDictionaries +CacheDictionary +CacheDictionaryThreads +CacheDictionaryThreadsActive +CacheDictionaryUpdateQueueBatches +CacheDictionaryUpdateQueueKeys +CacheFileSegments +cacheSessions +cachesize +caConfig camelCase +CamelCase +CapContains capn +Cap'n capnproto +CapnProto +CapUnion +cardinalities cardinality +cartesian cassandra +casted +catboost +CatBoost +catboostEvaluate +categoricalinformationvalue +categoricalInformationValue +cathetus cbindgen +cbrt ccache +CCTOOLS cctz +CDATA +CDFs +CDMA +ceil +CellAreaM +CellAreaRads +CellsIntersect +CentOS +centroid +certificateFile +CertificateHandler +CESU +cetera cfg +chadmin +Chadmin changelog changelogs +ChannelID charset charsets +chconn checkouting checksummed checksumming checksums +childern +chproxy +chunksize +cickhouse +Cidr +CIDR +CIDRToRange +cipherList +ciphertext +Ciphertext cityhash +cityHash +CityHash +CLang cli +ClickableSquare +clickcat +ClickCat clickhouse +ClickHouse +ClickHouseClient +clickhousedb +ClickHouseMigrator +ClickHouseNIO +ClickHouse's +ClickHouseVapor +clickhousex +clickmate clickstream +clickvisual +ClickVisual +CLion +CLOB +clockhour +cLoki +CloudDetails +clusterAllReplicas cmake +CMake +CMakeLists +CMPLNT codebase +CodeBlock codec +codecs +Codecs +CODECS +CodeLLDB +codepoint +codepoints +collapsingmergetree +CollapsingMergeTree +combinator +combinators +Combinators +commnents comparising +Compat +compatitalbe +CompiledExpressionCacheBytes +CompiledExpressionCacheCount +ComplexKeyCache +ComplexKeyDirect +ComplexKeyHashed +compressability +concat +concatAssumeInjective +concatWithSeparator +concatWithSeparatorAssumeInjective +concurenly +cond +conf config +Config configs +congruential +conjuction +conjuctive +ConnectionDetails +const +Const +ContextLockWait contrib +Contrib +convergance +convertCharset coroutines +cosineDistance +countDigits +countEqual +countMatches +countSubstrings +covariates +covarpop +covarPop +covarsamp +covarSamp +covid +Covid +COVID cpp cppkafka cpu +CPUFrequencyMHz +CPUs +Cramer's +cramersv +cramersV +cramersvbiascorrected +cramersVBiasCorrected +criteo +Criteo crlf croaring cronjob +Crotty +Crowdsourced +cryptocurrencies +cryptocurrency +cryptographic csv +CSVs csvwithnames +CSVWithNames csvwithnamesandtypes +CSVWithNamesAndTypes +CTEs +Ctrl +currentDatabase +CurrentMetrics +currentProfiles +currentRoles +currentUser +customizable +customizations customseparated +CustomSeparated customseparatedwithnames +CustomSeparatedWithNames customseparatedwithnamesandtypes +CustomSeparatedWithNamesAndTypes +cutFragment +cutIPv +cutQueryString +cutQueryStringAndFragment +cutToFirstSignificantSubdomain +cutToFirstSignificantSubdomainCustom +cutToFirstSignificantSubdomainCustomWithWWW +cutToFirstSignificantSubdomainWithWWW +cutURLParameter +cutWWW cyrus +DatabaseCatalog +DatabaseCatalogThreads +DatabaseCatalogThreadsActive +DatabaseOnDisk +DatabaseOnDiskThreads +DatabaseOnDiskThreadsActive +DatabaseOrdinaryThreads +DatabaseOrdinaryThreadsActive datacenter +datacenters datafiles +datagrip +DataGrip +datalens +DataLens +datanode dataset datasets +datasource +DataTime +datatypes +DataTypes +dateName datetime +dateTime +DateTime datetimes +DateTimes +dateTimeToSnowflake +dayofyear +dbal +DBAs +DbCL +dbeaver +DBeaver +dbgen dbms +DBMSs ddl +DDLWorker +DDLWORKER +DDLWorkerThreads +DDLWorkerThreadsActive deallocation +deallocations debian +decodeURLComponent +decodeURLFormComponent +decodeXMLComponent decompressor +decrypt +DECRYPT +decrypted +Decrypted +decrypts +deduplicate +Deduplicate +deduplicated +deduplicating +deduplication +Deduplication +defaultProfiles +defaultRoles +defaultValueOfArgumentType +defaultValueOfTypeName +DelayedInserts +DeliveryTag +deltalake +deltaLake +DeltaLake +deltasum +deltaSum +deltasumtimestamp +deltaSumTimestamp +demangle +denormalize +Denormalize +denormalized denormalizing denormals +DESC deserialization deserialized +deserializing +DestroyAggregatesThreads +DestroyAggregatesThreadsActive destructor destructors +detectCharset +detectLanguage +detectLanguageMixed +detectLanguageUnknown +determinator +deterministically +DictCacheRequests +dictGet +dictGetChildren +dictGetDescendant +dictGetHierarchy +dictGetOrDefault +dictGetOrNull +dictGetUUID +dictHas +dictIsIn +disableProtocols +disjunction +disjunctions +DiskAvailable +DiskObjectStorage +DiskObjectStorageAsyncThreads +DiskObjectStorageAsyncThreadsActive +DiskSpaceReservedForMerge +DiskTotal +DiskUnreserved +DiskUsed +displaySecretsInShowAndSelect +DistributedFilesToInsert +DistributedSend +distro +divideDecimal dmesg +DockerHub +DOGEFI +domainWithoutWWW dont +dotProduct +DoubleDelta +Doxygen +dplyr dragonbox +dropoff +dumpColumnStructure durations +ECMA +ecto +Ecto +EdgeAngle +EdgeLengthKm +EdgeLengthM +EmbeddedRocksDB +embeddings +Embeddings +emptyArray +emptyArrayDate +emptyArrayDateTime +emptyArrayFloat +emptyArrayInt +emptyArrayString +emptyArrayToSingle +emptyArrayUInt +enabledProfiles +enabledRoles +encodeURLComponent +encodeURLFormComponent +encodeXMLComponent encodings +Encodings +encryptions endian +endsWith +Engeneering enum +Enum +enum's +enums +Enums +Eoan +EphemeralNode +erfc +errorCodeToName +Ethereum +evalMLMethod +everytime +ExactEdgeLengthKm +ExactEdgeLengthM +ExactEdgeLengthRads +ExecutablePool exFAT +expiryMsec +exponentialmovingaverage +exponentialMovingAverage +expr +exprN +extendedVerification +extention +ExternalDistributed +extractAll +extractAllGroups +extractAllGroupsHorizontal +extractAllGroupsVertical +extractKeyValuePairs +extractKeyValuePairsWithEscaping +extractTextFromHTML +extractURLParameter +extractURLParameterNames +extractURLParameters +ExtType +failover +Failover +farmFingerprint +farmHash +FarmHash fastops fcoverage +FFFD filesystem +filesystemAvailable +FilesystemCacheBytes +FilesystemCacheElements +FilesystemCacheFiles +FilesystemCacheReadBuffers +FilesystemCacheSize +filesystemCapacity +filesystemFree +FilesystemLogsPathAvailableBytes +FilesystemLogsPathAvailableINodes +FilesystemLogsPathTotalBytes +FilesystemLogsPathTotalINodes +FilesystemLogsPathUsedBytes +FilesystemLogsPathUsedINodes +FilesystemMainPathAvailableBytes +FilesystemMainPathAvailableINodes +FilesystemMainPathTotalBytes +FilesystemMainPathTotalINodes +FilesystemMainPathUsedBytes +FilesystemMainPathUsedINodes filesystems +finalizeAggregation +fips +FIPS +firstSignificantSubdomain +firstSignificantSubdomainCustom +fixedstring +FixedString +flamegraph flatbuffers +flink +Flink +fluentd fmtlib +ForEach +formatDateTime +formatDateTimeInJoda +formatDateTimeInJodaSyntax +formated +formatReadableDecimalSize +formatReadableQuantity +formatReadableSize +formatReadableTimeDelta +formatRow +formatRowNoNewline formatschema formatter +FOSDEM +FQDN +FreeBSD +freezed +fromModifiedJulianDay +fromModifiedJulianDayOrNull +fromUnixTimestamp +fromUnixTimestampInJodaSyntax fsync +func +funtion +fuzzBits fuzzer +Fuzzer fuzzers -gRPC +Fuzzers +Gb +Gbit +Gcc +gccMurmurHash gcem +generateRandom +GenerateRandom +generateULID +generateUUIDv +geobase +geobases +Geobases +geocode +GeoCoord +geoDistance +geohash +Geohash +geohashDecode +geohashEncode +geohashesInBox +Geoid +geoip +geospatial +geoToH +geoToS +GetBaseCell +getblockinfo +GetDestinationIndexFromUnidirectionalEdge +getevents +GetFaces +GetIndexesFromUnidirectionalEdge +getMacro +GetNeighbors +GetOriginIndexFromUnidirectionalEdge +getOSKernelVersion +GetPentagonIndexes +GetRes +GetResolution +getServerPort +getSetting +getSizeOfEnumType +GetUnidirectionalEdge +GetUnidirectionalEdgeBoundary +GetUnidirectionalEdgesFromHexagon github +GitLab glibc +globalIn +globalNotIn +GlobalThread +GlobalThreadActive +glushkovds +GoLand +golang googletest +GoogleTest +grafana +Grafana +graphitemergetree +GraphiteMergeTree +graphouse +graphql +GraphQL +greatCircleAngle +greatCircleDistance +greaterOrEquals +greenspace +Greenwald +grouparray +groupArray +grouparrayinsertat +groupArrayInsertAt +grouparraylast +groupArrayLast +grouparraymovingavg +groupArrayMovingAvg +grouparraymovingsum +groupArrayMovingSum +grouparraysample +groupArraySample +groupbitand +groupBitAnd +groupbitmap +groupBitmap +groupbitmapand +groupBitmapAnd +groupbitmapor +groupBitmapOr +groupbitmapxor +groupBitmapXor +groupbitor +groupBitOr +groupbitxor +groupBitXor +groupuniqarray +groupUniqArray grpc +gRPC grpcio gtest +GTest +gtid +GTID +gzip +gzipped +hadoop +halfday +halfMD hardlinks +hasAll +hasAny +hasColumnInTable +HashedDictionary +HashedDictionaryThreads +HashedDictionaryThreadsActive +hashtables +hasSubstr +hasToken +hasTokenCaseInsensitive +hasTokenCaseInsensitiveOrNull +hasTokenOrNull +haversine +Haversine +have't +hdbc +HDDs hdfs +hdfsCluster heredoc +Heredoc heredocs +HexAreaKm +HexAreaM +HexRing +HHMM +Hight +hiveHash +HMAC +holistics +Holistics homebrew +Homebrew +Homebrew's +hopEnd +hopStart +horgh +HorizontalDivide +hostname +hostName +Hostname +hostnames +houseops +HouseOps +hsts +HSTS +html http +HTTPConnection https +HTTPThreads +hudi +Hudi +HyperLogLog hyperscan +hypot +Hypot +hyvor +IANA +icosahedron icudata +idempotency +identifiant +Identifiant +ifNotFinite +ifNull +iframe +ilike +IMDS +implicitely +incrementing +Incrementing +incremnt +IndexesAreNeighbors +indexHint +indexOf +infi +INFILE +InfluxDB +initializeAggregation +initialQueryID +injective +innogames +inodes +INSERTed +INSERTs +Instana instantiation +intDiv +intDivOrZero integrational integrations +Integrations +IntelliJ interserver +InterserverConnection +InterserverThreads +intervalLengthSum +intExp +intHash +IntN +introspections +invalidCertificateHandler invariants +invertedindexes +IOPrefetchThreads +IOPrefetchThreadsActive +IOThreads +IOThreadsActive +IOUringInFlightEvents +IOUringPendingEvents +IOWriterThreads +IOWriterThreadsActive +IPTrie +IPv +isConstant +isDecimalOverflow +isFinite +isInfinite +isIPAddressInRange +isIPv +isNaN +isNotNull +isNull +IsPentagon +IsResClassIII +IsValid +isValidJSON +isValidUTF +iteratively +Jaeger +Jannis +javaHash +JavaHash +javaHashUTF +jbod +JBOD jdbc jemalloc +Jemalloc +Jepsen +JetBrains +Jitter +Joda +JOINed +joinGet +JOINs json +JSONArrayLength +JSONAsObject jsonasstring +JSONAsString jsoncolumns +JSONColumns jsoncolumnsmonoblock +JSONColumnsWithMetadata jsoncompact +JSONCompact jsoncompactcolumns +JSONCompactColumns jsoncompacteachrow +JSONCompactEachRow jsoncompacteachrowwithnames +JSONCompactEachRowWithNames jsoncompacteachrowwithnamesandtypes +JSONCompactEachRowWithNamesAndTypes jsoncompactstrings +JSONCompactStrings jsoncompactstringseachrow +JSONCompactStringsEachRow jsoncompactstringseachrowwithnames +JSONCompactStringsEachRowWithNames jsoncompactstringseachrowwithnamesandtypes +JSONCompactStringsEachRowWithNamesAndTypes jsoneachrow +JSONEachRow jsoneachrowwithprogress +JSONEachRowWithProgress +JSONExtract +JSONExtractArrayRaw +JSONExtractBool +JSONExtractFloat +JSONExtractInt +JSONExtractKeys +JSONExtractKeysAndValues +JSONExtractKeysAndValuesRaw +JSONExtractRaw +JSONExtractString +JSONExtractUInt +JSONHas +JSONLength jsonobjecteachrow +JSONObjectEachRow +JSONs jsonstrings +JSONStrings jsonstringseachrow +JSONStringsEachRow jsonstringseachrowwithprogress +JSONStringsEachRowWithProgress +JSONType +jumpConsistentHash +JumpConsistentHash +Jupyter kafka +KafkaAssignedPartitions +KafkaBackgroundReads kafkacat +KafkaConsumers +KafkaConsumersInUse +KafkaConsumersWithAssignment +KafkaLibrdkafkaThreads +kafkaMurmurHash +KafkaProducers +KafkaWrites +Kahan +KDevelop +KeeperAliveConnections +keepermap +KeeperMap +KeeperOutstandingRequets +kerberized +kerberos +Kerberos +kernal +keyspace +keytab +Khanna +kittenhouse +KittenHouse +Klickhouse +Kolmogorov +kolmogorovsmirnovtest +kolmogorovSmirnovTest +kolya konsole +kRing +Kubernetes +kurtosis +kurtpop +kurtPop +kurtsamp +kurtSamp laion +lang +laravel latencies +ldap +LDAP +learing +leftPad +leftPadUTF +lemmatization +lemmatize +lemmatized +lengthUTF +lessOrEquals lexicographically -libFuzzer +lgamma +LGPL libc +libcatboost libcpuid libcxx libcxxabi libdivide libfarmhash libfuzzer +libFuzzer +LibFuzzer libgsasl libhdfs libmetrohash @@ -381,193 +1164,1405 @@ libs libunwind libuv libvirt +LightHouse linearizability linearizable +linearized lineasstring +LineAsString linefeeds lineorder +Linf +LinfDistance +LinfNorm +LinfNormalize +LinksDeployment +Linq linux +LLDB llvm +LLVM's +LoadAverage +loadDefaultCAFile localhost +localread +LocalThread +LocalThreadActive +LOCALTIME +LOCALTIMESTAMP +logagent +loghouse +LogQL +Logstash +logTrace +london +LONGLONG +LookML +lowcardinality +LowCardinality +lowercased +lowerUTF +LpDistance +LpNorm +LpNormalize +Luebbe +Lyft +lzma +MacBook +MACNumToString macOS +MacOS +MACStringToNum +MACStringToOUI +mailrugo +mailto +makeDate +makeDateTime +mannwhitneyutest +mannWhitneyUTest +mapAdd +mapAll +mapApply +mapConcat +mapContains +mapContainsKeyLike +mapExists +mapExtractKeyLike +mapFilter +mapFromArrays +mapKeys +mappedfile +mapPopulateSeries +mapReverseSort +mapSort +mapSubtract +mapUpdate +mapValues mariadb +MarkCacheBytes +MarkCacheFiles +MarksLoaderThreads +MarksLoaderThreadsActive +matcher +MaterializedMySQL +MaterializedPostgreSQL +materializedview +MaterializedView +MaxDDLEntryID +maxintersections +maxIntersections +maxintersectionsposition +maxIntersectionsPosition +maxmap +maxMap +maxmind +MaxMind +MaxPartCountForPartition +MaxPushedDDLEntryID +Mbps mdadm +meanztest +meanZTest +mebibytes +MEDIUMINT +Memcheck +MemoryCode +MemoryDataAndStack +MemoryResident +MemorySanitizer +MemoryShared +MemoryTracking +MemoryVirtual +MEMTABLE +mergeable +MergeJoin +MergeState +mergetree +MergeTree +MergeTreeAllRangesAnnouncementsSent +MergeTreeBackgroundExecutor +MergeTreeBackgroundExecutorThreads +MergeTreeBackgroundExecutorThreadsActive +MergeTreeDataSelectExecutor +MergeTreeDataSelectExecutorThreads +MergeTreeDataSelectExecutorThreadsActive +MergeTreePartsCleanerThreads +MergeTreePartsCleanerThreadsActive +MergeTreePartsLoaderThreads +MergeTreePartsLoaderThreadsActive +MergeTreeReadTaskRequestsSent +MergeTreeSettings +messageID +MessagePack +metacharacters +Metastore +metasymbols +metrica +metroHash +MetroHash +mfedotov +MiB +Milli +Milovidov +mindsdb +MindsDB +MinHash +minimalistic +mininum +MinIO miniselect +minmap +minMap +minmax +MinMax +mins +misconfiguration +mispredictions +mmap +MMapCacheCells +mmapped +MMappedAllocBytes +MMappedAllocs +MMappedFileBytes +MMappedFiles +moduloOrZero +mongodb +Mongodb +monthName +moscow +MSan msgpack +MsgPack msgpk +MSSQL +multibyte +multiFuzzyMatchAllIndices +multiFuzzyMatchAny +multiFuzzyMatchAnyIndex +multiIf multiline +Multiline +multiMatchAllIndices +multiMatchAny +multiMatchAnyIndex +multiplyDecimal +multipolygon +MultiPolygon +Multiqueries +multiSearchAllPositions +multiSearchAllPositionsUTF +multiSearchAny +multiSearchFirstIndex +multiSearchFirstPosition +multisets multithread +Multithreading +multiword +Multiword +munmap murmurhash +murmurHash +MurmurHash +musqldump mutex +MVCC +mydb +myfilter mysql +MySQLConnection mysqldump +MySQLDump mysqljs +MySQLThreads +mytable +Nagios +namedatabases +namenetworks +namenode +Namenode +namepassword +nameprofile +namequota +NamesAndTypesList +namespaces +Nano +NaNs natively +nats +NATS +NCHAR +negtive +NEKUDOTAYIM +Nesterov +nestjs +netloc +NetworkReceive +NetworkReceiveBytes +NetworkReceiveDrop +NetworkReceiveErrors +NetworkReceivePackets +NetworkSend +NetworkSendBytes +NetworkSendDrop +NetworkSendErrors +NetworkSendPackets +NEWDATE +NEWDECIMAL +NFKC +NFKD +ngram +ngrambf +ngramDistance +ngramMinHash +ngramMinHashArg +ngramMinHashArgCaseInsensitive +ngramMinHashArgCaseInsensitiveUTF +ngramMinHashArgUTF +ngramMinHashCaseInsensitive +ngramMinHashCaseInsensitiveUTF +ngramMinHashUTF +ngrams +ngramSearch +ngramSimHash +ngramSimHashCaseInsensitive +ngramSimHashCaseInsensitiveUTF +ngramSimHashUTF +NodeJs +nonNegativeDerivative noop +normalizedQueryHash +normalizeQuery +normalizeUTF +notEmpty +notEquals +notILike +notIn +notLike +notretry +nowInBlock +ntile nullability nullable +nullables +nullIf +NULLIF num +NumberOfDatabases +NumberOfDetachedByUserParts +NumberOfDetachedParts +NumberOfTables +numerics +NumHexagons +NumToString +NumToStringClassC +NuRaft +NVMe +NVME +nypd +NYPD obfuscator +ObjectId +observability +Observability +Octonica odbc +OFNS ok -openSUSE +Ok +OLAP +OLTP +omclickhouse +onstraints +ontime +OnTime +OpenCelliD +OpenFileForRead +OpenFileForWrite openldap +opensky +OpenSky +openssl +openSSL +OpenSSL +OpenStack +openSUSE +OpenSUSE opentelemetry +OpenTelemetry +optinal +Optinal +OrDefault +OrNull +OrZero +OSContextSwitches +OSGuestNiceTime +OSGuestNiceTimeCPU +OSGuestNiceTimeNormalized +OSGuestTime +OSGuestTimeCPU +OSGuestTimeNormalized +OSIdleTime +OSIdleTimeCPU +OSIdleTimeNormalized +OSInterrupts +OSIOWaitTime +OSIOWaitTimeCPU +OSIOWaitTimeNormalized +OSIrqTime +OSIrqTimeCPU +OSIrqTimeNormalized +OSMemoryAvailable +OSMemoryBuffers +OSMemoryCached +OSMemoryFreePlusCached +OSMemoryFreeWithoutCached +OSMemoryTotal +OSNiceTime +OSNiceTimeCPU +OSNiceTimeNormalized +OSOpenFiles +OSProcessesBlocked +OSProcessesCreated +OSProcessesRunning +OSSoftIrqTime +OSSoftIrqTimeCPU +OSSoftIrqTimeNormalized +OSStealTime +OSStealTimeCPU +OSStealTimeNormalized +OSSystemTime +OSSystemTimeCPU +OSSystemTimeNormalized +OSThreadsRunnable +OSThreadsTotal +OSUptime +OSUserTime +OSUserTimeCPU +OSUserTimeNormalized +OTLP +outfile +OUTFILE overcommit +overcommitted +OvercommitTracker +overfitting +PAAMAYIM +packetpool +packetsize +PagerDuty +pageviews +pandahouse +ParallelFormattingOutputFormatThreads +ParallelFormattingOutputFormatThreadsActive parallelization parallelize parallelized +ParallelParsingInputFormat +ParallelParsingInputFormatThreads +ParallelParsingInputFormatThreadsActive +Parametrized +params +paratemer +ParquetMetadata +parsable +parseable +parseDateTime +parseDateTimeBestEffort +parseDateTimeBestEffortOrNull +parseDateTimeBestEffortOrZero +parseDateTimeBestEffortUS +parseDateTimeBestEffortUSOrNull +parseDateTimeBestEffortUSOrZero +parseDateTimeInJodaSyntax +parseDateTimeInJodaSyntaxOrNull +parseDateTimeInJodaSyntaxOrZero +parseDateTimeOrNull +parseDateTimeOrZero parsers +Parsers +parseTimeDelta +Partitioner +PartMutation +PartsActive +PartsCommitted +PartsCompact +PartsDeleteOnDestroy +PartsDeleting +PartsInMemory +PartsOutdated +PartsPreActive +PartsPreCommitted +PartsTemporary +PartsWide +pathFull pclmulqdq +pcre +PCRE +PendingAsyncInsert +Percona performant +perl +persistency +phpclickhouse +PhpStorm +pipelining +plaintext +plantuml +PlantUML poco +PointDistKm +PointDistM +PointDistRads +pointInEllipses +pointInPolygon +polygonAreaCartesian +polygonAreaSpherical +polygonConvexHullCartesian +polygonPerimeterCartesian +polygonPerimeterSpherical +polygonsDistanceCartesian +polygonsDistanceSpherical +polygonsEqualsCartesian +polygonsIntersectionCartesian +polygonsIntersectionSpherical +polygonsSymDifferenceCartesian +polygonsSymDifferenceSpherical +polygonsUnionCartesian +polygonsUnionSpherical +polygonsWithinCartesian +polygonsWithinSpherical popcnt +porthttps +positionCaseInsensitive +positionCaseInsensitiveUTF +positionUTF +positiveModulo postfix postfixes +Postgres postgresql +PostgreSQLConnection +PostgreSQLThreads +PostgresSQL pre +pread +preallocate prebuild prebuilt +Precompiled preemptable +preferServerCiphers +prefertch +prefetch +prefetchsize preloaded +prepend +prepended +prepends +preprocess +Preprocess preprocessed +preprocessing preprocessor presentational prestable prettycompact +PrettyCompact prettycompactmonoblock +PrettyCompactMonoBlock prettycompactnoescapes +PrettyCompactNoEscapes prettycompactnoescapesmonoblock +PrettyCompactNoEscapesMonoBlock prettyjsoneachrow +PrettyJSONEachRow prettymonoblock +PrettyMonoBlock prettynoescapes +PrettyNoEscapes prettynoescapesmonoblock +PrettyNoEscapesMonoBlock prettyspace +PrettySpace prettyspacemonoblock +PrettySpaceMonoBlock prettyspacenoescapes +PrettySpaceNoEscapes prettyspacenoescapesmonoblock +PrettySpaceNoEscapesMonoBlock +prewhere +Prewhere +PREWHERE +privateKeyFile +privateKeyPassphraseHandler +PrivateKeyPassphraseHandler prlimit +PROCESSLIST +procfs +ProfileEvents +profiler +Profiler +profuct +Proleptic prometheus +PromHouse +Promql +PromQL +Promtail proto protobuf +Protobuf protobufsingle +ProtobufSingle +proxied +ProxySQL +pseudorandom +pseudorandomize psql ptrs +publsh +pushdown +pwrite py +PyCharm +QEMU +qouta +qryn +QTCreator +quantile +Quantile +quantilebfloat +quantileBFloat +quantiledeterministic +quantileDeterministic +quantileexact +quantileExact +quantileExactExclusive +quantileExactHigh +quantileExactInclusive +quantileExactLow +quantileexactweighted +quantileExactWeighted +quantileGK +quantileInterpolatedWeighted +quantiles +quantilesExactExclusive +quantilesExactInclusive +quantilesGK +quantilesTimingWeighted +quantiletdigest +quantileTDigest +quantiletdigestweighted +quantileTDigestWeighted +quantiletiming +quantileTiming +quantiletimingweighted +quantileTimingWeighted +quartile +quaters +QueryCacheHits +QueryCacheMisses +queryID +QueryPreempted +queryString +queryStringAndFragment +QueryThread +QuoteMeta +rabbitmq +RabbitMQ +raduis +randBernoulli +randBinomial +randCanonical +randChiSquared +randConstant +randExponential +randFisherF +randLogNormal +randNegativeBinomial +randNormal +randomFixedString +randomPrintableASCII +randomString +randomStringUTF +randPoisson +randStudentT +randUniform +RangeHashed +rankCorr rapidjson rawblob +RawBLOB +RBAC +RClickHouse readahead readline readme readonly +ReadonlyReplica +ReadTaskRequestsSent +readWKTMultiPolygon +readWKTPolygon +reate +rebalance rebalanced +recency +RecipeNLG +recompress +recompressed +recompressing +Recompressing +recompression +Recompression +reconnection +RectAdd +RectContains +RectIntersection +RectUnion +recurse +redash +Redash +reddit +Reddit +RedHat +redisstreams +ReDoS +Refactorings +refcounter +ReferenceKeyed +regexpExtract +regexpQuoteMeta +RegexpTree +regionHierarchy +regionIn +regionToArea +regionToCity +regionToContinent +regionToCountry +regionToDistrict +regionToName +regionToPopulation +regionToTopContinent +reinitialization +reinitializing +reinterpretAs +reinterpretAsDate +reinterpretAsDateTime +reinterpretAsFixedString +reinterpretAsFloat +reinterpretAsInt +reinterpretAsString +reinterpretAsUInt +reinterpretAsUUID +RemoteRead +remoteSecure +replaceAll +replaceOne +replaceRegexpAll +replaceRegexpOne +replacingmergetree +ReplacingMergeTree +ReplicasMaxAbsoluteDelay +ReplicasMaxInsertsInQueue +ReplicasMaxMergesInQueue +ReplicasMaxQueueSize +ReplicasMaxRelativeDelay +ReplicasSumInsertsInQueue +ReplicasSumMergesInQueue +ReplicasSumQueueSize +replicatable +ReplicatedAggregatingMergeTree +ReplicatedChecks +ReplicatedCollapsingMergeTree +ReplicatedFetch +ReplicatedGraphiteMergeTree +replicatedmergetree +ReplicatedMergeTree +ReplicatedReplacingMergeTree +ReplicatedSend +ReplicatedSummingMergeTree +ReplicatedVersionedCollapsingMergeTree replxx repo representable requestor +requireTLSv +Resample +resharding +reshards +RestartReplicaThreads +RestartReplicaThreadsActive +RestoreThreads +RestoreThreadsActive resultset +retentions rethrow +retransmit retriable +retuned +Returnes +reult +reverseDNSQuery +reverseUTF +RHEL +rightPad +rightPadUTF risc riscv ro +roadmap +RoaringBitmap rocksdb -rowNumberInBlock +RocksDB +rollup +Rollup +ROLLUP +roundAge +roundBankers +roundDown +roundDuration +roundToExp +routineley rowbinary +RowBinary rowbinarywithnames +RowBinaryWithNames rowbinarywithnamesandtypes +RowBinaryWithNamesAndTypes +rowNumberInAllBlocks +rowNumberInBlock rsync +rsyslog runnable runningAccumulate +runningConcurrency +runningDifference +runningDifferenceStartingWithFirstValue runtime +Runtime russian rw +RWLock +RWLockActiveReaders +RWLockActiveWriters +RWLockWaitingReaders +RWLockWaitingWriters +SaaS +Sanjeev +Sankey sasl +SATA +satisfiable +scala +Scalable +Scatterplot +Schaefer schemas +Schemas +Schwartzian +searchin +SeasClick +seccessfully +seekable +seektable +SeekTable +SELECTs +SelfManaged +Sematext +SendExternalTables +SendScalars +separatelly +sequenceCount +sequenceMatch +sequenceNextNode +SERIALIZABLE +serverUUID +sessionCacheSize +sessionIdContext +sessionTimeout +seva +shardCount +sharded +sharding +shardNum +ShareAlike +shortcircuit +shoutout +SIGTERM +SIMD simdjson +Simhash +SimHash +simpleaggregatefunction +SimpleAggregateFunction +simplelinearregression +simpleLinearRegression +SimpleState +simpliest +simpod +singlepart +sinh +siphash +sipHash +SipHash +skewness +skewpop +skewPop +skewsamp +skewSamp skippingerrors +sleepEachRow +SLES +SLRU +SMALLINT +Smirnov's +Smirnov'test +snowflakeToDateTime +socketcache +soundex +Soundex +SpanKind +sparkbar sparsehash +Spearman's +speedscope +splitByChar +splitByNonAlpha +splitByRegexp +splitByString +splitByWhitespace +SPNEGO +SQEs sql +sqlalchemy +SQLAlchemy +SQLConsoleDetail sqlinsert +SQLInsert +sqlite +SQLSTATE +sqrt src +SSDCache +SSDComplexKeyCache +SSDs +SSLManager +SSRF +SSSE +stacktrace stacktraces +startsWith +StartTime +StartTLS +StartupSystemTables +StartupSystemTablesThreads +StartupSystemTablesThreadsActive statbox stateful +Stateful +stddev +stddevpop +stddevPop +stddevsamp +stddevSamp stderr stdin stdout +stochastically +stochasticlinearregression +stochasticLinearRegression +stochasticlogisticregression +stochasticLogisticRegression +StorageBufferBytes +StorageBufferRows +StorageDistributed +StorageDistributedThreads +StorageDistributedThreadsActive +StorageHive +StorageHiveThreads +StorageHiveThreadsActive +StorageODBC +storages +StorageS +storig +stringToH +StringToNum +StringToNumOrDefault +StringToNumOrNull +stripelog +Stripelog +StripeLog +Strohmeier strtod strtoll strtoull +struct structs +studentttest +studentTTest +subarray +subarrays +subBitmap +subcolumn +subcolumns +Subcolumns +subcribe subdirectories +subdirectory +subexpression +Subexpression subexpressions +subfolder +subinterval +subintervals +subkey +submatch submodule submodules +Submodules +subnet +subnetwork subpattern subpatterns subqueries +Subqueries subquery +subranges +subreddits subseconds +substracted substring +substrings +Substrings +substringUTF +subtitiles +subtractDays +subtractHours +subtractMinutes +subtractMonths +subtractQuarters +subtractSeconds +subtractWeeks +subtractYears subtree subtype sudo +sumcount +sumCount +sumkahan +sumKahan +summap +sumMap +sumMapFiltered +summingmergetree +SummingMergeTree +sumwithoverflow +sumWithOverflow +superaggregates +Superset +SuperSet +SupersetDocker +supremum symlink symlinks +synchronious syntaxes +syscall +syscalls +syslog +syslogd systemd +SystemReplicasThreads +SystemReplicasThreadsActive +TabItem +tabix +Tabix +TablesLoaderThreads +TablesLoaderThreadsActive +TablesToDropQueueSize +tablum +TABLUM tabseparated +TabSeparated tabseparatedraw +TabSeparatedRaw tabseparatedrawwithnames +TabSeparatedRawWithNames tabseparatedrawwithnamesandtypes +TabSeparatedRawWithNamesAndTypes tabseparatedwithnames +TabSeparatedWithNames tabseparatedwithnamesandtypes +TabSeparatedWithNamesAndTypes +TargetSpecific tcp +TCPConnection +tcpnodelay +tcpPort +TCPThreads +Telegraf templateignorespaces +TemplateIgnoreSpaces +TemporaryFilesForAggregation +TemporaryFilesForJoin +TemporaryFilesForSort +TemporaryFilesUnknown +Testflows +tgamma tgz +Tgz th +thats +Theil's +theilsu +theilsU +themself +threadpool +ThreadPoolFSReaderThreads +ThreadPoolFSReaderThreadsActive +ThreadPoolRemoteFSReaderThreads +ThreadPoolRemoteFSReaderThreadsActive +ThreadsActive +ThreadsInOvercommitTracker +throwIf +timeSlot +timeSlots +Timeunit +timeZone +timeZoneOf +timeZoneOffset +timezones +TINYINT +tinylog +TinyLog +Tkachenko +TKSV +TLSv tmp +ToCenterChild +ToChildren +toColumnTypeName +toDate +toDateOrDefault +toDateOrNull +toDateOrZero +toDateTime +toDateTimeOrDefault +toDateTimeOrNull +toDateTimeOrZero +toDayOfMonth +toDayOfWeek +toDayOfYear +toDecimal +toDecimalString +toFixedString +toFloat +ToGeo +ToGeoBoundary +toHour +toInt +toInterval +toIPv +ToIPv +toISOWeek +toISOYear +toJSONString +tokenbf tokenization +tokenized +tokenizer +toLastDayOfMonth +toLowCardinality +toMinute toml +toModifiedJulianDay +toModifiedJulianDayOrNull +toMonday +toMonth +toNullable toolchain toolset +Toolset +ToParent +topk +topK +TopK +topkweighted +topKWeighted +topLevelDomain +toQuarter +toRelativeDayNum +toRelativeHourNum +toRelativeMinuteNum +toRelativeMonthNum +toRelativeQuarterNum +toRelativeSecondNum +toRelativeWeekNum +toRelativeYearNum +toSecond +ToSnowflake +toStartOfDay +toStartOfFifteenMinutes +toStartOfFiveMinutes +toStartOfHour +toStartOfInterval +toStartOfISOYear +toStartOfMinute +toStartOfMonth +toStartOfQuarter +toStartOfSecond +toStartOfTenMinutes +toStartOfWeek +toStartOfYear +toString +ToString +toStringCutToZero +TotalBytesOfMergeTreeTables +TotalPartsOfMergeTreeTables +TotalRowsOfMergeTreeTables +TotalTemporaryFiles +toTime +toTimeZone +toType +toTypeName +toUInt +toUnixTimestamp +toUUID +toUUIDOrDefault +toUUIDOrNull +toUUIDOrZero +toValidUTF +toWeek +toYear +toYearWeek +toYYYYMM +toYYYYMMDD +toYYYYMMDDhhmmss +TPCH +Tradeoff transactional +Transactional transactionally +translateUTF +translocality +trie +trimBoth +trimLeft +trimRight +trunc +tryBase +tryDecrypt +TSan +TSDB tskv tsv +TSVRaw +TSVs +TThe tui +tumbleEnd +tumbleStart +tupleDivide +tupleDivideByNumber +tupleElement +tupleHammingDistance +tupleMinus +tupleMultiply +tupleMultiplyByNumber +tupleNegate +tuplePlus +tupleToNameValuePairs turbostat +TwoColumnList txt +typename +Uber +UBSan ubuntu +UDFs uint +Uint +UInt +UIntN +ulid +ULID +ULIDStringToDateTime +UMTS unary +unbin +uncomment +UncompressedCacheBytes +UncompressedCacheCells +underying +undrop +UNDROP +unencoded unencrypted +unescaped +unescaping +unhex +unicode +unidimensional +UnidirectionalEdgeIsValid +uniq +uniqcombined +uniqCombined +uniqexact +uniqExact +uniqhll +uniqHLL +uniqtheta +uniqTheta +uniqThetaIntersect +uniqThetaNot +uniqthetasketch +uniqThetaSketch +UniqThetaSketch +uniqThetaUnion +uniqUpTo +unix unixodbc +unixODBC unoptimized +unparsed +unrealiable +unreplicated +unresolvable +unrounded +untracked +untrusted +untuple +Updatable +UPDATEs +uploaders +Uppercased +upperUTF +uptime +Uptime +uptrace +Uptrace +uring +URIs url +urlCluster +URLHash +URLHierarchy +URLPathHierarchy +urls +URL's +UserID userspace userver +Util utils uuid +UUid +UUIDNumToString +UUIDs +UUIDStringToNum +Vadim +Valgrind +VARCHAR variadic varint +varpop +varPop +varsamp +varSamp vectorized +Vectorized vectorscan +verificationDepth +verificationMode +versionedcollapsingmergetree +VersionedCollapsingMergeTree +VersionInteger +vhost +VideoContainer +ViewAllLink +VIEWs +VirtualBox +virtualized +visibleWidth +visitParam +visitParamExtractBool +visitParamExtractFloat +visitParamExtractInt +visitParamExtractRaw +visitParamExtractString +visitParamExtractUInt +visitParamHas +WALs wchc wchs webpage webserver +weekyear +Welch's +welchttest +welchTTest +Werror +Wether wget +which's whitespace whitespaces +wikistat +WikiStat +windowFunnel +WindowView +WithNames +WithNamesAndTypes +Woboq +WordNet +wordshingleMinHash +wordShingleMinHash +wordShingleMinHashArg +wordShingleMinHashArgCaseInsensitive +wordShingleMinHashArgCaseInsensitiveUTF +wordShingleMinHashArgUTF +wordShingleMinHashCaseInsensitive +wordShingleMinHashCaseInsensitiveUTF +wordShingleMinHashUTF +wordShingleSimHash +wordShingleSimHashCaseInsensitive +wordShingleSimHashCaseInsensitiveUTF +wordShingleSimHashUTF +WriteBuffer +WriteBuffers wrt xcode +XCode +Xeon +xeus +XHTML +xkcd +xlarge xml +XORs +xxHash xz -zLib -zLinux +YAML +YAMLRegExpTree +yandex +Yandex +Yasm +youtube +YYYY +zabbix +Zabbix +Zipkin zkcopy zlib +zLib +zLinux +znode znodes +ZooKeeper +ZooKeeperRequest +ZooKeeper's +ZooKeepers +ZooKeeperSession +zookeeperSessionUptime +ZooKeeperWatch zstd diff --git a/utils/check-style/check-doc-aspell b/utils/check-style/check-doc-aspell index d39769aa930..952dbd5b507 100755 --- a/utils/check-style/check-doc-aspell +++ b/utils/check-style/check-doc-aspell @@ -1,5 +1,8 @@ #!/usr/bin/env bash +# force-enable double star globbing +shopt -s globstar + # Perform spell checking on the docs if [[ ${1:-} == "--help" ]] || [[ ${1:-} == "-h" ]]; then From c3a888a47bfd68c3737dee55fe1a0a2c92fa9341 Mon Sep 17 00:00:00 2001 From: Roman Vlasenko Date: Fri, 2 Jun 2023 15:41:43 +0300 Subject: [PATCH 245/308] Remove needless minus sign --- docs/ru/faq/integration/json-import.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/faq/integration/json-import.md b/docs/ru/faq/integration/json-import.md index bc65b5a614a..a3c89aed429 100644 --- a/docs/ru/faq/integration/json-import.md +++ b/docs/ru/faq/integration/json-import.md @@ -19,7 +19,7 @@ $ echo '{"foo":"bar"}' | curl 'http://localhost:8123/?query=INSERT%20INTO%20test При помощи [интефейса CLI](../../interfaces/cli.md): ``` bash -$ echo '{"foo":"bar"}' | clickhouse-client ---query="INSERT INTO test FORMAT JSONEachRow" +$ echo '{"foo":"bar"}' | clickhouse-client --query="INSERT INTO test FORMAT JSONEachRow" ``` Чтобы не вставлять данные вручную, используйте одну из [готовых библиотек](../../interfaces/index.md). @@ -31,4 +31,4 @@ $ echo '{"foo":"bar"}' | clickhouse-client ---query="INSERT INTO test FORMAT JS :::note "Примечание" В HTTP-интерфейсе настройки передаются через параметры `GET` запроса, в `CLI` interface — как дополнительные аргументы командной строки, начинающиеся с `--`. - ::: \ No newline at end of file + ::: From 079008058adbefea63edc8afffa0dd20d694e5e5 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 08:49:26 -0400 Subject: [PATCH 246/308] move from server to user settings --- .../settings.md | 51 ++----------------- docs/en/operations/settings/settings.md | 42 +++++++++++++++ 2 files changed, 46 insertions(+), 47 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 82be1c10dcc..d07fb80d1da 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -275,9 +275,9 @@ Type: UInt64 Default: 1000 -## max_concurrent_insert_queries +## max_concurrent_queries -Limit on total number of concurrent insert queries. Zero means Unlimited. +Limit on total number of concurrently executed queries. Zero means Unlimited. Note that limits on insert and select queries, and on the maximum number of queries for users must also be considered. See also max_concurrent_insert_queries, max_concurrent_select_queries, max_concurrent_queries_for_all_users. Zero means unlimited. :::note These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. @@ -287,9 +287,9 @@ Type: UInt64 Default: 0 -## max_concurrent_queries +## max_concurrent_insert_queries -Limit on total number of concurrently executed queries. Zero means Unlimited. Note that limits on insert and select queries, and on the maximum number of queries for users must also be considered. See also max_concurrent_insert_queries, max_concurrent_select_queries, max_concurrent_queries_for_all_users. Zero means unlimited. +Limit on total number of concurrent insert queries. Zero means Unlimited. :::note These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. @@ -1277,49 +1277,6 @@ For more information, see the section [Creating replicated tables](../../engines ``` - -## max_concurrent_queries_for_user {#max-concurrent-queries-for-user} - -The maximum number of simultaneously processed queries related to MergeTree table per user. - -Possible values: - -- Positive integer. -- 0 — No limit. - -Default value: `0`. - -**Example** - -``` xml -5 -``` - -## max_concurrent_queries_for_all_users {#max-concurrent-queries-for-all-users} - -Throw exception if the value of this setting is less or equal than the current number of simultaneously processed queries. - -Example: `max_concurrent_queries_for_all_users` can be set to 99 for all users and database administrator can set it to 100 for itself to run queries for investigation even when the server is overloaded. - -Modifying the setting for one query or user does not affect other queries. - -Possible values: - -- Positive integer. -- 0 — No limit. - -Default value: `0`. - -**Example** - -``` xml -99 -``` - -**See Also** - -- [max_concurrent_queries](#max-concurrent-queries) - ## max_open_files {#max-open-files} The maximum number of open files. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b868299aeff..374afb6bed7 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -646,6 +646,48 @@ Used for the same purpose as `max_block_size`, but it sets the recommended block However, the block size cannot be more than `max_block_size` rows. By default: 1,000,000. It only works when reading from MergeTree engines. +## max_concurrent_queries_for_user {#max-concurrent-queries-for-user} + +The maximum number of simultaneously processed queries related to MergeTree table per user. + +Possible values: + +- Positive integer. +- 0 — No limit. + +Default value: `0`. + +**Example** + +``` xml +5 +``` + +## max_concurrent_queries_for_all_users {#max-concurrent-queries-for-all-users} + +Throw exception if the value of this setting is less or equal than the current number of simultaneously processed queries. + +Example: `max_concurrent_queries_for_all_users` can be set to 99 for all users and database administrator can set it to 100 for itself to run queries for investigation even when the server is overloaded. + +Modifying the setting for one query or user does not affect other queries. + +Possible values: + +- Positive integer. +- 0 — No limit. + +Default value: `0`. + +**Example** + +``` xml +99 +``` + +**See Also** + +- [max_concurrent_queries](/docs/en/operations/server-configuration-parameters/settings.md/#max_concurrent_queries) + ## merge_tree_min_rows_for_concurrent_read {#setting-merge-tree-min-rows-for-concurrent-read} If the number of rows to be read from a file of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `merge_tree_min_rows_for_concurrent_read` then ClickHouse tries to perform a concurrent reading from this file on several threads. From e148c60d5a308b2ea86128021beba0e8321fef9b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 16:18:32 +0300 Subject: [PATCH 247/308] Fixes for MergeTree with readonly disks (#50244) * fixes for MergeTree with readonly disks * Automatic style fix * Update test.py * Automatic style fix * Update test.py * Update test.py * Automatic style fix * Update test.py --------- Co-authored-by: robot-clickhouse Co-authored-by: alesapin --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++ src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageMergeTree.cpp | 5 +- .../test_disk_over_web_server/test.py | 50 ++++++++++++++++--- ...02435_rollback_cancelled_queries.reference | 1 - .../02435_rollback_cancelled_queries.sh | 11 ++-- 6 files changed, 61 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 338a221e45e..32665429051 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4854,6 +4854,9 @@ void MergeTreeData::checkAlterPartitionIsPossible( void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition, ContextPtr local_context) { + if (!supportsReplication() && isStaticStorage()) + return; + DataPartsVector parts_to_remove; const auto * partition_ast = partition->as(); if (partition_ast && partition_ast->all) @@ -4874,6 +4877,9 @@ void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition, Context void MergeTreeData::checkPartCanBeDropped(const String & part_name) { + if (!supportsReplication() && isStaticStorage()) + return; + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}); if (!part) throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in committed state", part_name); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index fce7d989a2f..1c41de6fa19 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -865,7 +865,7 @@ public: DiskPtr tryGetDiskForDetachedPart(const String & part_name) const; DiskPtr getDiskForDetachedPart(const String & part_name) const; - bool storesDataOnDisk() const override { return true; } + bool storesDataOnDisk() const override { return !isStaticStorage(); } Strings getDataPaths() const override; /// Reserves space at least 1MB. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a721dd30cd7..3da4724471d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -114,7 +114,7 @@ StorageMergeTree::StorageMergeTree( loadDataParts(has_force_restore_data_flag); - if (!attach && !getDataPartsForInternalUsage().empty()) + if (!attach && !getDataPartsForInternalUsage().empty() && !isStaticStorage()) throw Exception(ErrorCodes::INCORRECT_DATA, "Data directory for table already containing data parts - probably " "it was unclean DROP table or manual intervention. " @@ -283,6 +283,9 @@ StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & met void StorageMergeTree::checkTableCanBeDropped() const { + if (!supportsReplication() && isStaticStorage()) + return; + auto table_id = getStorageID(); getContext()->checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes()); } diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index fd71389f71a..719de5e8bef 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -10,16 +10,22 @@ def cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance( - "node1", main_configs=["configs/storage_conf.xml"], with_nginx=True + "node1", + main_configs=["configs/storage_conf.xml"], + with_nginx=True, ) cluster.add_instance( "node2", main_configs=["configs/storage_conf_web.xml"], with_nginx=True, stay_alive=True, + with_zookeeper=True, ) cluster.add_instance( - "node3", main_configs=["configs/storage_conf_web.xml"], with_nginx=True + "node3", + main_configs=["configs/storage_conf_web.xml"], + with_nginx=True, + with_zookeeper=True, ) cluster.add_instance( @@ -95,7 +101,7 @@ def test_usage(cluster, node_name): for i in range(3): node2.query( """ - ATTACH TABLE test{} UUID '{}' + CREATE TABLE test{} UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( @@ -140,7 +146,7 @@ def test_incorrect_usage(cluster): global uuids node2.query( """ - ATTACH TABLE test0 UUID '{}' + CREATE TABLE test0 UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( @@ -173,7 +179,7 @@ def test_cache(cluster, node_name): for i in range(3): node2.query( """ - ATTACH TABLE test{} UUID '{}' + CREATE TABLE test{} UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'cached_web'; """.format( @@ -238,7 +244,7 @@ def test_unavailable_server(cluster): global uuids node2.query( """ - ATTACH TABLE test0 UUID '{}' + CREATE TABLE test0 UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( @@ -276,3 +282,35 @@ def test_unavailable_server(cluster): ) node2.start_clickhouse() node2.query("DROP TABLE test0 SYNC") + + +def test_replicated_database(cluster): + node1 = cluster.instances["node3"] + node1.query( + "CREATE DATABASE rdb ENGINE=Replicated('/test/rdb', 's1', 'r1')", + settings={"allow_experimental_database_replicated": 1}, + ) + + global uuids + node1.query( + """ + CREATE TABLE rdb.table0 UUID '{}' + (id Int32) ENGINE = MergeTree() ORDER BY id + SETTINGS storage_policy = 'web'; + """.format( + uuids[0] + ) + ) + + node2 = cluster.instances["node2"] + node2.query( + "CREATE DATABASE rdb ENGINE=Replicated('/test/rdb', 's1', 'r2')", + settings={"allow_experimental_database_replicated": 1}, + ) + node2.query("SYSTEM SYNC DATABASE REPLICA rdb") + + assert node1.query("SELECT count() FROM rdb.table0") == "5000000\n" + assert node2.query("SELECT count() FROM rdb.table0") == "5000000\n" + + node1.query("DROP DATABASE rdb SYNC") + node2.query("DROP DATABASE rdb SYNC") diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference b/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference index 2d32c17ec7c..38ff81b2371 100644 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference @@ -1,3 +1,2 @@ 1000000 0 -1 diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index 8f8e8cc7ee0..776d1f850b0 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -110,11 +110,12 @@ insert_data 1 $CLICKHOUSE_CLIENT --implicit_transaction=1 -q 'select throwIf(count() % 1000000 != 0 or count() = 0) from dedup_test' \ || $CLICKHOUSE_CLIENT -q "select name, rows, active, visible, creation_tid, creation_csn from system.parts where database=currentDatabase();" -# Ensure that thread_cancel actually did something -$CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( - message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes', - 'Query was cancelled or a client has unexpectedly dropped the connection') or - message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')" +# Ensure that thread_cancel actually did something (useful when editing this test) +# We cannot check it in the CI, because sometimes it fails due to randomization +# $CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( +# message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes', +# 'Query was cancelled or a client has unexpectedly dropped the connection') or +# message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')" wait_for_queries_to_finish 30 $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table dedup_test" From 8e076c33d51d6ecdfa209158d75305befd6ab308 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 2 Jun 2023 13:19:13 +0000 Subject: [PATCH 248/308] Try fix flaky test test_async_query_sending --- tests/integration/test_hedged_requests/configs/logger.xml | 5 +++++ tests/integration/test_hedged_requests/test.py | 5 ++++- 2 files changed, 9 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_hedged_requests/configs/logger.xml diff --git a/tests/integration/test_hedged_requests/configs/logger.xml b/tests/integration/test_hedged_requests/configs/logger.xml new file mode 100644 index 00000000000..48fb4e91428 --- /dev/null +++ b/tests/integration/test_hedged_requests/configs/logger.xml @@ -0,0 +1,5 @@ + + + 20 + + \ No newline at end of file diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 2ca37fbb7ee..be6cea80f87 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -23,7 +23,7 @@ def started_cluster(): NODES["node"] = cluster.add_instance( "node", stay_alive=True, - main_configs=["configs/remote_servers.xml"], + main_configs=["configs/remote_servers.xml", "configs/logger.xml"], user_configs=["configs/users.xml"], ) @@ -377,6 +377,9 @@ def test_async_connect(started_cluster): check_changing_replica_events(2) check_if_query_sending_was_not_suspended() + # Restart server to reset connection pool state + NODES["node"].restart_clickhouse() + NODES["node"].query( "SELECT hostName(), id FROM distributed_connect ORDER BY id LIMIT 1 SETTINGS prefer_localhost_replica = 0, connect_timeout_with_failover_ms=5000, async_query_sending_for_remote=1, max_threads=1" ) From d9a4f8115fb7f362e93dc59e38a8d649d016e0e7 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 2 Jun 2023 13:19:41 +0000 Subject: [PATCH 249/308] Add new line delimiter --- tests/integration/test_hedged_requests/configs/logger.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_hedged_requests/configs/logger.xml b/tests/integration/test_hedged_requests/configs/logger.xml index 48fb4e91428..b341b14d43c 100644 --- a/tests/integration/test_hedged_requests/configs/logger.xml +++ b/tests/integration/test_hedged_requests/configs/logger.xml @@ -2,4 +2,4 @@ 20 - \ No newline at end of file + From 54872f9e7ea314da7f766c8929c212fd3d07ec21 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 13:27:56 +0000 Subject: [PATCH 250/308] Typos: Follow-up to #50476 --- .../mergetree-family/mergetree.md | 2 +- .../settings.md | 2 +- .../operations/settings/settings-formats.md | 2 +- docs/en/operations/settings/settings.md | 8 ++-- docs/en/operations/system-tables/quotas.md | 4 +- .../parametric-functions.md | 2 +- .../reference/exponentialmovingaverage.md | 4 +- docs/en/sql-reference/dictionaries/index.md | 6 +-- .../functions/array-functions.md | 4 +- .../functions/distance-functions.md | 2 +- .../functions/encryption-functions.md | 2 +- .../sql-reference/functions/hash-functions.md | 2 +- .../sql-reference/functions/math-functions.md | 2 +- .../functions/other-functions.md | 4 +- .../functions/type-conversion-functions.md | 2 +- docs/en/sql-reference/operators/index.md | 2 +- .../table-functions/urlCluster.md | 4 +- .../aspell-ignore/en/aspell-dict.txt | 45 ------------------- 18 files changed, 27 insertions(+), 72 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 1ab0f4057ff..07f706af91d 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -779,7 +779,7 @@ Disks, volumes and storage policies should be declared inside the ` Date: Fri, 2 Jun 2023 13:31:14 +0000 Subject: [PATCH 251/308] Apply uncommitted state after snapshot deser --- src/Coordination/KeeperStateMachine.cpp | 5 ++ src/Coordination/KeeperStorage.cpp | 51 ++++++++---- src/Coordination/KeeperStorage.h | 5 ++ src/Coordination/tests/gtest_coordination.cpp | 77 +++++++++++++++++++ 4 files changed, 124 insertions(+), 14 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 6635c74149a..a4568cbbdd3 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -363,6 +363,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) else if (s.get_last_log_idx() < latest_snapshot_meta->get_last_log_idx()) { LOG_INFO(log, "A snapshot with a larger last log index ({}) was created, skipping applying this snapshot", latest_snapshot_meta->get_last_log_idx()); + return true; } latest_snapshot_ptr = latest_snapshot_buf; @@ -372,6 +373,10 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) std::lock_guard lock(storage_and_responses_lock); auto snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(s.get_last_log_idx())); + + /// maybe some logs were preprocessed with log idx larger than the snapshot idx + /// we have to apply them to the new storage + storage->applyUncommittedState(*snapshot_deserialization_result.storage, s.get_last_log_idx()); storage = std::move(snapshot_deserialization_result.storage); latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta; cluster_config = snapshot_deserialization_result.cluster_config; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 7a1a5e42632..66d6b0f5843 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -375,23 +375,26 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) delta.operation); } +void KeeperStorage::UncommittedState::addDelta(Delta new_delta) +{ + const auto & added_delta = deltas.emplace_back(std::move(new_delta)); + + if (!added_delta.path.empty()) + { + deltas_for_path[added_delta.path].push_back(&added_delta); + applyDelta(added_delta); + } + else if (const auto * auth_delta = std::get_if(&added_delta.operation)) + { + auto & uncommitted_auth = session_and_auth[auth_delta->session_id]; + uncommitted_auth.emplace_back(&auth_delta->auth_id); + } +} + void KeeperStorage::UncommittedState::addDeltas(std::vector new_deltas) { for (auto & delta : new_deltas) - { - const auto & added_delta = deltas.emplace_back(std::move(delta)); - - if (!added_delta.path.empty()) - { - deltas_for_path[added_delta.path].push_back(&added_delta); - applyDelta(added_delta); - } - else if (const auto * auth_delta = std::get_if(&added_delta.operation)) - { - auto & uncommitted_auth = session_and_auth[auth_delta->session_id]; - uncommitted_auth.emplace_back(&auth_delta->auth_id); - } - } + addDelta(std::move(delta)); } void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) @@ -602,6 +605,26 @@ namespace } +void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_zxid) +{ + for (const auto & transaction : uncommitted_transactions) + { + if (transaction.zxid <= last_zxid) + continue; + other.uncommitted_transactions.push_back(transaction); + } + + auto it = uncommitted_state.deltas.begin(); + + for (; it != uncommitted_state.deltas.end(); ++it) + { + if (it->zxid <= last_zxid) + continue; + + other.uncommitted_state.addDelta(*it); + } +} + Coordination::Error KeeperStorage::commit(int64_t commit_zxid) { // Deltas are added with increasing ZXIDs diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 7eb10be3847..844cbf85c1e 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -222,6 +222,7 @@ public: { explicit UncommittedState(KeeperStorage & storage_) : storage(storage_) { } + void addDelta(Delta new_delta); void addDeltas(std::vector new_deltas); void commit(int64_t commit_zxid); void rollback(int64_t rollback_zxid); @@ -310,6 +311,10 @@ public: UncommittedState uncommitted_state{*this}; + // Apply uncommitted state to another storage using only transactions + // with zxid > last_zxid + void applyUncommittedState(KeeperStorage & other, int64_t last_zxid); + Coordination::Error commit(int64_t zxid); // Create node in the storage diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 62217fb2dd3..453fd0f2e60 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2524,6 +2524,83 @@ TEST_P(CoordinationTest, TestCheckNotExistsRequest) } } +TEST_P(CoordinationTest, TestReapplyingDeltas) +{ + using namespace DB; + using namespace Coordination; + + static constexpr int64_t initial_zxid = 100; + + const auto create_request = std::make_shared(); + create_request->path = "/test/data"; + create_request->is_sequential = true; + + const auto process_create = [](KeeperStorage & storage, const auto & request, int64_t zxid) + { + storage.preprocessRequest(request, 1, 0, zxid); + auto responses = storage.processRequest(request, 1, zxid); + EXPECT_GE(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Error::ZOK); + }; + + const auto commit_initial_data = [&](auto & storage) + { + int64_t zxid = 1; + + const auto root_create = std::make_shared(); + root_create->path = "/test"; + process_create(storage, root_create, zxid); + ++zxid; + + for (; zxid <= initial_zxid; ++zxid) + process_create(storage, create_request, zxid); + }; + + KeeperStorage storage1{500, "", keeper_context}; + commit_initial_data(storage1); + + for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) + storage1.preprocessRequest(create_request, 1, 0, zxid); + + /// create identical new storage + KeeperStorage storage2{500, "", keeper_context}; + commit_initial_data(storage2); + + storage1.applyUncommittedState(storage2, initial_zxid); + + const auto commit_unprocessed = [&](KeeperStorage & storage) + { + for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) + { + auto responses = storage.processRequest(create_request, 1, zxid); + EXPECT_GE(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Error::ZOK); + } + }; + + commit_unprocessed(storage1); + commit_unprocessed(storage2); + + const auto get_children = [&](KeeperStorage & storage) + { + const auto list_request = std::make_shared(); + list_request->path = "/test"; + auto responses = storage.processRequest(list_request, 1, std::nullopt, /*check_acl=*/true, /*is_local=*/true); + EXPECT_EQ(responses.size(), 1); + const auto * list_response = dynamic_cast(responses[0].response.get()); + EXPECT_TRUE(list_response); + return list_response->names; + }; + + auto children1 = get_children(storage1); + std::unordered_set children1_set(children1.begin(), children1.end()); + + auto children2 = get_children(storage2); + std::unordered_set children2_set(children2.begin(), children2.end()); + + ASSERT_TRUE(children1_set == children2_set); +} + INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite, CoordinationTest, ::testing::ValuesIn(std::initializer_list{ From 2a4f1c82756b8937a2f87453f1bd76bc09a45114 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 14:00:57 +0000 Subject: [PATCH 252/308] Fix typos --- docs/en/sql-reference/functions/other-functions.md | 6 +----- docs/en/sql-reference/functions/random-functions.md | 2 +- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 6d49527fd0d..254297451f7 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2006,11 +2006,7 @@ isDecimalOverflow(d, [p]) **Arguments** - `d` — value. [Decimal](../../sql-reference/data-types/decimal.md). -<<<<<<< HEAD -- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. This paratemer can be helpful to migrate data from/to another database or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). -======= -- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. Using of this parameter could be helpful for data extraction to another DBMS or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). ->>>>>>> rschu1ze/master +- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. This parameter can be helpful to migrate data from/to another database or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). **Returned values** diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index 63d5174b494..e593d9458f0 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -67,7 +67,7 @@ randUniform(min, max) **Arguments** - `min` - `Float64` - left boundary of the range, -- `max` - `Float64` - reight boundary of the range. +- `max` - `Float64` - right boundary of the range. **Returned value** From 17cca6ed756eaaa58eae7ef6aa89e43dcda8ce24 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 10:08:48 -0400 Subject: [PATCH 253/308] add direct join docs --- .../integrations/embedded-rocksdb.md | 85 +++++++++++++++++++ 1 file changed, 85 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index a3604b3c332..dab741a9f63 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -120,3 +120,88 @@ Values can be updated using the `ALTER TABLE` query. The primary key cannot be u ```sql ALTER TABLE test UPDATE v1 = v1 * 10 + 2 WHERE key LIKE 'some%' AND v3 > 3.1; ``` + +### Joins + +A special `direct` join with EmbeddedRocksDB tables is supported. +This direct join avoids forming a hash table in memory and accesses +the data directly from the EmbeddedRocksDB. + +To enable direct joins: +```sql +SET join_algorithm = 'direct' +``` + +:::tip +When the `join_algorithm` is set to `direct`, direct joins will be used +when possible. However, direct joins are not used for RIGHT or FULL JOINs. +ClickHouse will choose another join algorithm when direct joins are not possible. +::: + +#### Example + +##### Create and populate an EmbeddedRocksDB table: +```sql +CREATE TABLE rdb +( + `key` UInt32, + `value` Array(UInt32), + `value2` String +) +ENGINE = EmbeddedRocksDB +PRIMARY KEY key +``` + +```sql +INSERT INTO rdb + SELECT + toUInt32(sipHash64(number) % 10) as key, + [key, key+1] as value, + ('val2' || toString(key)) as value2 + FROM numbers_mt(10); +``` + +##### Create and populate a table to join with table `rdb`: + +```sql +CREATE TABLE t2 +( + `k` UInt16 +) +ENGINE = TinyLog +``` + +```sql +INSERT INTO t2 SELECT number AS k +FROM numbers_mt(10) +``` + +##### Set the join algorithm to `direct`: + +```sql +SET join_algorithm = 'direct' +``` + +##### An INNER JOIN: +```sql +SELECT * +FROM +( + SELECT k AS key + FROM t2 +) AS t2 +INNER JOIN rdb ON rdb.key = t2.key +ORDER BY key ASC +``` +```response +┌─key─┬─rdb.key─┬─value──┬─value2─┐ +│ 0 │ 0 │ [0,1] │ val20 │ +│ 2 │ 2 │ [2,3] │ val22 │ +│ 3 │ 3 │ [3,4] │ val23 │ +│ 6 │ 6 │ [6,7] │ val26 │ +│ 7 │ 7 │ [7,8] │ val27 │ +│ 8 │ 8 │ [8,9] │ val28 │ +│ 9 │ 9 │ [9,10] │ val29 │ +└─────┴─────────┴────────┴────────┘ +``` + From ae497d398a5bcb48fef1d6b90e24fb0818e2bd05 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Jun 2023 14:23:04 +0000 Subject: [PATCH 254/308] Remove unneeded test --- .../__init__.py | 0 .../configs/enable_keeper1.xml | 41 -- .../configs/enable_keeper2.xml | 41 -- .../configs/enable_keeper3.xml | 41 -- .../configs/use_keeper.xml | 16 - .../test.py | 473 ------------------ 6 files changed, 612 deletions(-) delete mode 100644 tests/integration/test_keeper_multinode_blocade_leader/__init__.py delete mode 100644 tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml delete mode 100644 tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml delete mode 100644 tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml delete mode 100644 tests/integration/test_keeper_multinode_blocade_leader/configs/use_keeper.xml delete mode 100644 tests/integration/test_keeper_multinode_blocade_leader/test.py diff --git a/tests/integration/test_keeper_multinode_blocade_leader/__init__.py b/tests/integration/test_keeper_multinode_blocade_leader/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml deleted file mode 100644 index 17455ed12f5..00000000000 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - 9181 - 1 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - - - 5000 - 10000 - 75 - trace - - - - - 1 - node1 - 9234 - true - 3 - - - 2 - node2 - 9234 - true - true - 2 - - - 3 - node3 - 9234 - true - true - 1 - - - - diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml deleted file mode 100644 index 03a23984cc2..00000000000 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - 9181 - 2 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - - - 5000 - 10000 - 75 - trace - - - - - 1 - node1 - 9234 - true - 3 - - - 2 - node2 - 9234 - true - true - 2 - - - 3 - node3 - 9234 - true - true - 1 - - - - diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml deleted file mode 100644 index a3196ac3061..00000000000 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - 9181 - 3 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - - - 5000 - 10000 - 75 - trace - - - - - 1 - node1 - 9234 - true - 3 - - - 2 - node2 - 9234 - true - true - 2 - - - 3 - node3 - 9234 - true - true - 1 - - - - diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/use_keeper.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/use_keeper.xml deleted file mode 100644 index 384e984f210..00000000000 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/use_keeper.xml +++ /dev/null @@ -1,16 +0,0 @@ - - - - node1 - 9181 - - - node2 - 9181 - - - node3 - 9181 - - - diff --git a/tests/integration/test_keeper_multinode_blocade_leader/test.py b/tests/integration/test_keeper_multinode_blocade_leader/test.py deleted file mode 100644 index 3af0751b0fd..00000000000 --- a/tests/integration/test_keeper_multinode_blocade_leader/test.py +++ /dev/null @@ -1,473 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster -import helpers.keeper_utils as keeper_utils -import random -import string -import os -import time -from multiprocessing.dummy import Pool -from helpers.network import PartitionManager -from helpers.test_tools import assert_eq_with_retry - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", - main_configs=["configs/enable_keeper1.xml", "configs/use_keeper.xml"], - stay_alive=True, -) -node2 = cluster.add_instance( - "node2", - main_configs=["configs/enable_keeper2.xml", "configs/use_keeper.xml"], - stay_alive=True, -) -node3 = cluster.add_instance( - "node3", - main_configs=["configs/enable_keeper3.xml", "configs/use_keeper.xml"], - stay_alive=True, -) - -from kazoo.client import KazooClient, KazooState - -""" -In this test, we blockade RAFT leader and check that the whole system is -able to recover. It's not a good test because we use ClickHouse's replicated -tables to check connectivity, but they may require special operations (or a long -wait) after session expiration. We don't use kazoo, because this client pretends -to be very smart: SUSPEND sessions, try to recover them, and so on. The test -will be even less predictable than with ClickHouse tables. - -TODO find (or write) not so smart python client. -TODO remove this when jepsen tests will be written. -""" - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - - finally: - cluster.shutdown() - - -def smaller_exception(ex): - return "\n".join(str(ex).split("\n")[0:2]) - - -def get_fake_zk(nodename, timeout=30.0): - _fake_zk_instance = KazooClient( - hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout - ) - _fake_zk_instance.start() - return _fake_zk_instance - - -def wait_nodes(): - keeper_utils.wait_nodes(cluster, [node1, node2, node3]) - - -# in extremely rare case it can take more than 5 minutes in debug build with sanitizer -@pytest.mark.timeout(600) -def test_blocade_leader(started_cluster): - for i in range(100): - wait_nodes() - try: - for i, node in enumerate([node1, node2, node3]): - node.query( - "CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary", - settings={"allow_deprecated_database_ordinary": 1}, - ) - node.query( - "CREATE TABLE IF NOT EXISTS ordinary.t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format( - i + 1 - ) - ) - break - except Exception as ex: - print("Got exception from node", smaller_exception(ex)) - time.sleep(0.1) - - node2.query( - "INSERT INTO ordinary.t1 SELECT number FROM numbers(10) SETTINGS insert_keeper_max_retries = 0" - ) - - node1.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) - node3.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) - - assert_eq_with_retry(node1, "SELECT COUNT() FROM ordinary.t1", "10") - assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t1", "10") - assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t1", "10") - - with PartitionManager() as pm: - pm.partition_instances(node2, node1) - pm.partition_instances(node3, node1) - - for i in range(100): - try: - restart_replica_for_sure( - node2, "ordinary.t1", "/clickhouse/t1/replicas/2" - ) - node2.query( - "INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - break - except Exception as ex: - try: - node2.query("ATTACH TABLE ordinary.t1") - except Exception as attach_ex: - print("Got exception node2", smaller_exception(attach_ex)) - print("Got exception node2", smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t1", "/clickhouse/t1/replicas/{}".format(num + 1) - ) - assert False, "Cannot insert anything node2" - - for i in range(100): - try: - restart_replica_for_sure( - node3, "ordinary.t1", "/clickhouse/t1/replicas/3" - ) - node3.query( - "INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - break - except Exception as ex: - try: - node3.query("ATTACH TABLE ordinary.t1") - except Exception as attach_ex: - print("Got exception node3", smaller_exception(attach_ex)) - print("Got exception node3", smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t1", "/clickhouse/t1/replicas/{}".format(num + 1) - ) - assert False, "Cannot insert anything node3" - - for n, node in enumerate([node1, node2, node3]): - for i in range(100): - try: - restart_replica_for_sure( - node, "ordinary.t1", "/clickhouse/t1/replicas/{}".format(n + 1) - ) - break - except Exception as ex: - try: - node.query("ATTACH TABLE ordinary.t1") - except Exception as attach_ex: - print( - "Got exception node{}".format(n + 1), - smaller_exception(attach_ex), - ) - - print("Got exception node{}".format(n + 1), smaller_exception(ex)) - time.sleep(0.5) - else: - assert False, "Cannot reconnect for node{}".format(n + 1) - - for i in range(100): - try: - node1.query( - "INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - break - except Exception as ex: - print("Got exception node1", smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t1", "/clickhouse/t1/replicas/{}".format(num + 1) - ) - assert False, "Cannot insert anything node1" - - for n, node in enumerate([node1, node2, node3]): - for i in range(100): - try: - restart_replica_for_sure( - node, "ordinary.t1", "/clickhouse/t1/replicas/{}".format(n + 1) - ) - node.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) - break - except Exception as ex: - try: - node.query("ATTACH TABLE ordinary.t1") - except Exception as attach_ex: - print( - "Got exception node{}".format(n + 1), - smaller_exception(attach_ex), - ) - - print("Got exception node{}".format(n + 1), smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t1", "/clickhouse/t1/replicas/{}".format(num + 1) - ) - assert False, "Cannot sync replica node{}".format(n + 1) - - if node1.query("SELECT COUNT() FROM ordinary.t1") != "310\n": - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t1", "/clickhouse/t1/replicas/{}".format(num + 1) - ) - - assert_eq_with_retry(node1, "SELECT COUNT() FROM ordinary.t1", "310") - assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t1", "310") - assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t1", "310") - - -def dump_zk(node, zk_path, replica_path): - print(node.query("SELECT * FROM system.replication_queue FORMAT Vertical")) - print("Replicas") - print(node.query("SELECT * FROM system.replicas FORMAT Vertical")) - print("Replica 2 info") - print( - node.query( - "SELECT * FROM system.zookeeper WHERE path = '{}' FORMAT Vertical".format( - zk_path - ) - ) - ) - print("Queue") - print( - node.query( - "SELECT * FROM system.zookeeper WHERE path = '{}/queue' FORMAT Vertical".format( - replica_path - ) - ) - ) - print("Log") - print( - node.query( - "SELECT * FROM system.zookeeper WHERE path = '{}/log' FORMAT Vertical".format( - zk_path - ) - ) - ) - print("Parts") - print( - node.query( - "SELECT name FROM system.zookeeper WHERE path = '{}/parts' FORMAT Vertical".format( - replica_path - ) - ) - ) - - -def restart_replica_for_sure(node, table_name, zk_replica_path): - fake_zk = None - try: - node.query("DETACH TABLE {}".format(table_name)) - fake_zk = get_fake_zk(node.name) - if fake_zk.exists(zk_replica_path + "/is_active") is not None: - fake_zk.delete(zk_replica_path + "/is_active") - - node.query("ATTACH TABLE {}".format(table_name)) - except Exception as ex: - print("Exception", ex) - raise ex - finally: - if fake_zk: - fake_zk.stop() - fake_zk.close() - - -# in extremely rare case it can take more than 5 minutes in debug build with sanitizer -@pytest.mark.timeout(600) -def test_blocade_leader_twice(started_cluster): - for i in range(100): - wait_nodes() - try: - for i, node in enumerate([node1, node2, node3]): - node.query( - "CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary", - settings={"allow_deprecated_database_ordinary": 1}, - ) - node.query( - "CREATE TABLE IF NOT EXISTS ordinary.t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format( - i + 1 - ) - ) - break - except Exception as ex: - print("Got exception from node", smaller_exception(ex)) - time.sleep(0.1) - - node2.query( - "INSERT INTO ordinary.t2 SELECT number FROM numbers(10) SETTINGS insert_keeper_max_retries = 0" - ) - - node1.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - node3.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - - assert_eq_with_retry(node1, "SELECT COUNT() FROM ordinary.t2", "10") - assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t2", "10") - assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t2", "10") - - with PartitionManager() as pm: - pm.partition_instances(node2, node1) - pm.partition_instances(node3, node1) - - for i in range(100): - try: - restart_replica_for_sure( - node2, "ordinary.t2", "/clickhouse/t2/replicas/2" - ) - node2.query( - "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - break - except Exception as ex: - try: - node2.query("ATTACH TABLE ordinary.t2") - except Exception as attach_ex: - print("Got exception node2", smaller_exception(attach_ex)) - print("Got exception node2", smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t2", "/clickhouse/t2/replicas/{}".format(num + 1) - ) - assert False, "Cannot reconnect for node2" - - for i in range(100): - try: - restart_replica_for_sure( - node3, "ordinary.t2", "/clickhouse/t2/replicas/3" - ) - node3.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - node3.query( - "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - break - except Exception as ex: - try: - node3.query("ATTACH TABLE ordinary.t2") - except Exception as attach_ex: - print("Got exception node3", smaller_exception(attach_ex)) - print("Got exception node3", smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t2", "/clickhouse/t2/replicas/{}".format(num + 1) - ) - assert False, "Cannot reconnect for node3" - - node2.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - - assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t2", "210") - assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t2", "210") - - # Total network partition - pm.partition_instances(node3, node2) - - for i in range(10): - try: - node3.query( - "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - assert False, "Node3 became leader?" - except Exception as ex: - time.sleep(0.5) - - for i in range(10): - try: - node2.query( - "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - assert False, "Node2 became leader?" - except Exception as ex: - time.sleep(0.5) - - for n, node in enumerate([node1, node2, node3]): - for i in range(100): - try: - restart_replica_for_sure( - node, "ordinary.t2", "/clickhouse/t2/replicas/{}".format(n + 1) - ) - break - except Exception as ex: - try: - node.query("ATTACH TABLE ordinary.t2") - except Exception as attach_ex: - print( - "Got exception node{}".format(n + 1), - smaller_exception(attach_ex), - ) - - print("Got exception node{}".format(n + 1), smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t2", "/clickhouse/t2/replicas/{}".format(num + 1) - ) - assert False, "Cannot reconnect for node{}".format(n + 1) - - for n, node in enumerate([node1, node2, node3]): - for i in range(100): - try: - node.query( - "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - break - except Exception as ex: - print("Got exception node{}".format(n + 1), smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t2", "/clickhouse/t2/replicas/{}".format(num + 1) - ) - assert False, "Cannot reconnect for node{}".format(n + 1) - - for i in range(100): - all_done = True - for n, node in enumerate([node1, node2, node3]): - try: - restart_replica_for_sure( - node, "ordinary.t2", "/clickhouse/t2/replicas/{}".format(n + 1) - ) - node.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - break - except Exception as ex: - all_done = False - try: - node.query("ATTACH TABLE ordinary.t2") - except Exception as attach_ex: - print( - "Got exception node{}".format(n + 1), - smaller_exception(attach_ex), - ) - - print("Got exception node{}".format(n + 1), smaller_exception(ex)) - time.sleep(0.5) - - if all_done: - break - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t2", "/clickhouse/t2/replicas/{}".format(num + 1) - ) - assert False, "Cannot reconnect in i {} retries".format(i) - - assert_eq_with_retry(node1, "SELECT COUNT() FROM ordinary.t2", "510") - if node2.query("SELECT COUNT() FROM ordinary.t2") != "510\n": - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t2", "/clickhouse/t2/replicas/{}".format(num + 1) - ) - - assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t2", "510") - assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t2", "510") From dcd5579851243a38ba9062636b72c2b60ebfadd1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Jun 2023 16:51:04 +0200 Subject: [PATCH 255/308] Mention ON CLUSTER options for SYSTEM queries --- docs/en/sql-reference/statements/system.md | 54 ++++++++++++++-------- docs/ru/sql-reference/statements/system.md | 48 +++++++++++-------- docs/zh/sql-reference/statements/system.md | 40 +++++++++------- 3 files changed, 87 insertions(+), 55 deletions(-) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index c5596b7ba5f..65a35f03fbe 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -119,19 +119,35 @@ The compiled expression cache is enabled/disabled with the query/user/profile-le Resets the [query cache](../../operations/query-cache.md). +```sql +SYSTEM DROP QUERY CACHE [ON CLUSTER cluster_name] +``` + ## FLUSH LOGS Flushes buffered log messages to system tables, e.g. system.query_log. Mainly useful for debugging since most system tables have a default flush interval of 7.5 seconds. This will also create system tables even if message queue is empty. +```sql +SYSTEM FLUSH LOGS [ON CLUSTER cluster_name] +``` + ## RELOAD CONFIG Reloads ClickHouse configuration. Used when configuration is stored in ZooKeeper. Note that `SYSTEM RELOAD CONFIG` does not reload `USER` configuration stored in ZooKeeper, it only reloads `USER` configuration that is stored in `users.xml`. To reload all `USER` config use `SYSTEM RELOAD USERS` +```sql +SYSTEM RELOAD CONFIG [ON CLUSTER cluster_name] +``` + ## RELOAD USERS Reloads all access storages, including: users.xml, local disk access storage, replicated (in ZooKeeper) access storage. +```sql +SYSTEM RELOAD USERS [ON CLUSTER cluster_name] +``` + ## SHUTDOWN Normally shuts down ClickHouse (like `service clickhouse-server stop` / `kill {$pid_clickhouse-server}`) @@ -149,7 +165,7 @@ ClickHouse can manage [distributed](../../engines/table-engines/special/distribu Disables background data distribution when inserting data into distributed tables. ``` sql -SYSTEM STOP DISTRIBUTED SENDS [db.] +SYSTEM STOP DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_name] ``` ### FLUSH DISTRIBUTED @@ -157,7 +173,7 @@ SYSTEM STOP DISTRIBUTED SENDS [db.] Forces ClickHouse to send data to cluster nodes synchronously. If any nodes are unavailable, ClickHouse throws an exception and stops query execution. You can retry the query until it succeeds, which will happen when all nodes are back online. ``` sql -SYSTEM FLUSH DISTRIBUTED [db.] +SYSTEM FLUSH DISTRIBUTED [db.] [ON CLUSTER cluster_name] ``` ### START DISTRIBUTED SENDS @@ -165,7 +181,7 @@ SYSTEM FLUSH DISTRIBUTED [db.] Enables background data distribution when inserting data into distributed tables. ``` sql -SYSTEM START DISTRIBUTED SENDS [db.] +SYSTEM START DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_name] ``` ## Managing MergeTree Tables @@ -177,7 +193,7 @@ ClickHouse can manage background processes in [MergeTree](../../engines/table-en Provides possibility to stop background merges for tables in the MergeTree family: ``` sql -SYSTEM STOP MERGES [ON VOLUME | [db.]merge_tree_family_table_name] +SYSTEM STOP MERGES [ON CLUSTER cluster_name] [ON VOLUME | [db.]merge_tree_family_table_name] ``` :::note @@ -189,7 +205,7 @@ SYSTEM STOP MERGES [ON VOLUME | [db.]merge_tree_family_table_name] Provides possibility to start background merges for tables in the MergeTree family: ``` sql -SYSTEM START MERGES [ON VOLUME | [db.]merge_tree_family_table_name] +SYSTEM START MERGES [ON CLUSTER cluster_name] [ON VOLUME | [db.]merge_tree_family_table_name] ``` ### STOP TTL MERGES @@ -198,7 +214,7 @@ Provides possibility to stop background delete old data according to [TTL expres Returns `Ok.` even if table does not exist or table has not MergeTree engine. Returns error when database does not exist: ``` sql -SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name] +SYSTEM STOP TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### START TTL MERGES @@ -207,7 +223,7 @@ Provides possibility to start background delete old data according to [TTL expre Returns `Ok.` even if table does not exist. Returns error when database does not exist: ``` sql -SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name] +SYSTEM START TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### STOP MOVES @@ -216,7 +232,7 @@ Provides possibility to stop background move data according to [TTL table expres Returns `Ok.` even if table does not exist. Returns error when database does not exist: ``` sql -SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] +SYSTEM STOP MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### START MOVES @@ -225,7 +241,7 @@ Provides possibility to start background move data according to [TTL table expre Returns `Ok.` even if table does not exist. Returns error when database does not exist: ``` sql -SYSTEM START MOVES [[db.]merge_tree_family_table_name] +SYSTEM START MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### SYSTEM UNFREEZE {#query_language-system-unfreeze} @@ -241,7 +257,7 @@ SYSTEM UNFREEZE WITH NAME Wait until all asynchronously loading data parts of a table (outdated data parts) will became loaded. ``` sql -SYSTEM WAIT LOADING PARTS [db.]merge_tree_family_table_name +SYSTEM WAIT LOADING PARTS [ON CLUSTER cluster_name] [db.]merge_tree_family_table_name ``` ## Managing ReplicatedMergeTree Tables @@ -254,7 +270,7 @@ Provides possibility to stop background fetches for inserted parts for tables in Always returns `Ok.` regardless of the table engine and even if table or database does not exist. ``` sql -SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START FETCHES @@ -263,7 +279,7 @@ Provides possibility to start background fetches for inserted parts for tables i Always returns `Ok.` regardless of the table engine and even if table or database does not exist. ``` sql -SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name] +SYSTEM START FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### STOP REPLICATED SENDS @@ -271,7 +287,7 @@ SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name] Provides possibility to stop background sends to other replicas in cluster for new inserted parts for tables in the `ReplicatedMergeTree` family: ``` sql -SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START REPLICATED SENDS @@ -279,7 +295,7 @@ SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] Provides possibility to start background sends to other replicas in cluster for new inserted parts for tables in the `ReplicatedMergeTree` family: ``` sql -SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] +SYSTEM START REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### STOP REPLICATION QUEUES @@ -287,7 +303,7 @@ SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] Provides possibility to stop background fetch tasks from replication queues which stored in Zookeeper for tables in the `ReplicatedMergeTree` family. Possible background tasks types - merges, fetches, mutation, DDL statements with ON CLUSTER clause: ``` sql -SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START REPLICATION QUEUES @@ -295,7 +311,7 @@ SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] Provides possibility to start background fetch tasks from replication queues which stored in Zookeeper for tables in the `ReplicatedMergeTree` family. Possible background tasks types - merges, fetches, mutation, DDL statements with ON CLUSTER clause: ``` sql -SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] +SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### SYNC REPLICA @@ -318,7 +334,7 @@ Provides possibility to reinitialize Zookeeper session's state for `ReplicatedMe Initialization of replication queue based on ZooKeeper data happens in the same way as for `ATTACH TABLE` statement. For a short time, the table will be unavailable for any operations. ``` sql -SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name +SYSTEM RESTART REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name ``` ### RESTORE REPLICA @@ -384,7 +400,7 @@ Provides possibility to reinitialize Zookeeper sessions state for all `Replicate Allows to drop filesystem cache. ```sql -SYSTEM DROP FILESYSTEM CACHE +SYSTEM DROP FILESYSTEM CACHE [ON CLUSTER cluster_name] ``` ### SYNC FILE CACHE @@ -396,5 +412,5 @@ It's too heavy and has potential for misuse. Will do sync syscall. ```sql -SYSTEM SYNC FILE CACHE +SYSTEM SYNC FILE CACHE [ON CLUSTER cluster_name] ``` diff --git a/docs/ru/sql-reference/statements/system.md b/docs/ru/sql-reference/statements/system.md index 22a74648eab..ec30a031643 100644 --- a/docs/ru/sql-reference/statements/system.md +++ b/docs/ru/sql-reference/statements/system.md @@ -39,7 +39,7 @@ SELECT name, status FROM system.dictionaries; **Синтаксис** ```sql -SYSTEM RELOAD MODELS +SYSTEM RELOAD MODELS [ON CLUSTER cluster_name] ``` ## RELOAD MODEL {#query_language-system-reload-model} @@ -49,7 +49,7 @@ SYSTEM RELOAD MODELS **Синтаксис** ```sql -SYSTEM RELOAD MODEL +SYSTEM RELOAD MODEL [ON CLUSTER cluster_name] ``` ## RELOAD FUNCTIONS {#query_language-system-reload-functions} @@ -59,8 +59,8 @@ SYSTEM RELOAD MODEL **Синтаксис** ```sql -RELOAD FUNCTIONS -RELOAD FUNCTION function_name +RELOAD FUNCTIONS [ON CLUSTER cluster_name] +RELOAD FUNCTION function_name [ON CLUSTER cluster_name] ``` ## DROP DNS CACHE {#query_language-system-drop-dns-cache} @@ -106,10 +106,18 @@ Cкомпилированные выражения используются ко Записывает буферы логов в системные таблицы (например system.query_log). Позволяет не ждать 7.5 секунд при отладке. Если буфер логов пустой, то этот запрос просто создаст системные таблицы. +```sql +SYSTEM FLUSH LOGS [ON CLUSTER cluster_name] +``` + ## RELOAD CONFIG {#query_language-system-reload-config} Перечитывает конфигурацию настроек ClickHouse. Используется при хранении конфигурации в zookeeper. +```sql +SYSTEM RELOAD CONFIG [ON CLUSTER cluster_name] +``` + ## SHUTDOWN {#query_language-system-shutdown} Штатно завершает работу ClickHouse (аналог `service clickhouse-server stop` / `kill {$pid_clickhouse-server}`) @@ -127,7 +135,7 @@ ClickHouse может оперировать [распределёнными](.. Отключает фоновую отправку при вставке данных в распределённые таблицы. ``` sql -SYSTEM STOP DISTRIBUTED SENDS [db.] +SYSTEM STOP DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_name] ``` ### FLUSH DISTRIBUTED {#query_language-system-flush-distributed} @@ -135,7 +143,7 @@ SYSTEM STOP DISTRIBUTED SENDS [db.] В синхронном режиме отправляет все данные на узлы кластера. Если какие-либо узлы недоступны, ClickHouse генерирует исключение и останавливает выполнение запроса. Такой запрос можно повторять до успешного завершения, что будет означать возвращение связанности с остальными узлами кластера. ``` sql -SYSTEM FLUSH DISTRIBUTED [db.] +SYSTEM FLUSH DISTRIBUTED [db.] [ON CLUSTER cluster_name] ``` ### START DISTRIBUTED SENDS {#query_language-system-start-distributed-sends} @@ -143,7 +151,7 @@ SYSTEM FLUSH DISTRIBUTED [db.] Включает фоновую отправку при вставке данных в распределенные таблицы. ``` sql -SYSTEM START DISTRIBUTED SENDS [db.] +SYSTEM START DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_name] ``` ## Managing MergeTree Tables {#query-language-system-mergetree} @@ -155,7 +163,7 @@ ClickHouse может управлять фоновыми процессами Позволяет остановить фоновые мержи для таблиц семейства MergeTree: ``` sql -SYSTEM STOP MERGES [ON VOLUME | [db.]merge_tree_family_table_name] +SYSTEM STOP MERGES [ON CLUSTER cluster_name] [ON VOLUME | [db.]merge_tree_family_table_name] ``` :::note @@ -166,7 +174,7 @@ SYSTEM STOP MERGES [ON VOLUME | [db.]merge_tree_family_table_name] Включает фоновые мержи для таблиц семейства MergeTree: ``` sql -SYSTEM START MERGES [ON VOLUME | [db.]merge_tree_family_table_name] +SYSTEM START MERGES [ON CLUSTER cluster_name] [ON VOLUME | [db.]merge_tree_family_table_name] ``` ### STOP TTL MERGES {#query_language-stop-ttl-merges} @@ -175,7 +183,7 @@ SYSTEM START MERGES [ON VOLUME | [db.]merge_tree_family_table_name Возвращает `Ok.` даже если указана несуществующая таблица или таблица имеет тип отличный от MergeTree. Возвращает ошибку если указана не существующая база данных: ``` sql -SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name] +SYSTEM STOP TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### START TTL MERGES {#query_language-start-ttl-merges} @@ -184,7 +192,7 @@ SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name] Возвращает `Ok.` даже если указана несуществующая таблица или таблица имеет тип отличный от MergeTree. Возвращает ошибку если указана не существующая база данных: ``` sql -SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name] +SYSTEM START TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### STOP MOVES {#query_language-stop-moves} @@ -193,7 +201,7 @@ SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name] Возвращает `Ok.` даже если указана несуществующая таблица или таблица имеет тип отличный от MergeTree. Возвращает ошибку если указана не существующая база данных: ``` sql -SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] +SYSTEM STOP MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### START MOVES {#query_language-start-moves} @@ -202,7 +210,7 @@ SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] Возвращает `Ok.` даже если указана несуществующая таблица или таблица имеет тип отличный от MergeTree. Возвращает ошибку если указана не существующая база данных: ``` sql -SYSTEM START MOVES [[db.]merge_tree_family_table_name] +SYSTEM START MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### SYSTEM UNFREEZE {#query_language-system-unfreeze} @@ -223,7 +231,7 @@ ClickHouse может управлять фоновыми процессами Всегда возвращает `Ok.` вне зависимости от типа таблицы и даже если таблица или база данных не существет. ``` sql -SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START FETCHES {#query_language-system-start-fetches} @@ -232,7 +240,7 @@ SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name] Всегда возвращает `Ok.` вне зависимости от типа таблицы и даже если таблица или база данных не существет. ``` sql -SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name] +SYSTEM START FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### STOP REPLICATED SENDS {#query_language-system-start-replicated-sends} @@ -240,7 +248,7 @@ SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name] Позволяет остановить фоновые процессы отсылки новых вставленных кусков данных другим репликам в кластере для таблиц семейства `ReplicatedMergeTree`: ``` sql -SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START REPLICATED SENDS {#query_language-system-start-replicated-sends} @@ -248,7 +256,7 @@ SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] Позволяет запустить фоновые процессы отсылки новых вставленных кусков данных другим репликам в кластере для таблиц семейства `ReplicatedMergeTree`: ``` sql -SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] +SYSTEM START REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### STOP REPLICATION QUEUES {#query_language-system-stop-replication-queues} @@ -256,7 +264,7 @@ SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] Останавливает фоновые процессы разбора заданий из очереди репликации которая хранится в Zookeeper для таблиц семейства `ReplicatedMergeTree`. Возможные типы заданий - merges, fetches, mutation, DDL запросы с ON CLUSTER: ``` sql -SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START REPLICATION QUEUES {#query_language-system-start-replication-queues} @@ -264,7 +272,7 @@ SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] Запускает фоновые процессы разбора заданий из очереди репликации которая хранится в Zookeeper для таблиц семейства `ReplicatedMergeTree`. Возможные типы заданий - merges, fetches, mutation, DDL запросы с ON CLUSTER: ``` sql -SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] +SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### SYNC REPLICA {#query_language-system-sync-replica} @@ -287,7 +295,7 @@ SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT | LIGHT Инициализация очереди репликации на основе данных ZooKeeper происходит так же, как при `ATTACH TABLE`. Некоторое время таблица будет недоступна для любых операций. ``` sql -SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name +SYSTEM RESTART REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name ``` ### RESTORE REPLICA {#query_language-system-restore-replica} diff --git a/docs/zh/sql-reference/statements/system.md b/docs/zh/sql-reference/statements/system.md index 8fd2dd74d26..b41b62d72c3 100644 --- a/docs/zh/sql-reference/statements/system.md +++ b/docs/zh/sql-reference/statements/system.md @@ -71,10 +71,18 @@ SYSTEM DROP REPLICA 'replica_name' FROM ZKPATH '/path/to/table/in/zk'; 将日志信息缓冲数据刷入系统表(例如system.query_log)。调试时允许等待不超过7.5秒。当信息队列为空时,会创建系统表。 +```sql +SYSTEM FLUSH LOGS [ON CLUSTER cluster_name] +``` + ## RELOAD CONFIG {#query_language-system-reload-config} 重新加载ClickHouse的配置。用于当配置信息存放在ZooKeeper时。 +```sql +SYSTEM RELOAD CONFIG [ON CLUSTER cluster_name] +``` + ## SHUTDOWN {#query_language-system-shutdown} 关闭ClickHouse服务(类似于 `service clickhouse-server stop` / `kill {$pid_clickhouse-server}`) @@ -93,7 +101,7 @@ ClickHouse可以管理 [distribute](../../engines/table-engines/special/distribu 当向分布式表插入数据时,禁用后台的分布式数据分发。 ``` sql -SYSTEM STOP DISTRIBUTED SENDS [db.] +SYSTEM STOP DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_name] ``` ### FLUSH DISTRIBUTED {#query_language-system-flush-distributed} @@ -101,7 +109,7 @@ SYSTEM STOP DISTRIBUTED SENDS [db.] 强制让ClickHouse同步向集群节点同步发送数据。如果有节点失效,ClickHouse抛出异常并停止插入操作。当所有节点都恢复上线时,你可以重试之前的操作直到成功执行。 ``` sql -SYSTEM FLUSH DISTRIBUTED [db.] +SYSTEM FLUSH DISTRIBUTED [db.] [ON CLUSTER cluster_name] ``` ### START DISTRIBUTED SENDS {#query_language-system-start-distributed-sends} @@ -109,7 +117,7 @@ SYSTEM FLUSH DISTRIBUTED [db.] 当向分布式表插入数据时,允许后台的分布式数据分发。 ``` sql -SYSTEM START DISTRIBUTED SENDS [db.] +SYSTEM START DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_name] ``` ## Managing MergeTree Tables {#query-language-system-mergetree} @@ -121,7 +129,7 @@ ClickHouse可以管理 [MergeTree](../../engines/table-engines/mergetree-family/ 为MergeTree系列引擎表停止后台合并操作。 ``` sql -SYSTEM STOP MERGES [[db.]merge_tree_family_table_name] +SYSTEM STOP MERGES [ON CLUSTER cluster_name] [ON VOLUME | [db.]merge_tree_family_table_name] ``` :::note @@ -133,7 +141,7 @@ SYSTEM STOP MERGES [[db.]merge_tree_family_table_name] 为MergeTree系列引擎表启动后台合并操作。 ``` sql -SYSTEM START MERGES [[db.]merge_tree_family_table_name] +SYSTEM START MERGES [ON CLUSTER cluster_name] [ON VOLUME | [db.]merge_tree_family_table_name] ``` ### STOP TTL MERGES {#query_language-stop-ttl-merges} @@ -142,7 +150,7 @@ SYSTEM START MERGES [[db.]merge_tree_family_table_name] 不管表存在与否,都返回 `OK.`。当数据库不存在时返回错误。 ``` sql -SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name] +SYSTEM STOP TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### START TTL MERGES {#query_language-start-ttl-merges} @@ -151,7 +159,7 @@ SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name] ``` sql -SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name] +SYSTEM START TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### STOP MOVES {#query_language-stop-moves} @@ -160,7 +168,7 @@ SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name] ``` sql -SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] +SYSTEM STOP MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### START MOVES {#query_language-start-moves} @@ -169,7 +177,7 @@ SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] ``` sql -SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] +SYSTEM START MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### SYSTEM UNFREEZE {#query_language-system-unfreeze} @@ -190,7 +198,7 @@ SYSTEM UNFREEZE WITH NAME 不管表引擎类型如何或表/数据库是否存,都返回 `OK.`。 ``` sql -SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START FETCHES {#query_language-system-start-fetches} @@ -199,7 +207,7 @@ SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name] 不管表引擎类型如何或表/数据库是否存,都返回 `OK.`。 ``` sql -SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name] +SYSTEM START FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### STOP REPLICATED SENDS {#query_language-system-start-replicated-sends} @@ -207,7 +215,7 @@ SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name] 停止通过后台分发 `ReplicatedMergeTree`系列引擎表中新插入的数据块到集群的其它副本节点。 ``` sql -SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START REPLICATED SENDS {#query_language-system-start-replicated-sends} @@ -215,7 +223,7 @@ SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] 启动通过后台分发 `ReplicatedMergeTree`系列引擎表中新插入的数据块到集群的其它副本节点。 ``` sql -SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] +SYSTEM START REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### STOP REPLICATION QUEUES {#query_language-system-stop-replication-queues} @@ -224,7 +232,7 @@ SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] 停止从Zookeeper中获取 `ReplicatedMergeTree`系列表的复制队列的后台任务。可能的后台任务类型包含:merges, fetches, mutation,带有 `ON CLUSTER`的ddl语句 ``` sql -SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START REPLICATION QUEUES {#query_language-system-start-replication-queues} @@ -232,7 +240,7 @@ SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] 启动从Zookeeper中获取 `ReplicatedMergeTree`系列表的复制队列的后台任务。可能的后台任务类型包含:merges, fetches, mutation,带有 `ON CLUSTER`的ddl语句 ``` sql -SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] +SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### SYNC REPLICA {#query_language-system-sync-replica} @@ -250,7 +258,7 @@ SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT | LIGHT ``` sql -SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name +SYSTEM RESTART REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name ``` ### RESTART REPLICAS {#query_language-system-restart-replicas} From 963641b20ffd09c01107fe8b210d38218ca85161 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 18:24:56 +0300 Subject: [PATCH 256/308] disable 00534_functions_bad_arguments with msan (#50481) --- tests/queries/0_stateless/00534_functions_bad_arguments10.sh | 2 +- tests/queries/0_stateless/00534_functions_bad_arguments5.sh | 2 +- tests/queries/0_stateless/00534_functions_bad_arguments6.sh | 2 +- tests/queries/0_stateless/00534_functions_bad_arguments9.sh | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00534_functions_bad_arguments10.sh b/tests/queries/0_stateless/00534_functions_bad_arguments10.sh index 8525b63a989..b9733f92812 100755 --- a/tests/queries/0_stateless/00534_functions_bad_arguments10.sh +++ b/tests/queries/0_stateless/00534_functions_bad_arguments10.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-debug +# Tags: no-tsan, no-debug, no-msan # Tag no-tsan: Too long for TSan # shellcheck disable=SC2016 diff --git a/tests/queries/0_stateless/00534_functions_bad_arguments5.sh b/tests/queries/0_stateless/00534_functions_bad_arguments5.sh index a8b0ce77677..812ba9f97fa 100755 --- a/tests/queries/0_stateless/00534_functions_bad_arguments5.sh +++ b/tests/queries/0_stateless/00534_functions_bad_arguments5.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-debug, no-fasttest +# Tags: no-tsan, no-debug, no-fasttest, no-msan # Tag no-tsan: Too long for TSan # shellcheck disable=SC2016 diff --git a/tests/queries/0_stateless/00534_functions_bad_arguments6.sh b/tests/queries/0_stateless/00534_functions_bad_arguments6.sh index b0080c3b418..6626a6dfe55 100755 --- a/tests/queries/0_stateless/00534_functions_bad_arguments6.sh +++ b/tests/queries/0_stateless/00534_functions_bad_arguments6.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-debug +# Tags: no-tsan, no-debug, no-msan # Tag no-tsan: Too long for TSan # shellcheck disable=SC2016 diff --git a/tests/queries/0_stateless/00534_functions_bad_arguments9.sh b/tests/queries/0_stateless/00534_functions_bad_arguments9.sh index 2975643020b..c7659db8621 100755 --- a/tests/queries/0_stateless/00534_functions_bad_arguments9.sh +++ b/tests/queries/0_stateless/00534_functions_bad_arguments9.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-debug +# Tags: no-tsan, no-debug, no-msan # Tag no-tsan: Too long for TSan # shellcheck disable=SC2016 From 4209ccfc088f832c49d7c530b517e73a2661504a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 17:42:04 +0200 Subject: [PATCH 257/308] fix --- .../MergeTree/ReplicatedMergeTreeSink.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 53 +++++++++++-------- tests/config/config.d/merge_tree.xml | 2 + tests/integration/test_ttl_replicated/test.py | 6 +-- 4 files changed, 37 insertions(+), 25 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 3efd364fc9c..8d9e2e14129 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -87,6 +87,7 @@ private: size_t checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper); /// Rename temporary part and commit to ZooKeeper. + /// Returns a list of conflicting async blocks and true if the whole parts was deduplicated std::pair, bool> commitPart( const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 44403fc708b..0a61369e163 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1936,6 +1936,30 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::executeFetchShared( } } +static void paranoidCheckForCoveredPartsInZooKeeper(const ZooKeeperPtr & zookeeper, const String & replica_path, + MergeTreeDataFormatVersion format_version, const String & covering_part_name) +{ +#ifdef ABORT_ON_LOGICAL_ERROR + constexpr bool paranoid_check_for_covered_parts_default = true; +#else + constexpr bool paranoid_check_for_covered_parts_default = false; +#endif + + bool paranoid_check_for_covered_parts = Context::getGlobalContextInstance()->getConfigRef().getBool( + "replicated_merge_tree_paranoid_check_on_drop_range", paranoid_check_for_covered_parts_default); + if (paranoid_check_for_covered_parts) + { + auto drop_range_info = MergeTreePartInfo::fromPartName(covering_part_name, format_version); + Strings parts_remain = zookeeper->getChildren(replica_path + "/parts"); + for (const auto & part_name : parts_remain) + { + auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); + if (drop_range_info.contains(part_info)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Part {} remains in ZooKeeper after DROP_RANGE {}", part_name, covering_part_name); + } + } +} void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) { @@ -1992,16 +2016,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) /// Forcibly remove parts from ZooKeeper removePartsFromZooKeeperWithRetries(parts_to_remove); - -#ifdef ABORT_ON_LOGICAL_ERROR - Strings parts_remain = getZooKeeper()->getChildren(replica_path + "/parts"); - for (const auto & part_name : parts_remain) - { - auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); - if (drop_range_info.contains(part_info)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} remains in ZooKeeper after DROP_RANGE {}", part_name, entry.new_part_name); - } -#endif + paranoidCheckForCoveredPartsInZooKeeper(getZooKeeper(), replica_path, format_version, entry.new_part_name); if (entry.detach) LOG_DEBUG(log, "Detached {} parts inside {}.", parts_to_remove.size(), entry.new_part_name); @@ -2137,6 +2152,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) { LOG_INFO(log, "All parts from REPLACE PARTITION command have been already attached"); removePartsFromZooKeeperWithRetries(parts_to_remove); + if (replace) + paranoidCheckForCoveredPartsInZooKeeper(getZooKeeper(), replica_path, format_version, entry_replace.drop_range_part_name); return true; } @@ -2433,6 +2450,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } removePartsFromZooKeeperWithRetries(parts_to_remove); + if (replace) + paranoidCheckForCoveredPartsInZooKeeper(getZooKeeper(), replica_path, format_version, entry_replace.drop_range_part_name); res_parts.clear(); parts_to_remove.clear(); cleanup_thread.wakeup(); @@ -7133,7 +7152,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom( clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); } - PartsToRemoveFromZooKeeper parts_to_remove; Coordination::Responses op_results; try @@ -7184,7 +7202,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( auto data_parts_lock = lockParts(); transaction.commit(&data_parts_lock); if (replace) - parts_to_remove = removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper(NO_TRANSACTION_RAW, drop_range, data_parts_lock); + removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, data_parts_lock); } PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed(), profile_events_scope.getSnapshot())); @@ -7204,11 +7222,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom( for (auto & lock : ephemeral_locks) lock.assumeUnlocked(); - /// Forcibly remove replaced parts from ZooKeeper - removePartsFromZooKeeperWithRetries(parts_to_remove); - - /// Speedup removing of replaced parts from filesystem - parts_to_remove.clear(); cleanup_thread.wakeup(); lock2.reset(); @@ -7377,7 +7390,6 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); - PartsToRemoveFromZooKeeper parts_to_remove; Coordination::Responses op_results; try @@ -7414,7 +7426,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta else zkutil::KeeperMultiException::check(code, ops, op_results); - parts_to_remove = removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper(NO_TRANSACTION_RAW, drop_range, src_data_parts_lock); + removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, src_data_parts_lock); transaction.commit(&src_data_parts_lock); } @@ -7436,9 +7448,6 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta for (auto & lock : ephemeral_locks) lock.assumeUnlocked(); - removePartsFromZooKeeperWithRetries(parts_to_remove); - - parts_to_remove.clear(); cleanup_thread.wakeup(); lock2.reset(); diff --git a/tests/config/config.d/merge_tree.xml b/tests/config/config.d/merge_tree.xml index bf2da9b09a2..bee9812274c 100644 --- a/tests/config/config.d/merge_tree.xml +++ b/tests/config/config.d/merge_tree.xml @@ -3,4 +3,6 @@ 1 8 + + 1 diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index a3e7d6e4b8b..39d66d857ff 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -517,7 +517,7 @@ def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_delete_{suff}', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) TTL date + INTERVAL 3 SECOND - SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100 + SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100, remove_empty_parts=0 """.format( suff=num_run, replica=node.name ) @@ -529,7 +529,7 @@ def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_group_by_{suff}', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) TTL date + INTERVAL 3 SECOND GROUP BY id SET val = sum(val) - SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100 + SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100, remove_empty_parts=0 """.format( suff=num_run, replica=node.name ) @@ -541,7 +541,7 @@ def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_where_{suff}', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) TTL date + INTERVAL 3 SECOND DELETE WHERE id % 2 = 1 - SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100 + SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100, remove_empty_parts=0 """.format( suff=num_run, replica=node.name ) From 179b6aca6aec1962fe3690d037aa0dae27531f5d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 17:52:57 +0200 Subject: [PATCH 258/308] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0a61369e163..5b7616d5f28 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2846,6 +2846,8 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo } LOG_DEBUG(log, "Copied {} queue entries, {} entries ignored", total_entries_to_copy, source_queue.size() - total_entries_to_copy); + LOG_TRACE(log, "Parts in ZooKeeper after mimic: {}", fmt::join(zookeeper->getChildren(replica_path + "/parts"), ", ")); + LOG_TRACE(log, "Enqueued fetches after mimic: {}", fmt::join(created_get_parts, ", ")); } From 163b2f32da72a66d44967439446331b9943361f3 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Jun 2023 18:13:46 +0200 Subject: [PATCH 259/308] Update src/Common/AsynchronousMetrics.cpp --- src/Common/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index cf8d451385b..8cd33521cbb 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -948,7 +948,7 @@ void AsynchronousMetrics::update(TimePoint update_time) std::string line; readText(line, *cgroupcpu_max); - auto space = line.find_first_of(" "); + auto space = line.find(" "); if (line.rfind("max", space) == std::string::npos) { From 9d0a63bd9263ba9dc9f9181d2fdd9898d269530f Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Jun 2023 18:15:20 +0200 Subject: [PATCH 260/308] Update src/Common/AsynchronousMetrics.cpp --- src/Common/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 8cd33521cbb..6821647a180 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -948,7 +948,7 @@ void AsynchronousMetrics::update(TimePoint update_time) std::string line; readText(line, *cgroupcpu_max); - auto space = line.find(" "); + auto space = line.find(' '); if (line.rfind("max", space) == std::string::npos) { From 71c5b1d9c67b8a8fca252aa56932b13deeaa12f3 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 12:42:35 -0400 Subject: [PATCH 261/308] add svg function docs --- docs/en/sql-reference/functions/geo/svg.md | 52 ++++++++++++++++++++++ 1 file changed, 52 insertions(+) create mode 100644 docs/en/sql-reference/functions/geo/svg.md diff --git a/docs/en/sql-reference/functions/geo/svg.md b/docs/en/sql-reference/functions/geo/svg.md new file mode 100644 index 00000000000..9081ac71338 --- /dev/null +++ b/docs/en/sql-reference/functions/geo/svg.md @@ -0,0 +1,52 @@ +--- +slug: /en/sql-reference/functions/geo/svg +sidebar_label: SVG +title: "Functions for Generating SVG images from Geo data" +--- + +## Syntax + +``` sql +SVG(geometry,[style]) +``` + +### Parameters + +- `geometry` — Geo data +- `style` — Optional style name + +### Returned value + +- The SVG representation of the geometry: + - SVG circle + - SVG polygon + - SVG path + +Type: String + +## Examples + +### Circle +```sql +SELECT SVG((0., 0.)) +``` +```response + +``` + +### Polygon +```sql +SELECT SVG([(0., 0.), (10, 0), (10, 10), (0, 10)]) +``` +```response + +``` + +### Path +```sql +SELECT SVG([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]]) +``` +```response + +``` + From 9b8975194821fe44018ed5bcbc9d5ae088b970f5 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 12:52:06 -0400 Subject: [PATCH 262/308] add docs for parallel view processing --- docs/en/operations/settings/settings.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 6c951739d41..5730503a670 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4229,6 +4229,12 @@ Default value: `2000` If it's enabled, in hedged requests we can start new connection until receiving first data packet even if we have already made some progress (but progress haven't updated for `receive_data_timeout` timeout), otherwise we disable changing replica after the first time we made progress. +## parallel_view_processing + +Enables pushing to attached views concurrently instead of sequentially. + +Default value: `false`. + ## partial_result_on_first_cancel {#partial_result_on_first_cancel} When set to `true` and the user wants to interrupt a query (for example using `Ctrl+C` on the client), then the query continues execution only on data that was already read from the table. Afterwards, it will return a partial result of the query for the part of the table that was read. To fully stop the execution of a query without a partial result, the user should send 2 cancel requests. From 0d98a46326ca671cb0ff0540972c6eba5280d565 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Jun 2023 13:02:18 -0400 Subject: [PATCH 263/308] Fix KeyError in cherry-pick --- tests/ci/cherry_pick.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index d36315151aa..07cdcc76c3a 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -71,11 +71,11 @@ This pull-request will be merged automatically as it reaches the mergeable state ### If the PR was closed and then reopened If it stuck, check {pr_url} for `{backport_created_label}` and delete it if \ -necessary. Manually merging will do nothing, since `{label_backports_created}` \ +necessary. Manually merging will do nothing, since `{backport_created_label}` \ prevents the original PR {pr_url} from being processed. If you want to recreate the PR: delete the `{label_cherrypick}` label and delete this branch. -You may also need to delete the `{label_backports_created}` label from the original PR. +You may also need to delete the `{backport_created_label}` label from the original PR. """ BACKPORT_DESCRIPTION = """This pull-request is a last step of an automated \ backporting. From 09b5b0c3f7a1265e5b21f2a818ec05c9afdc48e4 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 13:31:02 -0400 Subject: [PATCH 264/308] add word --- 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 2feb7981fcc..0455556ae96 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1453,6 +1453,7 @@ gRPC gccMurmurHash gcem generateRandom +generateRandomStructure generateULID generateUUIDv geoDistance From 85ded501d798a067fd9d3b1bdd0e2d6d8cbcc14b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 2 Jun 2023 20:02:14 +0200 Subject: [PATCH 265/308] 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 2feb7981fcc..0455556ae96 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1453,6 +1453,7 @@ gRPC gccMurmurHash gcem generateRandom +generateRandomStructure generateULID generateUUIDv geoDistance From a3ed86a52879367308f4425bef5617f98486a1bb Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Jun 2023 14:36:44 -0400 Subject: [PATCH 266/308] Documentation --- .../utilities/clickhouse-keeper-client.md | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) create mode 100644 docs/en/operations/utilities/clickhouse-keeper-client.md diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md new file mode 100644 index 00000000000..77f816fe428 --- /dev/null +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -0,0 +1,53 @@ +--- +slug: /en/operations/utilities/clickhouse-keeper-client +sidebar_label: clickhouse-keeper-client +--- + +# clickhouse-keeper-client + +A client application to interact with clickhouse-keeper by its native protocol. + +## Keys {#clickhouse-keeper-client} + +- `-q QUERY`, `--query=QUERY` — Query to execute. If this parameter is not passed, `clickhouse-keeper-client` will start in interactive mode. +- `-h HOST`, `--host=HOST` — Server host. Default value: `localhost`. +- `-p N`, `--port=N` — Server port. Default value: 2181 +- `--connection-timeout=TIMEOUT` — Set connection timeout in seconds. Default value: 10s. +- `--session-timeout=TIMEOUT` — Set session timeout in seconds. Default value: 10s. +- `--operation-timeout=TIMEOUT` — Set operation timeout in seconds. Default value: 10s. +- `--history-file=FILE_PATH` — Set path of history file. Default value: `~/.keeper-client-history`. +- `--help` — Shows the help message. + +## Example {#clickhouse-keeper-client-example} + +```bash +./clickhouse-keeper-client -h localhost:2181 --connection-timeout 30 --session-timeout 30 --operation-timeout 30 +Connected to ZooKeeper at [::1]:2181 with session_id 137 +/ :) ls +keeper foo bar +/ :) cd keeper +/keeper :) ls +api_version +/keeper :) cd api_version +/keeper/api_version :) ls + +/keeper/api_version :) cd xyz +Path /keeper/api_version/xyz does not exists +/keeper/api_version :) cd ../../ +/ :) ls +keeper foo bar +/ :) get keeper/api_version +2 +``` + +## Commands {#clickhouse-keeper-client-commands} + +- `ls [path]` -- Lists the nodes for the given path (default: cwd) +- `cd [path]` -- Change the working path (default `.`) +- `set [version]` -- Updates the node's value. Only update if version matches (default: -1) +- `create ` -- Creates new node +- `get ` -- Returns the node's value +- `remove ` -- Remove the node +- `rmr ` -- Recursively deletes path. Confirmation required +- `flwc ` -- Executes four-letter-word command +- `help` -- Prints this message From 991d1b97fc1b0959d1cb1659ee46a893b693716f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 2 Jun 2023 20:48:31 +0200 Subject: [PATCH 267/308] less traces in logs --- src/IO/WriteBufferFromS3.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 462cf2674c3..210cea02a36 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -245,10 +245,8 @@ WriteBufferFromS3::~WriteBufferFromS3() LOG_INFO(log, "WriteBufferFromS3 is not finalized in destructor. " "It could be if an exception occurs. File is not written to S3. " - "{}. " - "Stack trace: {}", - getLogDetails(), - StackTrace().toString()); + "{}.", + getLogDetails()); } task_tracker->safeWaitAll(); From 87eaaa0f7bf43a7145c24e726af8b3b912f38eea Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 16:30:18 -0400 Subject: [PATCH 268/308] address review comments --- .../table-engines/integrations/embedded-rocksdb.md | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index dab741a9f63..6664b6a4613 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -127,15 +127,17 @@ A special `direct` join with EmbeddedRocksDB tables is supported. This direct join avoids forming a hash table in memory and accesses the data directly from the EmbeddedRocksDB. +With large joins you may see much lower memory usage with direct joins +because the hash table is not created. + To enable direct joins: ```sql -SET join_algorithm = 'direct' +SET join_algorithm = 'direct, hash' ``` :::tip -When the `join_algorithm` is set to `direct`, direct joins will be used -when possible. However, direct joins are not used for RIGHT or FULL JOINs. -ClickHouse will choose another join algorithm when direct joins are not possible. +When the `join_algorithm` is set to `direct, hash`, direct joins will be used +when possible, and hash otherwise. ::: #### Example @@ -205,3 +207,6 @@ ORDER BY key ASC └─────┴─────────┴────────┴────────┘ ``` +### More information on Joins +- [`join_algorithm` setting](/docs/en/operations/settings/settings.md#settings-join_algorithm) +- [JOIN clause](/docs/en/sql-reference/statements/select/join.md) From b091d85bb1b38bd2bcccf8ed1c1588a56e06b7a7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Jun 2023 23:30:47 +0200 Subject: [PATCH 269/308] Remove strange object storage methods --- .../AzureBlobStorage/AzureObjectStorage.cpp | 70 ++++++++++++------- .../AzureBlobStorage/AzureObjectStorage.h | 4 +- .../Cached/CachedObjectStorage.cpp | 4 +- .../Cached/CachedObjectStorage.h | 2 +- .../DiskObjectStorageMetadata.cpp | 8 +-- .../DiskObjectStorageMetadata.h | 4 +- ...jectStorageRemoteMetadataRestoreHelper.cpp | 12 ++-- src/Disks/ObjectStorages/IObjectStorage.cpp | 26 +++++-- src/Disks/ObjectStorages/IObjectStorage.h | 68 +++++++----------- .../MetadataStorageFromDisk.cpp | 4 +- .../MetadataStorageFromPlainObjectStorage.cpp | 52 +++++++------- .../ObjectStorages/S3/S3ObjectStorage.cpp | 68 +++++------------- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 7 +- 13 files changed, 156 insertions(+), 173 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index c4dd0161c70..0044f465081 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -67,6 +67,49 @@ bool AzureObjectStorage::exists(const StoredObject & object) const return false; } +void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const +{ + auto client_ptr = client.get(); + + /// What a shame, no Exists method... + Azure::Storage::Blobs::ListBlobsOptions options; + options.Prefix = path; + if (max_keys) + options.PageSizeHint = max_keys; + else + options.PageSizeHint = settings.get()->list_object_keys_size; + Azure::Storage::Blobs::ListBlobsPagedResponse blob_list_response; + + while (true) + { + blob_list_response = client_ptr->ListBlobs(options); + auto blobs_list = blob_list_response.Blobs; + + for (const auto & blob : blobs_list) + { + children.emplace_back( + blob.Name, + ObjectMetadata{ + static_cast(blob.BlobSize), + blob.Details.LastModified.time_since_epoch().count(), + {}}); + } + + if (max_keys) + { + int keys_left = max_keys - static_cast(children.size()); + if (keys_left <= 0) + break; + options.PageSizeHint = keys_left; + } + + if (blob_list_response.HasPage()) + options.ContinuationToken = blob_list_response.NextPageToken; + else + break; + } +} + std::unique_ptr AzureObjectStorage::readObject( /// NOLINT const StoredObject & object, const ReadSettings & read_settings, @@ -146,33 +189,6 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO patchSettings(write_settings)); } -void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const -{ - auto client_ptr = client.get(); - - Azure::Storage::Blobs::ListBlobsOptions blobs_list_options; - blobs_list_options.Prefix = path; - if (max_keys) - blobs_list_options.PageSizeHint = max_keys; - else - blobs_list_options.PageSizeHint = settings.get()->list_object_keys_size; - - auto blobs_list_response = client_ptr->ListBlobs(blobs_list_options); - for (;;) - { - auto blobs_list = blobs_list_response.Blobs; - - for (const auto & blob : blobs_list) - children.emplace_back(blob.Name, blob.BlobSize); - - if (max_keys && children.size() >= static_cast(max_keys)) - break; - if (!blobs_list_response.HasPage()) - break; - blobs_list_response.MoveToNextPage(); - } -} - /// Remove file. Throws exception if file doesn't exists or it's a directory. void AzureObjectStorage::removeObject(const StoredObject & object) { diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 0c2aecd5c62..a36a03bcda4 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -58,6 +58,8 @@ public: AzureClientPtr && client_, SettingsPtr && settings_); + void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + DataSourceDescription getDataSourceDescription() const override { return data_source_description; } std::string getName() const override { return "AzureObjectStorage"; } @@ -84,8 +86,6 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const override; - /// Remove file. Throws exception if file doesn't exists or it's a directory. void removeObject(const StoredObject & object) override; diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index acf9430e85c..1d24d9d5411 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -201,9 +201,9 @@ std::unique_ptr CachedObjectStorage::cloneObjectStorage( return object_storage->cloneObjectStorage(new_namespace, config, config_prefix, context); } -void CachedObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const +void CachedObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { - object_storage->findAllFiles(path, children, max_keys); + object_storage->listObjects(path, children, max_keys); } ObjectMetadata CachedObjectStorage::getObjectMetadata(const std::string & path) const diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index f8e346e1aed..b5186d39c32 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -71,7 +71,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; ObjectMetadata getObjectMetadata(const std::string & path) const override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index c3284b635da..19d5a8e3567 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -54,7 +54,7 @@ void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf) assertChar('\n', buf); storage_objects[i].relative_path = object_relative_path; - storage_objects[i].bytes_size = object_size; + storage_objects[i].metadata.size_bytes = object_size; } readIntText(ref_count, buf); @@ -93,9 +93,9 @@ void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const writeIntText(total_size, buf); writeChar('\n', buf); - for (const auto & [object_relative_path, object_size] : storage_objects) + for (const auto & [object_relative_path, object_metadata] : storage_objects) { - writeIntText(object_size, buf); + writeIntText(object_metadata.size_bytes, buf); writeChar('\t', buf); writeEscapedString(object_relative_path, buf); writeChar('\n', buf); @@ -139,7 +139,7 @@ DiskObjectStorageMetadata::DiskObjectStorageMetadata( void DiskObjectStorageMetadata::addObject(const String & path, size_t size) { total_size += size; - storage_objects.emplace_back(path, size); + storage_objects.emplace_back(path, ObjectMetadata{size, {}, {}}); } diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h index a2d0653e4aa..6dced85d0b1 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h @@ -21,7 +21,7 @@ private: const std::string & common_metadata_path; /// Relative paths of blobs. - RelativePathsWithSize storage_objects; + RelativePathsWithMetadata storage_objects; const std::string object_storage_root_path; @@ -63,7 +63,7 @@ public: return object_storage_root_path; } - RelativePathsWithSize getBlobsRelativePaths() const + RelativePathsWithMetadata getBlobsRelativePaths() const { return storage_objects; } diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp index 4cca89b9a4f..74d1698bf01 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp @@ -356,7 +356,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * LOG_INFO(disk->log, "Starting restore files for disk {}", disk->name); std::vector> results; - auto restore_files = [this, &source_object_storage, &restore_information, &results](const RelativePathsWithSize & objects) + auto restore_files = [this, &source_object_storage, &restore_information, &results](const RelativePathsWithMetadata & objects) { std::vector keys_names; for (const auto & object : objects) @@ -389,8 +389,8 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * return true; }; - RelativePathsWithSize children; - source_object_storage->findAllFiles(restore_information.source_path, children, /* max_keys= */ 0); + RelativePathsWithMetadata children; + source_object_storage->listObjects(restore_information.source_path, children, /* max_keys= */ 0); restore_files(children); @@ -472,7 +472,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject || disk->object_storage_root_path != restore_information.source_path; std::set renames; - auto restore_file_operations = [this, &source_object_storage, &restore_information, &renames, &send_metadata](const RelativePathsWithSize & objects) + auto restore_file_operations = [this, &source_object_storage, &restore_information, &renames, &send_metadata](const RelativePathsWithMetadata & objects) { const String rename = "rename"; const String hardlink = "hardlink"; @@ -539,8 +539,8 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject return true; }; - RelativePathsWithSize children; - source_object_storage->findAllFiles(restore_information.source_path + "operations/", children, /* max_keys= */ 0); + RelativePathsWithMetadata children; + source_object_storage->listObjects(restore_information.source_path + "operations/", children, /* max_keys= */ 0); restore_file_operations(children); if (restore_information.detached) diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index a810db0cdf8..a5903f9d429 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -16,15 +16,29 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void IObjectStorage::findAllFiles(const std::string &, RelativePathsWithSize &, int) const +bool IObjectStorage::existsOrHasAnyChild(const std::string & path) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "findAllFiles() is not supported"); + RelativePathsWithMetadata files; + listObjects(path, files, 1); + return !files.empty(); } -void IObjectStorage::getDirectoryContents(const std::string &, - RelativePathsWithSize &, - std::vector &) const + +void IObjectStorage::listObjects(const std::string &, RelativePathsWithMetadata &, int) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getDirectoryContents() is not supported"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "listObjects() is not supported"); +} + + +std::optional IObjectStorage::tryGetObjectMetadata(const std::string & path) const +{ + try + { + return getObjectMetadata(path); + } + catch (...) + { + return {}; + } } ThreadPool & IObjectStorage::getThreadPoolWriter() diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 8babb2fbf1a..3a0bf1834a1 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -30,19 +30,6 @@ class WriteBufferFromFileBase; using ObjectAttributes = std::map; -struct RelativePathWithSize -{ - String relative_path; - size_t bytes_size; - - RelativePathWithSize() = default; - - RelativePathWithSize(const String & relative_path_, size_t bytes_size_) - : relative_path(relative_path_), bytes_size(bytes_size_) {} -}; -using RelativePathsWithSize = std::vector; - - struct ObjectMetadata { uint64_t size_bytes; @@ -50,6 +37,22 @@ struct ObjectMetadata std::optional attributes; }; +struct RelativePathWithMetadata +{ + String relative_path; + ObjectMetadata metadata{}; + + RelativePathWithMetadata() = default; + + RelativePathWithMetadata(const String & relative_path_, const ObjectMetadata & metadata_) + : relative_path(relative_path_), metadata(metadata_) + {} +}; + +using RelativePathsWithMetadata = std::vector; + + + /// Base class for all object storages which implement some subset of ordinary filesystem operations. /// /// Examples of object storages are S3, Azure Blob Storage, HDFS. @@ -65,36 +68,17 @@ public: /// Object exists or not virtual bool exists(const StoredObject & object) const = 0; - /// List all objects with specific prefix. - /// - /// For example if you do this over filesystem, you should skip folders and - /// return files only, so something like on local filesystem: - /// - /// find . -type f - /// - /// @param children - out files (relative paths) with their sizes. - /// @param max_keys - return not more then max_keys children - /// NOTE: max_keys is not the same as list_object_keys_size (disk property) - /// - if max_keys is set not more then max_keys keys should be returned - /// - however list_object_keys_size determine the size of the batch and should return all keys - /// - /// NOTE: It makes sense only for real object storages (S3, Azure), since - /// it is used only for one of the following: - /// - send_metadata (to restore metadata) - /// - see DiskObjectStorage::restoreMetadataIfNeeded() - /// - MetadataStorageFromPlainObjectStorage - only for s3_plain disk - virtual void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const; + /// Object exists or any child on the specified path exists. + /// We have this method because object storages are flat for example + /// /a/b/c/d may exist but /a/b/c may not. So this method will return true for + /// /, /a, /a/b, /a/b/c, /a/b/c/d while exists will return true only for /a/b/c/d + virtual bool existsOrHasAnyChild(const std::string & path) const; - /// Analog of directory content for object storage (object storage does not - /// have "directory" definition, but it can be emulated with usage of - /// "delimiter"), so this is analog of: - /// - /// find . -maxdepth 1 $path - /// - /// Return files in @files and directories in @directories - virtual void getDirectoryContents(const std::string & path, - RelativePathsWithSize & files, - std::vector & directories) const; + virtual void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const; + + /// Get object metadata if supported. It should be possible to receive + /// at least size of object + virtual std::optional tryGetObjectMetadata(const std::string & path) const; /// Get object metadata if supported. It should be possible to receive /// at least size of object diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp index 6adf24b5bda..9461a82845f 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp @@ -142,10 +142,10 @@ StoredObjects MetadataStorageFromDisk::getStorageObjects(const std::string & pat object_storage_paths.reserve(object_storage_relative_paths.size()); /// Relative paths -> absolute. - for (auto & [object_relative_path, size] : object_storage_relative_paths) + for (auto & [object_relative_path, object_meta] : object_storage_relative_paths) { auto object_path = fs::path(metadata->getBlobsCommonPrefix()) / object_relative_path; - StoredObject object{ object_path, size, path }; + StoredObject object{ object_path, object_meta.size_bytes, path }; object_storage_paths.push_back(object); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index a680a344746..2459fa38da3 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -39,11 +39,10 @@ std::filesystem::path MetadataStorageFromPlainObjectStorage::getAbsolutePath(con bool MetadataStorageFromPlainObjectStorage::exists(const std::string & path) const { - RelativePathsWithSize children; /// NOTE: exists() cannot be used here since it works only for existing /// key, and does not work for some intermediate path. - object_storage->findAllFiles(getAbsolutePath(path), children, 1); - return !children.empty(); + std::string abs_path = getAbsolutePath(path); + return object_storage->existsOrHasAnyChild(abs_path); } bool MetadataStorageFromPlainObjectStorage::isFile(const std::string & path) const @@ -55,44 +54,47 @@ bool MetadataStorageFromPlainObjectStorage::isFile(const std::string & path) con bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path) const { std::string directory = getAbsolutePath(path); - trimRight(directory); - directory += "/"; + if (!directory.ends_with('/')) + directory += '/'; - /// NOTE: This check is far from ideal, since it work only if the directory - /// really has files, and has excessive API calls - RelativePathsWithSize files; - std::vector directories; - object_storage->getDirectoryContents(directory, files, directories); - return !files.empty() || !directories.empty(); + RelativePathsWithMetadata files; + object_storage->listObjects(directory, files, 1); + return !files.empty(); } uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const { - RelativePathsWithSize children; - object_storage->findAllFiles(getAbsolutePath(path), children, 1); - if (children.empty()) - return 0; - if (children.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "findAllFiles() return multiple paths ({}) for {}", children.size(), path); - return children.front().bytes_size; + RelativePathsWithMetadata children; + auto metadata = object_storage->tryGetObjectMetadata(getAbsolutePath(path)); + if (metadata) + return metadata->size_bytes; + return 0; } std::vector MetadataStorageFromPlainObjectStorage::listDirectory(const std::string & path) const { - RelativePathsWithSize files; - std::vector directories; - object_storage->getDirectoryContents(getAbsolutePath(path), files, directories); + RelativePathsWithMetadata files; + std::string abs_path = getAbsolutePath(path); + if (!abs_path.ends_with('/')) + abs_path += '/'; + + object_storage->listObjects(abs_path, files, 0); std::vector result; for (const auto & path_size : files) + { result.push_back(path_size.relative_path); - for (const auto & directory : directories) - result.push_back(directory); + } + for (auto & row : result) { - chassert(row.starts_with(object_storage_root_path)); - row.erase(0, object_storage_root_path.size()); + chassert(row.starts_with(abs_path)); + row.erase(0, abs_path.size()); + auto slash_pos = row.find_first_of('/'); + if (slash_pos != std::string::npos) + row.erase(slash_pos, row.size() - slash_pos); } + return result; } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 79e9e1141bb..fc7ca4c35b5 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -183,7 +183,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN disk_write_settings); } -void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const +void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { auto settings_ptr = s3_settings.get(); auto client_ptr = client.get(); @@ -211,7 +211,7 @@ void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSi break; for (const auto & object : objects) - children.emplace_back(object.GetKey(), object.GetSize()); + children.emplace_back(object.GetKey(), ObjectMetadata{static_cast(object.GetSize()), object.GetLastModified().Millis() / 1000, {}}); if (max_keys) { @@ -225,54 +225,6 @@ void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSi } while (outcome.GetResult().GetIsTruncated()); } -void S3ObjectStorage::getDirectoryContents(const std::string & path, - RelativePathsWithSize & files, - std::vector & directories) const -{ - auto settings_ptr = s3_settings.get(); - auto client_ptr = client.get(); - - S3::ListObjectsV2Request request; - request.SetBucket(bucket); - /// NOTE: if you do "ls /foo" instead of "ls /foo/" over S3 with this API - /// it will return only "/foo" itself without any underlying nodes. - if (path.ends_with("/")) - request.SetPrefix(path); - else - request.SetPrefix(path + "/"); - request.SetMaxKeys(settings_ptr->list_object_keys_size); - request.SetDelimiter("/"); - - Aws::S3::Model::ListObjectsV2Outcome outcome; - do - { - ProfileEvents::increment(ProfileEvents::S3ListObjects); - ProfileEvents::increment(ProfileEvents::DiskS3ListObjects); - outcome = client_ptr->ListObjectsV2(request); - throwIfError(outcome); - - auto result = outcome.GetResult(); - auto result_objects = result.GetContents(); - auto result_common_prefixes = result.GetCommonPrefixes(); - - if (result_objects.empty() && result_common_prefixes.empty()) - break; - - for (const auto & object : result_objects) - files.emplace_back(object.GetKey(), object.GetSize()); - - for (const auto & common_prefix : result_common_prefixes) - { - std::string directory = common_prefix.GetPrefix(); - /// Make it compatible with std::filesystem::path::filename() - trimRight(directory, '/'); - directories.emplace_back(directory); - } - - request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - } while (outcome.GetResult().GetIsTruncated()); -} - void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exists) { auto client_ptr = client.get(); @@ -359,6 +311,22 @@ void S3ObjectStorage::removeObjectsIfExist(const StoredObjects & objects) removeObjectsImpl(objects, true); } +std::optional S3ObjectStorage::tryGetObjectMetadata(const std::string & path) const +{ + auto settings_ptr = s3_settings.get(); + auto object_info = S3::getObjectInfo(*client.get(), bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true, /* throw_on_error= */ false); + + if (object_info.size == 0 && object_info.last_modification_time == 0 && object_info.metadata.empty()) + return {}; + + ObjectMetadata result; + result.size_bytes = object_info.size; + result.last_modified = object_info.last_modification_time; + result.attributes = object_info.metadata; + + return result; +} + ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 70ed899586e..b0eb01aec0d 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -100,10 +100,7 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const override; - void getDirectoryContents(const std::string & path, - RelativePathsWithSize & files, - std::vector & directories) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; /// Uses `DeleteObjectRequest`. void removeObject(const StoredObject & object) override; @@ -121,6 +118,8 @@ public: ObjectMetadata getObjectMetadata(const std::string & path) const override; + std::optional tryGetObjectMetadata(const std::string & path) const override; + void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, From c73c836f9dfc74dbf5e9d213843c03a97e2a5735 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Jun 2023 23:33:04 +0200 Subject: [PATCH 270/308] Better --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 2459fa38da3..1131ec313e4 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -86,6 +86,7 @@ std::vector MetadataStorageFromPlainObjectStorage::listDirectory(co result.push_back(path_size.relative_path); } + std::unordered_set duplicates_filter; for (auto & row : result) { chassert(row.starts_with(abs_path)); @@ -93,9 +94,10 @@ std::vector MetadataStorageFromPlainObjectStorage::listDirectory(co auto slash_pos = row.find_first_of('/'); if (slash_pos != std::string::npos) row.erase(slash_pos, row.size() - slash_pos); + duplicates_filter.insert(row); } - return result; + return std::vector(duplicates_filter.begin(), duplicates_filter.end()); } DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(const std::string & path) const From 30be723a9aca0a585e92dbaee71d15905d6dc490 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Jun 2023 05:11:02 +0200 Subject: [PATCH 271/308] Fix bad code around metadata in RocksDB --- src/Common/ProfileEvents.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 9f4fc2d135b..2e0c4b82bba 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -429,10 +429,10 @@ The server successfully detected this situation and will download merged part fr M(AggregationPreallocatedElementsInHashTables, "How many elements were preallocated in hash tables for aggregation.") \ M(AggregationHashTablesInitializedAsTwoLevel, "How many hash tables were inited as two-level for aggregation.") \ \ - M(MergeTreeMetadataCacheGet, "Number of rocksdb reads(used for merge tree metadata cache)") \ - M(MergeTreeMetadataCachePut, "Number of rocksdb puts(used for merge tree metadata cache)") \ - M(MergeTreeMetadataCacheDelete, "Number of rocksdb deletes(used for merge tree metadata cache)") \ - M(MergeTreeMetadataCacheSeek, "Number of rocksdb seeks(used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheGet, "Number of rocksdb reads (used for merge tree metadata cache)") \ + M(MergeTreeMetadataCachePut, "Number of rocksdb puts (used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheDelete, "Number of rocksdb deletes (used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheSeek, "Number of rocksdb seeks (used for merge tree metadata cache)") \ M(MergeTreeMetadataCacheHit, "Number of times the read of meta file was done from MergeTree metadata cache") \ M(MergeTreeMetadataCacheMiss, "Number of times the read of meta file was not done from MergeTree metadata cache") \ \ From ccba3500dd92eaecd38a56d92b09336af26f371c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Jun 2023 07:13:26 +0300 Subject: [PATCH 272/308] Update StoragePostgreSQL.cpp --- src/Storages/StoragePostgreSQL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index ab6660abe00..431285da60d 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -247,6 +247,7 @@ public: if (nested_type->isNullable()) nested_type = static_cast(nested_type.get())->getNestedType(); + /// UUIDs inside arrays are expected to be unquoted in PostgreSQL. const bool quoted = !isUUID(nested_type); writeChar('{', ostr); From 894457d6e1cf7f005e1eb3cae57dded0dd8c4699 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 3 Jun 2023 14:36:52 +0200 Subject: [PATCH 273/308] Style fxi --- src/Disks/ObjectStorages/IObjectStorage.h | 1 - .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 5 ----- 2 files changed, 6 deletions(-) diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 3a0bf1834a1..28de80a88cd 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -52,7 +52,6 @@ struct RelativePathWithMetadata using RelativePathsWithMetadata = std::vector; - /// Base class for all object storages which implement some subset of ordinary filesystem operations. /// /// Examples of object storages are S3, Azure Blob Storage, HDFS. diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 1131ec313e4..c119e9f3adc 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -10,11 +10,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - MetadataStorageFromPlainObjectStorage::MetadataStorageFromPlainObjectStorage( ObjectStoragePtr object_storage_, const std::string & object_storage_root_path_) From e61131c4bb7fda6a6883c4b3946e6d5862547728 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 3 Jun 2023 21:45:53 +0300 Subject: [PATCH 274/308] QueryNode small fix --- src/Analyzer/QueryNode.cpp | 20 +++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 4c10d76690a..51e3dac781d 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -202,15 +202,16 @@ bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && - cte_name == rhs_typed.cte_name && - projection_columns == rhs_typed.projection_columns && is_distinct == rhs_typed.is_distinct && is_limit_with_ties == rhs_typed.is_limit_with_ties && is_group_by_with_totals == rhs_typed.is_group_by_with_totals && is_group_by_with_rollup == rhs_typed.is_group_by_with_rollup && is_group_by_with_cube == rhs_typed.is_group_by_with_cube && is_group_by_with_grouping_sets == rhs_typed.is_group_by_with_grouping_sets && - is_group_by_all == rhs_typed.is_group_by_all; + is_group_by_all == rhs_typed.is_group_by_all && + cte_name == rhs_typed.cte_name && + projection_columns == rhs_typed.projection_columns && + settings_changes == rhs_typed.settings_changes; } void QueryNode::updateTreeHashImpl(HashState & state) const @@ -239,6 +240,18 @@ void QueryNode::updateTreeHashImpl(HashState & state) const state.update(is_group_by_with_cube); state.update(is_group_by_with_grouping_sets); state.update(is_group_by_all); + + state.update(settings_changes.size()); + + for (const auto & setting_change : settings_changes) + { + state.update(setting_change.name.size()); + state.update(setting_change.name); + + auto setting_change_value_dump = setting_change.value.dump(); + state.update(setting_change_value_dump.size()); + state.update(setting_change_value_dump); + } } QueryTreeNodePtr QueryNode::cloneImpl() const @@ -256,6 +269,7 @@ QueryTreeNodePtr QueryNode::cloneImpl() const result_query_node->is_group_by_all = is_group_by_all; result_query_node->cte_name = cte_name; result_query_node->projection_columns = projection_columns; + result_query_node->settings_changes = settings_changes; return result_query_node; } From 0f37be549236f4b891ec324abd73adfdcc68f0b0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 17:20:07 +0200 Subject: [PATCH 275/308] Fix description for BrokenDistributedFilesToInsert Signed-off-by: Azat Khuzhin --- src/Common/CurrentMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 82d68ca8185..63a23c30f84 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -146,7 +146,7 @@ M(OutdatedPartsLoadingThreads, "Number of threads in the threadpool for loading Outdated data parts.") \ M(OutdatedPartsLoadingThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \ M(DistributedFilesToInsert, "Number of pending files to process for asynchronous insertion into Distributed tables. Number of files for every shard is summed.") \ - M(BrokenDistributedFilesToInsert, "Number of files for asynchronous insertion into Distributed tables that has been marked as broken. This metric will starts from 0 on start. Number of files for every shard is summed.") \ + M(BrokenDistributedFilesToInsert, "Number of files for asynchronous insertion into Distributed tables that has been marked as broken. Number of files for every shard is summed.") \ M(TablesToDropQueueSize, "Number of dropped tables, that are waiting for background data removal.") \ M(MaxDDLEntryID, "Max processed DDL entry of DDLWorker.") \ M(MaxPushedDDLEntryID, "Max DDL entry of DDLWorker that pushed to zookeeper.") \ From 69aec7af9bfa475bbc6581f4c582eff6db04d3de Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 17:23:39 +0200 Subject: [PATCH 276/308] Add new metrics BrokenDistributedBytesToInsert/DistributedBytesToInsert Useful to see at the server status overall. Signed-off-by: Azat Khuzhin --- src/Common/CurrentMetrics.cpp | 2 ++ .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 9 +++++++++ .../Distributed/DistributedAsyncInsertDirectoryQueue.h | 2 ++ 3 files changed, 13 insertions(+) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 63a23c30f84..edfbbe17600 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -145,6 +145,8 @@ M(ParquetDecoderThreadsActive, "Number of threads in the ParquetBlockInputFormat thread pool.") \ M(OutdatedPartsLoadingThreads, "Number of threads in the threadpool for loading Outdated data parts.") \ M(OutdatedPartsLoadingThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \ + M(DistributedBytesToInsert, "Number of pending bytes to process for asynchronous insertion into Distributed tables. Number of bytes for every shard is summed.") \ + M(BrokenDistributedBytesToInsert, "Number of bytes for asynchronous insertion into Distributed tables that has been marked as broken. Number of bytes for every shard is summed.") \ M(DistributedFilesToInsert, "Number of pending files to process for asynchronous insertion into Distributed tables. Number of files for every shard is summed.") \ M(BrokenDistributedFilesToInsert, "Number of files for asynchronous insertion into Distributed tables that has been marked as broken. Number of files for every shard is summed.") \ M(TablesToDropQueueSize, "Number of dropped tables, that are waiting for background data removal.") \ diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 9a9a6651bc4..d8d9a0c9d1e 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -35,6 +35,8 @@ namespace CurrentMetrics extern const Metric DistributedSend; extern const Metric DistributedFilesToInsert; extern const Metric BrokenDistributedFilesToInsert; + extern const Metric DistributedBytesToInsert; + extern const Metric BrokenDistributedBytesToInsert; } namespace fs = std::filesystem; @@ -138,7 +140,9 @@ DistributedAsyncInsertDirectoryQueue::DistributedAsyncInsertDirectoryQueue( , max_sleep_time(storage.getDistributedSettingsRef().monitor_max_sleep_time_ms.totalMilliseconds()) , log(&Poco::Logger::get(getLoggerName())) , monitor_blocker(monitor_blocker_) + , metric_pending_bytes(CurrentMetrics::DistributedBytesToInsert, 0) , metric_pending_files(CurrentMetrics::DistributedFilesToInsert, 0) + , metric_broken_bytes(CurrentMetrics::BrokenDistributedBytesToInsert, 0) , metric_broken_files(CurrentMetrics::BrokenDistributedFilesToInsert, 0) { fs::create_directory(broken_path); @@ -357,6 +361,7 @@ void DistributedAsyncInsertDirectoryQueue::initializeFilesFromDisk() LOG_TRACE(log, "Files set to {}", pending_files.size()); LOG_TRACE(log, "Bytes set to {}", bytes_count); + metric_pending_bytes.changeTo(bytes_count); metric_pending_files.changeTo(pending_files.size()); status.files_count = pending_files.size(); status.bytes_count = bytes_count; @@ -380,6 +385,7 @@ void DistributedAsyncInsertDirectoryQueue::initializeFilesFromDisk() LOG_TRACE(log, "Broken bytes set to {}", broken_bytes_count); metric_broken_files.changeTo(broken_files); + metric_broken_bytes.changeTo(broken_bytes_count); status.broken_files_count = broken_files; status.broken_bytes_count = broken_bytes_count; } @@ -520,6 +526,7 @@ bool DistributedAsyncInsertDirectoryQueue::addFileAndSchedule(const std::string { std::lock_guard lock(status_mutex); metric_pending_files.add(); + metric_pending_bytes.add(file_size); status.bytes_count += file_size; ++status.files_count; } @@ -679,6 +686,7 @@ void DistributedAsyncInsertDirectoryQueue::markAsBroken(const std::string & file status.broken_bytes_count += file_size; metric_broken_files.add(); + metric_broken_bytes.add(file_size); } fs::rename(file_path, broken_file_path); @@ -692,6 +700,7 @@ void DistributedAsyncInsertDirectoryQueue::markAsSend(const std::string & file_p { std::lock_guard status_lock(status_mutex); metric_pending_files.sub(); + metric_pending_bytes.sub(file_size); --status.files_count; status.bytes_count -= file_size; } diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h index de8bb813824..9a8a235e265 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h @@ -149,7 +149,9 @@ private: BackgroundSchedulePoolTaskHolder task_handle; + CurrentMetrics::Increment metric_pending_bytes; CurrentMetrics::Increment metric_pending_files; + CurrentMetrics::Increment metric_broken_bytes; CurrentMetrics::Increment metric_broken_files; }; From 009fe3d25e8a755e86d45084f2af5784f4463523 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 3 Jun 2023 21:32:29 +0200 Subject: [PATCH 277/308] Add profile events for eviction --- src/Common/ProfileEvents.cpp | 3 +++ src/Interpreters/Cache/FileCache.cpp | 14 +++++++++++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 9f4fc2d135b..8146a5017ad 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -381,6 +381,9 @@ The server successfully detected this situation and will download merged part fr M(CachedWriteBufferCacheWriteBytes, "Bytes written from source (remote fs, etc) to filesystem cache") \ M(CachedWriteBufferCacheWriteMicroseconds, "Time spent writing data into filesystem cache") \ \ + M(FilesystemCacheEvictedBytes, "Number of bytes evicted from filesystem cache") \ + M(FilesystemCacheEvictedFileSegments, "Number of file segments evicted from filesystem cache") \ + \ M(RemoteFSSeeks, "Total number of seeks for async buffer") \ M(RemoteFSPrefetches, "Number of prefetches made with asynchronous reading from remote filesystem") \ M(RemoteFSCancelledPrefetches, "Number of cancelled prefecthes (because of seek)") \ diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index ba160a31b73..79a9765108f 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -17,6 +17,12 @@ namespace fs = std::filesystem; +namespace ProfileEvents +{ + extern const Event FilesystemCacheEvictedBytes; + extern const Event FilesystemCacheEvictedFileSegments; +} + namespace { @@ -643,7 +649,9 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) return PriorityIterationResult::CONTINUE; } - /// TODO: we can resize if partially downloaded instead. + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); + locked_key.removeFileSegment(segment->offset(), segment->lock()); return PriorityIterationResult::REMOVE_AND_CONTINUE; } @@ -721,6 +729,10 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) chassert(candidate->releasable()); const auto * segment = candidate->file_segment.get(); + + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); + locked_key->removeFileSegment(segment->offset(), segment->lock()); segment->getQueueIterator()->remove(cache_lock); From 57070227a8468f68f9caede43b4d93ad286a2d00 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 3 Jun 2023 23:44:19 +0200 Subject: [PATCH 278/308] Fxi --- .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp | 4 +++- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 0044f465081..0358b4e915a 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -91,7 +91,9 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith blob.Name, ObjectMetadata{ static_cast(blob.BlobSize), - blob.Details.LastModified.time_since_epoch().count(), + Poco::Timestamp::fromEpochTime( + std::chrono::duration_cast( + blob.Details.LastModified.time_since_epoch()).count()), {}}); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index fc7ca4c35b5..6e63efcc1e3 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -211,7 +211,7 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet break; for (const auto & object : objects) - children.emplace_back(object.GetKey(), ObjectMetadata{static_cast(object.GetSize()), object.GetLastModified().Millis() / 1000, {}}); + children.emplace_back(object.GetKey(), ObjectMetadata{static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), {}}); if (max_keys) { From 6f01e2ad9fd25d3d09828d7fb8847d7e730ccb3f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Jun 2023 02:49:45 +0200 Subject: [PATCH 279/308] Fix incorrect constant folding --- src/Parsers/ExpressionElementParsers.cpp | 6 ++- ..._formatting_and_constant_folding.reference | 4 ++ ...istent_formatting_and_constant_folding.sql | 41 +++++++++++++++++++ 3 files changed, 50 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.reference create mode 100644 tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.sql diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 28cef51e571..3a7e8790bb4 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -829,7 +829,11 @@ static bool parseNumber(char * buffer, size_t size, bool negative, int base, Fie if (pos_integer == buffer + size && errno != ERANGE && (!negative || uint_value <= (1ULL << 63))) { - if (negative) + /// -0 should be still parsed as UInt instead of Int, + /// because otherwise it is not preserved during formatting-parsing roundtrip + /// (the signedness is lost during formatting) + + if (negative && uint_value != 0) res = static_cast(-uint_value); else res = uint_value; diff --git a/tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.reference b/tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.reference new file mode 100644 index 00000000000..39d23aa2298 --- /dev/null +++ b/tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.reference @@ -0,0 +1,4 @@ +0 UInt8 -1 Int8 -0 Float64 +0 +0 +0 diff --git a/tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.sql b/tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.sql new file mode 100644 index 00000000000..31278862ab8 --- /dev/null +++ b/tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.sql @@ -0,0 +1,41 @@ +SELECT -0, toTypeName(-0), -1, toTypeName(-1), -0., toTypeName(-0.); + +DROP TABLE IF EXISTS t4; +DROP TABLE IF EXISTS t7; + +create table t4 (c26 String) engine = Log; +create view t7 as select max(ref_3.c26) as c_2_c46_1 from t4 as ref_3; + +select + c_7_c4585_14 as c_4_c4593_5 + from + (select + avg(0) as c_7_c4572_1, + max(-0) as c_7_c4585_14 + from + t7 as ref_0 + group by ref_0.c_2_c46_1) as subq_0 +where c_4_c4593_5 <= multiIf(true, 1, exp10(c_4_c4593_5) <= 1, 1, 1); + +select x as c + from + (select 1 AS k, + max(0) as a, + max(-0) as x + from + t7 GROUP BY k) +where NOT ignore(c); + +SELECT x +FROM +( + SELECT + avg(0) AS c_7_c4572_1, + max(-0) AS x + FROM t7 AS ref_0 + GROUP BY ref_0.c_2_c46_1 +) +WHERE x <= multiIf(true, 1, exp10(x) <= 1, 1, 1); + +DROP TABLE t7; +DROP TABLE t4; From c62558f982366ba2b2bc03cd410bf70840358fcc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Jun 2023 04:44:51 +0200 Subject: [PATCH 280/308] Control memory usage in generateRandom --- src/Storages/StorageGenerateRandom.cpp | 85 ++++++++++++++++++- ...0416_pocopatch_progress_in_http_headers.sh | 1 - 2 files changed, 83 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index e48d3187cb2..293beca9c24 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -29,6 +29,7 @@ #include #include +#include #include #include @@ -81,6 +82,66 @@ void fillBufferWithRandomData(char * __restrict data, size_t limit, size_t size_ } +size_t estimateValueSize( + const DataTypePtr type, + UInt64 max_array_length, + UInt64 max_string_length) +{ + if (type->haveMaximumSizeOfValue()) + return type->getMaximumSizeOfValueInMemory(); + + TypeIndex idx = type->getTypeId(); + + switch (idx) + { + case TypeIndex::String: + { + return max_string_length + sizeof(size_t) + 1; + } + + /// The logic in this function should reflect the logic of fillColumnWithRandomData. + case TypeIndex::Array: + { + auto nested_type = typeid_cast(*type).getNestedType(); + return sizeof(size_t) + estimateValueSize(nested_type, max_array_length / 2, max_string_length); + } + + case TypeIndex::Map: + { + const DataTypePtr & nested_type = typeid_cast(*type).getNestedType(); + return sizeof(size_t) + estimateValueSize(nested_type, max_array_length / 2, max_string_length); + } + + case TypeIndex::Tuple: + { + auto elements = typeid_cast(type.get())->getElements(); + const size_t tuple_size = elements.size(); + size_t res = 0; + + for (size_t i = 0; i < tuple_size; ++i) + res += estimateValueSize(elements[i], max_array_length, max_string_length); + + return res; + } + + case TypeIndex::Nullable: + { + auto nested_type = typeid_cast(*type).getNestedType(); + return 1 + estimateValueSize(nested_type, max_array_length, max_string_length); + } + + case TypeIndex::LowCardinality: + { + auto nested_type = typeid_cast(*type).getDictionaryType(); + return sizeof(size_t) + estimateValueSize(nested_type, max_array_length, max_string_length); + } + + default: + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "The 'GenerateRandom' is not implemented for type {}", type->getName()); + } +} + + ColumnPtr fillColumnWithRandomData( const DataTypePtr type, UInt64 limit, @@ -192,7 +253,8 @@ ColumnPtr fillColumnWithRandomData( offsets[i] = offset; } - auto data_column = fillColumnWithRandomData(nested_type, offset, max_array_length, max_string_length, rng, context); + /// This division by two makes the size growth subexponential on depth. + auto data_column = fillColumnWithRandomData(nested_type, offset, max_array_length / 2, max_string_length, rng, context); return ColumnArray::create(data_column, std::move(offsets_column)); } @@ -200,7 +262,7 @@ ColumnPtr fillColumnWithRandomData( case TypeIndex::Map: { const DataTypePtr & nested_type = typeid_cast(*type).getNestedType(); - auto nested_column = fillColumnWithRandomData(nested_type, limit, max_array_length, max_string_length, rng, context); + auto nested_column = fillColumnWithRandomData(nested_type, limit, max_array_length / 2, max_string_length, rng, context); return ColumnMap::create(nested_column); } @@ -597,6 +659,25 @@ Pipe StorageGenerateRandom::read( block_header.insert({std::move(column), name_type.type, name_type.name}); } + /// Correction of block size for wide tables. + size_t preferred_block_size_bytes = context->getSettingsRef().preferred_block_size_bytes; + if (preferred_block_size_bytes) + { + size_t estimated_row_size_bytes = estimateValueSize(std::make_shared(block_header.getDataTypes()), max_array_length, max_string_length); + + size_t estimated_block_size_bytes = 0; + if (common::mulOverflow(max_block_size, estimated_row_size_bytes, estimated_block_size_bytes)) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large estimated block size in GenerateRandom table: its estimation leads to 64bit overflow"); + chassert(estimated_block_size_bytes != 0); + + if (estimated_block_size_bytes > preferred_block_size_bytes) + { + max_block_size = static_cast(max_block_size * (static_cast(preferred_block_size_bytes) / estimated_block_size_bytes)); + if (max_block_size == 0) + max_block_size = 1; + } + } + /// Will create more seed values for each source from initial seed. pcg64 generate(random_seed); diff --git a/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh b/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh index 6e9814cbca8..b2189ab0cc2 100755 --- a/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh +++ b/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh @@ -5,7 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=5&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT max(number) FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' -# This test will fail with external poco (progress not supported) ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" -d 'SELECT number FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&enable_http_compression=1" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d From 7ceedbd1806348f3af80d135430db5ca245a341a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Jun 2023 04:45:55 +0200 Subject: [PATCH 281/308] Update tests --- tests/queries/0_stateless/02539_generate_random_map.reference | 4 ++-- .../0_stateless/02586_generate_random_structure.reference | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02539_generate_random_map.reference b/tests/queries/0_stateless/02539_generate_random_map.reference index c0dc175c3cd..25e7f55667e 100644 --- a/tests/queries/0_stateless/02539_generate_random_map.reference +++ b/tests/queries/0_stateless/02539_generate_random_map.reference @@ -1,2 +1,2 @@ -1 -20 +0 +10 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index d2929fb4564..abe168c59e8 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -13,7 +13,7 @@ c3.e6 Array(Int256) c4 FixedString(183) c5 IPv4 c6 UInt256 -Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10','e1V0','e1V10','e1V14','e1V10','e1V14'] [{-13:777622572,102:-1122882357,62:1647813163,-94:2094022166},{-32:1448633509},{},{},{34:1536340393,19:-2049677851,74:65643868,-46:-1990799930,97:-531041081,46:-2634833,14:1581632600,89:-771229823,-105:1238603584},{47:1458809010,109:1640682510,86:1945730198,85:1505847247,35:-35189402}] [153363749503.3642648494826450951141750747382772821825909005880434540971999557,79828591186.7378041015337066268618633118713347614941338787453473118807106292,81672688565.9633830721322966111551266731935181670389237071708068971548883315,573768486971.1812413548839655834002608768736215115033958693122764224003897029,-393925092368.4893467278351090742501814120269109477445490969167853713051140487,46027399426.0865278566391382610843315130162915324295037009704113636499519839] [755855942,1804001770,-78103159,-866181765,731736602,-79599206] [5253556148991564114,4681434929596395351,-7302160004580855709,-3686747220178471318,6288582051009949273,646864891160092871] [17035203905051045016266537043565487029724162173062647021612805252288722534904,-42105881403933504641593145676742477006499618886131028341247993701618141933523,45346626822580305846120377917274679004279343244238782744860626882886217433843,-3660165069803677989574889324494857545543653453780976182221584349306428201647,-23316760935816288837287058499520670431785615691220162210524162590241529297823,6184785563808848524970564618169964412151721224362412457508264894603779018817] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 -Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10','e1V0','e1V10','e1V14','e1V10','e1V14'] [{-13:777622572,102:-1122882357,62:1647813163,-94:2094022166},{-32:1448633509},{},{},{34:1536340393,19:-2049677851,74:65643868,-46:-1990799930,97:-531041081,46:-2634833,14:1581632600,89:-771229823,-105:1238603584},{47:1458809010,109:1640682510,86:1945730198,85:1505847247,35:-35189402}] [153363749503.3642648494826450951141750747382772821825909005880434540971999557,79828591186.7378041015337066268618633118713347614941338787453473118807106292,81672688565.9633830721322966111551266731935181670389237071708068971548883315,573768486971.1812413548839655834002608768736215115033958693122764224003897029,-393925092368.4893467278351090742501814120269109477445490969167853713051140487,46027399426.0865278566391382610843315130162915324295037009704113636499519839] [755855942,1804001770,-78103159,-866181765,731736602,-79599206] [5253556148991564114,4681434929596395351,-7302160004580855709,-3686747220178471318,6288582051009949273,646864891160092871] [17035203905051045016266537043565487029724162173062647021612805252288722534904,-42105881403933504641593145676742477006499618886131028341247993701618141933523,45346626822580305846120377917274679004279343244238782744860626882886217433843,-3660165069803677989574889324494857545543653453780976182221584349306428201647,-23316760935816288837287058499520670431785615691220162210524162590241529297823,6184785563808848524970564618169964412151721224362412457508264894603779018817] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 +Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10','e1V0','e1V10','e1V14','e1V10','e1V14'] [{-13:237920722},{102:1831835481},{},{},{62:-1960618666,-94:-1219892650},{-32:777622572}] [-431683920736.2529974565693898437068698973660186760023819586689844248966618581,59756148020.9162673945900094878560562068677573399624031604279613893604002735,586508082308.9120376291637372686770291666437007489038136467093669000000245094,-542964250950.8072472621084212227729061994250148872708538789242780170470779236,102348230986.0528243362965635624540408239649901816289081513971979913007157412,856260707339.0308016734722383288485766011293159337170215574648625303012038641] [700742145,583743175,-694731143,205377506,-47073316,524775483] [-1523104801326101990,-3813742700509249040,-198797568561120929,1360848130483946585,-2034445041726915230,6445693522245970031] [-42294009754395695750286152418877456026522055877244789929409825751148328749462,-30114637036117781224059264373564439361653611308928312514969460679379590706382,50790691897328530213645537587874962516097580703236937570724165115560305762147,-53982023979693597997184389721196431225054557132927198244547868871851761566844,-4746897427675718862552189488292169089453556841811840545196357111533622948298,-12014200750706123405006110782843469166121588186681927916239818910819762049960] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 +Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10','e1V0','e1V10','e1V14','e1V10','e1V14'] [{-13:237920722},{102:1831835481},{},{},{62:-1960618666,-94:-1219892650},{-32:777622572}] [-431683920736.2529974565693898437068698973660186760023819586689844248966618581,59756148020.9162673945900094878560562068677573399624031604279613893604002735,586508082308.9120376291637372686770291666437007489038136467093669000000245094,-542964250950.8072472621084212227729061994250148872708538789242780170470779236,102348230986.0528243362965635624540408239649901816289081513971979913007157412,856260707339.0308016734722383288485766011293159337170215574648625303012038641] [700742145,583743175,-694731143,205377506,-47073316,524775483] [-1523104801326101990,-3813742700509249040,-198797568561120929,1360848130483946585,-2034445041726915230,6445693522245970031] [-42294009754395695750286152418877456026522055877244789929409825751148328749462,-30114637036117781224059264373564439361653611308928312514969460679379590706382,50790691897328530213645537587874962516097580703236937570724165115560305762147,-53982023979693597997184389721196431225054557132927198244547868871851761566844,-4746897427675718862552189488292169089453556841811840545196357111533622948298,-12014200750706123405006110782843469166121588186681927916239818910819762049960] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10'] [{}] [825002272867.1157788721157301271303736024856710948164507982705676578804195475] [1865150610] [7514464811443271056] [33504961604882608369857530219353040639899064613284394558131808339620328539033] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 c1 LowCardinality(Nullable(UInt64)), c2 Date32, c3 LowCardinality(Nullable(Float64)), c4 Int256, c5 Date32 From 53ec091c8d0b8aae36b2ee533f77b6cecc8dadf5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Jun 2023 05:00:29 +0200 Subject: [PATCH 282/308] Disable skim (Rust library) under memory sanitizer --- rust/skim/CMakeLists.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/rust/skim/CMakeLists.txt b/rust/skim/CMakeLists.txt index 1e7a43aba7c..c2e406ec12f 100644 --- a/rust/skim/CMakeLists.txt +++ b/rust/skim/CMakeLists.txt @@ -14,6 +14,11 @@ if (OS_FREEBSD) return() endif() +if (SANITIZE STREQUAL "memory") + message(STATUS "skim is disabled under memory sanitizer, because the interop is not instrumented properly") + return() +endif() + clickhouse_import_crate(MANIFEST_PATH Cargo.toml) # -Wno-dollar-in-identifier-extension: cxx bridge complies names with '$' From 60c2245da4c1cb23a7e62088b6c409afe941bd56 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Jun 2023 10:58:05 +0200 Subject: [PATCH 283/308] Make 01565_query_loop_after_client_error slightly more robust MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit CI founds one failure of this test [1], which is quite interesting, the test itself should not takes too long: 2023.06.04 05:01:36.889144 [ 42287 ] {} DynamicQueryHandler: Request URI: /?query=SELECT+%27Running+test+stateless%2F01565_query_loop_after_client_error.expect+from+pid%3D1053%27&database=system&connect_timeout=30&receive_timeout=30&send_timeout=30&http_connection_timeout=30&http_receive_timeout=30&http_send_timeout=30&output_format_parallel_formatting=0 ... 2023.06.04 05:01:37.364595 [ 42844 ] {fa79939f-1fa0-4f3b-b599-fd2269122d6a} executeQuery: (from [::1]:40264) (comment: 01565_query_loop_after_client_error.expect) INSERT INTO t01565(c0, c1) VALUES (stage: Complete) 2023.06.04 05:01:37.366485 [ 42844 ] {fa79939f-1fa0-4f3b-b599-fd2269122d6a} TCPHandler: Change cancellation status from NOT_CANCELLED to FULLY_CANCELLED. Log message: Received 'Cancel' packet from the client, canceling the query. 2023.06.04 05:01:37.366810 [ 42844 ] {fa79939f-1fa0-4f3b-b599-fd2269122d6a} TCPHandler: Processed in 0.002539914 sec. But for the last INSERT the client itself works **very slow**, which seems was the reason why "\r" had been sent too early and was never interpreted: send: sending "INSERT INTO t01565(c0, c1) VALUES ('1', 1) ;\n" to { exp4 } expect: does " " (spawn_id exp4) match glob pattern "INSERT"? no I expect: does " \u001b[21GI\u001b[0m\u001b[J\u001b[22G" (spawn_id exp4) match glob pattern "INSERT"? no  expect: does " \u001b[21GI\u001b[0m\u001b[J\u001b[22G\u001b[21G" (spawn_id exp4) match glob pattern "INSERT"? no INSERT INTO t expect: does " \u001b[21GI\u001b[0m\u001b[J\u001b[22G\u001b[21GINSERT INTO t\u001b[0m" (spawn_id exp4) match glob pattern "INSERT"? yes expect: set expect_out(0,string) "INSERT" expect: set expect_out(spawn_id) "exp4" expect: set expect_out(buffer) " \u001b[21GI\u001b[0m\u001b[J\u001b[22G\u001b[21GINSERT" send: sending "\r" to { exp4 } expect: does " INTO t\u001b[0m" (spawn_id exp4) match glob pattern "Ok."? no  expect: does " INTO t\u001b[0m\u001b[J" (spawn_id exp4) match glob pattern "Ok."? no  expect: does " INTO t\u001b[0m\u001b[J\u001b[34G" (spawn_id exp4) match glob pattern "Ok."? no  expect: does " INTO t\u001b[0m\u001b[J\u001b[34G\u001b[21G" (spawn_id exp4) match glob pattern "Ok."? no INSERT INTO t01565(c0, c1) VALUES ('1', 1) ;  expect: does " INTO t\u001b[0m\u001b[J\u001b[34G\u001b[21G\u001b[JINSERT INTO t01565\u001b[0;22;33m(\u001b[0mc0\u001b[0;1m,\u001b[0m c1\u001b[0;22;33m)\u001b[0m VALUES \u001b[0;22;33m(\u001b[0;22;36m'1'\u001b[0;1m,\u001b[0m \u001b[0;22;32m1\u001b[0;22;33m)\u001b[0m \u001b[0;1m;\u001b[0m\r\n\r\n\u001b[0m\u001b[1G" (spawn_id exp4) match glob pattern "Ok."? no expect: timed out Here you can see that it matched "INSERT" and then expect receive data from the client almost byte by byte. So I hope that expecting the last part of the query should fix the problem. [1]: https://s3.amazonaws.com/clickhouse-test-reports/50429/228ebab86db95dca1e29967061d245985bc86a0f/stateless_tests__release__s3_storage__[2_2].html Signed-off-by: Azat Khuzhin --- .../01565_query_loop_after_client_error.expect | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01565_query_loop_after_client_error.expect b/tests/queries/0_stateless/01565_query_loop_after_client_error.expect index 0faf8f0192b..e718fd99b7f 100755 --- a/tests/queries/0_stateless/01565_query_loop_after_client_error.expect +++ b/tests/queries/0_stateless/01565_query_loop_after_client_error.expect @@ -24,24 +24,24 @@ expect "\n:) " send -- "DROP TABLE IF EXISTS t01565;\n" # NOTE: this is important for -mn mode, you should send "\r" only after reading echoed command -expect "DROP" +expect "\r\n" send -- "\r" expect "\nOk." expect "\n:)" send -- "CREATE TABLE t01565 (c0 String, c1 Int32) ENGINE = Memory() ;\n" -expect "CREATE" +expect "\r\n" send -- "\r" expect "\nOk." expect "\n:) " send -- "INSERT INTO t01565(c0, c1) VALUES (\"1\",1) ;\n" -expect "INSERT" +expect "\r\n" send -- "\r" expect "\n:) " send -- "INSERT INTO t01565(c0, c1) VALUES ('1', 1) ;\n" -expect "INSERT" +expect "\r\n" send -- "\r" expect "\nOk." expect "\n:) " From db806bd394c7b7dfe42f225f3c1ad7b1be1f2ea9 Mon Sep 17 00:00:00 2001 From: auxten Date: Sun, 4 Jun 2023 17:44:29 +0800 Subject: [PATCH 284/308] Resize underlying vector only pos_offset == vector.size() --- src/IO/WriteBufferFromVector.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index 4b2a3581625..c793a34b406 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -86,7 +86,10 @@ private: size_t old_size = vector.size(); /// pos may not be equal to vector.data() + old_size, because WriteBuffer::next() can be used to flush data size_t pos_offset = pos - reinterpret_cast(vector.data()); - vector.resize(old_size * size_multiplier); + if (pos_offset == vector.size()) + { + vector.resize(old_size * size_multiplier); + } internal_buffer = Buffer(reinterpret_cast(vector.data() + pos_offset), reinterpret_cast(vector.data() + vector.size())); working_buffer = internal_buffer; } From 63eab2783e4e0c23fdbe3e5e233b4e3c0773f40a Mon Sep 17 00:00:00 2001 From: Sergey Kazmin <43613813+yerseg@users.noreply.github.com> Date: Fri, 2 Jun 2023 17:06:19 +0000 Subject: [PATCH 285/308] Make typeid_cast for pointers noexcept --- src/Common/typeid_cast.h | 38 ++++++++++++-------------------------- 1 file changed, 12 insertions(+), 26 deletions(-) diff --git a/src/Common/typeid_cast.h b/src/Common/typeid_cast.h index baee3aaf632..e134aac09e4 100644 --- a/src/Common/typeid_cast.h +++ b/src/Common/typeid_cast.h @@ -25,14 +25,14 @@ namespace DB */ template requires std::is_reference_v -To typeid_cast(From & from) +To typeid_cast(From & from) noexcept(false) { try { if ((typeid(From) == typeid(To)) || (typeid(from) == typeid(To))) return static_cast(from); } - catch (const std::exception & e) + catch (const std::bad_typeid & e) { throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR); } @@ -44,19 +44,12 @@ To typeid_cast(From & from) template requires std::is_pointer_v -To typeid_cast(From * from) +To typeid_cast(From * from) noexcept { - try - { - if ((typeid(From) == typeid(std::remove_pointer_t)) || (from && typeid(*from) == typeid(std::remove_pointer_t))) - return static_cast(from); - else - return nullptr; - } - catch (const std::exception & e) - { - throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR); - } + if ((typeid(From) == typeid(std::remove_pointer_t)) || (from && typeid(*from) == typeid(std::remove_pointer_t))) + return static_cast(from); + else + return nullptr; } namespace detail @@ -79,17 +72,10 @@ inline constexpr bool is_shared_ptr_v = is_shared_ptr::value; template requires detail::is_shared_ptr_v -To typeid_cast(const std::shared_ptr & from) +To typeid_cast(const std::shared_ptr & from) noexcept { - try - { - if ((typeid(From) == typeid(typename To::element_type)) || (from && typeid(*from) == typeid(typename To::element_type))) - return std::static_pointer_cast(from); - else - return nullptr; - } - catch (const std::exception & e) - { - throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR); - } + if ((typeid(From) == typeid(typename To::element_type)) || (from && typeid(*from) == typeid(typename To::element_type))) + return std::static_pointer_cast(from); + else + return nullptr; } From a3a12834671c95914103e59c24cf68fadc40f68f Mon Sep 17 00:00:00 2001 From: Sergey Kazmin <43613813+yerseg@users.noreply.github.com> Date: Sat, 3 Jun 2023 19:36:11 +0300 Subject: [PATCH 286/308] remove try-catch from the impl of typeid_cast for refs --- src/Common/typeid_cast.h | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/src/Common/typeid_cast.h b/src/Common/typeid_cast.h index e134aac09e4..f1ded97a9f1 100644 --- a/src/Common/typeid_cast.h +++ b/src/Common/typeid_cast.h @@ -27,15 +27,8 @@ template requires std::is_reference_v To typeid_cast(From & from) noexcept(false) { - try - { - if ((typeid(From) == typeid(To)) || (typeid(from) == typeid(To))) - return static_cast(from); - } - catch (const std::bad_typeid & e) - { - throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR); - } + if ((typeid(From) == typeid(To)) || (typeid(from) == typeid(To))) + return static_cast(from); throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Bad cast from type {} to {}", demangle(typeid(from).name()), demangle(typeid(To).name())); From 0f4dd26cebbcf9201124287e8d31acda92a9e9f7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 4 Jun 2023 16:03:44 +0200 Subject: [PATCH 287/308] Add async iteration to object storage --- src/Common/CurrentMetrics.cpp | 4 + .../AzureBlobStorage/AzureObjectStorage.cpp | 69 ++++++++++++++++++ .../AzureBlobStorage/AzureObjectStorage.h | 2 + src/Disks/ObjectStorages/IObjectStorage.cpp | 9 +++ src/Disks/ObjectStorages/IObjectStorage.h | 7 ++ .../ObjectStorages/ObjectStorageIterator.cpp | 20 +++++ .../ObjectStorages/ObjectStorageIterator.h | 53 ++++++++++++++ .../ObjectStorageIteratorAsync.cpp | 64 ++++++++++++++++ .../ObjectStorageIteratorAsync.h | 58 +++++++++++++++ .../ObjectStorages/S3/S3ObjectStorage.cpp | 73 +++++++++++++++++++ src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 2 + 11 files changed, 361 insertions(+) create mode 100644 src/Disks/ObjectStorages/ObjectStorageIterator.cpp create mode 100644 src/Disks/ObjectStorages/ObjectStorageIterator.h create mode 100644 src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp create mode 100644 src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 82d68ca8185..4c858ee788d 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -131,6 +131,10 @@ M(DistributedInsertThreadsActive, "Number of threads used for INSERT into Distributed running a task.") \ M(StorageS3Threads, "Number of threads in the StorageS3 thread pool.") \ M(StorageS3ThreadsActive, "Number of threads in the StorageS3 thread pool running a task.") \ + M(ObjectStorageS3Threads, "Number of threads in the S3ObjectStorage thread pool.") \ + M(ObjectStorageS3ThreadsActive, "Number of threads in the S3ObjectStorage thread pool running a task.") \ + M(ObjectStorageAzureThreads, "Number of threads in the AzureObjectStorage thread pool.") \ + M(ObjectStorageAzureThreadsActive, "Number of threads in the AzureObjectStorage thread pool running a task.") \ M(MergeTreePartsLoaderThreads, "Number of threads in the MergeTree parts loader thread pool.") \ M(MergeTreePartsLoaderThreadsActive, "Number of threads in the MergeTree parts loader thread pool running a task.") \ M(MergeTreePartsCleanerThreads, "Number of threads in the MergeTree parts cleaner thread pool.") \ diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 0358b4e915a..23a0da39dd3 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -11,9 +11,16 @@ #include #include +#include #include #include +namespace CurrentMetrics +{ + extern const Metric ObjectStorageAzureThreads; + extern const Metric ObjectStorageAzureThreadsActive; + +} namespace DB { @@ -26,6 +33,60 @@ namespace ErrorCodes } +namespace +{ + +class AzureIteratorAsync final : public IObjectStorageIteratorAsync +{ +public: + AzureIteratorAsync( + const std::string & path_prefix, + std::shared_ptr client_, + size_t max_list_size) + : IObjectStorageIteratorAsync( + CurrentMetrics::ObjectStorageAzureThreads, + CurrentMetrics::ObjectStorageAzureThreadsActive, + "ListObjectAzure") + , client(client_) + { + + options.Prefix = path_prefix; + options.PageSizeHint = static_cast(max_list_size); + } + +private: + bool getBatchAndCheckNext(RelativePathsWithMetadata & batch) override + { + auto outcome = client->ListBlobs(options); + auto blob_list_response = client->ListBlobs(options); + auto blobs_list = blob_list_response.Blobs; + + for (const auto & blob : blobs_list) + { + batch.emplace_back( + blob.Name, + ObjectMetadata{ + static_cast(blob.BlobSize), + Poco::Timestamp::fromEpochTime( + std::chrono::duration_cast( + blob.Details.LastModified.time_since_epoch()).count()), + {}}); + } + + options.ContinuationToken = blob_list_response.NextPageToken; + if (blob_list_response.HasPage()) + return true; + + return false; + } + + std::shared_ptr client; + Azure::Storage::Blobs::ListBlobsOptions options; +}; + +} + + AzureObjectStorage::AzureObjectStorage( const String & name_, AzureClientPtr && client_, @@ -67,6 +128,14 @@ bool AzureObjectStorage::exists(const StoredObject & object) const return false; } +ObjectStorageIteratorPtr AzureObjectStorage::iterate(const std::string & path_prefix) const +{ + auto settings_ptr = settings.get(); + auto client_ptr = client.get(); + + return std::make_shared(path_prefix, client_ptr, settings_ptr->list_object_keys_size); +} + void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { auto client_ptr = client.get(); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index a36a03bcda4..5b08ceb80e3 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -60,6 +60,8 @@ public: void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + ObjectStorageIteratorPtr iterate(const std::string & path_prefix) const override; + DataSourceDescription getDataSourceDescription() const override { return data_source_description; } std::string getName() const override { return "AzureObjectStorage"; } diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index a5903f9d429..ea22294224c 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -29,6 +30,14 @@ void IObjectStorage::listObjects(const std::string &, RelativePathsWithMetadata } +ObjectStorageIteratorPtr IObjectStorage::iterate(const std::string & path_prefix) const +{ + RelativePathsWithMetadata files; + listObjects(path_prefix, files, 0); + + return std::make_shared(std::move(files)); +} + std::optional IObjectStorage::tryGetObjectMetadata(const std::string & path) const { try diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 28de80a88cd..32f9d1ba764 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -20,6 +20,9 @@ #include #include #include +#include +#include +#include namespace DB @@ -51,6 +54,8 @@ struct RelativePathWithMetadata using RelativePathsWithMetadata = std::vector; +class IObjectStorageIterator; +using ObjectStorageIteratorPtr = std::shared_ptr; /// Base class for all object storages which implement some subset of ordinary filesystem operations. /// @@ -75,6 +80,8 @@ public: virtual void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const; + virtual ObjectStorageIteratorPtr iterate(const std::string & path_prefix) const; + /// Get object metadata if supported. It should be possible to receive /// at least size of object virtual std::optional tryGetObjectMetadata(const std::string & path) const; diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.cpp b/src/Disks/ObjectStorages/ObjectStorageIterator.cpp new file mode 100644 index 00000000000..188b743958c --- /dev/null +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.cpp @@ -0,0 +1,20 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +RelativePathWithMetadata ObjectStorageIteratorFromList::current() const +{ + if (!isValid()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); + + return *batch_iterator; +} + +} diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.h b/src/Disks/ObjectStorages/ObjectStorageIterator.h new file mode 100644 index 00000000000..c3afd395a74 --- /dev/null +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class IObjectStorageIterator +{ +public: + virtual void next() = 0; + virtual bool isValid() const = 0; + virtual RelativePathWithMetadata current() const = 0; + virtual size_t getAccumulatedSize() const = 0; + + virtual ~IObjectStorageIterator() = default; +}; + +using ObjectStorageIteratorPtr = std::shared_ptr; + +class ObjectStorageIteratorFromList : public IObjectStorageIterator +{ +public: + explicit ObjectStorageIteratorFromList(RelativePathsWithMetadata && batch_) + : batch(std::move(batch_)) + , batch_iterator(batch.begin()) + { + } + + void next() override + { + if (isValid()) + ++batch_iterator; + } + + bool isValid() const override + { + return batch_iterator != batch.end(); + } + + RelativePathWithMetadata current() const override; + + size_t getAccumulatedSize() const override + { + return batch.size(); + } +private: + RelativePathsWithMetadata batch; + RelativePathsWithMetadata::iterator batch_iterator; +}; + +} diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp new file mode 100644 index 00000000000..766071cf815 --- /dev/null +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -0,0 +1,64 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +void IObjectStorageIteratorAsync::next() +{ + std::lock_guard lock(mutex); + + if (current_batch_iterator != current_batch.end()) + { + ++current_batch_iterator; + } + else if (!is_finished) + { + if (outcome_future.valid()) + { + BatchAndHasNext next_batch = outcome_future.get(); + current_batch = std::move(next_batch.batch); + accumulated_size.fetch_add(current_batch.size(), std::memory_order_relaxed); + current_batch_iterator = current_batch.begin(); + if (next_batch.has_next) + outcome_future = scheduleBatch(); + else + is_finished = true; + } + } +} + +std::future IObjectStorageIteratorAsync::scheduleBatch() +{ + return list_objects_scheduler([this] + { + BatchAndHasNext result; + result.has_next = getBatchAndCheckNext(result.batch); + return result; + }, Priority{}); +} + + +bool IObjectStorageIteratorAsync::isValid() const +{ + return current_batch_iterator != current_batch.end(); +} + +RelativePathWithMetadata IObjectStorageIteratorAsync::current() const +{ + if (!isValid()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); + + return *current_batch_iterator; +} + +size_t IObjectStorageIteratorAsync::getAccumulatedSize() const +{ + return accumulated_size.load(std::memory_order_relaxed); +} + +} diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h new file mode 100644 index 00000000000..81ba9bce137 --- /dev/null +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h @@ -0,0 +1,58 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class IObjectStorageIteratorAsync : public IObjectStorageIterator +{ +public: + IObjectStorageIteratorAsync( + CurrentMetrics::Metric threads_metric, + CurrentMetrics::Metric threads_active_metric, + const std::string & thread_name) + : list_objects_pool(threads_metric, threads_active_metric, 1) + , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, thread_name)) + { + } + + void next() override; + bool isValid() const override; + RelativePathWithMetadata current() const override; + size_t getAccumulatedSize() const override; + + ~IObjectStorageIteratorAsync() override + { + list_objects_pool.wait(); + } + +protected: + + virtual bool getBatchAndCheckNext(RelativePathsWithMetadata & batch) = 0; + + struct BatchAndHasNext + { + RelativePathsWithMetadata batch; + bool has_next; + }; + + std::future scheduleBatch(); + + bool is_finished{false}; + + std::mutex mutex; + ThreadPool list_objects_pool; + ThreadPoolCallbackRunner list_objects_scheduler; + std::future outcome_future; + RelativePathsWithMetadata current_batch; + RelativePathsWithMetadata::iterator current_batch_iterator; + std::atomic accumulated_size = 0; +}; + + +} diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 6e63efcc1e3..d19be20f920 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -3,6 +3,7 @@ #if USE_AWS_S3 #include +#include #include #include @@ -33,6 +34,13 @@ namespace ProfileEvents extern const Event DiskS3ListObjects; } +namespace CurrentMetrics +{ + extern const Metric ObjectStorageS3Threads; + extern const Metric ObjectStorageS3ThreadsActive; +} + + namespace DB { @@ -84,6 +92,62 @@ void logIfError(const Aws::Utils::Outcome & response, std::functi } +namespace +{ + +class S3IteratorAsync final : public IObjectStorageIteratorAsync +{ +public: + S3IteratorAsync( + const std::string & bucket, + const std::string & path_prefix, + std::shared_ptr client_, + size_t max_list_size) + : IObjectStorageIteratorAsync( + CurrentMetrics::ObjectStorageS3Threads, + CurrentMetrics::ObjectStorageS3ThreadsActive, + "ListObjectS3") + , client(client_) + { + request.SetBucket(bucket); + request.SetPrefix(path_prefix); + request.SetMaxKeys(static_cast(max_list_size)); + } + +private: + bool getBatchAndCheckNext(RelativePathsWithMetadata & batch) override + { + ProfileEvents::increment(ProfileEvents::S3ListObjects); + + bool result = false; + auto outcome = client->ListObjectsV2(request); + /// Outcome failure will be handled on the caller side. + if (outcome.IsSuccess()) + { + auto objects = outcome.GetResult().GetContents(); + + result = !objects.empty(); + + for (const auto & object : objects) + batch.emplace_back(object.GetKey(), ObjectMetadata{static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), {}}); + + if (result) + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + + return result; + } + + throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", + quoteString(request.GetBucket()), quoteString(request.GetPrefix()), + backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); + } + + std::shared_ptr client; + S3::ListObjectsV2Request request; +}; + +} + bool S3ObjectStorage::exists(const StoredObject & object) const { auto settings_ptr = s3_settings.get(); @@ -183,6 +247,15 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN disk_write_settings); } + +ObjectStorageIteratorPtr S3ObjectStorage::iterate(const std::string & path_prefix) const +{ + auto settings_ptr = s3_settings.get(); + auto client_ptr = client.get(); + + return std::make_shared(bucket, path_prefix, client_ptr, settings_ptr->list_object_keys_size); +} + void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { auto settings_ptr = s3_settings.get(); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index b0eb01aec0d..072e1354d38 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -102,6 +102,8 @@ public: void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + ObjectStorageIteratorPtr iterate(const std::string & path_prefix) const override; + /// Uses `DeleteObjectRequest`. void removeObject(const StoredObject & object) override; From 4bb44c7c72417a6e7a5f2ec7e1651b4360f9956e Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Sun, 4 Jun 2023 23:06:21 +0800 Subject: [PATCH 288/308] 1. skip extract darwin toolchain in builder when uncessary 2. update MacOSX SDK version in toolchain readme to match in builder --- cmake/toolchain/darwin-x86_64/README.txt | 4 ++-- docker/packager/binary/build.sh | 8 +++++--- docker/packager/packager | 2 ++ 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/cmake/toolchain/darwin-x86_64/README.txt b/cmake/toolchain/darwin-x86_64/README.txt index 65c9aba5be6..90ada960bfa 100644 --- a/cmake/toolchain/darwin-x86_64/README.txt +++ b/cmake/toolchain/darwin-x86_64/README.txt @@ -1,2 +1,2 @@ -wget https://github.com/phracker/MacOSX-SDKs/releases/download/10.15/MacOSX10.15.sdk.tar.xz -tar xJf MacOSX10.15.sdk.tar.xz --strip-components=1 +wget https://github.com/phracker/MacOSX-SDKs/releases/download/11.3/MacOSX11.0.sdk.tar.xz +tar xJf MacOSX11.0.sdk.tar.xz --strip-components=1 diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 2cd0a011013..ee1011a9cd5 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -11,9 +11,11 @@ ccache_status () { [ -O /build ] || git config --global --add safe.directory /build -mkdir -p /build/cmake/toolchain/darwin-x86_64 -tar xJf /MacOSX11.0.sdk.tar.xz -C /build/cmake/toolchain/darwin-x86_64 --strip-components=1 -ln -sf darwin-x86_64 /build/cmake/toolchain/darwin-aarch64 +if [ "$EXTRACT_TOOLCHAIN_DARWIN" = "1" ];then + mkdir -p /build/cmake/toolchain/darwin-x86_64 + tar xJf /MacOSX11.0.sdk.tar.xz -C /build/cmake/toolchain/darwin-x86_64 --strip-components=1 + ln -sf darwin-x86_64 /build/cmake/toolchain/darwin-aarch64 +fi # Uncomment to debug ccache. Don't put ccache log in /output right away, or it # will be confusingly packed into the "performance" package. diff --git a/docker/packager/packager b/docker/packager/packager index a894fe2d8e9..1b3df858cd2 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -167,6 +167,7 @@ def parse_env_variables( cmake_flags.append( "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/darwin/toolchain-x86_64.cmake" ) + result.append("EXTRACT_TOOLCHAIN_DARWIN=1") elif is_cross_darwin_arm: cc = compiler[: -len(DARWIN_ARM_SUFFIX)] cmake_flags.append("-DCMAKE_AR:FILEPATH=/cctools/bin/aarch64-apple-darwin-ar") @@ -181,6 +182,7 @@ def parse_env_variables( cmake_flags.append( "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/darwin/toolchain-aarch64.cmake" ) + result.append("EXTRACT_TOOLCHAIN_DARWIN=1") elif is_cross_arm: cc = compiler[: -len(ARM_SUFFIX)] cmake_flags.append( From 6fe0aa531e28c27287fb5b6f57536ede0016b20a Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 16:18:53 -0300 Subject: [PATCH 289/308] Update summap.md --- .../aggregate-functions/reference/summap.md | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/summap.md b/docs/en/sql-reference/aggregate-functions/reference/summap.md index 1acfde3783a..d63e8b81716 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/summap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/summap.md @@ -5,7 +5,9 @@ sidebar_position: 141 # sumMap -Syntax: `sumMap(key, value)` or `sumMap(Tuple(key, value))` +Syntax: [sumMap(key , value )](../../data-types/array.md) or [sumMap(Tuple(key , value ))](../../data-types/tuple.md) + +Alias: `sumMappedArrays`. Totals the `value` array according to the keys specified in the `key` array. @@ -27,6 +29,7 @@ CREATE TABLE sum_map( ), statusMapTuple Tuple(Array(Int32), Array(Int32)) ) ENGINE = Log; + INSERT INTO sum_map VALUES ('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, 10, 10], ([1, 2, 3], [10, 10, 10])), ('2000-01-01', '2000-01-01 00:00:00', [3, 4, 5], [10, 10, 10], ([3, 4, 5], [10, 10, 10])), @@ -47,3 +50,7 @@ GROUP BY timeslot │ 2000-01-01 00:01:00 │ ([4,5,6,7,8],[10,10,20,10,10]) │ ([4,5,6,7,8],[10,10,20,10,10]) │ └─────────────────────┴──────────────────────────────────────────────┴────────────────────────────────┘ ``` + +**See Also** + +- [-Map combinator for Map datatype](../combinators.md#-map) From 424a043c326cb8451bfc0da23fc2d1df2385fd4e Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 16:22:26 -0300 Subject: [PATCH 290/308] Update map.md --- docs/en/sql-reference/data-types/map.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/sql-reference/data-types/map.md b/docs/en/sql-reference/data-types/map.md index 0ea183d73d8..e0c8b98f9f8 100644 --- a/docs/en/sql-reference/data-types/map.md +++ b/docs/en/sql-reference/data-types/map.md @@ -108,6 +108,7 @@ Result: - [map()](../../sql-reference/functions/tuple-map-functions.md#function-map) function - [CAST()](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) function +- [-Map combinator for Map datatype](../aggregate-functions/combinators.md#-map) ## Related content From 7c64b1f26634acc8e93dd65c00b8b0dfb6419f1d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 16:32:22 -0300 Subject: [PATCH 291/308] Update combinators.md --- .../aggregate-functions/combinators.md | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index e1db5d8d23e..fd693430064 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -30,7 +30,34 @@ Example 2: `uniqArray(arr)` – Counts the number of unique elements in all ‘a The -Map suffix can be appended to any aggregate function. This will create an aggregate function which gets Map type as an argument, and aggregates values of each key of the map separately using the specified aggregate function. The result is also of a Map type. -Examples: `sumMap(map(1,1))`, `avgMap(map('a', 1))`. +**Example** + +```sql +CREATE TABLE map_map( + date Date, + timeslot DateTime, + status Map(String, UInt64) +) ENGINE = Log; + +INSERT INTO map_map VALUES + ('2000-01-01', '2000-01-01 00:00:00', (['a', 'b', 'c'], [10, 10, 10])), + ('2000-01-01', '2000-01-01 00:00:00', (['c', 'd', 'e'], [10, 10, 10])), + ('2000-01-01', '2000-01-01 00:01:00', (['d', 'e', 'f'], [10, 10, 10])), + ('2000-01-01', '2000-01-01 00:01:00', (['f', 'g', 'g'], [10, 10, 10])); + +SELECT + timeslot, + sumMap(status), + avgMap(status), + minMap(status) +FROM map_map +GROUP BY timeslot; + +┌────────────timeslot─┬─sumMap(status)───────────────────────┬─avgMap(status)───────────────────────┬─minMap(status)───────────────────────┐ +│ 2000-01-01 00:00:00 │ {'a':10,'b':10,'c':20,'d':10,'e':10} │ {'a':10,'b':10,'c':10,'d':10,'e':10} │ {'a':10,'b':10,'c':10,'d':10,'e':10} │ +│ 2000-01-01 00:01:00 │ {'d':10,'e':10,'f':20,'g':20} │ {'d':10,'e':10,'f':10,'g':10} │ {'d':10,'e':10,'f':10,'g':10} │ +└─────────────────────┴──────────────────────────────────────┴──────────────────────────────────────┴──────────────────────────────────────┘ +``` ## -SimpleState From d81884be82bf32440cbaae3f63349716898aa1e3 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:13:25 -0300 Subject: [PATCH 292/308] Update summap.md --- docs/en/sql-reference/aggregate-functions/reference/summap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/summap.md b/docs/en/sql-reference/aggregate-functions/reference/summap.md index d63e8b81716..0725b80c2a2 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/summap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/summap.md @@ -5,7 +5,7 @@ sidebar_position: 141 # sumMap -Syntax: [sumMap(key , value )](../../data-types/array.md) or [sumMap(Tuple(key , value ))](../../data-types/tuple.md) +Syntax: `sumMap(key , value )` [Array](../../data-types/array.md) or `sumMap(Tuple(key , value ))` [Tuple](../../data-types/tuple.md) Alias: `sumMappedArrays`. From d93ee14e328ce0c8026a1d94a4233b9905fed793 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:15:16 -0300 Subject: [PATCH 293/308] Update summap.md --- docs/en/sql-reference/aggregate-functions/reference/summap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/summap.md b/docs/en/sql-reference/aggregate-functions/reference/summap.md index 0725b80c2a2..aa12fad9559 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/summap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/summap.md @@ -5,7 +5,7 @@ sidebar_position: 141 # sumMap -Syntax: `sumMap(key , value )` [Array](../../data-types/array.md) or `sumMap(Tuple(key , value ))` [Tuple](../../data-types/tuple.md) +Syntax: `sumMap(key <[Array](../../data-types/array.md)>, value <[Array](../../data-types/array.md)>)` or `sumMap([Tuple](../../data-types/tuple.md)(key , value ))` Alias: `sumMappedArrays`. From 7ca719c73a41f54c3b2032297e8d0b3f9f6dd7a2 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:19:07 -0300 Subject: [PATCH 294/308] Update summap.md --- docs/en/sql-reference/aggregate-functions/reference/summap.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/summap.md b/docs/en/sql-reference/aggregate-functions/reference/summap.md index aa12fad9559..baa25edc250 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/summap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/summap.md @@ -5,7 +5,9 @@ sidebar_position: 141 # sumMap -Syntax: `sumMap(key <[Array](../../data-types/array.md)>, value <[Array](../../data-types/array.md)>)` or `sumMap([Tuple](../../data-types/tuple.md)(key , value ))` +Syntax: `sumMap(key , value )` [Array type](../../data-types/array.md) or `sumMap(Tuple(key , value ))` [Tuple type](../../data-types/tuple.md). + +Arguments: Alias: `sumMappedArrays`. From 0d9728410fd7631ff0e0755c749024de5b7ffdb1 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:31:12 -0300 Subject: [PATCH 295/308] Update first_value.md --- .../reference/first_value.md | 35 ++++++++++++++++--- 1 file changed, 30 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/first_value.md b/docs/en/sql-reference/aggregate-functions/reference/first_value.md index f343ca3f66c..15e0b113afd 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/first_value.md +++ b/docs/en/sql-reference/aggregate-functions/reference/first_value.md @@ -6,24 +6,32 @@ sidebar_position: 7 # first_value Selects the first encountered value, similar to `any`, but could accept NULL. +Mostly it should be used with [Window Functions](../../window-functions.md). +Without Window Functions the result will be random if the source stream is not ordered. ## examples ```sql -insert into test_data (a,b) values (1,null), (2,3), (4, 5), (6,null) +CREATE TABLE test_data +( + a Int64, + b Nullable(Int64) +) +ENGINE = Memory; + +INSERT INTO test_data (a, b) FORMAT Values (1,null), (2,3), (4, 5), (6,null); ``` ### example1 The NULL value is ignored at default. ```sql -select first_value(b) from test_data +select first_value(b) from test_data; ``` ```text ┌─first_value_ignore_nulls(b)─┐ │ 3 │ └─────────────────────────────┘ - ``` ### example2 @@ -36,7 +44,6 @@ select first_value(b) ignore nulls from test_data ┌─first_value_ignore_nulls(b)─┐ │ 3 │ └─────────────────────────────┘ - ``` ### example3 @@ -46,10 +53,28 @@ select first_value(b) respect nulls from test_data ``` ```text - ┌─first_value_respect_nulls(b)─┐ │ ᴺᵁᴸᴸ │ └──────────────────────────────┘ ``` +### example4 +Stabilized result using the sub-query with `ORDER BY`. +```sql +SELECT + first_value_respect_nulls(b), + first_value(b) +FROM +( + SELECT * + FROM test_data + ORDER BY a ASC +) +``` + +```text +┌─first_value_respect_nulls(b)─┬─first_value(b)─┐ +│ ᴺᵁᴸᴸ │ 3 │ +└──────────────────────────────┴────────────────┘ +``` From 2e26e84909098c97eab553eb7fcfa98b5a92bfae Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:34:31 -0300 Subject: [PATCH 296/308] Update last_value.md --- .../reference/last_value.md | 32 +++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/last_value.md b/docs/en/sql-reference/aggregate-functions/reference/last_value.md index 7b6e14e4a55..77b4f3d1b60 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/last_value.md +++ b/docs/en/sql-reference/aggregate-functions/reference/last_value.md @@ -6,12 +6,20 @@ sidebar_position: 8 # last_value Selects the last encountered value, similar to `anyLast`, but could accept NULL. - +Mostly it should be used with [Window Functions](../../window-functions.md). +Without Window Functions the result will be random if the source stream is not ordered. ## examples ```sql -insert into test_data (a,b) values (1,null), (2,3), (4, 5), (6,null) +CREATE TABLE test_data +( + a Int64, + b Nullable(Int64) +) +ENGINE = Memory; + +INSERT INTO test_data (a, b) Values (1,null), (2,3), (4, 5), (6,null) ``` ### example1 @@ -50,4 +58,24 @@ select last_value(b) respect nulls from test_data └─────────────────────────────┘ ``` +### example4 +Stabilized result using the sub-query with `ORDER BY`. +```sql +SELECT + last_value_respect_nulls(b), + last_value(b) +FROM +( + SELECT * + FROM test_data + ORDER BY a ASC +) +``` + +```text +┌─last_value_respect_nulls(b)─┬─last_value(b)─┐ +│ ᴺᵁᴸᴸ │ 5 │ +└─────────────────────────────┴───────────────┘ +``` + From bd7a593dd3ae4447f23c8658a04fb79d164b9d84 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:34:46 -0300 Subject: [PATCH 297/308] Update first_value.md --- .../sql-reference/aggregate-functions/reference/first_value.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/first_value.md b/docs/en/sql-reference/aggregate-functions/reference/first_value.md index 15e0b113afd..6b764ec5739 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/first_value.md +++ b/docs/en/sql-reference/aggregate-functions/reference/first_value.md @@ -19,7 +19,7 @@ CREATE TABLE test_data ) ENGINE = Memory; -INSERT INTO test_data (a, b) FORMAT Values (1,null), (2,3), (4, 5), (6,null); +INSERT INTO test_data (a, b) Values (1,null), (2,3), (4, 5), (6,null); ``` ### example1 From fb21a6907a3eef7a054b44f9c81c5bc3a05f5cb7 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:38:05 -0300 Subject: [PATCH 298/308] Update last_value.md --- .../sql-reference/aggregate-functions/reference/last_value.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/last_value.md b/docs/en/sql-reference/aggregate-functions/reference/last_value.md index 77b4f3d1b60..21a86a5f130 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/last_value.md +++ b/docs/en/sql-reference/aggregate-functions/reference/last_value.md @@ -6,7 +6,7 @@ sidebar_position: 8 # last_value Selects the last encountered value, similar to `anyLast`, but could accept NULL. -Mostly it should be used with [Window Functions](../../window-functions.md). +Mostly it should be used with [Window Functions](../../window-functions/index.md). Without Window Functions the result will be random if the source stream is not ordered. ## examples From bcd89cbbf28c79e831f3e39a422319e5b6ea5915 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:38:38 -0300 Subject: [PATCH 299/308] Update first_value.md --- .../sql-reference/aggregate-functions/reference/first_value.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/first_value.md b/docs/en/sql-reference/aggregate-functions/reference/first_value.md index 6b764ec5739..c1965b23fe3 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/first_value.md +++ b/docs/en/sql-reference/aggregate-functions/reference/first_value.md @@ -6,7 +6,7 @@ sidebar_position: 7 # first_value Selects the first encountered value, similar to `any`, but could accept NULL. -Mostly it should be used with [Window Functions](../../window-functions.md). +Mostly it should be used with [Window Functions](../../window-functions/index.md). Without Window Functions the result will be random if the source stream is not ordered. ## examples From bf127f4e1e3a08de7ae822d0b53d25ad80899efa Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Jun 2023 21:31:43 +0200 Subject: [PATCH 300/308] MSan support for Rust Previously you have to unpoison memory from the Rust, however Rust does supports MSan, so let's simply use it. But for this we need nightly Rust and recompile standard library. Signed-off-by: Azat Khuzhin --- docker/packager/binary/Dockerfile | 4 +++- rust/.cargo/config.toml.in | 7 +++++++ rust/BLAKE3/include/blake3.h | 2 -- rust/BLAKE3/src/lib.rs | 25 ------------------------- rust/CMakeLists.txt | 10 ++++++++++ src/Functions/FunctionsHashing.h | 7 +------ 6 files changed, 21 insertions(+), 34 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index fa860b2207f..dd21c8552d3 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -46,10 +46,12 @@ ENV CXX=clang++-${LLVM_VERSION} # Rust toolchain and libraries ENV RUSTUP_HOME=/rust/rustup ENV CARGO_HOME=/rust/cargo -ENV PATH="/rust/cargo/env:${PATH}" ENV PATH="/rust/cargo/bin:${PATH}" RUN curl https://sh.rustup.rs -sSf | bash -s -- -y && \ chmod 777 -R /rust && \ + rustup toolchain install nightly && \ + rustup default nightly && \ + rustup component add rust-src && \ rustup target add aarch64-unknown-linux-gnu && \ rustup target add x86_64-apple-darwin && \ rustup target add x86_64-unknown-freebsd && \ diff --git a/rust/.cargo/config.toml.in b/rust/.cargo/config.toml.in index a1dd966117b..db83145d449 100644 --- a/rust/.cargo/config.toml.in +++ b/rust/.cargo/config.toml.in @@ -1,3 +1,10 @@ [env] CFLAGS = "@RUST_CFLAGS@" CXXFLAGS = "@RUST_CXXFLAGS@" + +[build] +rustflags = @RUSTFLAGS@ +rustdocflags = @RUSTFLAGS@ + +[unstable] +@RUST_CARGO_BUILD_STD@ diff --git a/rust/BLAKE3/include/blake3.h b/rust/BLAKE3/include/blake3.h index 85572506d43..5dc7d5bd902 100644 --- a/rust/BLAKE3/include/blake3.h +++ b/rust/BLAKE3/include/blake3.h @@ -8,8 +8,6 @@ extern "C" { char *blake3_apply_shim(const char *begin, uint32_t _size, uint8_t *out_char_data); -char *blake3_apply_shim_msan_compat(const char *begin, uint32_t size, uint8_t *out_char_data); - void blake3_free_char_pointer(char *ptr_to_free); } // extern "C" diff --git a/rust/BLAKE3/src/lib.rs b/rust/BLAKE3/src/lib.rs index 2b54787589f..011145d2f71 100644 --- a/rust/BLAKE3/src/lib.rs +++ b/rust/BLAKE3/src/lib.rs @@ -3,7 +3,6 @@ extern crate libc; use std::ffi::{CStr, CString}; use std::os::raw::c_char; -use std::mem; #[no_mangle] pub unsafe extern "C" fn blake3_apply_shim( @@ -24,30 +23,6 @@ pub unsafe extern "C" fn blake3_apply_shim( std::ptr::null_mut() } -#[no_mangle] -pub unsafe extern "C" fn blake3_apply_shim_msan_compat( - mut begin: *const c_char, - size: u32, - out_char_data: *mut u8, -) -> *mut c_char { - if begin.is_null() { - let err_str = CString::new("input was a null pointer").unwrap(); - return err_str.into_raw(); - } - libc::memset(out_char_data as *mut libc::c_void, 0, mem::size_of::()); - let mut hasher = blake3::Hasher::new(); - let mut vec = Vec::::new(); - for _ in 0..size { - vec.push(*begin as u8); - begin = begin.add(1); - } - let input_res = vec.as_mut_slice(); - hasher.update(input_res); - let mut reader = hasher.finalize_xof(); - reader.fill(std::slice::from_raw_parts_mut(out_char_data, blake3::OUT_LEN)); - std::ptr::null_mut() -} - // Freeing memory according to docs: https://doc.rust-lang.org/std/ffi/struct.CString.html#method.into_raw #[no_mangle] pub unsafe extern "C" fn blake3_free_char_pointer(ptr_to_free: *mut c_char) { diff --git a/rust/CMakeLists.txt b/rust/CMakeLists.txt index d229894791a..6700ead9786 100644 --- a/rust/CMakeLists.txt +++ b/rust/CMakeLists.txt @@ -14,8 +14,18 @@ macro(configure_rustc) set(RUST_CFLAGS "${RUST_CFLAGS} --sysroot ${CMAKE_SYSROOT}") endif() + set(RUSTFLAGS "[]") + set(RUST_CARGO_BUILD_STD "") + # For more info: https://doc.rust-lang.org/beta/unstable-book/compiler-flags/sanitizer.html#memorysanitizer + if (SANITIZE STREQUAL "memory") + set(RUST_CARGO_BUILD_STD "build-std = [\"std\", \"panic_abort\", \"core\", \"alloc\"]") + set(RUSTFLAGS "[\"-Zsanitizer=memory\", \"-Zsanitizer-memory-track-origins\"]") + endif() + message(STATUS "RUST_CFLAGS: ${RUST_CFLAGS}") message(STATUS "RUST_CXXFLAGS: ${RUST_CXXFLAGS}") + message(STATUS "RUSTFLAGS: ${RUSTFLAGS}") + message(STATUS "RUST_CARGO_BUILD_STD: ${RUST_CARGO_BUILD_STD}") # NOTE: requires RW access for the source dir configure_file("${CMAKE_CURRENT_SOURCE_DIR}/.cargo/config.toml.in" "${CMAKE_CURRENT_SOURCE_DIR}/.cargo/config.toml" @ONLY) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 32e3fbbd4ea..a4d4fbd085d 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -816,12 +816,7 @@ struct ImplBLAKE3 #else static void apply(const char * begin, const size_t size, unsigned char* out_char_data) { -# if defined(MEMORY_SANITIZER) - auto err_msg = blake3_apply_shim_msan_compat(begin, safe_cast(size), out_char_data); - __msan_unpoison(out_char_data, length); -# else - auto err_msg = blake3_apply_shim(begin, safe_cast(size), out_char_data); -# endif + auto err_msg = blake3_apply_shim(begin, safe_cast(size), out_char_data); if (err_msg != nullptr) { auto err_st = std::string(err_msg); From 045573e92565a6a58a7eae80cc11bb686807e9ee Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 18:10:34 -0300 Subject: [PATCH 301/308] Update datetime64.md --- .../en/sql-reference/data-types/datetime64.md | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/data-types/datetime64.md b/docs/en/sql-reference/data-types/datetime64.md index 2d4035831fa..da3d9dc4f65 100644 --- a/docs/en/sql-reference/data-types/datetime64.md +++ b/docs/en/sql-reference/data-types/datetime64.md @@ -63,7 +63,7 @@ SELECT * FROM dt WHERE timestamp = toDateTime64('2019-01-01 00:00:00', 3, 'Asia/ ``` text ┌───────────────timestamp─┬─event_id─┐ -│ 2019-01-01 00:00:00.000 │ 2 │ +│ 2019-01-01 00:00:00.000 │ 3 │ └─────────────────────────┴──────────┘ ``` @@ -75,8 +75,8 @@ SELECT * FROM dt WHERE timestamp = toDateTime64(1546300800.123, 3); ``` text ┌───────────────timestamp─┬─event_id─┐ -│ 2019-01-01 00:00:00.123 │ 1 │ -│ 2019-01-01 00:00:00.123 │ 2 │ +│ 2019-01-01 03:00:00.123 │ 1 │ +│ 2019-01-01 03:00:00.123 │ 2 │ └─────────────────────────┴──────────┘ ``` @@ -91,7 +91,7 @@ SELECT toDateTime64(now(), 3, 'Asia/Istanbul') AS column, toTypeName(column) AS ``` text ┌──────────────────column─┬─x──────────────────────────────┐ -│ 2019-10-16 04:12:04.000 │ DateTime64(3, 'Asia/Istanbul') │ +│ 2023-06-05 00:09:52.000 │ DateTime64(3, 'Asia/Istanbul') │ └─────────────────────────┴────────────────────────────────┘ ``` @@ -100,13 +100,14 @@ SELECT toDateTime64(now(), 3, 'Asia/Istanbul') AS column, toTypeName(column) AS ``` sql SELECT toDateTime64(timestamp, 3, 'Europe/London') as lon_time, -toDateTime64(timestamp, 3, 'Asia/Istanbul') as mos_time +toDateTime64(timestamp, 3, 'Asia/Istanbul') as istanbul_time FROM dt; ``` ``` text -┌───────────────lon_time──┬────────────────mos_time─┐ -│ 2019-01-01 00:00:00.000 │ 2019-01-01 03:00:00.000 │ +┌────────────────lon_time─┬───────────istanbul_time─┐ +│ 2019-01-01 00:00:00.123 │ 2019-01-01 03:00:00.123 │ +│ 2019-01-01 00:00:00.123 │ 2019-01-01 03:00:00.123 │ │ 2018-12-31 21:00:00.000 │ 2019-01-01 00:00:00.000 │ └─────────────────────────┴─────────────────────────┘ ``` @@ -115,10 +116,9 @@ FROM dt; - [Type conversion functions](../../sql-reference/functions/type-conversion-functions.md) - [Functions for working with dates and times](../../sql-reference/functions/date-time-functions.md) -- [Functions for working with arrays](../../sql-reference/functions/array-functions.md) -- [The `date_time_input_format` setting](../../operations/settings/settings.md#settings-date_time_input_format) -- [The `date_time_output_format` setting](../../operations/settings/settings.md#settings-date_time_output_format) +- [The `date_time_input_format` setting](../../operations/settings/formats.md#date_time_input_format) +- [The `date_time_output_format` setting](../../operations/settings/formats.md#date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) -- [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-datetime) +- [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-for-working-with-dates-and-times) - [`Date` data type](../../sql-reference/data-types/date.md) - [`DateTime` data type](../../sql-reference/data-types/datetime.md) From 136efd68257ccaac503b2e47957803e283165afc Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 18:58:24 -0300 Subject: [PATCH 302/308] Update datetime64.md --- docs/en/sql-reference/data-types/datetime64.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/datetime64.md b/docs/en/sql-reference/data-types/datetime64.md index da3d9dc4f65..793691850b1 100644 --- a/docs/en/sql-reference/data-types/datetime64.md +++ b/docs/en/sql-reference/data-types/datetime64.md @@ -116,8 +116,8 @@ FROM dt; - [Type conversion functions](../../sql-reference/functions/type-conversion-functions.md) - [Functions for working with dates and times](../../sql-reference/functions/date-time-functions.md) -- [The `date_time_input_format` setting](../../operations/settings/formats.md#date_time_input_format) -- [The `date_time_output_format` setting](../../operations/settings/formats.md#date_time_output_format) +- [The `date_time_input_format` setting](../../operations/settings/settings-formats.md#date_time_input_format) +- [The `date_time_output_format` setting](../../operations/settings/settings-formats.md#date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) - [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-for-working-with-dates-and-times) - [`Date` data type](../../sql-reference/data-types/date.md) From e5c95add52ed86c56249fe85d8f7c02132736ae3 Mon Sep 17 00:00:00 2001 From: auxten Date: Mon, 5 Jun 2023 08:43:55 +0800 Subject: [PATCH 303/308] use old_size Co-authored-by: Alexey Milovidov --- src/IO/WriteBufferFromVector.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index c793a34b406..a2ecc34f1ab 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -86,7 +86,7 @@ private: size_t old_size = vector.size(); /// pos may not be equal to vector.data() + old_size, because WriteBuffer::next() can be used to flush data size_t pos_offset = pos - reinterpret_cast(vector.data()); - if (pos_offset == vector.size()) + if (pos_offset == old_size) { vector.resize(old_size * size_multiplier); } From 4234c4f36addd2607ecc16131ec67ef1089d10ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Jun 2023 02:51:11 +0200 Subject: [PATCH 304/308] Remove flaky test --- tests/integration/test_merge_tree_s3/test.py | 25 -------------------- 1 file changed, 25 deletions(-) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 7730bfcf7b2..2ccd517923a 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -739,31 +739,6 @@ def test_cache_with_full_disk_space(cluster, node_name): check_no_objects_after_drop(cluster, node_name=node_name) -@pytest.mark.parametrize("node_name", ["node"]) -def test_store_cleanup_disk_s3(cluster, node_name): - node = cluster.instances[node_name] - node.query("DROP TABLE IF EXISTS s3_test SYNC") - node.query( - "CREATE TABLE s3_test UUID '00000000-1000-4000-8000-000000000001' (n UInt64) Engine=MergeTree() ORDER BY n SETTINGS storage_policy='s3';" - ) - node.query("INSERT INTO s3_test SELECT 1") - - node.stop_clickhouse(kill=True) - path_to_data = "/var/lib/clickhouse/" - node.exec_in_container(["rm", f"{path_to_data}/metadata/default/s3_test.sql"]) - node.start_clickhouse() - - node.wait_for_log_line( - "Removing unused directory", timeout=90, look_behind_lines=1000 - ) - node.wait_for_log_line("directories from store") - node.query( - "CREATE TABLE s3_test UUID '00000000-1000-4000-8000-000000000001' (n UInt64) Engine=MergeTree() ORDER BY n SETTINGS storage_policy='s3';" - ) - node.query("INSERT INTO s3_test SELECT 1") - check_no_objects_after_drop(cluster) - - @pytest.mark.parametrize("node_name", ["node"]) def test_cache_setting_compatibility(cluster, node_name): node = cluster.instances[node_name] From 47379ac03965f4834bf6aaa00ce777dec731a3c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Jun 2023 03:58:42 +0300 Subject: [PATCH 305/308] Update build.sh --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index ee1011a9cd5..c0803c74147 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -11,7 +11,7 @@ ccache_status () { [ -O /build ] || git config --global --add safe.directory /build -if [ "$EXTRACT_TOOLCHAIN_DARWIN" = "1" ];then +if [ "$EXTRACT_TOOLCHAIN_DARWIN" = "1" ]; then mkdir -p /build/cmake/toolchain/darwin-x86_64 tar xJf /MacOSX11.0.sdk.tar.xz -C /build/cmake/toolchain/darwin-x86_64 --strip-components=1 ln -sf darwin-x86_64 /build/cmake/toolchain/darwin-aarch64 From 5fc8838b04d37d26207fff488bd60127f9eedaa8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Jun 2023 04:58:29 +0300 Subject: [PATCH 306/308] Update KeyCondition.cpp --- src/Storages/MergeTree/KeyCondition.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 923e5237420..16bd555092e 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -943,6 +943,13 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & return {field.columns, field.row_idx, result_idx}; } +/** When table's key has expression with these functions from a column, + * and when a column in a query is compared with a constant, such as: + * CREATE TABLE (x String) ORDER BY toDate(x) + * SELECT ... WHERE x LIKE 'Hello%' + * we want to apply the function to the constant for index analysis, + * but should modify it to pass on unparseable values. + */ static std::set date_time_parsing_functions = { "toDate", "toDate32", From 3c5bd78856d1848cd457a30c2d8320b3f65a41d9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Jun 2023 06:13:39 +0200 Subject: [PATCH 307/308] Fix typo --- src/Storages/MergeTree/KeyCondition.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index ab5820be90a..02ef7e6bebd 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -948,7 +948,7 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & * CREATE TABLE (x String) ORDER BY toDate(x) * SELECT ... WHERE x LIKE 'Hello%' * we want to apply the function to the constant for index analysis, - * but should modify it to pass on unparseable values. + * but should modify it to pass on unparsable values. */ static std::set date_time_parsing_functions = { "toDate", From f1058d2d9d2201f21882b487499ea4f4212fec0b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Jun 2023 09:51:16 +0300 Subject: [PATCH 308/308] Revert "Disable skim (Rust library) under memory sanitizer" --- rust/skim/CMakeLists.txt | 5 ----- 1 file changed, 5 deletions(-) diff --git a/rust/skim/CMakeLists.txt b/rust/skim/CMakeLists.txt index c2e406ec12f..1e7a43aba7c 100644 --- a/rust/skim/CMakeLists.txt +++ b/rust/skim/CMakeLists.txt @@ -14,11 +14,6 @@ if (OS_FREEBSD) return() endif() -if (SANITIZE STREQUAL "memory") - message(STATUS "skim is disabled under memory sanitizer, because the interop is not instrumented properly") - return() -endif() - clickhouse_import_crate(MANIFEST_PATH Cargo.toml) # -Wno-dollar-in-identifier-extension: cxx bridge complies names with '$'