From 067bfb884406d05722edec2d4fef82c2b65844ec Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 17:47:14 +0000 Subject: [PATCH 01/75] 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 02/75] 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 03/75] 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 04/75] 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 05/75] 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 06/75] 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 07/75] 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 c93202cca4cec2e83c51cb6b3cb56dc820965caa Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 9 Mar 2023 21:23:57 -0500 Subject: [PATCH 08/75] 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 09/75] 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 10/75] 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 11/75] 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 12/75] 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 13/75] 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 14/75] 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 15/75] 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 16/75] 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 17/75] 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 18/75] 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 19/75] 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 20/75] 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 892e436046f3f7bb135c5df8b18a1951833dd29f Mon Sep 17 00:00:00 2001 From: pufit Date: Sun, 2 Apr 2023 16:51:10 -0400 Subject: [PATCH 21/75] 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 8bef8fc1de5acf9910f83b978c8b91768da7f670 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Apr 2023 00:54:28 -0400 Subject: [PATCH 22/75] 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 23/75] 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 2426c445b0f17a0c98be86463efda8bd552d18de Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Apr 2023 20:10:32 -0400 Subject: [PATCH 24/75] 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 25/75] 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 26/75] 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 27/75] 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 28/75] 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 29/75] 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 30/75] 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 31/75] 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 32/75] 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 33/75] 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 34/75] 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 35/75] 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 36/75] 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 37/75] 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 38/75] 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 39/75] 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 602b9a740ec2bf3064d2970c54fbc92da9304991 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 May 2023 19:39:33 +0000 Subject: [PATCH 40/75] 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 41/75] 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 42/75] 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 43/75] 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 44/75] 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 45/75] 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 46/75] 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 47/75] 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 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 48/75] 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 194ce2d881aa6c3598f24e93cce29671ec9f67c3 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 17 May 2023 13:13:57 +0000 Subject: [PATCH 49/75] 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 50/75] 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 51/75] 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 c93836b9620f2bd424d5f6132404a455c94a39dd Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 19 May 2023 22:26:53 +0000 Subject: [PATCH 52/75] 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 646eeb63a4cc720b05ff9de48364be32a6936d94 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 May 2023 19:46:05 +0000 Subject: [PATCH 53/75] 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 c0bc75eacd0624e38d2a1581e19906778ea8b676 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 May 2023 09:50:34 +0000 Subject: [PATCH 54/75] 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 58610f11ab90a017a1275e1a9a0b843d17e948d8 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 May 2023 10:54:02 +0000 Subject: [PATCH 55/75] 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 0580859e6fa70102d3cde058040c4722d51170fc Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 May 2023 14:05:44 +0000 Subject: [PATCH 56/75] 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 be62c8faba774c8c594a6d0368efba0c7a540576 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 May 2023 15:27:39 +0200 Subject: [PATCH 57/75] 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 65586c50f500dd5daa51de60087c54acfdf5f914 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Wed, 31 May 2023 00:21:13 -0700 Subject: [PATCH 58/75] 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 a96ee7411b0fdd28d9a77d127f74848b889a73f6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 31 May 2023 14:28:36 +0000 Subject: [PATCH 59/75] Bump From 495580918a285ff46ddb3fb91f3b66885b6e2138 Mon Sep 17 00:00:00 2001 From: alekar Date: Wed, 31 May 2023 10:00:19 -0700 Subject: [PATCH 60/75] 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 61/75] 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 6334b6d5698f81429f5ae1351172ae89f5d7d2ea Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Wed, 31 May 2023 17:54:30 -0700 Subject: [PATCH 62/75] 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 08d98329b07ff772812999059a45af03352be030 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 10:12:09 +0200 Subject: [PATCH 63/75] 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 02e986a9e7c1c33bf8818411de538c58af8a5198 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 1 Jun 2023 19:06:11 +0200 Subject: [PATCH 64/75] 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 16be379fa55c6dc8172004460799be93b7a52b88 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Jun 2023 11:18:46 +0200 Subject: [PATCH 65/75] 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 66/75] 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 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 67/75] 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 163b2f32da72a66d44967439446331b9943361f3 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Jun 2023 18:13:46 +0200 Subject: [PATCH 68/75] 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 69/75] 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 70/75] 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 0d98a46326ca671cb0ff0540972c6eba5280d565 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Jun 2023 13:02:18 -0400 Subject: [PATCH 71/75] 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 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 72/75] 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 73/75] 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 74/75] 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 ccba3500dd92eaecd38a56d92b09336af26f371c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Jun 2023 07:13:26 +0300 Subject: [PATCH 75/75] 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);