From 10335aec136c541416979fee3b59c78a3221202c Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 14 Oct 2022 15:09:35 +0000 Subject: [PATCH 01/82] Improve using structure from insertion table in table functions --- src/Core/Settings.h | 2 +- src/Interpreters/Context.cpp | 73 +++++++++++++++++-- src/Interpreters/Context.h | 11 ++- src/Interpreters/JoinedTables.cpp | 11 +-- src/Interpreters/JoinedTables.h | 3 +- src/Interpreters/executeQuery.cpp | 3 +- src/TableFunctions/ITableFunction.h | 6 +- src/TableFunctions/ITableFunctionFileLike.cpp | 5 ++ src/TableFunctions/ITableFunctionFileLike.h | 2 + src/TableFunctions/TableFunctionS3.cpp | 5 ++ src/TableFunctions/TableFunctionS3.h | 2 + ...e_structure_from_insertion_table.reference | 9 +++ ...458_use_structure_from_insertion_table.sql | 39 ++++++++++ 13 files changed, 152 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference create mode 100644 tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5dedc6117aa..e87a2b99c91 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -609,7 +609,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(Bool, load_marks_asynchronously, false, "Load MergeTree marks asynchronously", 0) \ \ - M(Bool, use_structure_from_insertion_table_in_table_functions, false, "Use structure from insertion table instead of schema inference from data", 0) \ + M(UInt64, use_structure_from_insertion_table_in_table_functions, 2, "Use structure from insertion table instead of schema inference from data. Possible values: 0 - disabled, 1 - enabled, 2 - auto", 0) \ \ M(UInt64, http_max_tries, 10, "Max attempts to read via http.", 0) \ M(UInt64, http_retry_initial_backoff_ms, 100, "Min milliseconds for backoff, when retrying read via http", 0) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 321a46baff6..793135477d0 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -75,6 +75,8 @@ #include #include #include +#include +#include #include #include #include @@ -1184,7 +1186,7 @@ void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String } -StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) +StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const ASTSelectQuery * select_query_hint) { auto hash = table_expression->getTreeHash(); String key = toString(hash.first) + '_' + toString(hash.second); @@ -1194,15 +1196,61 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) if (!res) { TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression, shared_from_this()); - if (getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint()) + if (getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint() && hasInsertionTable()) { - const auto & insertion_table = getInsertionTable(); - if (!insertion_table.empty()) + const auto & structure_hint = DatabaseCatalog::instance().getTable(getInsertionTable(), shared_from_this())->getInMemoryMetadataPtr()->getColumns(); + bool use_columns_from_insert_query = true; + + /// use_structure_from_insertion_table_in_table_functions=2 means `auto` + if (select_query_hint && getSettingsRef().use_structure_from_insertion_table_in_table_functions == 2) { - const auto & structure_hint - = DatabaseCatalog::instance().getTable(insertion_table, shared_from_this())->getInMemoryMetadataPtr()->columns; - table_function_ptr->setStructureHint(structure_hint); + const auto * expression_list = select_query_hint->select()->as(); + Names columns_names; + bool have_asteriks = false; + /// First, check if we have only identifiers, asteriks and literals in select expression, + /// and if no, we cannot use the structure from insertion table. + for (const auto & expression : expression_list->children) + { + if (auto * identifier = expression->as()) + { + columns_names.push_back(identifier->name()); + } + else if (expression->as()) + { + have_asteriks = true; + } + else if (!expression->as()) + { + use_columns_from_insert_query = false; + break; + } + } + + /// Check that all identifiers are column names from insertion table. + for (const auto & column_name : columns_names) + { + if (!structure_hint.has(column_name)) + { + use_columns_from_insert_query = false; + break; + } + } + + /// If we don't have asteriks but only subset of columns, we should use + /// structure from insertion table only in case when table function + /// supports reading subset of columns from data. + if (use_columns_from_insert_query && !have_asteriks && !columns_names.empty()) + { + /// For input function we should check if input format supports reading subset of columns. + if (table_function_ptr->getName() == "input") + use_columns_from_insert_query = FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(getInsertFormat()); + else + use_columns_from_insert_query = table_function_ptr->supportsReadingSubsetOfColumns(); + } } + + if (use_columns_from_insert_query) + table_function_ptr->setStructureHint(structure_hint); } res = table_function_ptr->execute(table_expression, shared_from_this(), table_function_ptr->getName()); @@ -1432,12 +1480,21 @@ String Context::getDefaultFormat() const return default_format.empty() ? "TabSeparated" : default_format; } - void Context::setDefaultFormat(const String & name) { default_format = name; } +String Context::getInsertFormat() const +{ + return insert_format; +} + +void Context::setInsertFormat(const String & name) +{ + insert_format = name; +} + MultiVersion::Version Context::getMacros() const { return shared->macros.get(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index a9984e32c1b..0cc4b17968f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -232,6 +233,9 @@ private: String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification. /// Thus, used in HTTP interface. If not specified - then some globally default format is used. + + String insert_format; /// Format, used in insert query. + TemporaryTablesMapping external_tables_mapping; Scalars scalars; /// Used to store constant values which are different on each instance during distributed plan, such as _shard_num. @@ -592,7 +596,9 @@ public: const QueryFactoriesInfo & getQueryFactoriesInfo() const { return query_factories_info; } void addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const; - StoragePtr executeTableFunction(const ASTPtr & table_expression); + /// For table functions s3/file/url/hdfs/input we can use structure from + /// insertion table depending on select expression. + StoragePtr executeTableFunction(const ASTPtr & table_expression, const ASTSelectQuery * select_query_hint = nullptr); void addViewSource(const StoragePtr & storage); StoragePtr getViewSource() const; @@ -621,6 +627,9 @@ public: String getDefaultFormat() const; /// If default_format is not specified, some global default format is returned. void setDefaultFormat(const String & name); + String getInsertFormat() const; + void setInsertFormat(const String & name); + MultiVersion::Version getMacros() const; void setMacros(std::unique_ptr && macros); diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index f4a98ada199..b88bb5d1caf 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -173,12 +173,13 @@ using RenameQualifiedIdentifiersVisitor = InDepthNodeVisitorgetQueryContext()->executeTableFunction(left_table_expression); + return context->getQueryContext()->executeTableFunction(left_table_expression, &select_query); StorageID table_id = StorageID::createEmpty(); if (left_db_and_table) diff --git a/src/Interpreters/JoinedTables.h b/src/Interpreters/JoinedTables.h index 9d01c081e9f..7562dbc9ac5 100644 --- a/src/Interpreters/JoinedTables.h +++ b/src/Interpreters/JoinedTables.h @@ -22,7 +22,7 @@ using StorageMetadataPtr = std::shared_ptr; class JoinedTables { public: - JoinedTables(ContextPtr context, const ASTSelectQuery & select_query, bool include_all_columns_ = false); + JoinedTables(ContextPtr context, const ASTSelectQuery & select_query_, bool include_all_columns_ = false); void reset(const ASTSelectQuery & select_query); @@ -52,6 +52,7 @@ private: /// Legacy (duplicated left table values) ASTPtr left_table_expression; std::optional left_db_and_table; + const ASTSelectQuery & select_query; }; } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 7a633242904..253de6ea3ac 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -450,6 +450,7 @@ static std::tuple executeQueryImpl( if (insert_query) { + context->setInsertFormat(insert_query->format); if (insert_query->data) query_end = insert_query->data; else @@ -579,7 +580,7 @@ static std::tuple executeQueryImpl( insert_query->tryFindInputFunction(input_function); if (input_function) { - StoragePtr storage = context->executeTableFunction(input_function); + StoragePtr storage = context->executeTableFunction(input_function, insert_query->select->as()); auto & input_storage = dynamic_cast(*storage); auto input_metadata_snapshot = input_storage.getInMemoryMetadataPtr(); auto pipe = getSourceFromASTInsertQuery( diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index 4b9a87b93f1..d586946da04 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -54,15 +54,17 @@ public: virtual ColumnsDescription getActualTableStructure(ContextPtr /*context*/) const = 0; /// Check if table function needs a structure hint from SELECT query in case of - /// INSERT INTO FUNCTION ... SELECT ... + /// INSERT INTO FUNCTION ... SELECT ... and INSERT INTO ... SELECT ... FROM table_function(...) /// It's used for schema inference. virtual bool needStructureHint() const { return false; } /// Set a structure hint from SELECT query in case of - /// INSERT INTO FUNCTION ... SELECT ... + /// INSERT INTO FUNCTION ... SELECT ... and INSERT INTO ... SELECT ... FROM table_function(...) /// This hint could be used not to repeat schema in function arguments. virtual void setStructureHint(const ColumnsDescription &) {} + virtual bool supportsReadingSubsetOfColumns() { return true; } + /// Create storage according to the query. StoragePtr execute(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns_ = {}, bool use_global_context = false) const; diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 8be2341b81d..d62e44a16cc 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -34,6 +34,11 @@ String ITableFunctionFileLike::getFormatFromFirstArgument() return FormatFactory::instance().getFormatFromFileName(filename, true); } +bool ITableFunctionFileLike::supportsReadingSubsetOfColumns() +{ + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format); +} + void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, ContextPtr context) { /// Parse args diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index c2f32eb0aa3..589fce67638 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -18,6 +18,8 @@ public: void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } + bool supportsReadingSubsetOfColumns() override; + protected: void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; virtual void parseFirstArguments(const ASTPtr & arg, const ContextPtr & context); diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 0bf33007760..44b6f087d6b 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -146,6 +146,11 @@ ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) return parseColumnsListFromString(configuration.structure, context); } +bool TableFunctionS3::supportsReadingSubsetOfColumns() +{ + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); +} + StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { Poco::URI uri (configuration.url); diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index be84bc4d8ab..05d3ebea316 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -30,6 +30,8 @@ public: void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } + bool supportsReadingSubsetOfColumns() override; + protected: friend class TableFunctionS3Cluster; diff --git a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference new file mode 100644 index 00000000000..0ca28640270 --- /dev/null +++ b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference @@ -0,0 +1,9 @@ +\N 1 +1 2 +\N 42 +\N 42 +\N 42 +\N 42 +\N +\N +\N diff --git a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql new file mode 100644 index 00000000000..a609dc361fe --- /dev/null +++ b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql @@ -0,0 +1,39 @@ +-- Tags: no-parallel, no-fasttest + +insert into function file(02458_data.jsonl) select NULL as x, 42 as y settings engine_file_truncate_on_insert=1; +insert into function file(02458_data.jsoncompacteachrow) select NULL as x, 42 as y settings engine_file_truncate_on_insert=1; +drop table if exists test; +create table test (x Nullable(UInt32), y UInt32) engine=Memory(); + +set use_structure_from_insertion_table_in_table_functions=2; +insert into test select * from file(02458_data.jsonl); +insert into test select x, 1 from file(02458_data.jsonl); +insert into test select x, y from file(02458_data.jsonl); +insert into test select x + 1, y from file(02458_data.jsonl); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select x, z from file(02458_data.jsonl); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} + +insert into test select * from file(02458_data.jsoncompacteachrow); +insert into test select x, 1 from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select x, y from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select x + 1, y from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select x, z from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} + +insert into test select * from input() format CSV 1,2 +insert into test select x, y from input() format CSV 1,2 -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} +insert into test select x, y from input() format JSONEachRow {"x" : null, "y" : 42} + +select * from test order by y; + +drop table test; +create table test (x Nullable(UInt32)) engine=Memory(); +insert into test select * from file(02458_data.jsonl); +insert into test select x from file(02458_data.jsonl); +insert into test select y from file(02458_data.jsonl); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select y as x from file(02458_data.jsonl); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} + +insert into test select c1 from input() format CSV 1,2; -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} +insert into test select x from input() format JSONEachRow {"x" : null, "y" : 42} + +select * from test order by x; + +drop table test; From 2b150a575e263d7c91dbbc25f489452d901383c4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 14 Oct 2022 15:15:03 +0000 Subject: [PATCH 02/82] Add docs --- docs/en/operations/settings/settings.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 31609fe24be..82fb825df45 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3403,6 +3403,17 @@ Use schema from cache for URL with last modification time validation (for urls w Default value: `true`. +## use_structure_from_insertion_table_in_table_functions {use_structure_from_insertion_table_in_table_functions} + +Use structure from insertion table instead of schema inference from data. + +Possible values: +- 0 - disabled +- 1 - enabled +- 2 - auto + +Default value: 2. + ## compatibility {#compatibility} This setting changes other settings according to provided ClickHouse version. From 11a9b5977a30ec409ef2113ae0e1ac636b34a9f0 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 14 Oct 2022 19:22:05 +0200 Subject: [PATCH 03/82] Fix typos --- src/Interpreters/Context.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6ca6e987436..b5002b1df19 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1246,8 +1246,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const { const auto * expression_list = select_query_hint->select()->as(); Names columns_names; - bool have_asteriks = false; - /// First, check if we have only identifiers, asteriks and literals in select expression, + bool have_asterisk = false; + /// First, check if we have only identifiers, asterisk and literals in select expression, /// and if no, we cannot use the structure from insertion table. for (const auto & expression : expression_list->children) { @@ -1257,7 +1257,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const } else if (expression->as()) { - have_asteriks = true; + have_asterisk = true; } else if (!expression->as()) { @@ -1276,10 +1276,10 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const } } - /// If we don't have asteriks but only subset of columns, we should use + /// If we don't have asterisk but only subset of columns, we should use /// structure from insertion table only in case when table function /// supports reading subset of columns from data. - if (use_columns_from_insert_query && !have_asteriks && !columns_names.empty()) + if (use_columns_from_insert_query && !have_asterisk && !columns_names.empty()) { /// For input function we should check if input format supports reading subset of columns. if (table_function_ptr->getName() == "input") From 798e6281b4e7ceac3e5b5ca0de1c150c364be039 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Oct 2022 13:19:07 +0000 Subject: [PATCH 04/82] Fix test --- tests/queries/0_stateless/01825_type_json_schema_inference.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01825_type_json_schema_inference.sh b/tests/queries/0_stateless/01825_type_json_schema_inference.sh index 36991bd8069..f6917ab18ad 100755 --- a/tests/queries/0_stateless/01825_type_json_schema_inference.sh +++ b/tests/queries/0_stateless/01825_type_json_schema_inference.sh @@ -46,7 +46,7 @@ ${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_inference (obj JSON, map Map(String echo '{"map": {"k1": 1, "k2": 2}, "obj": {"k1": 1, "k2": 2}}' > $filename -${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_inference SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json', 'JSONEachRow')" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_inference SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json', 'JSONEachRow')" --allow_experimental_object_type 1 --use_structure_from_insertion_table_in_table_functions 1 ${CLICKHOUSE_CLIENT} -q "SELECT * FROM t_json_inference FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 ${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(obj) FROM t_json_inference LIMIT 1" From 93d48a24032e5e0f4036a0d96a5293e65f898e0b Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Mon, 24 Oct 2022 11:04:45 +0000 Subject: [PATCH 05/82] add functions for powerBI connection --- src/Functions/UTCTimestamp.cpp | 117 +++++++++++++++++ src/Functions/timeDiff.cpp | 228 +++++++++++++++++++++++++++++++++ 2 files changed, 345 insertions(+) create mode 100644 src/Functions/UTCTimestamp.cpp create mode 100644 src/Functions/timeDiff.cpp diff --git a/src/Functions/UTCTimestamp.cpp b/src/Functions/UTCTimestamp.cpp new file mode 100644 index 00000000000..9b114754061 --- /dev/null +++ b/src/Functions/UTCTimestamp.cpp @@ -0,0 +1,117 @@ +#include + +#include +#include +#include +#include + +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +namespace +{ + +/// Get the UTC time. (It is a constant, it is evaluated once for the entire query.) +class ExecutableFunctionUTCTimestamp : public IExecutableFunction +{ +public: + explicit ExecutableFunctionUTCTimestamp(time_t time_) : time_value(time_) {} + + String getName() const override { return "UTCTimestamp"; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + return DataTypeDateTime().createColumnConst( + input_rows_count, + static_cast(time_value)); + } + +private: + time_t time_value; +}; + +class FunctionBaseUTCTimestamp : public IFunctionBase +{ +public: + explicit FunctionBaseUTCTimestamp(time_t time_, DataTypes argument_types_, DataTypePtr return_type_) + : time_value(time_), argument_types(std::move(argument_types_)), return_type(std::move(return_type_)) {} + + String getName() const override { return "UTCTimestamp"; } + + const DataTypes & getArgumentTypes() const override + { + return argument_types; + } + + const DataTypePtr & getResultType() const override + { + return return_type; + } + + ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override + { + return std::make_unique(time_value); + } + + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + +private: + time_t time_value; + DataTypes argument_types; + DataTypePtr return_type; +}; + +class UTCTimestampOverloadResolver : public IFunctionOverloadResolver +{ +public: + static constexpr auto name = "UTCTimestamp"; + + String getName() const override { return name; } + + bool isDeterministic() const override { return false; } + + bool isVariadic() const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique(); } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() > 0) + { + throw Exception("Arguments size of function " + getName() + " should be 0", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + return std::make_shared(); + } + + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override + { + if (arguments.size() > 0) + { + throw Exception("Arguments size of function " + getName() + " should be 0", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + return std::make_unique(time(nullptr), DataTypes(), std::make_shared("UTC")); + } +}; + +} + +/// UTC_timestamp for MySQL interface support +REGISTER_FUNCTION(UTCTimestamp) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerAlias("UTC_timestamp", UTCTimestampOverloadResolver::name, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/timeDiff.cpp b/src/Functions/timeDiff.cpp new file mode 100644 index 00000000000..9081351998f --- /dev/null +++ b/src/Functions/timeDiff.cpp @@ -0,0 +1,228 @@ +#include +#include +#include +#include +#include +#include + +#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; +} + +namespace +{ + +/** TimeDiff(t1, t2) + * t1 and t2 can be Date or DateTime + */ +class FunctionTimeDiff : public IFunction +{ + using ColumnDateTime64 = ColumnDecimal; +public: + static constexpr auto name = "TimeDiff"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() != 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!isDate(arguments[0]) && !isDateTime(arguments[0]) && !isDateTime64(arguments[1])) + throw Exception("First argument for function " + getName() + " must be Date or DateTime", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isDate(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[2])) + throw Exception("Second argument for function " + getName() + " must be Date or DateTime", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const IColumn & x = *arguments[0].column; + const IColumn & y = *arguments[1].column; + + size_t rows = input_rows_count; + auto res = ColumnInt64::create(rows); + + dispatchForColumns(x, y, DateLUT::instance(), DateLUT::instance(), res->getData()); + + return res; + } + +private: + template + void dispatchForColumns( + const IColumn & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * x_vec_16 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_32 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_64 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); + else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); + else + throw Exception("Illegal column for first argument of function " + getName() + ", must be Date, DateTime or DateTime64", ErrorCodes::ILLEGAL_COLUMN); + } + + template + void dispatchForSecondColumn( + const LeftColumnType & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); + else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); + else + throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); + } + + template + void dispatchConstForSecondColumn( + T1 x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_64, timezone_x, timezone_y, result); + else + throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); + } + + template + void vectorVector( + const LeftColumnType & x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & x_data = x.getData(); + const auto & y_data = y.getData(); + + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); + } + + template + void vectorConstant( + const LeftColumnType & x, T2 y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & x_data = x.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto y_value = stripDecimalFieldValue(y); + + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); + } + + template + void constantVector( + T1 x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & y_data = y.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto x_value = stripDecimalFieldValue(x); + + for (size_t i = 0, size = y.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); + } + + template + Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const + { + return static_cast(transform_y.execute(y, timezone_y)) + - static_cast(transform_x.execute(x, timezone_x)); + } + + template + static UInt32 getScale(const T & v) + { + if constexpr (std::is_same_v) + return v.getScale(); + else if constexpr (std::is_same_v>) + return v.getScale(); + + return 0; + } + template + static auto stripDecimalFieldValue(T && v) + { + if constexpr (std::is_same_v, DecimalField>) + return v.getValue(); + else + return v; + } +}; + +} + +REGISTER_FUNCTION(TimeDiff) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} From e0c87b2ae2c6f2eef84a9b4cdf642b77a1fd47ea Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Mon, 24 Oct 2022 12:53:10 +0000 Subject: [PATCH 06/82] small fix --- src/Functions/UTCTimestamp.cpp | 2 +- src/Functions/timeDiff.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/UTCTimestamp.cpp b/src/Functions/UTCTimestamp.cpp index 9b114754061..218c7fa5c1c 100644 --- a/src/Functions/UTCTimestamp.cpp +++ b/src/Functions/UTCTimestamp.cpp @@ -79,7 +79,7 @@ public: bool isDeterministic() const override { return false; } - bool isVariadic() const override { return true; } + bool isVariadic() const override { return false; } size_t getNumberOfArguments() const override { return 0; } static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique(); } diff --git a/src/Functions/timeDiff.cpp b/src/Functions/timeDiff.cpp index 9081351998f..0353dbf7c51 100644 --- a/src/Functions/timeDiff.cpp +++ b/src/Functions/timeDiff.cpp @@ -47,9 +47,9 @@ public: return name; } - bool isVariadic() const override { return true; } + bool isVariadic() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - size_t getNumberOfArguments() const override { return 0; } + size_t getNumberOfArguments() const override { return 2; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { From 1d405788090f20e30747939666a51934e87a4e12 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 18:35:18 +0200 Subject: [PATCH 07/82] Remove cruft from Docker --- docker/server/Dockerfile.ubuntu | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index c6254b898ed..3a04728864f 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -80,6 +80,16 @@ RUN arch=${TARGETARCH:-amd64} \ && mkdir -p /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client \ && chmod ugo+Xrw -R /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client +# Remove as much of Ubuntu as possible. +# ClickHouse does not need Ubuntu. It can run on top of Linux kernel without any OS distribution. +# ClickHouse does not need Docker at all. ClickHouse is above all that. +# It does not care about Ubuntu, Docker, or other cruft and you should neither. +# The fact that this Docker image is based on Ubuntu is just a misconception. +# Some vulnerability scanners are arguing about Ubuntu, which is not relevant to ClickHouse at all. +# ClickHouse does not care when you report false vulnerabilities by running some Docker scanners. + +RUN sudo apt remove -y libksba8 + # we need to allow "others" access to clickhouse folder, because docker container # can be started with arbitrary uid (openshift usecase) From d0a7f855ef19e3baf7de96d04ec99497308d212c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Oct 2022 05:32:11 +0300 Subject: [PATCH 08/82] Update docker/server/Dockerfile.ubuntu Co-authored-by: Nikita Mikhaylov --- docker/server/Dockerfile.ubuntu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 3a04728864f..9deec86b746 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -88,7 +88,7 @@ RUN arch=${TARGETARCH:-amd64} \ # Some vulnerability scanners are arguing about Ubuntu, which is not relevant to ClickHouse at all. # ClickHouse does not care when you report false vulnerabilities by running some Docker scanners. -RUN sudo apt remove -y libksba8 +RUN apt-get remove --purge -y libksba8 && apt-get autoremove # we need to allow "others" access to clickhouse folder, because docker container # can be started with arbitrary uid (openshift usecase) From 8db31beee61e4f28fd5ab4e824751a2afa2cfe6b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 30 Jul 2022 09:19:58 +0300 Subject: [PATCH 09/82] Remove covered parts for fetched part Here is an example that I found on production, simplified. Consider the following queue (nothing of this had been processed on this replica): - GET_PART all_0_0_0 (queue-0000000001) - GET_PART all_1_1_0 (queue-0000000002) ... - GET_PART all_0_1_1 (queue-0000000003) - GET_PART all_2_2_0 (queue-0000000004) ... - MERGE_PARTS from [all_0_1_1, all_2_2_0] to all_0_2_2 (queue-0000000005) And now queue-0000000005 started to executing (either because of reording, or because at that time GET_PART fails), and it does not have any required parts, so it will fetch them, but not all_0_0_0 and all_1_1_0, so this replica delay will set to the time of min(queue-0000000001, queue-0000000002), while it is not true, since it already have parts that covers those parts. and since MERGE_PARTS takes 30min, it increased the replica delay eventually to 30min, for the time range of 30min, which is pretty huge. Signed-off-by: Azat Khuzhin --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 12 +++++- .../MergeTree/ReplicatedMergeTreeQueue.h | 6 ++- src/Storages/StorageReplicatedMergeTree.cpp | 40 ++++++++++++++++--- src/Storages/StorageReplicatedMergeTree.h | 5 ++- 4 files changed, 51 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index d6d937ce66f..e60403149b5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1015,7 +1015,8 @@ bool ReplicatedMergeTreeQueue::checkReplaceRangeCanBeRemoved(const MergeTreePart void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, - const std::optional & covering_entry) + const std::optional & covering_entry, + const String & fetch_entry_znode) { /// TODO is it possible to simplify it? Queue to_wait; @@ -1029,10 +1030,17 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( [[maybe_unused]] bool called_from_alter_query_directly = covering_entry && covering_entry->replace_range_entry && covering_entry->replace_range_entry->columns_version < 0; [[maybe_unused]] bool called_for_broken_part = !covering_entry; - assert(currently_executing_drop_replace_ranges.contains(part_info) || called_from_alter_query_directly || called_for_broken_part); + assert(currently_executing_drop_replace_ranges.contains(part_info) || called_from_alter_query_directly || called_for_broken_part || !fetch_entry_znode.empty()); for (Queue::iterator it = queue.begin(); it != queue.end();) { + /// Skipping currently processing entry + if (!fetch_entry_znode.empty() && (*it)->znode_name == fetch_entry_znode) + { + ++it; + continue; + } + auto type = (*it)->type; bool is_simple_producing_op = type == LogEntry::GET_PART || type == LogEntry::ATTACH_PART || diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 2ebdd604af2..32421f91b04 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -336,8 +336,10 @@ public: * And also wait for the completion of their execution, if they are now being executed. * covering_entry is as an entry that caused removal of entries in range (usually, DROP_RANGE) */ - void removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, - const std::optional & covering_entry); + void removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, + const MergeTreePartInfo & part_info, + const std::optional & covering_entry, + const String & fetch_entry_znode); /** In the case where there are not enough parts to perform the merge in part_name * - move actions with merged parts to the end of the queue diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ef56ade2d58..0e4e73f0e3c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1731,8 +1731,18 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che if (!entry.actual_new_part_name.empty()) LOG_DEBUG(log, "Will fetch part {} instead of {}", entry.actual_new_part_name, entry.new_part_name); - if (!fetchPart(part_name, metadata_snapshot, fs::path(zookeeper_path) / "replicas" / replica, false, entry.quorum)) + String source_replica_path = fs::path(zookeeper_path) / "replicas" / replica; + if (!fetchPart(part_name, + metadata_snapshot, + source_replica_path, + /* to_detached= */ false, + entry.quorum, + /* zookeeper_ */ nullptr, + /* try_fetch_shared= */ true, + entry.znode_name)) + { return false; + } } catch (Exception & e) { @@ -1815,7 +1825,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) LOG_TRACE(log, "Executing DROP_RANGE {}", entry.new_part_name); auto drop_range_info = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); getContext()->getMergeList().cancelInPartition(getStorageID(), drop_range_info.partition_id, drop_range_info.max_block); - queue.removePartProducingOpsInRange(getZooKeeper(), drop_range_info, entry); + queue.removePartProducingOpsInRange(getZooKeeper(), drop_range_info, entry, /* fetch_entry_znode= */ {}); part_check_thread.cancelRemovedPartsCheck(drop_range_info); /// Delete the parts contained in the range to be deleted. @@ -1884,7 +1894,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (replace) { getContext()->getMergeList().cancelInPartition(getStorageID(), drop_range.partition_id, drop_range.max_block); - queue.removePartProducingOpsInRange(getZooKeeper(), drop_range, entry); + queue.removePartProducingOpsInRange(getZooKeeper(), drop_range, entry, /* fetch_entry_znode= */ {}); part_check_thread.cancelRemovedPartsCheck(drop_range); } else @@ -3431,7 +3441,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n /// so GET_PART all_1_42_5 (and all source parts) is useless. The only thing we can do is to fetch all_1_42_5_63. /// 2. If all_1_42_5_63 is lost, then replication may stuck waiting for all_1_42_5_63 to appear, /// because we may have some covered parts (more precisely, parts with the same min and max blocks) - queue.removePartProducingOpsInRange(zookeeper, broken_part_info, {}); + queue.removePartProducingOpsInRange(zookeeper, broken_part_info, /* covering_entry= */ {}, /* fetch_entry_znode= */ {}); String part_path = fs::path(replica_path) / "parts" / part_name; @@ -3831,8 +3841,15 @@ bool StorageReplicatedMergeTree::partIsLastQuorumPart(const MergeTreePartInfo & } -bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const StorageMetadataPtr & metadata_snapshot, - const String & source_replica_path, bool to_detached, size_t quorum, zkutil::ZooKeeper::Ptr zookeeper_, bool try_fetch_shared) +bool StorageReplicatedMergeTree::fetchPart( + const String & part_name, + const StorageMetadataPtr & metadata_snapshot, + const String & source_replica_path, + bool to_detached, + size_t quorum, + zkutil::ZooKeeper::Ptr zookeeper_, + bool try_fetch_shared, + String entry_znode) { auto zookeeper = zookeeper_ ? zookeeper_ : getZooKeeper(); const auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); @@ -4031,6 +4048,17 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora ProfileEvents::increment(ProfileEvents::ObsoleteReplicatedParts); } + /// It is possible that fetched parts may cover other parts (see + /// findReplicaHavingCoveringPart()), and if those covered parts + /// cannot be executed right now (due to MERGE_PARTS that covers + /// them is in progress), replica delay will be increased until + /// those entries will be executed (if covered operations + /// finishes) in other words until MERGE_PARTS is in progress, + /// while this can take awhile. + /// + /// So let's just remove them from the queue. + queue.removePartProducingOpsInRange(zookeeper, part->info, /* covering_entry= */ {}, entry_znode); + write_part_log({}); } else diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 436defebe1d..b15350d077b 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -672,11 +672,12 @@ private: bool fetchPart( const String & part_name, const StorageMetadataPtr & metadata_snapshot, - const String & replica_path, + const String & source_replica_path, bool to_detached, size_t quorum, zkutil::ZooKeeper::Ptr zookeeper_ = nullptr, - bool try_fetch_shared = true); + bool try_fetch_shared = true, + String entry_znode = ""); /** Download the specified part from the specified replica. * Used for replace local part on the same s3-shared part in hybrid storage. From 6cbf50a8295cbf7e21aabc461ebf2322d167ad75 Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Fri, 28 Oct 2022 21:06:32 +0000 Subject: [PATCH 10/82] extract common part --- src/Functions/dateDiff.cpp | 335 ++++++++++++++++++++++--------------- src/Functions/timeDiff.cpp | 228 ------------------------- 2 files changed, 204 insertions(+), 359 deletions(-) delete mode 100644 src/Functions/timeDiff.cpp diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index b8bf3c11698..5673fd23173 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -33,6 +33,147 @@ namespace ErrorCodes namespace { +class DateDiffImpl +{ +using ColumnDateTime64 = ColumnDecimal; +public: +explicit DateDiffImpl(const String& name_) : name(name_) {} + +template +void dispatchForColumns( + const IColumn & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const +{ + if (const auto * x_vec_16 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_32 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_64 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); + else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); + else + throw Exception("Illegal column for first argument of function " + name + ", must be Date, DateTime or DateTime64", ErrorCodes::ILLEGAL_COLUMN); +} + +template +void dispatchForSecondColumn( + const LeftColumnType & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const +{ + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); + else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); + else + throw Exception("Illegal column for second argument of function " + name + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); +} + +template +void dispatchConstForSecondColumn( + T1 x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const +{ + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_64, timezone_x, timezone_y, result); + else + throw Exception("Illegal column for second argument of function " + name + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); +} + +template +void vectorVector( + const LeftColumnType & x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const +{ + const auto & x_data = x.getData(); + const auto & y_data = y.getData(); + + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); +} + +template +void vectorConstant( + const LeftColumnType & x, T2 y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const +{ + const auto & x_data = x.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto y_value = stripDecimalFieldValue(y); + + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); +} + +template +void constantVector( + T1 x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const +{ + const auto & y_data = y.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto x_value = stripDecimalFieldValue(x); + + for (size_t i = 0, size = y.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); +} + +template +Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const +{ + return static_cast(transform_y.execute(y, timezone_y)) + - static_cast(transform_x.execute(x, timezone_x)); +} + +template +static UInt32 getScale(const T & v) +{ + if constexpr (std::is_same_v) + return v.getScale(); + else if constexpr (std::is_same_v>) + return v.getScale(); + + return 0; +} +template +static auto stripDecimalFieldValue(T && v) +{ + if constexpr (std::is_same_v, DecimalField>) + return v.getValue(); + else + return v; +} +private: + String name; +}; + + /** dateDiff('unit', t1, t2, [timezone]) * t1 and t2 can be Date or DateTime * @@ -105,158 +246,85 @@ public: const auto & timezone_y = extractTimeZoneFromFunctionArguments(arguments, 3, 2); if (unit == "year" || unit == "yy" || unit == "yyyy") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "quarter" || unit == "qq" || unit == "q") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "month" || unit == "mm" || unit == "m") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "week" || unit == "wk" || unit == "ww") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "day" || unit == "dd" || unit == "d") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "hour" || unit == "hh" || unit == "h") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "minute" || unit == "mi" || unit == "n") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "second" || unit == "ss" || unit == "s") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else throw Exception("Function " + getName() + " does not support '" + unit + "' unit", ErrorCodes::BAD_ARGUMENTS); return res; } - private: - template - void dispatchForColumns( - const IColumn & x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const + DateDiffImpl impl{name}; +}; + + +/** TimeDiff(t1, t2) + * t1 and t2 can be Date or DateTime + */ +class FunctionTimeDiff : public IFunction +{ + using ColumnDateTime64 = ColumnDecimal; +public: + static constexpr auto name = "TimeDiff"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { - if (const auto * x_vec_16 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_32 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_64 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); - else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); - else - throw Exception("Illegal column for first argument of function " + getName() + ", must be Date, DateTime or DateTime64", ErrorCodes::ILLEGAL_COLUMN); + return name; } - template - void dispatchForSecondColumn( - const LeftColumnType & x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const + bool isVariadic() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (const auto * y_vec_16 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_64 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); - else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); - else - throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); + if (arguments.size() != 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!isDate(arguments[0]) && !isDateTime(arguments[0]) && !isDateTime64(arguments[1])) + throw Exception("First argument for function " + getName() + " must be Date or DateTime", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isDate(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[2])) + throw Exception("Second argument for function " + getName() + " must be Date or DateTime", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); } - template - void dispatchConstForSecondColumn( - T1 x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - if (const auto * y_vec_16 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_64 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_64, timezone_x, timezone_y, result); - else - throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); - } - - template - void vectorVector( - const LeftColumnType & x, const RightColumnType & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - const auto & x_data = x.getData(); - const auto & y_data = y.getData(); - - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - for (size_t i = 0, size = x.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); - } - - template - void vectorConstant( - const LeftColumnType & x, T2 y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - const auto & x_data = x.getData(); - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - const auto y_value = stripDecimalFieldValue(y); - - for (size_t i = 0, size = x.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); - } - - template - void constantVector( - T1 x, const RightColumnType & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - const auto & y_data = y.getData(); - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - const auto x_value = stripDecimalFieldValue(x); - - for (size_t i = 0, size = y.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); - } - - template - Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const - { - return static_cast(transform_y.execute(y, timezone_y)) - - static_cast(transform_x.execute(x, timezone_x)); - } - - template - static UInt32 getScale(const T & v) - { - if constexpr (std::is_same_v) - return v.getScale(); - else if constexpr (std::is_same_v>) - return v.getScale(); - - return 0; - } - template - static auto stripDecimalFieldValue(T && v) - { - if constexpr (std::is_same_v, DecimalField>) - return v.getValue(); - else - return v; + const IColumn & x = *arguments[0].column; + const IColumn & y = *arguments[1].column; + + size_t rows = input_rows_count; + auto res = ColumnInt64::create(rows); + + impl.dispatchForColumns(x, y, DateLUT::instance(), DateLUT::instance(), res->getData()); + + return res; } +private: + DateDiffImpl impl{name}; }; } @@ -266,4 +334,9 @@ REGISTER_FUNCTION(DateDiff) factory.registerFunction({}, FunctionFactory::CaseInsensitive); } +REGISTER_FUNCTION(TimeDiff) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + } diff --git a/src/Functions/timeDiff.cpp b/src/Functions/timeDiff.cpp deleted file mode 100644 index 0353dbf7c51..00000000000 --- a/src/Functions/timeDiff.cpp +++ /dev/null @@ -1,228 +0,0 @@ -#include -#include -#include -#include -#include -#include - -#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; -} - -namespace -{ - -/** TimeDiff(t1, t2) - * t1 and t2 can be Date or DateTime - */ -class FunctionTimeDiff : public IFunction -{ - using ColumnDateTime64 = ColumnDecimal; -public: - static constexpr auto name = "TimeDiff"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - String getName() const override - { - return name; - } - - bool isVariadic() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - size_t getNumberOfArguments() const override { return 2; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments.size() != 2) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 2", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - if (!isDate(arguments[0]) && !isDateTime(arguments[0]) && !isDateTime64(arguments[1])) - throw Exception("First argument for function " + getName() + " must be Date or DateTime", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!isDate(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[2])) - throw Exception("Second argument for function " + getName() + " must be Date or DateTime", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override - { - const IColumn & x = *arguments[0].column; - const IColumn & y = *arguments[1].column; - - size_t rows = input_rows_count; - auto res = ColumnInt64::create(rows); - - dispatchForColumns(x, y, DateLUT::instance(), DateLUT::instance(), res->getData()); - - return res; - } - -private: - template - void dispatchForColumns( - const IColumn & x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - if (const auto * x_vec_16 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_32 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_64 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); - else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); - else - throw Exception("Illegal column for first argument of function " + getName() + ", must be Date, DateTime or DateTime64", ErrorCodes::ILLEGAL_COLUMN); - } - - template - void dispatchForSecondColumn( - const LeftColumnType & x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - if (const auto * y_vec_16 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_64 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); - else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); - else - throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); - } - - template - void dispatchConstForSecondColumn( - T1 x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - if (const auto * y_vec_16 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_64 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_64, timezone_x, timezone_y, result); - else - throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); - } - - template - void vectorVector( - const LeftColumnType & x, const RightColumnType & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - const auto & x_data = x.getData(); - const auto & y_data = y.getData(); - - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - for (size_t i = 0, size = x.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); - } - - template - void vectorConstant( - const LeftColumnType & x, T2 y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - const auto & x_data = x.getData(); - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - const auto y_value = stripDecimalFieldValue(y); - - for (size_t i = 0, size = x.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); - } - - template - void constantVector( - T1 x, const RightColumnType & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - const auto & y_data = y.getData(); - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - const auto x_value = stripDecimalFieldValue(x); - - for (size_t i = 0, size = y.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); - } - - template - Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const - { - return static_cast(transform_y.execute(y, timezone_y)) - - static_cast(transform_x.execute(x, timezone_x)); - } - - template - static UInt32 getScale(const T & v) - { - if constexpr (std::is_same_v) - return v.getScale(); - else if constexpr (std::is_same_v>) - return v.getScale(); - - return 0; - } - template - static auto stripDecimalFieldValue(T && v) - { - if constexpr (std::is_same_v, DecimalField>) - return v.getValue(); - else - return v; - } -}; - -} - -REGISTER_FUNCTION(TimeDiff) -{ - factory.registerFunction({}, FunctionFactory::CaseInsensitive); -} - -} From 330f18194a90fc473a5630117a7630d345d8a783 Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Sun, 30 Oct 2022 14:44:13 +0000 Subject: [PATCH 11/82] fix test --- .../02415_all_new_functions_must_be_documented.reference | 2 ++ 1 file changed, 2 insertions(+) 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 040a8c8d317..ae701d9ae83 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 @@ -51,9 +51,11 @@ LpNormalize MACNumToString MACStringToNum MACStringToOUI +TimeDiff URLHash URLHierarchy URLPathHierarchy +UTCTimestamp UUIDNumToString UUIDStringToNum _CAST From 81c1576e61fcac935bba37701fd5e072367be6c1 Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Mon, 31 Oct 2022 13:03:13 +0000 Subject: [PATCH 12/82] add tests --- src/Functions/dateDiff.cpp | 290 +++++++++--------- .../02474_timeDiff_UTCTimestamp.reference | 9 + .../02474_timeDiff_UTCTimestamp.sql | 12 + 3 files changed, 168 insertions(+), 143 deletions(-) create mode 100644 tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference create mode 100644 tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index e62f0e2c517..c653db3ddf2 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -36,156 +36,157 @@ namespace class DateDiffImpl { -using ColumnDateTime64 = ColumnDecimal; public: -explicit DateDiffImpl(const String& name_) : name(name_) {} + using ColumnDateTime64 = ColumnDecimal; -template -void dispatchForColumns( - const IColumn & x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const -{ - if (const auto * x_vec_16 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_32 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_32_s = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_32_s, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_64 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); - else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_32_s = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_32_s->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column for first argument of function {}, must be Date, Date32, DateTime or DateTime64", - name); -} + explicit DateDiffImpl(const String & name_) : name(name_) {} -template -void dispatchForSecondColumn( - const LeftColumnType & x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const -{ - if (const auto * y_vec_16 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_32_s, timezone_x, timezone_y, result); - else if (const auto * y_vec_64 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); - else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_32_s = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_32_s->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column for second argument of function {}, must be Date, Date32, DateTime or DateTime64", - name); -} + template + void dispatchForColumns( + const IColumn & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * x_vec_16 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_32 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_32_s = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_32_s, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_64 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); + else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_32_s = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_32_s->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column for first argument of function {}, must be Date, Date32, DateTime or DateTime64", + name); + } -template -void dispatchConstForSecondColumn( - T1 x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const -{ - if (const auto * y_vec_16 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) - constantVector(x, *y_vec_32_s, timezone_x, timezone_y, result); - else if (const auto * y_vec_64 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_64, timezone_x, timezone_y, result); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column for second argument of function {}, must be Date, Date32, DateTime or DateTime64", - name); -} + template + void dispatchForSecondColumn( + const LeftColumnType & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_32_s, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); + else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_32_s = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_32_s->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column for second argument of function {}, must be Date, Date32, DateTime or DateTime64", + name); + } -template -void vectorVector( - const LeftColumnType & x, const RightColumnType & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const -{ - const auto & x_data = x.getData(); - const auto & y_data = y.getData(); + template + void dispatchConstForSecondColumn( + T1 x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) + constantVector(x, *y_vec_32_s, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_64, timezone_x, timezone_y, result); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column for second argument of function {}, must be Date, Date32, DateTime or DateTime64", + name); + } - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - for (size_t i = 0, size = x.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); -} + template + void vectorVector( + const LeftColumnType & x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & x_data = x.getData(); + const auto & y_data = y.getData(); -template -void vectorConstant( - const LeftColumnType & x, T2 y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const -{ - const auto & x_data = x.getData(); - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - const auto y_value = stripDecimalFieldValue(y); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); + } - for (size_t i = 0, size = x.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); -} + template + void vectorConstant( + const LeftColumnType & x, T2 y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & x_data = x.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto y_value = stripDecimalFieldValue(y); -template -void constantVector( - T1 x, const RightColumnType & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const -{ - const auto & y_data = y.getData(); - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - const auto x_value = stripDecimalFieldValue(x); + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); + } - for (size_t i = 0, size = y.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); -} + template + void constantVector( + T1 x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & y_data = y.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto x_value = stripDecimalFieldValue(x); -template -Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const -{ - return static_cast(transform_y.execute(y, timezone_y)) - - static_cast(transform_x.execute(x, timezone_x)); -} + for (size_t i = 0, size = y.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); + } -template -static UInt32 getScale(const T & v) -{ - if constexpr (std::is_same_v) - return v.getScale(); - else if constexpr (std::is_same_v>) - return v.getScale(); + template + Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const + { + return static_cast(transform_y.execute(y, timezone_y)) + - static_cast(transform_x.execute(x, timezone_x)); + } - return 0; -} -template -static auto stripDecimalFieldValue(T && v) -{ - if constexpr (std::is_same_v, DecimalField>) - return v.getValue(); - else - return v; -} + template + static UInt32 getScale(const T & v) + { + if constexpr (std::is_same_v) + return v.getScale(); + else if constexpr (std::is_same_v>) + return v.getScale(); + + return 0; + } + template + static auto stripDecimalFieldValue(T && v) + { + if constexpr (std::is_same_v, DecimalField>) + return v.getValue(); + else + return v; + } private: String name; }; @@ -321,13 +322,16 @@ public: + toString(arguments.size()) + ", should be 2", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!isDate(arguments[0]) && !isDateTime(arguments[0]) && !isDateTime64(arguments[1])) - throw Exception("First argument for function " + getName() + " must be Date or DateTime", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (!isDate(arguments[0]) && !isDate32(arguments[0]) && !isDateTime(arguments[0]) && !isDateTime64(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be Date, Date32, DateTime or DateTime64", + getName()); - if (!isDate(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[2])) - throw Exception("Second argument for function " + getName() + " must be Date or DateTime", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (!isDate(arguments[1]) && !isDate32(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[1])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Second argument for function {} must be Date, Date32, DateTime or DateTime64", + getName() + ); return std::make_shared(); } diff --git a/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference new file mode 100644 index 00000000000..bc6ff2b709c --- /dev/null +++ b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference @@ -0,0 +1,9 @@ +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql new file mode 100644 index 00000000000..eac5edae483 --- /dev/null +++ b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql @@ -0,0 +1,12 @@ +-- all tests should be equal to zero as timediff is same as dateDiff('second', ... ) +SELECT dateDiff('second', toDate32('1927-01-01'), toDate32('1927-01-02')) - timeDiff(toDate32('1927-01-01'), toDate32('1927-01-02')); +SELECT dateDiff('second', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)) - timeDiff(toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); +SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')) - timeDiff(toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); +SELECT dateDiff('second', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')) - timeDiff(toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')); +SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')) - timeDiff(toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')); +SELECT dateDiff('second', toDate32('2015-08-18'), toDate('2015-08-19')) - timeDiff(toDate32('2015-08-18'), toDate('2015-08-19')); +SELECT dateDiff('second', toDate('2015-08-18'), toDate32('2015-08-19')) - timeDiff(toDate('2015-08-18'), toDate32('2015-08-19')); + +-- UTCTimestamp equals to now('UTC') +SELECT dateDiff('s', UTCTimestamp(), now('UTC')); +SELECT timeDiff(UTCTimestamp(), now('UTC')); From 9dbb7e335cc4323689e46dc50d049fe528c02728 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 1 Nov 2022 14:02:25 +0100 Subject: [PATCH 13/82] Fix tests --- tests/queries/0_stateless/01825_type_json_schema_inference.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01825_type_json_schema_inference.sh b/tests/queries/0_stateless/01825_type_json_schema_inference.sh index f6917ab18ad..a524eaec20a 100755 --- a/tests/queries/0_stateless/01825_type_json_schema_inference.sh +++ b/tests/queries/0_stateless/01825_type_json_schema_inference.sh @@ -46,7 +46,7 @@ ${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_inference (obj JSON, map Map(String echo '{"map": {"k1": 1, "k2": 2}, "obj": {"k1": 1, "k2": 2}}' > $filename -${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_inference SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json', 'JSONEachRow')" --allow_experimental_object_type 1 --use_structure_from_insertion_table_in_table_functions 1 +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_inference SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json', 'JSONEachRow')" --allow_experimental_object_type 1 --use_structure_from_insertion_table_in_table_functions 0 ${CLICKHOUSE_CLIENT} -q "SELECT * FROM t_json_inference FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 ${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(obj) FROM t_json_inference LIMIT 1" From 34a33d182138a145ed2560ffe9fbc467f69a858e Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Tue, 1 Nov 2022 15:09:05 +0000 Subject: [PATCH 14/82] add documentation for functions --- src/Functions/UTCTimestamp.cpp | 18 +++++++++++++----- src/Functions/dateDiff.cpp | 11 ++++++++++- 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/src/Functions/UTCTimestamp.cpp b/src/Functions/UTCTimestamp.cpp index 218c7fa5c1c..275e5c278fa 100644 --- a/src/Functions/UTCTimestamp.cpp +++ b/src/Functions/UTCTimestamp.cpp @@ -5,8 +5,6 @@ #include #include -#include - namespace DB { @@ -86,7 +84,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() > 0) + if (!arguments.empty()) { throw Exception("Arguments size of function " + getName() + " should be 0", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } @@ -96,7 +94,7 @@ public: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override { - if (arguments.size() > 0) + if (!arguments.empty()) { throw Exception("Arguments size of function " + getName() + " should be 0", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } @@ -110,7 +108,17 @@ public: /// UTC_timestamp for MySQL interface support REGISTER_FUNCTION(UTCTimestamp) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({ + R"( +Returns the current date and time at the moment of query analysis. The function is a constant expression. +Same as `now('UTC')`. Was added only for MySQL support. `now` is prefered. + +Example: +[example:typical] +)", + Documentation::Examples{ + {"typical", "SELECT UTCTimestamp();"}}, + Documentation::Categories{"Dates and Times"}}, FunctionFactory::CaseInsensitive); factory.registerAlias("UTC_timestamp", UTCTimestampOverloadResolver::name, FunctionFactory::CaseInsensitive); } diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index c653db3ddf2..039d0736a7c 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -364,7 +364,16 @@ REGISTER_FUNCTION(DateDiff) REGISTER_FUNCTION(TimeDiff) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({R"( +Returns the difference between two dates or dates with time values. The difference is calculated in seconds units (see toRelativeSecondNum). +It is same as `dateDiff` and was added only for MySQL support. `dateDiff` is prefered. + +Example: +[example:typical] +)", + Documentation::Examples{ + {"typical", "SELECT timeDiff(UTCTimestamp(), now());"}}, + Documentation::Categories{"Dates and Times"}}, FunctionFactory::CaseInsensitive); } } From 84c88ddc914ffd77a6d9c296bd35ce3cb753b100 Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Tue, 1 Nov 2022 17:29:15 +0000 Subject: [PATCH 15/82] fix typos --- src/Functions/UTCTimestamp.cpp | 2 +- src/Functions/dateDiff.cpp | 2 +- .../02415_all_new_functions_must_be_documented.reference | 2 -- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Functions/UTCTimestamp.cpp b/src/Functions/UTCTimestamp.cpp index 275e5c278fa..27b5b33bc6e 100644 --- a/src/Functions/UTCTimestamp.cpp +++ b/src/Functions/UTCTimestamp.cpp @@ -111,7 +111,7 @@ REGISTER_FUNCTION(UTCTimestamp) factory.registerFunction({ R"( Returns the current date and time at the moment of query analysis. The function is a constant expression. -Same as `now('UTC')`. Was added only for MySQL support. `now` is prefered. +Same as `now('UTC')`. Was added only for MySQL support. `now` is preferred. Example: [example:typical] diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index 039d0736a7c..ec9c9df8e49 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -366,7 +366,7 @@ REGISTER_FUNCTION(TimeDiff) { factory.registerFunction({R"( Returns the difference between two dates or dates with time values. The difference is calculated in seconds units (see toRelativeSecondNum). -It is same as `dateDiff` and was added only for MySQL support. `dateDiff` is prefered. +It is same as `dateDiff` and was added only for MySQL support. `dateDiff` is preferred. Example: [example:typical] 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 ae701d9ae83..040a8c8d317 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 @@ -51,11 +51,9 @@ LpNormalize MACNumToString MACStringToNum MACStringToOUI -TimeDiff URLHash URLHierarchy URLPathHierarchy -UTCTimestamp UUIDNumToString UUIDStringToNum _CAST From 774a86021f0cd020eb0c1765fe859a46fbc5df83 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 2 Nov 2022 19:16:51 +0000 Subject: [PATCH 16/82] Fix datetime schema inference in case of empty string --- src/Formats/EscapingRuleUtils.cpp | 3 +++ .../0_stateless/02475_date_time_schema_inference_bug.reference | 1 + .../0_stateless/02475_date_time_schema_inference_bug.sql | 1 + 3 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/02475_date_time_schema_inference_bug.reference create mode 100644 tests/queries/0_stateless/02475_date_time_schema_inference_bug.sql diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index e80ab50968d..bba94e98e49 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -464,6 +464,9 @@ bool tryInferDate(const std::string_view & field) bool tryInferDateTime(const std::string_view & field, const FormatSettings & settings) { + if (field.empty()) + return false; + ReadBufferFromString buf(field); Float64 tmp_float; /// Check if it's just a number, and if so, don't try to infer DateTime from it, diff --git a/tests/queries/0_stateless/02475_date_time_schema_inference_bug.reference b/tests/queries/0_stateless/02475_date_time_schema_inference_bug.reference new file mode 100644 index 00000000000..8b137891791 --- /dev/null +++ b/tests/queries/0_stateless/02475_date_time_schema_inference_bug.reference @@ -0,0 +1 @@ + diff --git a/tests/queries/0_stateless/02475_date_time_schema_inference_bug.sql b/tests/queries/0_stateless/02475_date_time_schema_inference_bug.sql new file mode 100644 index 00000000000..1aea4a8024c --- /dev/null +++ b/tests/queries/0_stateless/02475_date_time_schema_inference_bug.sql @@ -0,0 +1 @@ +select * from format(CSV, '""'); From eeb9c57c5995d255610a71fe70aa3f79af97214e Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Thu, 3 Nov 2022 20:21:02 +0800 Subject: [PATCH 17/82] to fix issue 42856 and MaterializedMySQL improvement --- src/Core/MySQL/MySQLReplication.cpp | 7 +++++-- src/Core/MySQL/MySQLReplication.h | 3 ++- src/DataTypes/convertMySQLDataType.cpp | 2 +- src/Parsers/MySQL/ASTAlterCommand.cpp | 8 +++++++- src/Parsers/MySQL/ASTDeclareColumn.cpp | 2 ++ .../materialize_with_ddl.py | 10 ++++++++++ 6 files changed, 27 insertions(+), 5 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 6f3ac1b40e9..45437ea5843 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -116,9 +116,10 @@ namespace MySQLReplication if (!query.starts_with("XA COMMIT")) transaction_complete = false; } - else if (query.starts_with("SAVEPOINT")) + else if (query.starts_with("SAVEPOINT") || query.starts_with("ROLLBACK") + || query.starts_with("RELEASE SAVEPOINT")) { - throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::LOGICAL_ERROR); + typ = QUERY_EVENT_OTHER; } } @@ -941,6 +942,8 @@ namespace MySQLReplication { case QUERY_EVENT_MULTI_TXN_FLAG: case QUERY_EVENT_XA: + /// Ignore queries that have no impact on the data + case QUERY_EVENT_OTHER: { event = std::make_shared(std::move(query->header)); break; diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index 8900eee0102..d4bb3fb86f9 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -368,7 +368,8 @@ namespace MySQLReplication { QUERY_EVENT_DDL = 0, QUERY_EVENT_MULTI_TXN_FLAG = 1, - QUERY_EVENT_XA = 2 + QUERY_EVENT_XA = 2, + QUERY_EVENT_OTHER = 3 }; class QueryEvent : public EventBase diff --git a/src/DataTypes/convertMySQLDataType.cpp b/src/DataTypes/convertMySQLDataType.cpp index 64633c6fd7b..307ff317204 100644 --- a/src/DataTypes/convertMySQLDataType.cpp +++ b/src/DataTypes/convertMySQLDataType.cpp @@ -55,7 +55,7 @@ DataTypePtr convertMySQLDataType(MultiEnum type_support, else res = std::make_shared(); } - else if (type_name == "int" || type_name == "mediumint") + else if (type_name == "int" || type_name == "mediumint" || type_name == "integer") { if (is_unsigned) res = std::make_shared(); diff --git a/src/Parsers/MySQL/ASTAlterCommand.cpp b/src/Parsers/MySQL/ASTAlterCommand.cpp index b5b36ff3c74..10446d40172 100644 --- a/src/Parsers/MySQL/ASTAlterCommand.cpp +++ b/src/Parsers/MySQL/ASTAlterCommand.cpp @@ -267,7 +267,12 @@ static inline bool parseRenameCommand(IParser::Pos & pos, ASTPtr & node, Expecte } else { - return false; + if (!ParserCompoundIdentifier(true).parse(pos, new_name, expected)) + return false; + auto new_table_id = new_name->as()->getTableId(); + alter_command->type = ASTAlterCommand::RENAME_TABLE; + alter_command->new_table_name = new_table_id.table_name; + alter_command->new_database_name = new_table_id.database_name; } node = alter_command; @@ -306,6 +311,7 @@ static inline bool parseOtherCommand(IParser::Pos & pos, ASTPtr & node, Expected OptionDescribe("CONVERT TO CHARACTER SET", "charset", std::make_shared()), OptionDescribe("CHARACTER SET", "charset", std::make_shared()), OptionDescribe("DEFAULT CHARACTER SET", "charset", std::make_shared()), + OptionDescribe("COMMENT", "", std::make_shared()), OptionDescribe("LOCK", "lock", std::make_shared()) } }; diff --git a/src/Parsers/MySQL/ASTDeclareColumn.cpp b/src/Parsers/MySQL/ASTDeclareColumn.cpp index 89085ef989d..e585dcb670c 100644 --- a/src/Parsers/MySQL/ASTDeclareColumn.cpp +++ b/src/Parsers/MySQL/ASTDeclareColumn.cpp @@ -52,6 +52,7 @@ static inline bool parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node, OptionDescribe("KEY", "primary_key", std::make_unique()), OptionDescribe("COMMENT", "comment", std::make_unique()), OptionDescribe("CHARACTER SET", "charset_name", std::make_unique()), + OptionDescribe("CHARSET", "charset", std::make_unique()), OptionDescribe("COLLATE", "collate", std::make_unique()), OptionDescribe("COLUMN_FORMAT", "column_format", std::make_unique()), OptionDescribe("STORAGE", "storage", std::make_unique()), @@ -59,6 +60,7 @@ static inline bool parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node, OptionDescribe("GENERATED ALWAYS AS", "generated", std::make_unique()), OptionDescribe("STORED", "is_stored", std::make_unique()), OptionDescribe("VIRTUAL", "is_stored", std::make_unique()), + OptionDescribe("INVISIBLE", "", std::make_unique()), OptionDescribe("", "reference", std::make_unique()), OptionDescribe("", "constraint", std::make_unique()), } diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index 22d4633685e..bfb6d2bc6dd 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -875,6 +875,16 @@ def alter_rename_table_with_materialized_mysql_database( "1\n2\n3\n4\n5\n", ) + mysql_node.query("ALTER TABLE test_database_rename_table.test_table_4 RENAME test_database_rename_table.test_table_5") + mysql_node.query("ALTER TABLE test_database_rename_table.test_table_5 RENAME TO test_database_rename_table.test_table_6") + mysql_node.query("ALTER TABLE test_database_rename_table.test_table_6 RENAME AS test_database_rename_table.test_table_7") + + check_query( + clickhouse_node, + "SELECT * FROM test_database_rename_table.test_table_7 ORDER BY id FORMAT TSV", + "1\n2\n3\n4\n5\n", + ) + clickhouse_node.query("DROP DATABASE test_database_rename_table") mysql_node.query("DROP DATABASE test_database_rename_table") From 6603df8e136393ee49866d016d767c74a185ba80 Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Mon, 7 Nov 2022 11:28:21 +0000 Subject: [PATCH 18/82] fix tests --- .../02474_timeDiff_UTCTimestamp.reference | 18 +++++++++--------- .../02474_timeDiff_UTCTimestamp.sql | 18 +++++++++--------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference index bc6ff2b709c..bb0850568bb 100644 --- a/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference +++ b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference @@ -1,9 +1,9 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql index eac5edae483..231bebc1503 100644 --- a/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql +++ b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql @@ -1,12 +1,12 @@ -- all tests should be equal to zero as timediff is same as dateDiff('second', ... ) -SELECT dateDiff('second', toDate32('1927-01-01'), toDate32('1927-01-02')) - timeDiff(toDate32('1927-01-01'), toDate32('1927-01-02')); -SELECT dateDiff('second', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)) - timeDiff(toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); -SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')) - timeDiff(toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); -SELECT dateDiff('second', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')) - timeDiff(toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')); -SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')) - timeDiff(toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')); -SELECT dateDiff('second', toDate32('2015-08-18'), toDate('2015-08-19')) - timeDiff(toDate32('2015-08-18'), toDate('2015-08-19')); -SELECT dateDiff('second', toDate('2015-08-18'), toDate32('2015-08-19')) - timeDiff(toDate('2015-08-18'), toDate32('2015-08-19')); +SELECT dateDiff('second', toDate32('1927-01-01'), toDate32('1927-01-02')) - timeDiff(toDate32('1927-01-01'), toDate32('1927-01-02')) <= 2; +SELECT dateDiff('second', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)) - timeDiff(toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)) <= 2; +SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')) - timeDiff(toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')) <= 2; +SELECT dateDiff('second', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')) - timeDiff(toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')) <= 2; +SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')) - timeDiff(toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')) <= 2; +SELECT dateDiff('second', toDate32('2015-08-18'), toDate('2015-08-19')) - timeDiff(toDate32('2015-08-18'), toDate('2015-08-19')) <= 2; +SELECT dateDiff('second', toDate('2015-08-18'), toDate32('2015-08-19')) - timeDiff(toDate('2015-08-18'), toDate32('2015-08-19')) <= 2; -- UTCTimestamp equals to now('UTC') -SELECT dateDiff('s', UTCTimestamp(), now('UTC')); -SELECT timeDiff(UTCTimestamp(), now('UTC')); +SELECT dateDiff('s', UTCTimestamp(), now('UTC')) <= 2; +SELECT timeDiff(UTCTimestamp(), now('UTC')) <= 2; From e94e087187c74c1b88c0772635f3186448990844 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 9 Nov 2022 14:08:16 +0100 Subject: [PATCH 19/82] Add generic named collections class and factory --- src/Common/ErrorCodes.cpp | 1 + src/Storages/NamedCollections.cpp | 272 ++++++++++++++++++ src/Storages/NamedCollections.h | 120 ++++++++ .../tests/gtest_named_collections.cpp | 75 +++++ 4 files changed, 468 insertions(+) create mode 100644 src/Storages/NamedCollections.cpp create mode 100644 src/Storages/NamedCollections.h create mode 100644 src/Storages/tests/gtest_named_collections.cpp diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index e80ad5c141a..26f837f9fe5 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -637,6 +637,7 @@ M(666, CANNOT_USE_CACHE) \ M(667, NOT_INITIALIZED) \ M(668, INVALID_STATE) \ + M(669, UNKNOWN_NAMED_COLLECTION) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/NamedCollections.cpp b/src/Storages/NamedCollections.cpp new file mode 100644 index 00000000000..94870507b59 --- /dev/null +++ b/src/Storages/NamedCollections.cpp @@ -0,0 +1,272 @@ +#include "NamedCollections.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +static constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections."; + +namespace ErrorCodes +{ + extern const int UNKNOWN_NAMED_COLLECTION; +} + +namespace +{ + std::string getCollectionPrefix(const std::string & collection_name) + { + return NAMED_COLLECTIONS_CONFIG_PREFIX + collection_name; + } +} + +NamedCollectionFactory & NamedCollectionFactory::instance() +{ + static NamedCollectionFactory instance; + return instance; +} + +void NamedCollectionFactory::initialize(const Poco::Util::AbstractConfiguration & server_config) +{ + std::lock_guard lock(mutex); + if (is_initialized) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Named collection factory is already initialzied"); + } + + config = &server_config; + is_initialized = true; +} + +void NamedCollectionFactory::assertInitialized(std::lock_guard & /* lock */) const +{ + if (!is_initialized) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Named collection factory must be initialized before used"); + } +} + +bool NamedCollectionFactory::exists(const std::string & collection_name) const +{ + std::lock_guard lock(mutex); + assertInitialized(lock); + return config->has(getCollectionPrefix(collection_name)); +} + +NamedCollectionPtr NamedCollectionFactory::get( + const std::string & collection_name, + const NamedCollectionInfo & collection_info) const +{ + std::lock_guard lock(mutex); + assertInitialized(lock); + + if (!exists(collection_name)) + throw Exception( + ErrorCodes::UNKNOWN_NAMED_COLLECTION, + "There is no named collection `{}` in config", + collection_name); + + return getImpl(collection_name, collection_info, lock); +} + +NamedCollectionPtr NamedCollectionFactory::tryGet( + const std::string & collection_name, + const NamedCollectionInfo & collection_info) const +{ + std::lock_guard lock(mutex); + assertInitialized(lock); + + if (!exists(collection_name)) + return nullptr; + + return getImpl(collection_name, collection_info, lock); +} + +NamedCollectionPtr NamedCollectionFactory::getImpl( + const std::string & collection_name, + const NamedCollectionInfo & collection_info, + std::lock_guard & /* lock */) const +{ + auto it = named_collections.find(collection_name); + if (it == named_collections.end()) + { + const auto collection_prefix = getCollectionPrefix(collection_name); + const auto collection_view = config->createView(collection_prefix); + + auto collection = std::make_unique(); + collection->initialize(*collection_view, collection_info); + it = named_collections.emplace(collection_name, std::move(collection)).first; + } + return it->second; +} + +struct NamedCollection::Impl +{ + std::unordered_map collection; + + ImplPtr copy() const + { + auto impl = std::make_unique(); + impl->collection = collection; + return impl; + } + + Value get(const Key & key) const + { + auto it = collection.find(key); + if (it == collection.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no key: {}", key); + return it->second; + } + + void replace(const Key & key, const Value & value) + { + auto it = collection.find(key); + if (it == collection.end()) + collection.emplace(key, value); + else + it->second = value; + } + + void initialize( + const Poco::Util::AbstractConfiguration & config, + const NamedCollectionInfo & collection_info) + { + for (const auto & [key, key_info] : collection_info) + { + const auto & default_value = key_info.default_value; + const bool has_value = config.has(key); + + if (!default_value && !has_value) + continue; + + Field value; + switch (key_info.type) + { + case Field::Types::Which::String: + value = has_value ? config.getString(key) : default_value->get(); + break; + case Field::Types::Which::UInt64: + value = has_value ? config.getUInt64(key) : default_value->get(); + break; + case Field::Types::Which::Int64: + value = has_value ? config.getInt64(key) : default_value->get(); + break; + case Field::Types::Which::Float64: + value = has_value ? config.getDouble(key) : default_value->get(); + break; + default: + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Unsupported type: {}", toString(key_info.type)); + } + + collection.emplace(key, value); + } + } + + static void validate( + const Poco::Util::AbstractConfiguration & config, + const NamedCollectionInfo & collection_info) + { + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys("", config_keys); + + auto get_suggestion = [&](bool only_required_keys) + { + std::string suggestion; + for (const auto & [key, info] : collection_info) + { + if (only_required_keys && info.is_required) + continue; + + if (!suggestion.empty()) + suggestion += ", "; + + suggestion += key; + } + return suggestion; + }; + + std::set required_keys; + for (const auto & [key, info] : collection_info) + { + if (info.is_required) + required_keys.insert(key); + } + + for (const auto & key : config_keys) + { + if (!collection_info.contains(key)) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Unknown key `{}`, expected one of: {}", + key, get_suggestion(false)); + } + required_keys.erase(key); + } + + if (!required_keys.empty()) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Keys `{}` are required, but was not found in config. List of required keys: {}", + fmt::join(required_keys, ", "), get_suggestion(true)); + } + } +}; + +NamedCollection::NamedCollection() +{ + pimpl = std::make_unique(); +} + +NamedCollection::NamedCollection(ImplPtr pimpl_) +{ + pimpl = std::move(pimpl_); +} + +NamedCollection::Value NamedCollection::get(const Key & key) const +{ + return pimpl->get(key); +} + +std::shared_ptr NamedCollection::copy() const +{ + return std::make_shared(pimpl->copy()); +} + +void NamedCollection::validate( + const Poco::Util::AbstractConfiguration & config, + const NamedCollectionInfo & collection_info) const +{ + pimpl->validate(config, collection_info); +} + +void NamedCollection::initialize( + const Poco::Util::AbstractConfiguration & config, + const NamedCollectionInfo & collection_info) +{ + pimpl->initialize(config, collection_info); +} + +void NamedCollection::replace(const Key & key, const Value & value) +{ + pimpl->replace(key, value); +} + +} diff --git a/src/Storages/NamedCollections.h b/src/Storages/NamedCollections.h new file mode 100644 index 00000000000..870d167889d --- /dev/null +++ b/src/Storages/NamedCollections.h @@ -0,0 +1,120 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class NamedCollection; +using NamedCollectionPtr = std::shared_ptr; +struct NamedCollectionValueInfo; +using NamedCollectionInfo = std::unordered_map; + +/** + * A factory of immutable named collections. + * Named collections are defined in server config as arbitrary + * structure configurations: + * + * + * ... + * + * ... + * + * In order to get a named collection, you need to know it's name + * and expected structure of the collection defined as NamedCollectionInfo. + */ +class NamedCollectionFactory : boost::noncopyable +{ +public: + void initialize(const Poco::Util::AbstractConfiguration & server_config); + + bool exists(const std::string & collection_name) const; + + NamedCollectionPtr get( + const std::string & collection_name, + const NamedCollectionInfo & collection_info) const; + + NamedCollectionPtr tryGet( + const std::string & collection_name, + const NamedCollectionInfo & collection_info) const; + + static NamedCollectionFactory & instance(); + +private: + NamedCollectionPtr getImpl( + const std::string & collection_name, + const NamedCollectionInfo & collection_info, + std::lock_guard & lock) const; + + using NamedCollections = std::unordered_map; + mutable NamedCollections named_collections; + +private: + const Poco::Util::AbstractConfiguration * config; + + void assertInitialized(std::lock_guard & lock) const; + + bool is_initialized = false; + mutable std::mutex mutex; +}; + + +class NamedCollection +{ +friend class NamedCollectionFactory; + +private: + struct Impl; + using ImplPtr = std::unique_ptr; + + ImplPtr pimpl; + +public: + using Key = std::string; + using Value = Field; + + Value get(const Key & key) const; + + void replace(const Key & key, const Value & value); + + /// Copy current named collection to allow modification as + /// NamedConnectionFactory returns immutable collections. + std::shared_ptr copy() const; + + NamedCollection(); + explicit NamedCollection(ImplPtr pimpl_); + +protected: + /// Initialize from config. `config` must be a view to the required collection, + /// e.g. the root of `config` is the root of collection. + void initialize( + const Poco::Util::AbstractConfiguration & config, + const NamedCollectionInfo & collection_info); + + /// Validate named collection in config. + /// Throws exception if named collection keys in config are not the same as + /// expected (contains unknown keys or misses required keys) + void validate( + const Poco::Util::AbstractConfiguration & config, + const NamedCollectionInfo & collection_info) const; +}; + + +/** + * Named collection info which allows to parse config. + * Contains a mapping key_path -> value_info. + */ +struct NamedCollectionValueInfo +{ + /// Type of the value. One of: String, UInt64, Int64, Double. + Field::Types::Which type; + /// Optional default value for the case if there is no such key in config. + std::optional default_value; + /// Is this value required or optional? Throw exception if the value is + /// required, but is not specified in config. + bool is_required = true; +}; + +} diff --git a/src/Storages/tests/gtest_named_collections.cpp b/src/Storages/tests/gtest_named_collections.cpp new file mode 100644 index 00000000000..669e7bae25b --- /dev/null +++ b/src/Storages/tests/gtest_named_collections.cpp @@ -0,0 +1,75 @@ +#include +#include +#include +#include +#include + +using namespace DB; + +TEST(NamedCollections, Simple) +{ + std::string xml(R"CONFIG( + + + value1 + 2 + -3 + 4.4 + + + value4 + 5 + 6.6 + + +)CONFIG"); + + Poco::XML::DOMParser dom_parser; + Poco::AutoPtr document = dom_parser.parseString(xml); + Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); + + NamedCollectionFactory::instance().initialize(*config); + + ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1")); + ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection2")); + ASSERT_TRUE(NamedCollectionFactory::instance().tryGet("collection3", {}) == nullptr); + + using ValueInfo = NamedCollectionValueInfo; + ValueInfo string_def{Field::Types::Which::String, std::nullopt, true}; + ValueInfo uint_def{Field::Types::Which::UInt64, std::nullopt, true}; + ValueInfo int_def{Field::Types::Which::Int64, std::nullopt, true}; + ValueInfo double_def{Field::Types::Which::Float64, std::nullopt, true}; + + NamedCollectionInfo collection1_info; + collection1_info.emplace("key1", string_def); + collection1_info.emplace("key2", uint_def); + collection1_info.emplace("key3", int_def); + collection1_info.emplace("key4", double_def); + + auto collection1 = NamedCollectionFactory::instance().get("collection1", collection1_info); + ASSERT_TRUE(collection1 != nullptr); + + ASSERT_TRUE(collection1->get("key1").safeGet() == "value1"); + ASSERT_TRUE(collection1->get("key2").safeGet() == 2); + ASSERT_TRUE(collection1->get("key3").safeGet() == -3); + ASSERT_TRUE(collection1->get("key4").safeGet() == 4.4); + + NamedCollectionInfo collection2_info; + collection2_info.emplace("key4", string_def); + collection2_info.emplace("key5", uint_def); + collection2_info.emplace("key6", double_def); + + auto collection2 = NamedCollectionFactory::instance().get("collection2", collection2_info); + ASSERT_TRUE(collection2 != nullptr); + + ASSERT_TRUE(collection2->get("key4").safeGet() == "value4"); + ASSERT_TRUE(collection2->get("key5").safeGet() == 5); + ASSERT_TRUE(collection2->get("key6").safeGet() == 6.6); + + auto mutable_collection2 = collection2->copy(); + mutable_collection2->replace("key4", UInt64(4)); + mutable_collection2->replace("key7", "value7"); + + ASSERT_TRUE(mutable_collection2->get("key4").safeGet() == 4); + ASSERT_TRUE(mutable_collection2->get("key7").safeGet() == "value7"); +} From 3eb6c03b471e1e1f5dd883e451762d9b0daaad83 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 9 Nov 2022 19:46:47 +0000 Subject: [PATCH 20/82] Disable clickhouse local and clickhouse client non-interactive progress by default. --- programs/client/Client.cpp | 2 +- programs/local/LocalServer.cpp | 2 +- src/Client/ClientBase.cpp | 2 +- ...ient_INSERT_progress_profile_events.expect | 2 +- ...ocal_INSERT_progress_profile_events.expect | 2 +- .../0_stateless/02456_progress_tty.expect | 27 ++++++------------- 6 files changed, 13 insertions(+), 24 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 93136df2a5b..b262c5a0db1 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1088,7 +1088,7 @@ void Client::processConfig() } else { - std::string progress = config().getString("progress", "tty"); + std::string progress = config().getString("progress", "off"); need_render_progress = (Poco::icompare(progress, "off") && Poco::icompare(progress, "no") && Poco::icompare(progress, "false") && Poco::icompare(progress, "0")); echo_queries = config().getBool("echo", false); ignore_error = config().getBool("ignore-error", false); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 3ac9c1e7c37..fea82b456cd 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -489,7 +489,7 @@ void LocalServer::processConfig() } else { - std::string progress = config().getString("progress", "tty"); + std::string progress = config().getString("progress", "off"); need_render_progress = (Poco::icompare(progress, "off") && Poco::icompare(progress, "no") && Poco::icompare(progress, "false") && Poco::icompare(progress, "0")); echo_queries = config().hasOption("echo") || config().hasOption("verbose"); ignore_error = config().getBool("ignore-error", false); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 95018f8a632..c8fcf560859 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2324,7 +2324,7 @@ void ClientBase::init(int argc, char ** argv) ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") ("query_id", po::value(), "query_id") - ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::TTY, "tty"), "Print progress of queries execution - to TTY (default): tty|on|1|true|yes; to STDERR: err; OFF: off|0|false|no") + ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::OFF, "off"), "Print progress of queries execution - to TTY (default): tty|on|1|true|yes; to STDERR: err; OFF: off|0|false|no") ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") diff --git a/tests/queries/0_stateless/02310_clickhouse_client_INSERT_progress_profile_events.expect b/tests/queries/0_stateless/02310_clickhouse_client_INSERT_progress_profile_events.expect index 20333ae7960..07815e57610 100755 --- a/tests/queries/0_stateless/02310_clickhouse_client_INSERT_progress_profile_events.expect +++ b/tests/queries/0_stateless/02310_clickhouse_client_INSERT_progress_profile_events.expect @@ -24,7 +24,7 @@ expect_after { spawn bash send "source $basedir/../shell_config.sh\r" -send "yes | head -n10000000 | \$CLICKHOUSE_CLIENT --query \"insert into function null('foo String') format TSV\" >/dev/null\r" +send "yes | head -n10000000 | \$CLICKHOUSE_CLIENT --progress --query \"insert into function null('foo String') format TSV\" >/dev/null\r" expect "Progress: " send "\3" diff --git a/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect b/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect index 5c95e17aefd..3333ee93468 100755 --- a/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect +++ b/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect @@ -24,7 +24,7 @@ expect_after { spawn bash send "source $basedir/../shell_config.sh\r" -send "yes | head -n10000000 | \$CLICKHOUSE_LOCAL --query \"insert into function null('foo String') format TSV\" >/dev/null\r" +send "yes | head -n10000000 | \$CLICKHOUSE_LOCAL --progress --query \"insert into function null('foo String') format TSV\" >/dev/null\r" expect "Progress: " send "\3" diff --git a/tests/queries/0_stateless/02456_progress_tty.expect b/tests/queries/0_stateless/02456_progress_tty.expect index 9daa6caa3fa..45156c6b0f5 100755 --- a/tests/queries/0_stateless/02456_progress_tty.expect +++ b/tests/queries/0_stateless/02456_progress_tty.expect @@ -17,32 +17,21 @@ expect_after { spawn bash send "source $basedir/../shell_config.sh\r" -# Progress is displayed by default -send "\$CLICKHOUSE_LOCAL --query 'SELECT sum(sleep(1) = 0) FROM numbers(3) SETTINGS max_block_size = 1' >/dev/null\r" -expect "Progress: " -expect "█" -send "\3" - -# It is true even if we redirect both stdout and stderr to /dev/null -send "\$CLICKHOUSE_LOCAL --query 'SELECT sum(sleep(1) = 0) FROM numbers(3) SETTINGS max_block_size = 1' >/dev/null 2>&1\r" -expect "Progress: " -expect "█" +# Progress is not displayed by default +send "\$CLICKHOUSE_LOCAL --query 'SELECT sleep(1), \$\$Hello\$\$ FROM numbers(3) SETTINGS max_block_size = 1' 2>/dev/null\r" +expect -exact "0\tHello\r\n" send "\3" # The option --progress has implicit value of true -send "\$CLICKHOUSE_LOCAL --progress --query 'SELECT sum(sleep(1) = 0) FROM numbers(3) SETTINGS max_block_size = 1' >/dev/null 2>&1\r" +send "\$CLICKHOUSE_LOCAL --progress --query 'SELECT sum(sleep(1) = 0) FROM numbers(3) SETTINGS max_block_size = 1' >/dev/null\r" expect "Progress: " expect "█" send "\3" -# But we can set it to false -send "\$CLICKHOUSE_LOCAL --progress false --query 'SELECT sleep(1), \$\$Hello\$\$ FROM numbers(3) SETTINGS max_block_size = 1' 2>/dev/null\r" -expect -exact "0\tHello\r\n" -send "\3" - -# As well as to 0 for the same effect -send "\$CLICKHOUSE_LOCAL --progress 0 --query 'SELECT sleep(1), \$\$Hello\$\$ FROM numbers(3) SETTINGS max_block_size = 1' 2>/dev/null\r" -expect -exact "0\tHello\r\n" +# It works even if we redirect both stdout and stderr to /dev/null +send "\$CLICKHOUSE_LOCAL --progress --query 'SELECT sum(sleep(1) = 0) FROM numbers(3) SETTINGS max_block_size = 1' >/dev/null 2>&1\r" +expect "Progress: " +expect "█" send "\3" # If we set it to 1, the progress will be displayed as well From 981ece08edf71b4bdfd16b5f71a102e0aed4fd24 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 10 Nov 2022 00:55:39 +0000 Subject: [PATCH 21/82] Rewrite tests for memory overcommit --- .../configs/global_overcommit_tracker.xml | 3 - .../test_global_overcommit_tracker/test.py | 52 ------------ .../__init__.py | 0 .../configs/global_overcommit_tracker.xml | 3 + .../test_overcommit_tracker/test.py | 85 +++++++++++++++++++ .../02104_overcommit_memory.reference | 1 - .../0_stateless/02104_overcommit_memory.sh | 49 ----------- 7 files changed, 88 insertions(+), 105 deletions(-) delete mode 100644 tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml delete mode 100644 tests/integration/test_global_overcommit_tracker/test.py rename tests/integration/{test_global_overcommit_tracker => test_overcommit_tracker}/__init__.py (100%) create mode 100644 tests/integration/test_overcommit_tracker/configs/global_overcommit_tracker.xml create mode 100644 tests/integration/test_overcommit_tracker/test.py delete mode 100644 tests/queries/0_stateless/02104_overcommit_memory.reference delete mode 100755 tests/queries/0_stateless/02104_overcommit_memory.sh diff --git a/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml b/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml deleted file mode 100644 index a05d8865a6b..00000000000 --- a/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 50000000 - \ No newline at end of file diff --git a/tests/integration/test_global_overcommit_tracker/test.py b/tests/integration/test_global_overcommit_tracker/test.py deleted file mode 100644 index 871f9ca983e..00000000000 --- a/tests/integration/test_global_overcommit_tracker/test.py +++ /dev/null @@ -1,52 +0,0 @@ -import pytest - -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance( - "node", main_configs=["configs/global_overcommit_tracker.xml"] -) - - -@pytest.fixture(scope="module", autouse=True) -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -TEST_QUERY_A = "SELECT number FROM numbers(1000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator_for_user=1, memory_usage_overcommit_max_wait_microseconds=500" -TEST_QUERY_B = "SELECT number FROM numbers(1000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator_for_user=2, memory_usage_overcommit_max_wait_microseconds=500" - - -def test_overcommited_is_killed(): - node.query("CREATE USER A") - node.query("GRANT ALL ON *.* TO A") - node.query("CREATE USER B") - node.query("GRANT ALL ON *.* TO B") - - responses_A = list() - responses_B = list() - for _ in range(500): - responses_A.append(node.get_query_request(TEST_QUERY_A, user="A")) - responses_B.append(node.get_query_request(TEST_QUERY_B, user="B")) - - overcommited_killed = False - for response in responses_A: - _, err = response.get_answer_and_error() - if "MEMORY_LIMIT_EXCEEDED" in err: - overcommited_killed = True - finished = False - for response in responses_B: - _, err = response.get_answer_and_error() - if err == "": - finished = True - - assert overcommited_killed, "no overcommited task was killed" - assert finished, "all tasks are killed" - - node.query("DROP USER IF EXISTS A") - node.query("DROP USER IF EXISTS B") diff --git a/tests/integration/test_global_overcommit_tracker/__init__.py b/tests/integration/test_overcommit_tracker/__init__.py similarity index 100% rename from tests/integration/test_global_overcommit_tracker/__init__.py rename to tests/integration/test_overcommit_tracker/__init__.py diff --git a/tests/integration/test_overcommit_tracker/configs/global_overcommit_tracker.xml b/tests/integration/test_overcommit_tracker/configs/global_overcommit_tracker.xml new file mode 100644 index 00000000000..6f83a570ccc --- /dev/null +++ b/tests/integration/test_overcommit_tracker/configs/global_overcommit_tracker.xml @@ -0,0 +1,3 @@ + + 2000000000 + \ No newline at end of file diff --git a/tests/integration/test_overcommit_tracker/test.py b/tests/integration/test_overcommit_tracker/test.py new file mode 100644 index 00000000000..0592915e036 --- /dev/null +++ b/tests/integration/test_overcommit_tracker/test.py @@ -0,0 +1,85 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", main_configs=["configs/global_overcommit_tracker.xml"] +) + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +GLOBAL_TEST_QUERY_A = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS memory_overcommit_ratio_denominator_for_user=1" +GLOBAL_TEST_QUERY_B = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS memory_overcommit_ratio_denominator_for_user=80000000" + +USER_TEST_QUERY_A = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS max_memory_usage_for_user=2000000000,memory_overcommit_ratio_denominator=1" +USER_TEST_QUERY_B = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS max_memory_usage_for_user=2000000000,memory_overcommit_ratio_denominator=80000000" + + +def test_user_overcommit(): + node.query("CREATE USER IF NOT EXISTS A") + node.query("GRANT ALL ON *.* TO A") + + responses_A = list() + responses_B = list() + for i in range(100): + if i % 2 == 0: + responses_A.append(node.get_query_request(USER_TEST_QUERY_A, user="A")) + else: + responses_B.append(node.get_query_request(USER_TEST_QUERY_B, user="A")) + + overcommited_killed = False + for response in responses_A: + _, err = response.get_answer_and_error() + if "MEMORY_LIMIT_EXCEEDED" in err: + overcommited_killed = True + finished = False + for response in responses_B: + _, err = response.get_answer_and_error() + if err == "": + finished = True + + assert overcommited_killed, "no overcommited task was killed" + assert finished, "all tasks are killed" + + node.query("DROP USER IF EXISTS A") + + +def test_global_overcommit(): + node.query("CREATE USER IF NOT EXISTS A") + node.query("GRANT ALL ON *.* TO A") + node.query("CREATE USER IF NOT EXISTS B") + node.query("GRANT ALL ON *.* TO B") + + responses_A = list() + responses_B = list() + for i in range(100): + if i % 2 == 0: + responses_A.append(node.get_query_request(GLOBAL_TEST_QUERY_A, user="A")) + else: + responses_B.append(node.get_query_request(GLOBAL_TEST_QUERY_B, user="B")) + + overcommited_killed = False + for response in responses_A: + _, err = response.get_answer_and_error() + if "MEMORY_LIMIT_EXCEEDED" in err: + overcommited_killed = True + finished = False + for response in responses_B: + _, err = response.get_answer_and_error() + if err == "": + finished = True + + assert overcommited_killed, "no overcommited task was killed" + assert finished, "all tasks are killed" + + node.query("DROP USER IF EXISTS A") + node.query("DROP USER IF EXISTS B") diff --git a/tests/queries/0_stateless/02104_overcommit_memory.reference b/tests/queries/0_stateless/02104_overcommit_memory.reference deleted file mode 100644 index b108f48e0fa..00000000000 --- a/tests/queries/0_stateless/02104_overcommit_memory.reference +++ /dev/null @@ -1 +0,0 @@ -OVERCOMMITED WITH USER LIMIT WAS KILLED diff --git a/tests/queries/0_stateless/02104_overcommit_memory.sh b/tests/queries/0_stateless/02104_overcommit_memory.sh deleted file mode 100755 index 384e30b1283..00000000000 --- a/tests/queries/0_stateless/02104_overcommit_memory.sh +++ /dev/null @@ -1,49 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-parallel, no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -$CLICKHOUSE_CLIENT -q 'CREATE USER IF NOT EXISTS u02104 IDENTIFIED WITH no_password' -$CLICKHOUSE_CLIENT -q 'GRANT ALL ON *.* TO u02104' - -function overcommited() -{ - while true; do - $CLICKHOUSE_CLIENT -u u02104 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator=1,memory_usage_overcommit_max_wait_microseconds=500' 2>&1 \ - | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo "OVERCOMMITED WITH USER LIMIT IS KILLED" - done -} - -function expect_execution() -{ - while true; do - $CLICKHOUSE_CLIENT -u u02104 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_memory_usage_for_user=5000000,memory_overcommit_ratio_denominator=2,memory_usage_overcommit_max_wait_microseconds=500' >/dev/null 2>/dev/null - done -} - -export -f overcommited -export -f expect_execution - -function user_test() -{ - for _ in {1..10}; - do - timeout 10 bash -c overcommited & - timeout 10 bash -c expect_execution & - done; - - wait -} - -output=$(user_test) - -if test -z "$output" -then - echo "OVERCOMMITED WITH USER LIMIT WAS NOT KILLED" -else - echo "OVERCOMMITED WITH USER LIMIT WAS KILLED" -fi - -$CLICKHOUSE_CLIENT -q 'DROP USER IF EXISTS u02104' From eb612a699d8acdd9f363c321b1d553e1a857d501 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 10 Nov 2022 01:08:12 +0000 Subject: [PATCH 22/82] Automatic style fix --- tests/integration/test_overcommit_tracker/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_overcommit_tracker/test.py b/tests/integration/test_overcommit_tracker/test.py index 0592915e036..5c18ee950d5 100644 --- a/tests/integration/test_overcommit_tracker/test.py +++ b/tests/integration/test_overcommit_tracker/test.py @@ -8,6 +8,7 @@ node = cluster.add_instance( "node", main_configs=["configs/global_overcommit_tracker.xml"] ) + @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: From a5a1f5abf7ca4084d66ede084cede34362d5f698 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Nov 2022 07:33:54 +0000 Subject: [PATCH 23/82] fix client.py - should use '--progress', return tests - they are working, fix help string --- src/Client/ClientBase.cpp | 2 +- tests/queries/0_stateless/02456_progress_tty.expect | 10 ++++++++++ tests/queries/0_stateless/helpers/client.py | 2 +- 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index c8fcf560859..5bd65ac2334 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2324,7 +2324,7 @@ void ClientBase::init(int argc, char ** argv) ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") ("query_id", po::value(), "query_id") - ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::OFF, "off"), "Print progress of queries execution - to TTY (default): tty|on|1|true|yes; to STDERR: err; OFF: off|0|false|no") + ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::OFF, "off"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR: err; OFF (default): off|0|false|no") ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") diff --git a/tests/queries/0_stateless/02456_progress_tty.expect b/tests/queries/0_stateless/02456_progress_tty.expect index 45156c6b0f5..ba6cc0537eb 100755 --- a/tests/queries/0_stateless/02456_progress_tty.expect +++ b/tests/queries/0_stateless/02456_progress_tty.expect @@ -34,6 +34,16 @@ expect "Progress: " expect "█" send "\3" +# But we can set it to false +send "\$CLICKHOUSE_LOCAL --progress false --query 'SELECT sleep(1), \$\$Hello\$\$ FROM numbers(3) SETTINGS max_block_size = 1' 2>/dev/null\r" +expect -exact "0\tHello\r\n" +send "\3" + +# As well as to 0 for the same effect +send "\$CLICKHOUSE_LOCAL --progress 0 --query 'SELECT sleep(1), \$\$Hello\$\$ FROM numbers(3) SETTINGS max_block_size = 1' 2>/dev/null\r" +expect -exact "0\tHello\r\n" +send "\3" + # If we set it to 1, the progress will be displayed as well send "\$CLICKHOUSE_LOCAL --progress 1 --query 'SELECT sum(sleep(1) = 0) FROM numbers(3) SETTINGS max_block_size = 1' >/dev/null 2>&1\r" expect "Progress: " diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index 5c8589dfca1..184b6b22cb6 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -16,7 +16,7 @@ class client(object): def __init__(self, command=None, name="", log=None): self.client = uexpect.spawn(["/bin/bash", "--noediting"]) if command is None: - command = os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client" + command = os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client --progress" self.client.command = command self.client.eol("\r") self.client.logger(log, prefix=name) From 055e655145ae9b7f1062cc76dc23b0ebe9684509 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 10 Nov 2022 07:41:13 +0000 Subject: [PATCH 24/82] Automatic style fix --- tests/queries/0_stateless/helpers/client.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index 184b6b22cb6..7c9b2c1c83f 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -16,7 +16,9 @@ class client(object): def __init__(self, command=None, name="", log=None): self.client = uexpect.spawn(["/bin/bash", "--noediting"]) if command is None: - command = os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client --progress" + command = ( + os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client --progress" + ) self.client.command = command self.client.eol("\r") self.client.logger(log, prefix=name) From c50e7957886b6fb7c8584a97d3bbd256316473ad Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 10 Nov 2022 11:04:51 +0000 Subject: [PATCH 25/82] Fix expected ZXID --- tests/integration/test_keeper_four_word_command/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 4559904f8b7..c425c18158b 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -367,7 +367,7 @@ def test_cmd_stat(started_cluster): assert result["Received"] == "10" assert result["Sent"] == "10" assert int(result["Connections"]) == 1 - assert int(result["Zxid"]) > 14 + assert int(result["Zxid"]) >= 10 assert result["Mode"] == "leader" assert result["Node count"] == "13" From 71437851b18f155b5e2620ab31480649427ab5fc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 10 Nov 2022 12:56:00 +0100 Subject: [PATCH 26/82] Update client.py --- tests/queries/0_stateless/helpers/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index 7c9b2c1c83f..26c82cad0d1 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -17,7 +17,7 @@ class client(object): self.client = uexpect.spawn(["/bin/bash", "--noediting"]) if command is None: command = ( - os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client --progress" + os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client" ) self.client.command = command self.client.eol("\r") From 98bb346a29af47aa323eec4788575dae4957893b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 10 Nov 2022 12:05:41 +0000 Subject: [PATCH 27/82] Automatic style fix --- tests/queries/0_stateless/helpers/client.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index 26c82cad0d1..5c8589dfca1 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -16,9 +16,7 @@ class client(object): def __init__(self, command=None, name="", log=None): self.client = uexpect.spawn(["/bin/bash", "--noediting"]) if command is None: - command = ( - os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client" - ) + command = os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client" self.client.command = command self.client.eol("\r") self.client.logger(log, prefix=name) From e2de80953285c0ec93da2eb4a8ed349c9be6f432 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 14 Sep 2022 15:40:45 +0200 Subject: [PATCH 28/82] Add typing to commit_status_helper.py --- tests/ci/commit_status_helper.py | 33 ++++++++++++++++---------------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 8b9d28502c1..cbd0fd4b3ce 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -3,19 +3,19 @@ import csv import os import time -from typing import Optional +from typing import List import logging from ci_config import CI_CONFIG, REQUIRED_CHECKS from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL from github import Github from github.Commit import Commit -from pr_info import SKIP_MERGEABLE_CHECK_LABEL +from pr_info import PRInfo, SKIP_MERGEABLE_CHECK_LABEL RETRY = 5 -def override_status(status, check_name, invert=False): +def override_status(status: str, check_name: str, invert=False) -> str: if CI_CONFIG["tests_config"].get(check_name, {}).get("force_tests", False): return "success" @@ -27,24 +27,23 @@ def override_status(status, check_name, invert=False): return status -def get_commit( - gh: Github, commit_sha: str, retry_count: int = RETRY -) -> Optional[Commit]: +def get_commit(gh: Github, commit_sha: str, retry_count: int = RETRY) -> Commit: for i in range(retry_count): try: repo = gh.get_repo(GITHUB_REPOSITORY) commit = repo.get_commit(commit_sha) - return commit + break except Exception as ex: if i == retry_count - 1: raise ex time.sleep(i) - # just suppress warning - return None + return commit -def post_commit_status(gh, sha, check_name, description, state, report_url): +def post_commit_status( + gh: Github, sha: str, check_name: str, description: str, state: str, report_url: str +): for i in range(RETRY): try: commit = get_commit(gh, sha, 1) @@ -61,7 +60,9 @@ def post_commit_status(gh, sha, check_name, description, state, report_url): time.sleep(i) -def post_commit_status_to_file(file_path, description, state, report_url): +def post_commit_status_to_file( + file_path: str, description: str, state: str, report_url: str +): if os.path.exists(file_path): raise Exception(f'File "{file_path}" already exists!') with open(file_path, "w", encoding="utf-8") as f: @@ -69,21 +70,21 @@ def post_commit_status_to_file(file_path, description, state, report_url): out.writerow([state, report_url, description]) -def remove_labels(gh, pr_info, labels_names): +def remove_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]): repo = gh.get_repo(GITHUB_REPOSITORY) pull_request = repo.get_pull(pr_info.number) for label in labels_names: pull_request.remove_from_labels(label) -def post_labels(gh, pr_info, labels_names): +def post_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]): repo = gh.get_repo(GITHUB_REPOSITORY) pull_request = repo.get_pull(pr_info.number) for label in labels_names: pull_request.add_to_labels(label) -def fail_mergeable_check(commit, description): +def fail_mergeable_check(commit: Commit, description: str): commit.create_status( context="Mergeable Check", description=description, @@ -92,7 +93,7 @@ def fail_mergeable_check(commit, description): ) -def reset_mergeable_check(commit, description=""): +def reset_mergeable_check(commit: Commit, description: str = ""): commit.create_status( context="Mergeable Check", description=description, @@ -101,7 +102,7 @@ def reset_mergeable_check(commit, description=""): ) -def update_mergeable_check(gh, pr_info, check_name): +def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str): if SKIP_MERGEABLE_CHECK_LABEL in pr_info.labels: return From eec74e46e42a41b11eb80555acb2cb2a0aaaf8eb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 14 Sep 2022 16:00:46 +0200 Subject: [PATCH 29/82] Move get_commit_filtered_statuses to commit_status_helper --- tests/ci/commit_status_helper.py | 18 ++++++++++++++++++ tests/ci/rerun_helper.py | 26 ++++---------------------- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index cbd0fd4b3ce..185dc64daa9 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -10,9 +10,11 @@ from ci_config import CI_CONFIG, REQUIRED_CHECKS from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL from github import Github from github.Commit import Commit +from github.CommitStatus import CommitStatus from pr_info import PRInfo, SKIP_MERGEABLE_CHECK_LABEL RETRY = 5 +CommitStatuses = List[CommitStatus] def override_status(status: str, check_name: str, invert=False) -> str: @@ -70,6 +72,22 @@ def post_commit_status_to_file( out.writerow([state, report_url, description]) +def get_commit_filtered_statuses(commit: Commit) -> CommitStatuses: + """ + Squash statuses to latest state + 1. context="first", state="success", update_time=1 + 2. context="second", state="success", update_time=2 + 3. context="first", stat="failure", update_time=3 + =========> + 1. context="second", state="success" + 2. context="first", stat="failure" + """ + filtered = {} + for status in sorted(commit.get_statuses(), key=lambda x: x.updated_at): + filtered[status.context] = status + return list(filtered.values()) + + def remove_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]): repo = gh.get_repo(GITHUB_REPOSITORY) pull_request = repo.get_pull(pr_info.number) diff --git a/tests/ci/rerun_helper.py b/tests/ci/rerun_helper.py index c4ae70eadb9..fa73256d759 100644 --- a/tests/ci/rerun_helper.py +++ b/tests/ci/rerun_helper.py @@ -1,14 +1,13 @@ #!/usr/bin/env python3 -from typing import List, Optional +from typing import Optional -from commit_status_helper import get_commit +from commit_status_helper import get_commit, get_commit_filtered_statuses from github import Github from github.CommitStatus import CommitStatus from pr_info import PRInfo -CommitStatuses = List[CommitStatus] - +# TODO: move it to commit_status_helper class RerunHelper: def __init__(self, gh: Github, pr_info: PRInfo, check_name: str): self.gh = gh @@ -18,7 +17,7 @@ class RerunHelper: if commit is None: raise ValueError(f"unable to receive commit for {pr_info.sha}") self.pygh_commit = commit - self.statuses = self.ger_filtered_statuses() + self.statuses = get_commit_filtered_statuses(commit) def is_already_finished_by_status(self) -> bool: # currently we agree even for failed statuses @@ -35,20 +34,3 @@ class RerunHelper: if self.check_name in status.context: return status return None - - def ger_filtered_statuses(self) -> CommitStatuses: - """ - Squash statuses to latest state - 1. context="first", state="success", update_time=1 - 2. context="second", state="success", update_time=2 - 3. context="first", stat="failure", update_time=3 - =========> - 1. context="second", state="success" - 2. context="first", stat="failure" - """ - filt = {} - for status in sorted( - self.pygh_commit.get_statuses(), key=lambda x: x.updated_at - ): - filt[status.context] = status - return list(filt.values()) From f8c907c463f5a53c79589d129a6e9a32f8b8fc6e Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Thu, 10 Nov 2022 13:14:04 +0100 Subject: [PATCH 30/82] Revert "Revert " Keeper retries during insert (clean)"" --- src/Common/ZooKeeper/TestKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Core/Settings.h | 5 + .../MergeTree/EphemeralLockInZooKeeper.cpp | 39 +- .../MergeTree/EphemeralLockInZooKeeper.h | 15 +- ...ReplicatedMergeTreeMergeStrategyPicker.cpp | 4 + .../MergeTree/ReplicatedMergeTreeSink.cpp | 247 +++++--- .../MergeTree/ReplicatedMergeTreeSink.h | 19 +- src/Storages/MergeTree/ZooKeeperRetries.h | 265 +++++++++ .../MergeTree/ZooKeeperWithFaultInjection.h | 527 ++++++++++++++++++ src/Storages/StorageReplicatedMergeTree.cpp | 93 +++- src/Storages/StorageReplicatedMergeTree.h | 35 +- tests/config/install.sh | 1 + .../config/users.d/insert_keeper_retries.xml | 8 + .../__init__.py | 0 .../test_inserts_with_keeper_retries/test.py | 100 ++++ .../integration/test_restore_replica/test.py | 9 +- .../00121_drop_column_zookeeper.sql | 6 +- ...partitioning_replicated_zookeeper_long.sql | 47 +- .../00652_replicated_mutations_zookeeper.sh | 24 +- ...replicated_without_partition_zookeeper.sql | 8 +- ..._fetch_merged_or_mutated_part_zookeeper.sh | 9 +- ...ated_minimalistic_part_header_zookeeper.sh | 1 + .../01037_zookeeper_check_table_empty_pk.sql | 9 +- ...eeper_system_mutations_with_parts_names.sh | 3 +- ...eeper_mutations_and_insert_quorum_long.sql | 11 +- .../01135_default_and_alter_zookeeper.sql | 4 +- ...mutation_stuck_after_replace_partition.sql | 2 + .../0_stateless/01158_zookeeper_log_long.sql | 7 +- ...compact_part_replicated_zookeeper_long.sql | 4 + ...18_long_unsuccessful_mutation_zookeeper.sh | 9 +- ...artition_key_replicated_zookeeper_long.sql | 13 +- ...1417_freeze_partition_verbose_zookeeper.sh | 8 +- ...replicated_detach_drop_and_quorum_long.sql | 1 + ...01451_replicated_detach_drop_part_long.sql | 9 +- .../01459_manual_write_to_replicas.sh | 10 +- .../01459_manual_write_to_replicas_quorum.sh | 6 +- ...check_many_parallel_quorum_inserts_long.sh | 4 +- .../01509_parallel_quorum_and_merge_long.sh | 13 +- ...arallel_quorum_insert_no_replicas_long.sql | 13 +- ..._execute_merges_on_single_replica_long.sql | 2 + ...primary_key_without_order_by_zookeeper.sql | 12 +- ...6_replicated_mutations_empty_partition.sql | 7 +- .../01593_concurrent_alter_mutations_kill.sh | 6 +- ..._part_and_deduplication_zookeeper_long.sql | 5 +- ...kground_checker_blather_zookeeper_long.sql | 8 +- .../01810_max_part_removal_threads_long.sh | 3 + .../0_stateless/01825_type_json_3.sql.j2 | 2 + .../01870_modulo_partition_key.sql | 9 +- .../02228_merge_tree_insert_memory_usage.sql | 5 +- .../02340_parts_refcnt_mergetree.sh | 4 +- .../02369_lost_part_intersecting_merges.sh | 16 +- .../02448_clone_replica_lost_part.sql | 2 + ...456_keeper_retries_during_insert.reference | 2 + .../02456_keeper_retries_during_insert.sql | 26 + 55 files changed, 1451 insertions(+), 250 deletions(-) create mode 100644 src/Storages/MergeTree/ZooKeeperRetries.h create mode 100644 src/Storages/MergeTree/ZooKeeperWithFaultInjection.h create mode 100644 tests/config/users.d/insert_keeper_retries.xml create mode 100644 tests/integration/test_inserts_with_keeper_retries/__init__.py create mode 100644 tests/integration/test_inserts_with_keeper_retries/test.py create mode 100644 tests/queries/0_stateless/02456_keeper_retries_during_insert.reference create mode 100644 tests/queries/0_stateless/02456_keeper_retries_during_insert.sql diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index aad5131fcb5..62615afb4f7 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -34,7 +34,7 @@ using TestKeeperRequestPtr = std::shared_ptr; class TestKeeper final : public IKeeper { public: - TestKeeper(const zkutil::ZooKeeperArgs & args_); + explicit TestKeeper(const zkutil::ZooKeeperArgs & args_); ~TestKeeper() override; bool isExpired() const override { return expired; } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index c9529b78f8a..9de8241cfbe 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -156,7 +156,7 @@ public: using Ptr = std::shared_ptr; using ErrorsList = std::initializer_list; - ZooKeeper(const ZooKeeperArgs & args_, std::shared_ptr zk_log_ = nullptr); + explicit ZooKeeper(const ZooKeeperArgs & args_, std::shared_ptr zk_log_ = nullptr); /** Config of the form: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8c67cf94a61..45ca9845afb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -658,6 +658,11 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, multiple_joins_try_to_keep_original_names, false, "Do not add aliases to top level expression list on multiple joins rewrite", 0) \ M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, optimize_sorting_by_input_stream_properties, true, "Optimize sorting by sorting properties of input stream", 0) \ + M(UInt64, insert_keeper_max_retries, 0, "Max retries for keeper operations during insert", 0) \ + M(UInt64, insert_keeper_retry_initial_backoff_ms, 100, "Initial backoff timeout for keeper operations during insert", 0) \ + M(UInt64, insert_keeper_retry_max_backoff_ms, 10000, "Max backoff timeout for keeper operations during insert", 0) \ + M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ + M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 6ddb35d109e..6d75f523056 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -12,22 +13,22 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & path_) - : zookeeper(&zookeeper_), path_prefix(path_prefix_), path(path_) +EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & path_) + : zookeeper(zookeeper_), path_prefix(path_prefix_), path(path_) { if (path.size() <= path_prefix.size()) throw Exception("Logical error: name of the main node is shorter than prefix.", ErrorCodes::LOGICAL_ERROR); } std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path) + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path) { String path; if (deduplication_path.empty()) { String holder_path = temp_path + "/" + EphemeralLockInZooKeeper::LEGACY_LOCK_OTHER; - path = zookeeper_.create(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential); + path = zookeeper_->create(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential); } else { @@ -39,11 +40,15 @@ std::optional createEphemeralLockInZooKeeper( ops.emplace_back(zkutil::makeRemoveRequest(deduplication_path, -1)); ops.emplace_back(zkutil::makeCreateRequest(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential)); Coordination::Responses responses; - Coordination::Error e = zookeeper_.tryMulti(ops, responses); + Coordination::Error e = zookeeper_->tryMulti(ops, responses); if (e != Coordination::Error::ZOK) { if (responses[0]->error == Coordination::Error::ZNODEEXISTS) { + LOG_DEBUG( + &Poco::Logger::get("createEphemeralLockInZooKeeper"), + "Deduplication path already exists: deduplication_path={}", + deduplication_path); return {}; } else @@ -82,9 +87,31 @@ EphemeralLockInZooKeeper::~EphemeralLockInZooKeeper() { unlock(); } + catch (const zkutil::KeeperException & e) + { + if (Coordination::isHardwareError(e.code)) + LOG_DEBUG( + &Poco::Logger::get("EphemeralLockInZooKeeper"), + "ZooKeeper communication error during unlock: code={} message='{}'", + e.code, + e.message()); + else if (e.code == Coordination::Error::ZNONODE) + /// To avoid additional round-trip for unlocking, + /// ephemeral node can be deleted explicitly as part of another multi op request to ZK + /// and marked as such via assumeUnlocked() if we got successful response. + /// But it's possible that the multi op request can be executed on server side, and client will not get response due to network issue. + /// In such case, assumeUnlocked() will not be called, so we'll get ZNONODE error here since the noded is already deleted + LOG_DEBUG( + &Poco::Logger::get("EphemeralLockInZooKeeper"), + "ZooKeeper node was already deleted: code={} message={}", + e.code, + e.message()); + else + tryLogCurrentException("EphemeralLockInZooKeeper"); + } catch (...) { - tryLogCurrentException("~EphemeralLockInZooKeeper"); + tryLogCurrentException("EphemeralLockInZooKeeper"); } } diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h index 5945fa10d91..c630bcc4681 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h @@ -12,6 +12,8 @@ namespace DB { +class ZooKeeperWithFaultInjection; +using ZooKeeperWithFaultInjectionPtr = std::shared_ptr; namespace ErrorCodes { @@ -25,13 +27,14 @@ namespace ErrorCodes class EphemeralLockInZooKeeper : public boost::noncopyable { friend std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path); + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path); protected: - EphemeralLockInZooKeeper() = delete; - EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & path_); + EphemeralLockInZooKeeper(const String & path_prefix_, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & path_); public: + EphemeralLockInZooKeeper() = delete; + /// Fake "secondary node" names for blocks with and without "deduplication_path" static constexpr const char * LEGACY_LOCK_INSERT = "abandonable_lock-insert"; static constexpr const char * LEGACY_LOCK_OTHER = "abandonable_lock-other"; @@ -53,7 +56,7 @@ public: bool isLocked() const { - return zookeeper; + return zookeeper.get(); } String getPath() const @@ -91,13 +94,13 @@ public: ~EphemeralLockInZooKeeper(); private: - zkutil::ZooKeeper * zookeeper = nullptr; + ZooKeeperWithFaultInjectionPtr zookeeper; String path_prefix; String path; }; std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path); + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path); /// Acquires block number locks in all partitions. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp index 626295d7255..192f0d23f96 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp @@ -112,6 +112,8 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() && now - last_refresh_time < REFRESH_STATE_MINIMUM_INTERVAL_SECONDS) return; + LOG_DEBUG(storage.log, "Updating strategy picker state"); + auto zookeeper = storage.getZooKeeper(); auto all_replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas"); @@ -154,6 +156,8 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() last_refresh_time = now; current_replica_index = current_replica_index_tmp; active_replicas = active_replicas_tmp; + + LOG_DEBUG(storage.log, "Strategy picker state updated, current replica: {}, active replicas: [{}]", current_replica_index, fmt::join(active_replicas, ", ")); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 5482052e184..b6a486f14ee 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -9,7 +9,6 @@ #include #include - namespace ProfileEvents { extern const Event DuplicatedInsertedBlocks; @@ -32,6 +31,7 @@ namespace ErrorCodes extern const int DUPLICATE_DATA_PART; extern const int PART_IS_TEMPORARILY_LOCKED; extern const int LOGICAL_ERROR; + extern const int TABLE_IS_READ_ONLY; extern const int QUERY_WAS_CANCELLED; } @@ -84,7 +84,7 @@ ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( ReplicatedMergeTreeSink::~ReplicatedMergeTreeSink() = default; /// Allow to verify that the session in ZooKeeper is still alive. -static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) +static void assertSessionIsNotExpired(const zkutil::ZooKeeperPtr & zookeeper) { if (!zookeeper) throw Exception("No ZooKeeper session.", ErrorCodes::NO_ZOOKEEPER); @@ -93,7 +93,7 @@ static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) throw Exception("ZooKeeper session has been expired.", ErrorCodes::NO_ZOOKEEPER); } -size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper) +size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper) { if (!isQuorumEnabled()) return 0; @@ -103,6 +103,7 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & z Strings replicas = zookeeper->getChildren(fs::path(storage.zookeeper_path) / "replicas"); Strings exists_paths; + exists_paths.reserve(replicas.size()); for (const auto & replica : replicas) if (replica != storage.replica_name) exists_paths.emplace_back(fs::path(storage.zookeeper_path) / "replicas" / replica / "is_active"); @@ -110,20 +111,28 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & z auto exists_result = zookeeper->exists(exists_paths); auto get_results = zookeeper->get(Strings{storage.replica_path + "/is_active", storage.replica_path + "/host"}); + Coordination::Error keeper_error = Coordination::Error::ZOK; size_t active_replicas = 1; /// Assume current replica is active (will check below) for (size_t i = 0; i < exists_paths.size(); ++i) { - auto status = exists_result[i]; - if (status.error == Coordination::Error::ZOK) + auto error = exists_result[i].error; + if (error == Coordination::Error::ZOK) ++active_replicas; + else if (Coordination::isHardwareError(error)) + keeper_error = error; } size_t replicas_number = replicas.size(); size_t quorum_size = getQuorumSize(replicas_number); if (active_replicas < quorum_size) + { + if (Coordination::isHardwareError(keeper_error)) + throw Coordination::Exception("Failed to check number of alive replicas", keeper_error); + throw Exception(ErrorCodes::TOO_FEW_LIVE_REPLICAS, "Number of alive replicas ({}) is less than requested quorum ({}/{}).", active_replicas, quorum_size, replicas_number); + } /** Is there a quorum for the last part for which a quorum is needed? * Write of all the parts with the included quorum is linearly ordered. @@ -156,15 +165,34 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) { auto block = getHeader().cloneWithColumns(chunk.detachColumns()); - auto zookeeper = storage.getZooKeeper(); - assertSessionIsNotExpired(zookeeper); + const auto & settings = context->getSettingsRef(); + zookeeper_retries_info = ZooKeeperRetriesInfo( + "ReplicatedMergeTreeSink::consume", + settings.insert_keeper_max_retries ? log : nullptr, + settings.insert_keeper_max_retries, + settings.insert_keeper_retry_initial_backoff_ms, + settings.insert_keeper_retry_max_backoff_ms); + + ZooKeeperWithFaultInjectionPtr zookeeper = ZooKeeperWithFaultInjection::createInstance( + settings.insert_keeper_fault_injection_probability, + settings.insert_keeper_fault_injection_seed, + storage.getZooKeeper(), + "ReplicatedMergeTreeSink::consume", + log); /** If write is with quorum, then we check that the required number of replicas is now live, * and also that for all previous parts for which quorum is required, this quorum is reached. * And also check that during the insertion, the replica was not reinitialized or disabled (by the value of `is_active` node). * TODO Too complex logic, you can do better. */ - size_t replicas_num = checkQuorumPrecondition(zookeeper); + size_t replicas_num = 0; + ZooKeeperRetriesControl quorum_retries_ctl("checkQuorumPrecondition", zookeeper_retries_info); + quorum_retries_ctl.retryLoop( + [&]() + { + zookeeper->setKeeper(storage.getZooKeeper()); + replicas_num = checkQuorumPrecondition(zookeeper); + }); if (!storage_snapshot->object_columns.empty()) convertDynamicColumnsToTuples(block, storage_snapshot); @@ -176,7 +204,6 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) size_t streams = 0; bool support_parallel_write = false; - const Settings & settings = context->getSettingsRef(); for (auto & current_block : part_blocks) { @@ -256,7 +283,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) finishDelayedChunk(zookeeper); } -void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper) +void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) { if (!delayed_chunk) return; @@ -271,7 +298,7 @@ void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeepe try { - commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num); + commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num, false); last_block_is_duplicate = last_block_is_duplicate || part->is_duplicate; @@ -294,8 +321,9 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt { /// NOTE: No delay in this case. That's Ok. - auto zookeeper = storage.getZooKeeper(); - assertSessionIsNotExpired(zookeeper); + auto origin_zookeeper = storage.getZooKeeper(); + assertSessionIsNotExpired(origin_zookeeper); + auto zookeeper = std::make_shared(origin_zookeeper); size_t replicas_num = checkQuorumPrecondition(zookeeper); @@ -304,7 +332,7 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt try { part->version.setCreationTID(Tx::PrehistoricTID, nullptr); - commitPart(zookeeper, part, "", replicas_num); + commitPart(zookeeper, part, "", replicas_num, true); PartLog::addNewPart(storage.getContext(), part, watch.elapsed()); } catch (...) @@ -315,10 +343,11 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt } void ReplicatedMergeTreeSink::commitPart( - zkutil::ZooKeeperPtr & zookeeper, + const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id, - size_t replicas_num) + size_t replicas_num, + bool writing_existing_part) { /// It is possible that we alter a part with different types of source columns. /// In this case, if column was not altered, the result type will be different with what we have in metadata. @@ -326,8 +355,6 @@ void ReplicatedMergeTreeSink::commitPart( /// /// metadata_snapshot->check(part->getColumns()); - assertSessionIsNotExpired(zookeeper); - String temporary_part_relative_path = part->getDataPartStorage().getPartDirectory(); /// There is one case when we need to retry transaction in a loop. @@ -337,14 +364,65 @@ void ReplicatedMergeTreeSink::commitPart( bool is_already_existing_part = false; - while (true) + /// for retries due to keeper error + bool part_committed_locally_but_zookeeper = false; + Coordination::Error write_part_info_keeper_error = Coordination::Error::ZOK; + + ZooKeeperRetriesControl retries_ctl("commitPart", zookeeper_retries_info); + retries_ctl.retryLoop([&]() { + zookeeper->setKeeper(storage.getZooKeeper()); + if (storage.is_readonly) + { + /// stop retries if in shutdown + if (storage.shutdown_called) + throw Exception( + ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to shutdown: replica_path={}", storage.replica_path); + + /// When we attach existing parts it's okay to be in read-only mode + /// For example during RESTORE REPLICA. + if (!writing_existing_part) + { + retries_ctl.setUserError(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode: replica_path={}", storage.replica_path); + return; + } + } + + if (retries_ctl.isRetry()) + { + /// If we are retrying, check if last iteration was actually successful, + /// we could get network error on committing part to zk + /// but the operation could be completed by zk server + + /// If this flag is true, then part is in Active state, and we'll not retry anymore + /// we only check if part was committed to zk and return success or failure correspondingly + /// Note: if commit to zk failed then cleanup thread will mark the part as Outdated later + if (part_committed_locally_but_zookeeper) + { + /// check that info about the part was actually written in zk + if (zookeeper->exists(fs::path(storage.replica_path) / "parts" / part->name)) + { + LOG_DEBUG(log, "Part was successfully committed on previous iteration: part_id={}", part->name); + } + else + { + retries_ctl.setUserError( + ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, + "Insert failed due to zookeeper error. Please retry. Reason: {}", + Coordination::errorMessage(write_part_info_keeper_error)); + } + + retries_ctl.stopRetries(); + return; + } + } + /// Obtain incremental block number and lock it. The lock holds our intention to add the block to the filesystem. /// We remove the lock just after renaming the part. In case of exception, block number will be marked as abandoned. /// Also, make deduplication check. If a duplicate is detected, no nodes are created. /// Allocate new block number and check for duplicates - bool deduplicate_block = !block_id.empty(); + const bool deduplicate_block = !block_id.empty(); String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : ""; auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); ThreadFuzzer::maybeInjectSleep(); @@ -468,7 +546,13 @@ void ReplicatedMergeTreeSink::commitPart( else quorum_path = storage.zookeeper_path + "/quorum/status"; - waitForQuorum(zookeeper, existing_part_name, quorum_path, quorum_info.is_active_node_version, replicas_num); + if (!retries_ctl.callAndCatchAll( + [&]() + { + waitForQuorum( + zookeeper, existing_part_name, quorum_path, quorum_info.is_active_node_version, replicas_num); + })) + return; } else { @@ -477,6 +561,7 @@ void ReplicatedMergeTreeSink::commitPart( return; } + LOG_INFO(log, "Block with ID {} already exists on other replicas as part {}; will write it locally with that name.", block_id, existing_part_name); @@ -508,8 +593,7 @@ void ReplicatedMergeTreeSink::commitPart( } catch (const Exception & e) { - if (e.code() != ErrorCodes::DUPLICATE_DATA_PART - && e.code() != ErrorCodes::PART_IS_TEMPORARILY_LOCKED) + if (e.code() != ErrorCodes::DUPLICATE_DATA_PART && e.code() != ErrorCodes::PART_IS_TEMPORARILY_LOCKED) throw; } @@ -526,15 +610,26 @@ void ReplicatedMergeTreeSink::commitPart( part->name); } - ThreadFuzzer::maybeInjectSleep(); + try + { + ThreadFuzzer::maybeInjectSleep(); + storage.lockSharedData(*part, zookeeper, false, {}); + ThreadFuzzer::maybeInjectSleep(); + } + catch (const Exception &) + { + transaction.rollbackPartsToTemporaryState(); - storage.lockSharedData(*part, false, {}); + part->is_temp = true; + part->renameTo(temporary_part_relative_path, false); + + throw; + } ThreadFuzzer::maybeInjectSleep(); Coordination::Responses responses; Coordination::Error multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT - if (multi_code == Coordination::Error::ZOK) { transaction.commit(); @@ -549,18 +644,32 @@ void ReplicatedMergeTreeSink::commitPart( throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Insert query (for block {}) was cancelled by concurrent ALTER PARTITION", block_number_lock->getPath()); } - else if (multi_code == Coordination::Error::ZCONNECTIONLOSS - || multi_code == Coordination::Error::ZOPERATIONTIMEOUT) + else if (Coordination::isHardwareError(multi_code)) { + write_part_info_keeper_error = multi_code; /** If the connection is lost, and we do not know if the changes were applied, we can not delete the local part - * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. - */ + * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. + */ transaction.commit(); - storage.enqueuePartForCheck(part->name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); + + /// Setting this flag is point of no return + /// On next retry, we'll just check if actually operation succeed or failed + /// and return ok or error correspondingly + part_committed_locally_but_zookeeper = true; + + /// if all retries will be exhausted by accessing zookeeper on fresh retry -> we'll add committed part to queue in the action + /// here lambda capture part name, it's ok since we'll not generate new one for this insert, + /// see comments around 'part_committed_locally_but_zookeeper' flag + retries_ctl.actionAfterLastFailedRetry( + [&storage = storage, part_name = part->name]() + { storage.enqueuePartForCheck(part_name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); }); /// We do not know whether or not data has been inserted. - throw Exception("Unknown status, client must retry. Reason: " + String(Coordination::errorMessage(multi_code)), - ErrorCodes::UNKNOWN_STATUS_OF_INSERT); + retries_ctl.setUserError( + ErrorCodes::UNKNOWN_STATUS_OF_INSERT, + "Unknown status, client must retry. Reason: {}", + Coordination::errorMessage(multi_code)); + return; } else if (Coordination::isUserError(multi_code)) { @@ -580,62 +689,72 @@ void ReplicatedMergeTreeSink::commitPart( part->renameTo(temporary_part_relative_path, false); /// If this part appeared on other replica than it's better to try to write it locally one more time. If it's our part - /// than it will be ignored on the next itration. + /// than it will be ignored on the next iteration. ++loop_counter; if (loop_counter == max_iterations) { part->is_duplicate = true; /// Part is duplicate, just remove it from local FS throw Exception("Too many transaction retries - it may indicate an error", ErrorCodes::DUPLICATE_DATA_PART); } - continue; + retries_ctl.requestUnconditionalRetry(); /// we want one more iteration w/o counting it as a try and timeout + return; } else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == quorum_info.status_path) { - storage.unlockSharedData(*part); + storage.unlockSharedData(*part, zookeeper); transaction.rollback(); throw Exception("Another quorum insert has been already started", ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); } else { - storage.unlockSharedData(*part); + storage.unlockSharedData(*part, zookeeper); /// NOTE: We could be here if the node with the quorum existed, but was quickly removed. transaction.rollback(); - throw Exception("Unexpected logical error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + Coordination::errorMessage(multi_code) + ", path " + failed_op_path, - ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + throw Exception( + ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, + "Unexpected logical error while adding block {} with ID '{}': {}, path {}", + block_number, + block_id, + Coordination::errorMessage(multi_code), + failed_op_path); } } - else if (Coordination::isHardwareError(multi_code)) - { - storage.unlockSharedData(*part); - transaction.rollback(); - throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); - } else { - storage.unlockSharedData(*part); + storage.unlockSharedData(*part, zookeeper); transaction.rollback(); - throw Exception("Unexpected ZooKeeper error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + throw Exception( + ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, + "Unexpected ZooKeeper error while adding block {} with ID '{}': {}", + block_number, + block_id, + Coordination::errorMessage(multi_code)); } - - break; - } + }, + [&zookeeper]() { zookeeper->cleanupEphemeralNodes(); }); if (isQuorumEnabled()) { - if (is_already_existing_part) + ZooKeeperRetriesControl quorum_retries_ctl("waitForQuorum", zookeeper_retries_info); + quorum_retries_ctl.retryLoop([&]() { - /// We get duplicate part without fetch - /// Check if this quorum insert is parallel or not - if (zookeeper->exists(storage.zookeeper_path + "/quorum/parallel/" + part->name)) - storage.updateQuorum(part->name, true); - else if (zookeeper->exists(storage.zookeeper_path + "/quorum/status")) - storage.updateQuorum(part->name, false); - } + zookeeper->setKeeper(storage.getZooKeeper()); - waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_version, replicas_num); + if (is_already_existing_part) + { + /// We get duplicate part without fetch + /// Check if this quorum insert is parallel or not + if (zookeeper->exists(storage.zookeeper_path + "/quorum/parallel/" + part->name)) + storage.updateQuorum(part->name, true); + else if (zookeeper->exists(storage.zookeeper_path + "/quorum/status")) + storage.updateQuorum(part->name, false); + } + + if (!quorum_retries_ctl.callAndCatchAll( + [&]() + { waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_version, replicas_num); })) + return; + }); } } @@ -650,11 +769,11 @@ void ReplicatedMergeTreeSink::onFinish() { auto zookeeper = storage.getZooKeeper(); assertSessionIsNotExpired(zookeeper); - finishDelayedChunk(zookeeper); + finishDelayedChunk(std::make_shared(zookeeper)); } void ReplicatedMergeTreeSink::waitForQuorum( - zkutil::ZooKeeperPtr & zookeeper, + const ZooKeeperWithFaultInjectionPtr & zookeeper, const std::string & part_name, const std::string & quorum_path, Int32 is_active_node_version, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index e3ec5d14a52..2154c7693f2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace Poco { class Logger; } @@ -60,6 +62,7 @@ public: } private: + ZooKeeperRetriesInfo zookeeper_retries_info; struct QuorumInfo { String status_path; @@ -71,20 +74,24 @@ private: /// Checks active replicas. /// Returns total number of replicas. - size_t checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper); + size_t checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper); /// Rename temporary part and commit to ZooKeeper. void commitPart( - zkutil::ZooKeeperPtr & zookeeper, + const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id, - size_t replicas_num); + size_t replicas_num, + bool writing_existing_part); /// Wait for quorum to be satisfied on path (quorum_path) form part (part_name) /// Also checks that replica still alive. void waitForQuorum( - zkutil::ZooKeeperPtr & zookeeper, const std::string & part_name, - const std::string & quorum_path, int is_active_node_version, size_t replicas_num) const; + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const std::string & part_name, + const std::string & quorum_path, + int is_active_node_version, + size_t replicas_num) const; StorageReplicatedMergeTree & storage; StorageMetadataPtr metadata_snapshot; @@ -116,7 +123,7 @@ private: struct DelayedChunk; std::unique_ptr delayed_chunk; - void finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper); + void finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper); }; } diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h new file mode 100644 index 00000000000..22ace074245 --- /dev/null +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -0,0 +1,265 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int OK; +} + +struct ZooKeeperRetriesInfo +{ + ZooKeeperRetriesInfo() = default; + ZooKeeperRetriesInfo(std::string name_, Poco::Logger * logger_, UInt64 max_retries_, UInt64 initial_backoff_ms_, UInt64 max_backoff_ms_) + : name(std::move(name_)) + , logger(logger_) + , max_retries(max_retries_) + , curr_backoff_ms(std::min(initial_backoff_ms_, max_backoff_ms_)) + , max_backoff_ms(max_backoff_ms_) + { + } + + std::string name; + Poco::Logger * logger = nullptr; + UInt64 max_retries = 0; + UInt64 curr_backoff_ms = 0; + UInt64 max_backoff_ms = 0; + UInt64 retry_count = 0; +}; + +class ZooKeeperRetriesControl +{ +public: + ZooKeeperRetriesControl(std::string name_, ZooKeeperRetriesInfo & retries_info_) : name(std::move(name_)), retries_info(retries_info_) + { + } + + void retryLoop(auto && f) + { + retryLoop(f, []() {}); + } + + void retryLoop(auto && f, auto && iteration_cleanup) + { + while (canTry()) + { + try + { + f(); + iteration_cleanup(); + } + catch (const zkutil::KeeperException & e) + { + iteration_cleanup(); + + if (!Coordination::isHardwareError(e.code)) + throw; + + setKeeperError(e.code, e.message()); + } + catch (...) + { + iteration_cleanup(); + throw; + } + } + } + + bool callAndCatchAll(auto && f) + { + try + { + f(); + return true; + } + catch (const zkutil::KeeperException & e) + { + setKeeperError(e.code, e.message()); + } + catch (const Exception & e) + { + setUserError(e.code(), e.what()); + } + return false; + } + + void setUserError(int code, std::string message) + { + if (retries_info.logger) + LOG_TRACE( + retries_info.logger, "ZooKeeperRetriesControl: {}/{}: setUserError: error={} message={}", retries_info.name, name, code, message); + + /// if current iteration is already failed, keep initial error + if (!iteration_succeeded) + return; + + iteration_succeeded = false; + user_error.code = code; + user_error.message = std::move(message); + keeper_error = KeeperError{}; + } + + template + void setUserError(int code, fmt::format_string fmt, Args &&... args) + { + setUserError(code, fmt::format(fmt, std::forward(args)...)); + } + + void setKeeperError(Coordination::Error code, std::string message) + { + if (retries_info.logger) + LOG_TRACE( + retries_info.logger, "ZooKeeperRetriesControl: {}/{}: setKeeperError: error={} message={}", retries_info.name, name, code, message); + + /// if current iteration is already failed, keep initial error + if (!iteration_succeeded) + return; + + iteration_succeeded = false; + keeper_error.code = code; + keeper_error.message = std::move(message); + user_error = UserError{}; + } + + void stopRetries() { stop_retries = true; } + + void requestUnconditionalRetry() { unconditional_retry = true; } + + bool isLastRetry() const { return retries_info.retry_count >= retries_info.max_retries; } + + bool isRetry() const { return retries_info.retry_count > 0; } + + Coordination::Error getLastKeeperErrorCode() const { return keeper_error.code; } + + /// action will be called only once and only after latest failed retry + void actionAfterLastFailedRetry(std::function f) { action_after_last_failed_retry = std::move(f); } + +private: + struct KeeperError + { + using Code = Coordination::Error; + Code code = Code::ZOK; + std::string message; + }; + + struct UserError + { + int code = ErrorCodes::OK; + std::string message; + }; + + bool canTry() + { + ++iteration_count; + /// first iteration is ordinary execution, no further checks needed + if (0 == iteration_count) + return true; + + if (unconditional_retry) + { + unconditional_retry = false; + return true; + } + + /// iteration succeeded -> no need to retry + if (iteration_succeeded) + { + /// avoid unnecessary logs, - print something only in case of retries + if (retries_info.logger && iteration_count > 1) + LOG_DEBUG( + retries_info.logger, + "ZooKeeperRetriesControl: {}/{}: succeeded after: iterations={} total_retries={}", + retries_info.name, + name, + iteration_count, + retries_info.retry_count); + return false; + } + + if (stop_retries) + { + logLastError("stop retries on request"); + action_after_last_failed_retry(); + throwIfError(); + return false; + } + + if (retries_info.retry_count >= retries_info.max_retries) + { + logLastError("retry limit is reached"); + action_after_last_failed_retry(); + throwIfError(); + return false; + } + + /// retries + ++retries_info.retry_count; + logLastError("will retry due to error"); + sleepForMilliseconds(retries_info.curr_backoff_ms); + retries_info.curr_backoff_ms = std::min(retries_info.curr_backoff_ms * 2, retries_info.max_backoff_ms); + + /// reset the flag, it will be set to false in case of error + iteration_succeeded = true; + + return true; + } + + void throwIfError() const + { + if (user_error.code != ErrorCodes::OK) + throw Exception(user_error.code, user_error.message); + + if (keeper_error.code != KeeperError::Code::ZOK) + throw zkutil::KeeperException(keeper_error.code, keeper_error.message); + } + + void logLastError(std::string_view header) + { + if (user_error.code == ErrorCodes::OK) + { + if (retries_info.logger) + LOG_DEBUG( + retries_info.logger, + "ZooKeeperRetriesControl: {}/{}: {}: retry_count={} timeout={}ms error={} message={}", + retries_info.name, + name, + header, + retries_info.retry_count, + retries_info.curr_backoff_ms, + keeper_error.code, + keeper_error.message); + } + else + { + if (retries_info.logger) + LOG_DEBUG( + retries_info.logger, + "ZooKeeperRetriesControl: {}/{}: {}: retry_count={} timeout={}ms error={} message={}", + retries_info.name, + name, + header, + retries_info.retry_count, + retries_info.curr_backoff_ms, + user_error.code, + user_error.message); + } + } + + + std::string name; + ZooKeeperRetriesInfo & retries_info; + Int64 iteration_count = -1; + UserError user_error; + KeeperError keeper_error; + std::function action_after_last_failed_retry = []() {}; + bool unconditional_retry = false; + bool iteration_succeeded = true; + bool stop_retries = false; +}; + +} diff --git a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h new file mode 100644 index 00000000000..491f97b52bd --- /dev/null +++ b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h @@ -0,0 +1,527 @@ +#pragma once +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +class RandomFaultInjection +{ +public: + RandomFaultInjection(double probability, UInt64 seed_) : rndgen(seed_), distribution(probability) { } + + void beforeOperation() + { + if (distribution(rndgen)) + throw zkutil::KeeperException("Fault injection before operation", Coordination::Error::ZSESSIONEXPIRED); + } + void afterOperation() + { + if (distribution(rndgen)) + throw zkutil::KeeperException("Fault injection after operation", Coordination::Error::ZOPERATIONTIMEOUT); + } + +private: + std::mt19937_64 rndgen; + std::bernoulli_distribution distribution; +}; + +/// +/// ZooKeeperWithFaultInjection mimics ZooKeeper interface and inject failures according to failure policy if set +/// +class ZooKeeperWithFaultInjection +{ + using zk = zkutil::ZooKeeper; + + zk::Ptr keeper; + zk::Ptr keeper_prev; + std::unique_ptr fault_policy; + std::string name; + Poco::Logger * logger = nullptr; + UInt64 calls_total = 0; + UInt64 calls_without_fault_injection = 0; + const UInt64 seed = 0; + + std::vector ephemeral_nodes; + + ZooKeeperWithFaultInjection( + zk::Ptr const & keeper_, + double fault_injection_probability, + UInt64 fault_injection_seed, + std::string name_, + Poco::Logger * logger_) + : keeper(keeper_), name(std::move(name_)), logger(logger_), seed(fault_injection_seed) + { + fault_policy = std::make_unique(fault_injection_probability, fault_injection_seed); + + if (unlikely(logger)) + LOG_TRACE( + logger, + "ZooKeeperWithFaultInjection created: name={} seed={} fault_probability={}", + name, + seed, + fault_injection_probability); + } + +public: + using Ptr = std::shared_ptr; + + static ZooKeeperWithFaultInjection::Ptr createInstance( + double fault_injection_probability, UInt64 fault_injection_seed, const zk::Ptr & zookeeper, std::string name, Poco::Logger * logger) + { + /// validate all parameters here, constructor just accept everything + + if (fault_injection_probability < 0.0) + fault_injection_probability = .0; + else if (fault_injection_probability > 1.0) + fault_injection_probability = 1.0; + + if (0 == fault_injection_seed) + fault_injection_seed = randomSeed(); + + if (fault_injection_probability > 0.0) + return std::shared_ptr( + new ZooKeeperWithFaultInjection(zookeeper, fault_injection_probability, fault_injection_seed, std::move(name), logger)); + + /// if no fault injection provided, create instance which will not log anything + return std::make_shared(zookeeper); + } + + explicit ZooKeeperWithFaultInjection(zk::Ptr const & keeper_) : keeper(keeper_) { } + + ~ZooKeeperWithFaultInjection() + { + if (unlikely(logger)) + LOG_TRACE( + logger, + "ZooKeeperWithFaultInjection report: name={} seed={} calls_total={} calls_succeeded={} calls_failed={} failure_rate={}", + name, + seed, + calls_total, + calls_without_fault_injection, + calls_total - calls_without_fault_injection, + float(calls_total - calls_without_fault_injection) / calls_total); + } + + void setKeeper(zk::Ptr const & keeper_) { keeper = keeper_; } + bool isNull() const { return keeper.get() == nullptr; } + + /// + /// mirror ZooKeeper interface + /// + + Strings getChildren( + const std::string & path, + Coordination::Stat * stat = nullptr, + const zkutil::EventPtr & watch = nullptr, + Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) + { + return access("getChildren", path, [&]() { return keeper->getChildren(path, stat, watch, list_request_type); }); + } + + Coordination::Error tryGetChildren( + const std::string & path, + Strings & res, + Coordination::Stat * stat = nullptr, + const zkutil::EventPtr & watch = nullptr, + Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) + { + return access("tryGetChildren", path, [&]() { return keeper->tryGetChildren(path, res, stat, watch, list_request_type); }); + } + + zk::FutureExists asyncExists(const std::string & path, Coordination::WatchCallback watch_callback = {}) + { + return access("asyncExists", path, [&]() { return keeper->asyncExists(path, watch_callback); }); + } + + zk::FutureGet asyncTryGet(const std::string & path) + { + return access("asyncTryGet", path, [&]() { return keeper->asyncTryGet(path); }); + } + + bool tryGet( + const std::string & path, + std::string & res, + Coordination::Stat * stat = nullptr, + const zkutil::EventPtr & watch = nullptr, + Coordination::Error * code = nullptr) + { + return access("tryGet", path, [&]() { return keeper->tryGet(path, res, stat, watch, code); }); + } + + Coordination::Error tryMulti(const Coordination::Requests & requests, Coordination::Responses & responses) + { + constexpr auto method = "tryMulti"; + auto error = access( + method, + !requests.empty() ? requests.front()->getPath() : "", + [&]() { return keeper->tryMulti(requests, responses); }, + [&](const Coordination::Error & original_error) + { + if (original_error == Coordination::Error::ZOK) + faultInjectionPostAction(method, requests, responses); + }, + [&]() + { + responses.clear(); + for (size_t i = 0; i < requests.size(); ++i) + responses.emplace_back(std::make_shared()); + }); + + + /// collect ephemeral nodes when no fault was injected (to clean up on demand) + if (unlikely(fault_policy) && Coordination::Error::ZOK == error) + { + doForEachCreatedEphemeralNode( + method, requests, responses, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); + } + return error; + } + + Coordination::Error tryMultiNoThrow(const Coordination::Requests & requests, Coordination::Responses & responses) + { + constexpr auto method = "tryMultiNoThrow"; + constexpr auto no_throw = true; + constexpr auto inject_failure_before_op = false; + auto error = access( + method, + !requests.empty() ? requests.front()->getPath() : "", + [&]() { return keeper->tryMultiNoThrow(requests, responses); }, + [&](const Coordination::Error & original_error) + { + if (original_error == Coordination::Error::ZOK) + faultInjectionPostAction(method, requests, responses); + }, + [&]() + { + responses.clear(); + for (size_t i = 0; i < requests.size(); ++i) + responses.emplace_back(std::make_shared()); + }); + + /// collect ephemeral nodes when no fault was injected (to clean up later) + if (unlikely(fault_policy) && Coordination::Error::ZOK == error) + { + doForEachCreatedEphemeralNode( + method, requests, responses, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); + } + return error; + } + + std::string get(const std::string & path, Coordination::Stat * stat = nullptr, const zkutil::EventPtr & watch = nullptr) + { + return access("get", path, [&]() { return keeper->get(path, stat, watch); }); + } + + zkutil::ZooKeeper::MultiGetResponse get(const std::vector & paths) + { + return access("get", !paths.empty() ? paths.front() : "", [&]() { return keeper->get(paths); }); + } + + bool exists(const std::string & path, Coordination::Stat * stat = nullptr, const zkutil::EventPtr & watch = nullptr) + { + return access("exists", path, [&]() { return keeper->exists(path, stat, watch); }); + } + + zkutil::ZooKeeper::MultiExistsResponse exists(const std::vector & paths) + { + return access("exists", !paths.empty() ? paths.front() : "", [&]() { return keeper->exists(paths); }); + } + + std::string create(const std::string & path, const std::string & data, int32_t mode) + { + auto path_created = access( + "create", + path, + [&]() { return keeper->create(path, data, mode); }, + [&](std::string const & result_path) + { + try + { + if (mode == zkutil::CreateMode::EphemeralSequential || mode == zkutil::CreateMode::Ephemeral) + { + keeper->remove(result_path); + if (unlikely(logger)) + LOG_TRACE(logger, "ZooKeeperWithFaultInjection cleanup: seed={} func={} path={}", seed, "create", result_path); + } + } + catch (const zkutil::KeeperException & e) + { + if (unlikely(logger)) + LOG_TRACE( + logger, + "ZooKeeperWithFaultInjection cleanup FAILED: seed={} func={} path={} code={} message={} ", + seed, + "create", + result_path, + e.code, + e.message()); + } + }); + + /// collect ephemeral nodes when no fault was injected (to clean up later) + if (unlikely(fault_policy)) + { + if (mode == zkutil::CreateMode::EphemeralSequential || mode == zkutil::CreateMode::Ephemeral) + ephemeral_nodes.push_back(path_created); + } + + return path_created; + } + + Coordination::Responses multi(const Coordination::Requests & requests) + { + constexpr auto method = "multi"; + auto result = access( + method, + !requests.empty() ? requests.front()->getPath() : "", + [&]() { return keeper->multi(requests); }, + [&](Coordination::Responses & responses) { faultInjectionPostAction(method, requests, responses); }); + + /// collect ephemeral nodes to clean up + if (unlikely(fault_policy)) + { + doForEachCreatedEphemeralNode( + method, requests, result, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); + } + return result; + } + + void createAncestors(const std::string & path) + { + access("createAncestors", path, [&]() { return keeper->createAncestors(path); }); + } + + Coordination::Error tryRemove(const std::string & path, int32_t version = -1) + { + return access("tryRemove", path, [&]() { return keeper->tryRemove(path, version); }); + } + + void cleanupEphemeralNodes() + { + for (const auto & path : ephemeral_nodes) + { + try + { + if (keeper_prev) + keeper_prev->tryRemove(path); + } + catch (...) + { + if (unlikely(logger)) + tryLogCurrentException(logger, "Exception during ephemeral nodes clean up"); + } + } + + ephemeral_nodes.clear(); + } + +private: + void faultInjectionBefore(std::function fault_cleanup) + { + try + { + if (unlikely(fault_policy)) + fault_policy->beforeOperation(); + } + catch (const zkutil::KeeperException &) + { + fault_cleanup(); + throw; + } + } + void faultInjectionAfter(std::function fault_cleanup) + { + try + { + if (unlikely(fault_policy)) + fault_policy->afterOperation(); + } + catch (const zkutil::KeeperException &) + { + fault_cleanup(); + throw; + } + } + + void doForEachCreatedEphemeralNode( + const char * method, const Coordination::Requests & requests, const Coordination::Responses & responses, auto && action) + { + if (responses.empty()) + return; + + if (responses.size() != requests.size()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Number of responses doesn't match number of requests: method={} requests={} responses={}", + method, + requests.size(), + responses.size()); + + /// find create request with ephemeral flag + std::vector> create_requests; + for (size_t i = 0; i < requests.size(); ++i) + { + const auto * create_req = dynamic_cast(requests[i].get()); + if (create_req && create_req->is_ephemeral) + create_requests.emplace_back(i, create_req); + } + + for (auto && [i, req] : create_requests) + { + const auto * create_resp = dynamic_cast(responses.at(i).get()); + if (!create_resp) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Response should be CreateResponse: method={} index={} path={}", method, i, req->path); + + action(create_resp->path_created); + } + } + + void faultInjectionPostAction(const char * method, const Coordination::Requests & requests, Coordination::Responses & responses) + { + doForEachCreatedEphemeralNode(method, requests, responses, [&](const String & path_created) { keeper->remove(path_created); }); + } + + template + struct FaultCleanupTypeImpl + { + using Type = std::function; + }; + + template <> + struct FaultCleanupTypeImpl + { + using Type = std::function; + }; + + template + using FaultCleanupType = typename FaultCleanupTypeImpl::Type; + + template < + bool no_throw_access = false, + bool inject_failure_before_op = true, + int inject_failure_after_op = true, + typename Operation, + typename Result = std::invoke_result_t> + Result access( + const char * func_name, + const std::string & path, + Operation operation, + FaultCleanupType fault_after_op_cleanup = {}, + FaultCleanupType fault_before_op_cleanup = {}) + { + try + { + ++calls_total; + + if (!keeper) + throw zkutil::KeeperException( + "Session is considered to be expired due to fault injection", Coordination::Error::ZSESSIONEXPIRED); + + if constexpr (inject_failure_before_op) + { + faultInjectionBefore( + [&] + { + if (fault_before_op_cleanup) + fault_before_op_cleanup(); + }); + } + + if constexpr (!std::is_same_v) + { + Result res = operation(); + + /// if connectivity error occurred w/o fault injection -> just return it + if constexpr (std::is_same_v) + { + if (Coordination::isHardwareError(res)) + return res; + } + + if constexpr (inject_failure_after_op) + { + faultInjectionAfter( + [&] + { + if (fault_after_op_cleanup) + fault_after_op_cleanup(res); + }); + } + + ++calls_without_fault_injection; + + if (unlikely(logger)) + LOG_TRACE(logger, "ZooKeeperWithFaultInjection call SUCCEEDED: seed={} func={} path={}", seed, func_name, path); + + return res; + } + else + { + operation(); + + if constexpr (inject_failure_after_op) + { + faultInjectionAfter( + [&fault_after_op_cleanup] + { + if (fault_after_op_cleanup) + fault_after_op_cleanup(); + }); + } + + ++calls_without_fault_injection; + + if (unlikely(logger)) + LOG_TRACE(logger, "ZooKeeperWithFaultInjection call SUCCEEDED: seed={} func={} path={}", seed, func_name, path); + } + } + catch (const zkutil::KeeperException & e) + { + if (unlikely(logger)) + LOG_TRACE( + logger, + "ZooKeeperWithFaultInjection call FAILED: seed={} func={} path={} code={} message={} ", + seed, + func_name, + path, + e.code, + e.message()); + + /// save valid pointer to clean up ephemeral nodes later if necessary + if (keeper) + keeper_prev = keeper; + keeper.reset(); + + /// for try*NoThrow() methods + if constexpr (no_throw_access) + return e.code; + + if constexpr (std::is_same_v) + { + /// try*() methods throws at least on hardware error and return only on user errors + /// todo: the methods return only on subset of user errors, and throw on another errors + /// to mimic the methods exactly - we need to specify errors on which to return for each such method + if (Coordination::isHardwareError(e.code)) + throw; + + return e.code; + } + + throw; + } + } +}; + +using ZooKeeperWithFaultInjectionPtr = ZooKeeperWithFaultInjection::Ptr; +} diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3b85581a157..dde844c0ee0 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4479,9 +4479,16 @@ void StorageReplicatedMergeTree::assertNotReadonly() const SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - const auto storage_settings_ptr = getSettings(); - assertNotReadonly(); + /// If table is read-only because it doesn't have metadata in zk yet, then it's not possible to insert into it + /// Without this check, we'll write data parts on disk, and afterwards will remove them since we'll fail to commit them into zk + /// In case of remote storage like s3, it'll generate unnecessary PUT requests + if (is_readonly && (!has_metadata_in_zookeeper.has_value() || false == has_metadata_in_zookeeper.value())) + throw Exception( + ErrorCodes::TABLE_IS_READ_ONLY, + "Table is in readonly mode since table metadata was not found in zookeeper: replica_path={}", + replica_path); + const auto storage_settings_ptr = getSettings(); const Settings & query_settings = local_context->getSettingsRef(); bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; @@ -4996,8 +5003,7 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St Int64 mutation_version; { - auto zookeeper = getZooKeeper(); - delimiting_block_lock = allocateBlockNumber(partition_id, zookeeper); + delimiting_block_lock = allocateBlockNumber(partition_id, getZooKeeper()); right = delimiting_block_lock->getNumber(); /// Make sure we cover all parts in drop range. /// There might be parts with mutation version greater than current block number @@ -5278,7 +5284,7 @@ void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, c } -bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) const +bool StorageReplicatedMergeTree::existsNodeCached(const ZooKeeperWithFaultInjectionPtr & zookeeper, const std::string & path) const { { std::lock_guard lock(existing_nodes_cache_mutex); @@ -5286,7 +5292,7 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) cons return true; } - bool res = getZooKeeper()->exists(path); + bool res = zookeeper->exists(path); if (res) { @@ -5298,9 +5304,22 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) cons } -std::optional -StorageReplicatedMergeTree::allocateBlockNumber( - const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path, const String & zookeeper_path_prefix) const +std::optional StorageReplicatedMergeTree::allocateBlockNumber( + const String & partition_id, + const zkutil::ZooKeeperPtr & zookeeper, + const String & zookeeper_block_id_path, + const String & zookeeper_path_prefix) const +{ + return allocateBlockNumber( + partition_id, std::make_shared(zookeeper), zookeeper_block_id_path, zookeeper_path_prefix); +} + + +std::optional StorageReplicatedMergeTree::allocateBlockNumber( + const String & partition_id, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const String & zookeeper_block_id_path, + const String & zookeeper_path_prefix) const { String zookeeper_table_path; if (zookeeper_path_prefix.empty()) @@ -5311,7 +5330,7 @@ StorageReplicatedMergeTree::allocateBlockNumber( String block_numbers_path = fs::path(zookeeper_table_path) / "block_numbers"; String partition_path = fs::path(block_numbers_path) / partition_id; - if (!existsNodeCached(partition_path)) + if (!existsNodeCached(zookeeper, partition_path)) { Coordination::Requests ops; /// Check that table is not being dropped ("host" is the first node that is removed on replica drop) @@ -5329,10 +5348,9 @@ StorageReplicatedMergeTree::allocateBlockNumber( } return createEphemeralLockInZooKeeper( - fs::path(partition_path) / "block-", fs::path(zookeeper_table_path) / "temp", *zookeeper, zookeeper_block_id_path); + fs::path(partition_path) / "block-", fs::path(zookeeper_table_path) / "temp", zookeeper, zookeeper_block_id_path); } - Strings StorageReplicatedMergeTree::tryWaitForAllReplicasToProcessLogEntry( const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry, Int64 wait_for_inactive_timeout) { @@ -7085,7 +7103,7 @@ CancellationCode StorageReplicatedMergeTree::killPartMoveToShard(const UUID & ta void StorageReplicatedMergeTree::getCommitPartOps( Coordination::Requests & ops, - MutableDataPartPtr & part, + const DataPartPtr & part, const String & block_id_path) const { const String & part_name = part->name; @@ -7667,11 +7685,28 @@ void StorageReplicatedMergeTree::lockSharedDataTemporary(const String & part_nam String zookeeper_node = fs::path(zc_zookeeper_path) / id / replica_name; LOG_TRACE(log, "Set zookeeper temporary ephemeral lock {}", zookeeper_node); - createZeroCopyLockNode(zookeeper, zookeeper_node, zkutil::CreateMode::Ephemeral, false); + createZeroCopyLockNode( + std::make_shared(zookeeper), zookeeper_node, zkutil::CreateMode::Ephemeral, false); } } -void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part, bool replace_existing_lock, std::optional hardlinked_files) const +void StorageReplicatedMergeTree::lockSharedData( + const IMergeTreeDataPart & part, + bool replace_existing_lock, + std::optional hardlinked_files) const +{ + auto zookeeper = tryGetZooKeeper(); + if (zookeeper) + return lockSharedData(part, std::make_shared(zookeeper), replace_existing_lock, hardlinked_files); + else + return lockSharedData(part, std::make_shared(nullptr), replace_existing_lock, hardlinked_files); +} + +void StorageReplicatedMergeTree::lockSharedData( + const IMergeTreeDataPart & part, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + bool replace_existing_lock, + std::optional hardlinked_files) const { auto settings = getSettings(); @@ -7681,8 +7716,7 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part, if (!part.getDataPartStorage().supportZeroCopyReplication()) return; - zkutil::ZooKeeperPtr zookeeper = tryGetZooKeeper(); - if (!zookeeper) + if (zookeeper->isNull()) return; String id = part.getUniqueId(); @@ -7716,7 +7750,14 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part, } } -std::pair StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part) const +std::pair +StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part) const +{ + return unlockSharedData(part, std::make_shared(nullptr)); +} + +std::pair +StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part, const ZooKeeperWithFaultInjectionPtr & zookeeper) const { auto settings = getSettings(); if (!settings->allow_remote_fs_zero_copy_replication) @@ -7762,11 +7803,10 @@ std::pair StorageReplicatedMergeTree::unlockSharedData(const IMer /// We remove parts during table shutdown. If exception happen, restarting thread will be already turned /// off and nobody will reconnect our zookeeper connection. In this case we use zookeeper connection from /// context. - zkutil::ZooKeeperPtr zookeeper; if (shutdown_called.load()) - zookeeper = getZooKeeperIfTableShutDown(); + zookeeper->setKeeper(getZooKeeperIfTableShutDown()); else - zookeeper = getZooKeeper(); + zookeeper->setKeeper(getZooKeeper()); /// It can happen that we didn't had the connection to zookeeper during table creation, but actually /// table is completely dropped, so we can drop it without any additional checks. @@ -7791,7 +7831,7 @@ namespace /// But sometimes we need an opposite. When we deleting all_0_0_0_1 it can be non replicated to other replicas, so we are the only owner of this part. /// In this case when we will drop all_0_0_0_1 we will drop blobs for all_0_0_0. But it will lead to dataloss. For such case we need to check that other replicas /// still need parent part. -std::pair getParentLockedBlobs(zkutil::ZooKeeperPtr zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const std::string & part_info_str, MergeTreeDataFormatVersion format_version, Poco::Logger * log) +std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const std::string & part_info_str, MergeTreeDataFormatVersion format_version, Poco::Logger * log) { NameSet files_not_to_remove; @@ -7853,7 +7893,7 @@ std::pair getParentLockedBlobs(zkutil::ZooKeeperPtr zookeeper_ptr std::pair StorageReplicatedMergeTree::unlockSharedDataByID( String part_id, const String & table_uuid, const String & part_name, - const String & replica_name_, const std::string & disk_type, zkutil::ZooKeeperPtr zookeeper_ptr, const MergeTreeSettings & settings, + const String & replica_name_, const std::string & disk_type, const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const MergeTreeSettings & settings, Poco::Logger * logger, const String & zookeeper_path_old, MergeTreeDataFormatVersion data_format_version) { boost::replace_all(part_id, "/", "_"); @@ -7872,7 +7912,8 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( if (!files_not_to_remove_str.empty()) boost::split(files_not_to_remove, files_not_to_remove_str, boost::is_any_of("\n ")); - auto [has_parent, parent_not_to_remove] = getParentLockedBlobs(zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_name, data_format_version, logger); + auto [has_parent, parent_not_to_remove] = getParentLockedBlobs( + zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_name, data_format_version, logger); files_not_to_remove.insert(parent_not_to_remove.begin(), parent_not_to_remove.end()); String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / part_id; @@ -8387,7 +8428,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP void StorageReplicatedMergeTree::createZeroCopyLockNode( - const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node, int32_t mode, + const ZooKeeperWithFaultInjectionPtr & zookeeper, const String & zookeeper_node, int32_t mode, bool replace_existing_lock, const String & path_to_set_hardlinked_files, const NameSet & hardlinked_files) { /// In rare case other replica can remove path between createAncestors and createIfNotExists @@ -8504,7 +8545,7 @@ bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const St id, table_uuid, part_name, detached_replica_name, toString(disk->getDataSourceDescription().type), - zookeeper, local_context->getReplicatedMergeTreeSettings(), + std::make_shared(zookeeper), local_context->getReplicatedMergeTreeSettings(), &Poco::Logger::get("StorageReplicatedMergeTree"), detached_zookeeper_path, MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index fe2105ab4ec..d767d94889c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -83,6 +83,9 @@ namespace DB * as the time will take the time of creation the appropriate part on any of the replicas. */ +class ZooKeeperWithFaultInjection; +using ZooKeeperWithFaultInjectionPtr = std::shared_ptr; + class StorageReplicatedMergeTree final : public MergeTreeData { public: @@ -267,6 +270,11 @@ public: /// Lock part in zookeeper for use shared data in several nodes void lockSharedData(const IMergeTreeDataPart & part, bool replace_existing_lock, std::optional hardlinked_files) const override; + void lockSharedData( + const IMergeTreeDataPart & part, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + bool replace_existing_lock, + std::optional hardlinked_files) const; void lockSharedDataTemporary(const String & part_name, const String & part_id, const DiskPtr & disk) const; @@ -274,13 +282,23 @@ public: /// Return true if data unlocked /// Return false if data is still used by another node std::pair unlockSharedData(const IMergeTreeDataPart & part) const override; + std::pair + unlockSharedData(const IMergeTreeDataPart & part, const ZooKeeperWithFaultInjectionPtr & zookeeper) const; /// Unlock shared data part in zookeeper by part id /// Return true if data unlocked /// Return false if data is still used by another node - static std::pair unlockSharedDataByID(String part_id, const String & table_uuid, const String & part_name, const String & replica_name_, - const std::string & disk_type, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, Poco::Logger * logger, - const String & zookeeper_path_old, MergeTreeDataFormatVersion data_format_version); + static std::pair unlockSharedDataByID( + String part_id, + const String & table_uuid, + const String & part_name, + const String & replica_name_, + const std::string & disk_type, + const ZooKeeperWithFaultInjectionPtr & zookeeper_, + const MergeTreeSettings & settings, + Poco::Logger * logger, + const String & zookeeper_path_old, + MergeTreeDataFormatVersion data_format_version); /// Fetch part only if some replica has it on shared storage like S3 MutableDataPartStoragePtr tryToFetchIfShared(const IMergeTreeDataPart & part, const DiskPtr & disk, const String & path) override; @@ -534,7 +552,7 @@ private: bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override; - void getCommitPartOps(Coordination::Requests & ops, MutableDataPartPtr & part, const String & block_id_path = "") const; + void getCommitPartOps(Coordination::Requests & ops, const DataPartPtr & part, const String & block_id_path = "") const; /// Adds actions to `ops` that remove a part from ZooKeeper. /// Set has_children to true for "old-style" parts (those with /columns and /checksums child znodes). @@ -711,6 +729,11 @@ private: std::optional allocateBlockNumber( const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path = "", const String & zookeeper_path_prefix = "") const; + std::optional allocateBlockNumber( + const String & partition_id, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const String & zookeeper_block_id_path = "", + const String & zookeeper_path_prefix = "") const; /** Wait until all replicas, including this, execute the specified action from the log. * If replicas are added at the same time, it can not wait the added replica. @@ -748,7 +771,7 @@ private: /// Check for a node in ZK. If it is, remember this information, and then immediately answer true. mutable std::unordered_set existing_nodes_cache; mutable std::mutex existing_nodes_cache_mutex; - bool existsNodeCached(const std::string & path) const; + bool existsNodeCached(const ZooKeeperWithFaultInjectionPtr & zookeeper, const std::string & path) const; /// Cancels INSERTs in the block range by removing ephemeral block numbers void clearLockedBlockNumbersInPartition(zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num); @@ -836,7 +859,7 @@ private: const String & part_name, const String & zookeeper_path_old); static void createZeroCopyLockNode( - const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node, + const ZooKeeperWithFaultInjectionPtr & zookeeper, const String & zookeeper_node, int32_t mode = zkutil::CreateMode::Persistent, bool replace_existing_lock = false, const String & path_to_set_hardlinked_files = "", const NameSet & hardlinked_files = {}); diff --git a/tests/config/install.sh b/tests/config/install.sh index 876d05507fe..51865665f59 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -71,6 +71,7 @@ ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/filelog.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/enable_blobs_check.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/marks.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/insert_keeper_retries.xml $DEST_SERVER_PATH/users.d/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), diff --git a/tests/config/users.d/insert_keeper_retries.xml b/tests/config/users.d/insert_keeper_retries.xml new file mode 100644 index 00000000000..f7b652bf2ef --- /dev/null +++ b/tests/config/users.d/insert_keeper_retries.xml @@ -0,0 +1,8 @@ + + + + 0 + 0.0 + + + diff --git a/tests/integration/test_inserts_with_keeper_retries/__init__.py b/tests/integration/test_inserts_with_keeper_retries/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_inserts_with_keeper_retries/test.py b/tests/integration/test_inserts_with_keeper_retries/test.py new file mode 100644 index 00000000000..dbf18365053 --- /dev/null +++ b/tests/integration/test_inserts_with_keeper_retries/test.py @@ -0,0 +1,100 @@ +#!/usr/bin/env python3 + +import pytest +import time +import threading +from helpers.cluster import ClickHouseCluster +from multiprocessing.dummy import Pool +from helpers.network import PartitionManager +from helpers.client import QueryRuntimeException +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance("node1", with_zookeeper=True) + + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_replica_inserts_with_keeper_restart(started_cluster): + try: + node1.query( + "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '0') ORDER BY tuple()" + ) + + p = Pool(1) + zk_stopped_event = threading.Event() + + def zoo_restart(zk_stopped_event): + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + zk_stopped_event.set() + cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + + job = p.apply_async(zoo_restart, (zk_stopped_event,)) + + zk_stopped_event.wait(90) + + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=20" + ) + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10) SETTINGS insert_keeper_max_retries=20" + ) + + job.wait() + p.close() + p.join() + + assert node1.query("SELECT COUNT() FROM r") == "20\n" + + finally: + node1.query("DROP TABLE IF EXISTS r SYNC") + + +def test_replica_inserts_with_keeper_disconnect(started_cluster): + try: + node1.query( + "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '0') ORDER BY tuple()" + ) + + p = Pool(1) + disconnect_event = threading.Event() + + def keeper_disconnect(node, event): + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node) + event.set() + + job = p.apply_async( + keeper_disconnect, + ( + node1, + disconnect_event, + ), + ) + disconnect_event.wait(90) + + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=20" + ) + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10) SETTINGS insert_keeper_max_retries=20" + ) + + job.wait() + p.close() + p.join() + + assert node1.query("SELECT COUNT() FROM r") == "20\n" + + finally: + node1.query("DROP TABLE IF EXISTS r SYNC") diff --git a/tests/integration/test_restore_replica/test.py b/tests/integration/test_restore_replica/test.py index 0b11cdf7512..31c503f6184 100644 --- a/tests/integration/test_restore_replica/test.py +++ b/tests/integration/test_restore_replica/test.py @@ -7,6 +7,9 @@ from helpers.test_tools import assert_eq_with_retry def fill_nodes(nodes): + for node in nodes: + node.query("DROP TABLE IF EXISTS test SYNC") + for node in nodes: node.query( """ @@ -29,11 +32,7 @@ nodes = [node_1, node_2, node_3] def fill_table(): - node_1.query("TRUNCATE TABLE test") - - for node in nodes: - node.query("SYSTEM SYNC REPLICA test") - + fill_nodes(nodes) check_data(0, 0) # it will create multiple parts in each partition and probably cause merges diff --git a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql index f62f11c60fd..ed1f654f847 100644 --- a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql +++ b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql @@ -1,14 +1,14 @@ -- Tags: zookeeper, no-replicated-database -- Tag no-replicated-database: Old syntax is not allowed -DROP TABLE IF EXISTS alter_00121; +DROP TABLE IF EXISTS alter_00121 SYNC; set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE alter_00121 (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t1', 'r1', d, (d), 8192); INSERT INTO alter_00121 VALUES ('2014-01-01', 1); ALTER TABLE alter_00121 DROP COLUMN x; -DROP TABLE alter_00121; +DROP TABLE alter_00121 SYNC; CREATE TABLE alter_00121 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t2', 'r1', d, (d), 8192); @@ -23,4 +23,4 @@ SELECT * FROM alter_00121 ORDER BY d; ALTER TABLE alter_00121 DROP COLUMN x; SELECT * FROM alter_00121 ORDER BY d; -DROP TABLE alter_00121; +DROP TABLE alter_00121 SYNC; diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql index 8267a451728..0ee8ba07006 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql @@ -1,11 +1,12 @@ -- Tags: long, replica SET replication_alter_partitions_sync = 2; +SET insert_keeper_fault_injection_probability=0; SELECT '*** Not partitioned ***'; -DROP TABLE IF EXISTS not_partitioned_replica1_00502; -DROP TABLE IF EXISTS not_partitioned_replica2_00502; +DROP TABLE IF EXISTS not_partitioned_replica1_00502 SYNC; +DROP TABLE IF EXISTS not_partitioned_replica2_00502 SYNC; CREATE TABLE not_partitioned_replica1_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '1') ORDER BY x; CREATE TABLE not_partitioned_replica2_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '2') ORDER BY x; @@ -14,7 +15,7 @@ INSERT INTO not_partitioned_replica1_00502 VALUES (4), (5); SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'not_partitioned_replica1_00502' AND active ORDER BY name; -SYSTEM SYNC REPLICA not_partitioned_replica1_00502; +SYSTEM SYNC REPLICA not_partitioned_replica2_00502; OPTIMIZE TABLE not_partitioned_replica1_00502 PARTITION tuple() FINAL; SELECT 'Parts after OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'not_partitioned_replica2_00502' AND active ORDER BY name; @@ -25,13 +26,13 @@ ALTER TABLE not_partitioned_replica1_00502 DETACH PARTITION ID 'all'; SELECT 'Sum after DETACH PARTITION:'; SELECT sum(x) FROM not_partitioned_replica2_00502; -DROP TABLE not_partitioned_replica1_00502; -DROP TABLE not_partitioned_replica2_00502; +DROP TABLE not_partitioned_replica1_00502 SYNC; +DROP TABLE not_partitioned_replica2_00502 SYNC; SELECT '*** Partitioned by week ***'; -DROP TABLE IF EXISTS partitioned_by_week_replica1; -DROP TABLE IF EXISTS partitioned_by_week_replica2; +DROP TABLE IF EXISTS partitioned_by_week_replica1 SYNC; +DROP TABLE IF EXISTS partitioned_by_week_replica2 SYNC; CREATE TABLE partitioned_by_week_replica1(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '1') PARTITION BY toMonday(d) ORDER BY x; CREATE TABLE partitioned_by_week_replica2(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '2') PARTITION BY toMonday(d) ORDER BY x; @@ -41,7 +42,7 @@ INSERT INTO partitioned_by_week_replica1 VALUES ('2000-01-03', 4), ('2000-01-03' SELECT 'Parts before OPTIMIZE:'; -- Select parts on the first replica to avoid waiting for replication. SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_week_replica1' AND active ORDER BY name; -SYSTEM SYNC REPLICA partitioned_by_week_replica1; +SYSTEM SYNC REPLICA partitioned_by_week_replica2; OPTIMIZE TABLE partitioned_by_week_replica1 PARTITION '2000-01-03' FINAL; SELECT 'Parts after OPTIMIZE:'; -- After OPTIMIZE with replication_alter_partitions_sync=2 replicas must be in sync. SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_week_replica2' AND active ORDER BY name; @@ -52,13 +53,13 @@ ALTER TABLE partitioned_by_week_replica1 DROP PARTITION '1999-12-27'; SELECT 'Sum after DROP PARTITION:'; SELECT sum(x) FROM partitioned_by_week_replica2; -DROP TABLE partitioned_by_week_replica1; -DROP TABLE partitioned_by_week_replica2; +DROP TABLE partitioned_by_week_replica1 SYNC; +DROP TABLE partitioned_by_week_replica2 SYNC; SELECT '*** Partitioned by a (Date, UInt8) tuple ***'; -DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00502; -DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00502; +DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00502 SYNC; +DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00502 SYNC; CREATE TABLE partitioned_by_tuple_replica1_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '1') ORDER BY x PARTITION BY (d, x); CREATE TABLE partitioned_by_tuple_replica2_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '2') ORDER BY x PARTITION BY (d, x); @@ -67,7 +68,7 @@ INSERT INTO partitioned_by_tuple_replica1_00502 VALUES ('2000-01-02', 1, 4), ('2 SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_tuple_replica1_00502' AND active ORDER BY name; -SYSTEM SYNC REPLICA partitioned_by_tuple_replica1_00502; +SYSTEM SYNC REPLICA partitioned_by_tuple_replica2_00502; OPTIMIZE TABLE partitioned_by_tuple_replica1_00502 PARTITION ('2000-01-01', 1) FINAL; OPTIMIZE TABLE partitioned_by_tuple_replica1_00502 PARTITION ('2000-01-02', 1) FINAL; SELECT 'Parts after OPTIMIZE:'; @@ -79,13 +80,13 @@ ALTER TABLE partitioned_by_tuple_replica1_00502 DETACH PARTITION ID '20000101-1' SELECT 'Sum after DETACH PARTITION:'; SELECT sum(y) FROM partitioned_by_tuple_replica2_00502; -DROP TABLE partitioned_by_tuple_replica1_00502; -DROP TABLE partitioned_by_tuple_replica2_00502; +DROP TABLE partitioned_by_tuple_replica1_00502 SYNC; +DROP TABLE partitioned_by_tuple_replica2_00502 SYNC; SELECT '*** Partitioned by String ***'; -DROP TABLE IF EXISTS partitioned_by_string_replica1; -DROP TABLE IF EXISTS partitioned_by_string_replica2; +DROP TABLE IF EXISTS partitioned_by_string_replica1 SYNC; +DROP TABLE IF EXISTS partitioned_by_string_replica2 SYNC; CREATE TABLE partitioned_by_string_replica1(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '1') PARTITION BY s ORDER BY x; CREATE TABLE partitioned_by_string_replica2(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '2') PARTITION BY s ORDER BY x; @@ -105,13 +106,13 @@ ALTER TABLE partitioned_by_string_replica1 DROP PARTITION 'bbb'; SELECT 'Sum after DROP PARTITION:'; SELECT sum(x) FROM partitioned_by_string_replica2; -DROP TABLE partitioned_by_string_replica1; -DROP TABLE partitioned_by_string_replica2; +DROP TABLE partitioned_by_string_replica1 SYNC; +DROP TABLE partitioned_by_string_replica2 SYNC; SELECT '*** Table without columns with fixed size ***'; -DROP TABLE IF EXISTS without_fixed_size_columns_replica1; -DROP TABLE IF EXISTS without_fixed_size_columns_replica2; +DROP TABLE IF EXISTS without_fixed_size_columns_replica1 SYNC; +DROP TABLE IF EXISTS without_fixed_size_columns_replica2 SYNC; CREATE TABLE without_fixed_size_columns_replica1(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '1') PARTITION BY length(s) ORDER BY s; CREATE TABLE without_fixed_size_columns_replica2(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '2') PARTITION BY length(s) ORDER BY s; @@ -130,5 +131,5 @@ ALTER TABLE without_fixed_size_columns_replica1 DROP PARTITION 1; SELECT 'After DROP PARTITION:'; SELECT * FROM without_fixed_size_columns_replica2 ORDER BY s; -DROP TABLE without_fixed_size_columns_replica1; -DROP TABLE without_fixed_size_columns_replica2; +DROP TABLE without_fixed_size_columns_replica1 SYNC; +DROP TABLE without_fixed_size_columns_replica2 SYNC; diff --git a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh index 7a6c7609660..a1e7d8727c7 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh @@ -9,8 +9,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=./mergetree_mutations.lib . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r1 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r2 SYNC" ${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations_r1(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r1', d, intDiv(x, 10), 8192)" ${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations_r2(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r2', d, intDiv(x, 10), 8192)" @@ -19,9 +19,9 @@ ${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2" # Insert some data -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 1, 'a')" -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 2, 'b'), ('2000-01-01', 3, 'c'), ('2000-01-01', 4, 'd') \ ('2000-02-01', 2, 'b'), ('2000-02-01', 3, 'c'), ('2000-02-01', 4, 'd')" @@ -35,7 +35,7 @@ ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE s = 'd' SETT ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE m = 3 SETTINGS mutations_sync = 2" # Insert more data -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 5, 'e'), ('2000-02-01', 5, 'e')" ${CLICKHOUSE_CLIENT} --query "SYSTEM SYNC REPLICA mutations_r2" @@ -49,8 +49,8 @@ ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, block_numbers.partiti ${CLICKHOUSE_CLIENT} --query="SELECT '*** Test mutations cleaner ***'" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r1 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r2 SYNC" # Create 2 replicas with finished_mutations_to_keep = 2 ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_cleaner_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations_cleaner', 'r1') ORDER BY x SETTINGS \ @@ -63,7 +63,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_cleaner_r2(x UInt32) ENGINE cleanup_delay_period_random_add = 0" # Insert some data -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_cleaner_r1(x) VALUES (1), (2), (3), (4), (5)" +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_cleaner_r1(x) VALUES (1), (2), (3), (4), (5)" # Add some mutations and wait for their execution ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_cleaner_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2" @@ -80,8 +80,8 @@ sleep 1.5 # Check that the first mutation is cleaned ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, is_done FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner_r2' ORDER BY mutation_id" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r1 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r2 SYNC" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r1 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r2 SYNC" diff --git a/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql b/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql index 4cdd75f520c..8f694345d93 100644 --- a/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql +++ b/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql @@ -2,8 +2,8 @@ SET optimize_on_insert = 0; -DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00661; -DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00661; +DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00661 SYNC; +DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00661 SYNC; CREATE TABLE partitioned_by_tuple_replica1_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '1') PARTITION BY (d, x) ORDER BY (d, x, w); CREATE TABLE partitioned_by_tuple_replica2_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '2') PARTITION BY (d, x) ORDER BY (d, x, w); @@ -21,5 +21,5 @@ OPTIMIZE TABLE partitioned_by_tuple_replica1_00661 FINAL; SYSTEM SYNC REPLICA partitioned_by_tuple_replica2_00661; SELECT * FROM partitioned_by_tuple_replica2_00661 ORDER BY d, x, w, y; -DROP TABLE partitioned_by_tuple_replica1_00661; -DROP TABLE partitioned_by_tuple_replica2_00661; +DROP TABLE partitioned_by_tuple_replica1_00661 SYNC; +DROP TABLE partitioned_by_tuple_replica2_00661 SYNC; diff --git a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh index 4637b210194..3f384380f9b 100755 --- a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh +++ b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh @@ -9,8 +9,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -n --query=" - DROP TABLE IF EXISTS fetches_r1; - DROP TABLE IF EXISTS fetches_r2" + DROP TABLE IF EXISTS fetches_r1 SYNC; + DROP TABLE IF EXISTS fetches_r2 SYNC" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/fetches', 'r1') ORDER BY x" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/fetches', 'r2') ORDER BY x \ @@ -18,6 +18,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r2(x UInt32) ENGINE Replicate prefer_fetch_merged_part_size_threshold=0" ${CLICKHOUSE_CLIENT} -n --query=" + SET insert_keeper_fault_injection_probability=0; INSERT INTO fetches_r1 VALUES (1); INSERT INTO fetches_r1 VALUES (2); INSERT INTO fetches_r1 VALUES (3)" @@ -51,5 +52,5 @@ ${CLICKHOUSE_CLIENT} --query="SELECT '*** Check data after fetch/clone of mutate ${CLICKHOUSE_CLIENT} --query="SELECT _part, * FROM fetches_r2 ORDER BY x" ${CLICKHOUSE_CLIENT} -n --query=" - DROP TABLE fetches_r1; - DROP TABLE fetches_r2" + DROP TABLE fetches_r1 SYNC; + DROP TABLE fetches_r2 SYNC" diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh index 6f609065c01..5fc3fa460e6 100755 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh @@ -12,6 +12,7 @@ $CLICKHOUSE_CLIENT -nm -q " DROP TABLE IF EXISTS part_header_r1; DROP TABLE IF EXISTS part_header_r2; +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET replication_alter_partitions_sync = 2; CREATE TABLE part_header_r1(x UInt32, y UInt32) diff --git a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql index 703f8824055..b836f806170 100644 --- a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql +++ b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql @@ -1,9 +1,10 @@ -- Tags: zookeeper +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET check_query_single_value_result = 0; SET send_logs_level = 'fatal'; -DROP TABLE IF EXISTS mt_without_pk; +DROP TABLE IF EXISTS mt_without_pk SYNC; CREATE TABLE mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = MergeTree() ORDER BY tuple(); @@ -11,9 +12,9 @@ INSERT INTO mt_without_pk VALUES (1, 2); CHECK TABLE mt_without_pk; -DROP TABLE IF EXISTS mt_without_pk; +DROP TABLE IF EXISTS mt_without_pk SYNC; -DROP TABLE IF EXISTS replicated_mt_without_pk; +DROP TABLE IF EXISTS replicated_mt_without_pk SYNC; CREATE TABLE replicated_mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01037/replicated_mt_without_pk', '1') ORDER BY tuple(); @@ -21,4 +22,4 @@ INSERT INTO replicated_mt_without_pk VALUES (1, 2); CHECK TABLE replicated_mt_without_pk; -DROP TABLE IF EXISTS replicated_mt_without_pk; +DROP TABLE IF EXISTS replicated_mt_without_pk SYNC; diff --git a/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh b/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh index a438cf7badc..68c511b80ac 100755 --- a/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh +++ b/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh @@ -52,7 +52,8 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE replicated_table_for_mutations(k UInt ${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES replicated_table_for_mutations" -${CLICKHOUSE_CLIENT} --query="INSERT INTO replicated_table_for_mutations select number, number from numbers(100000)" +# test relays on part ids, which are non-deterministic with keeper fault injections, so disable it +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO replicated_table_for_mutations select number, number from numbers(100000)" ${CLICKHOUSE_CLIENT} --query="SELECT sum(v1) FROM replicated_table_for_mutations" diff --git a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql index 43ab053655a..e4acfed8a0c 100644 --- a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql +++ b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql @@ -1,8 +1,8 @@ -- Tags: long, zookeeper, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards -DROP TABLE IF EXISTS mutations_and_quorum1; -DROP TABLE IF EXISTS mutations_and_quorum2; +DROP TABLE IF EXISTS mutations_and_quorum1 SYNC; +DROP TABLE IF EXISTS mutations_and_quorum2 SYNC; CREATE TABLE mutations_and_quorum1 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '1') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '2') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); @@ -10,6 +10,9 @@ CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGI -- Should not be larger then 600e6 (default timeout in clickhouse-test) SET insert_quorum=2, insert_quorum_parallel=0, insert_quorum_timeout=300e3; +SET insert_keeper_max_retries=100; +SET insert_keeper_retry_max_backoff_ms=10; + INSERT INTO mutations_and_quorum1 VALUES ('2019-01-01', 'test1'), ('2019-02-01', 'test2'), ('2019-03-01', 'test3'), ('2019-04-01', 'test4'), ('2019-05-01', 'test1'), ('2019-06-01', 'test2'), ('2019-07-01', 'test3'), ('2019-08-01', 'test4'), ('2019-09-01', 'test1'), ('2019-10-01', 'test2'), ('2019-11-01', 'test3'), ('2019-12-01', 'test4'); ALTER TABLE mutations_and_quorum1 DELETE WHERE something = 'test1' SETTINGS mutations_sync=2; @@ -19,5 +22,5 @@ SELECT COUNT() FROM mutations_and_quorum2; SELECT COUNT() FROM system.mutations WHERE database = currentDatabase() AND table like 'mutations_and_quorum%' and is_done = 0; -DROP TABLE IF EXISTS mutations_and_quorum1; -DROP TABLE IF EXISTS mutations_and_quorum2; +DROP TABLE IF EXISTS mutations_and_quorum1 SYNC; +DROP TABLE IF EXISTS mutations_and_quorum2 SYNC; diff --git a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql index c370726c72f..2096942630e 100644 --- a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql +++ b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql @@ -1,6 +1,6 @@ -- Tags: zookeeper -DROP TABLE IF EXISTS default_table; +DROP TABLE IF EXISTS default_table SYNC; CREATE TABLE default_table ( @@ -26,4 +26,4 @@ ALTER TABLE default_table MODIFY COLUMN enum_column Enum8('undefined' = 0, 'fox' SHOW CREATE TABLE default_table; -DROP TABLE IF EXISTS default_table; +DROP TABLE IF EXISTS default_table SYNC; diff --git a/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql b/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql index 687c8051eed..bd11b24d568 100644 --- a/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql +++ b/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql @@ -1,5 +1,7 @@ -- Tags: zookeeper +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + set send_logs_level='error'; drop table if exists mt; drop table if exists rmt sync; diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.sql b/tests/queries/0_stateless/01158_zookeeper_log_long.sql index 5f6ca762ae7..cabb0ffe294 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.sql +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.sql @@ -1,13 +1,16 @@ -- Tags: long, zookeeper, no-replicated-database, no-polymorphic-parts -- Tag no-replicated-database: Fails due to additional replicas or shards -drop table if exists rmt; +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + +drop table if exists rmt sync; -- cleanup code will perform extra Exists -- (so the .reference will not match) create table rmt (n int) engine=ReplicatedMergeTree('/test/01158/{database}/rmt', '1') order by n settings cleanup_delay_period=86400, replicated_can_become_leader=0; system sync replica rmt; insert into rmt values (1); insert into rmt values (1); +system sync replica rmt; system flush logs; select 'log'; @@ -30,7 +33,7 @@ from system.zookeeper_log where (session_id, xid) in (select session_id, xid from system.zookeeper_log where path like '/test/01158/' || currentDatabase() || '/rmt/blocks/%' and op_num not in (1, 12, 500)) order by xid, type, request_idx; -drop table rmt; +drop table rmt sync; system flush logs; select 'duration_ms'; diff --git a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql index 875bd1f96de..78adbee612f 100644 --- a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql @@ -1,5 +1,9 @@ -- Tags: long, replica +-- in case of keeper fault injection on insert, set bigger number of retries because partitions +set insert_keeper_max_retries=100; +set insert_keeper_retry_max_backoff_ms=10; + -- Testing basic functionality with compact parts set replication_alter_partitions_sync = 2; drop table if exists mt_compact; diff --git a/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh b/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh index 34fa822b6ea..f7615974237 100755 --- a/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh +++ b/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table SYNC" $CLICKHOUSE_CLIENT --query " CREATE TABLE mutation_table( @@ -17,9 +17,10 @@ $CLICKHOUSE_CLIENT --query " PARTITION BY key % 10 " -$CLICKHOUSE_CLIENT --query "INSERT INTO mutation_table select number, toString(number) from numbers(100000) where number % 10 != 0" +# disable keeper fault injection during insert since test checks part names. Part names can differ in case of retries during insert +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO mutation_table select number, toString(number) from numbers(100000) where number % 10 != 0" -$CLICKHOUSE_CLIENT --query "INSERT INTO mutation_table VALUES(0, 'hello')" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO mutation_table VALUES(0, 'hello')" $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM mutation_table" @@ -71,4 +72,4 @@ $CLICKHOUSE_CLIENT --query "SELECT is_done, parts_to_do FROM system.mutations wh $CLICKHOUSE_CLIENT --query "SELECT type, new_part_name FROM system.replication_queue WHERE table='mutation_table' and database='$CLICKHOUSE_DATABASE'" -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table SYNC" diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql index 8bf5d4f2cf8..f20156fd9e3 100644 --- a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql @@ -1,7 +1,10 @@ -- Tags: long, replica -DROP TABLE IF EXISTS test; -DROP TABLE IF EXISTS test2; +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries +SET replication_alter_partitions_sync=2; + +DROP TABLE IF EXISTS test SYNC; +DROP TABLE IF EXISTS test2 SYNC; CREATE TABLE test (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01346/table', 'r1') PARTITION BY x ORDER BY y; CREATE TABLE test2 (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01346/table', 'r2') PARTITION BY x ORDER BY y; @@ -17,7 +20,6 @@ ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'goodbye' = 3); INSERT INTO test VALUES ('goodbye', 'test'); OPTIMIZE TABLE test FINAL; SELECT * FROM test ORDER BY x; -SYSTEM SYNC REPLICA test2; SELECT * FROM test2 ORDER BY x; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test' AND active ORDER BY partition; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test2' AND active ORDER BY partition; @@ -31,7 +33,6 @@ ALTER TABLE test MODIFY COLUMN x Int8; INSERT INTO test VALUES (111, 'abc'); OPTIMIZE TABLE test FINAL; SELECT * FROM test ORDER BY x; -SYSTEM SYNC REPLICA test2; SELECT * FROM test2 ORDER BY x; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test' AND active ORDER BY partition; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test2' AND active ORDER BY partition; @@ -48,5 +49,5 @@ ALTER TABLE test RENAME COLUMN y TO z; -- { serverError 524 } ALTER TABLE test DROP COLUMN x; -- { serverError 47 } ALTER TABLE test DROP COLUMN y; -- { serverError 47 } -DROP TABLE test; -DROP TABLE test2; +DROP TABLE test SYNC; +DROP TABLE test2 SYNC; diff --git a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh index 01678d1b500..4629450c1f9 100755 --- a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh +++ b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh @@ -12,9 +12,9 @@ FREEZE_OUT_STRUCTURE='backup_name String, backup_path String , part_backup_path # setup -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated;" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated SYNC;" ${CLICKHOUSE_CLIENT} --query "CREATE TABLE table_for_freeze_replicated (key UInt64, value String) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table_for_freeze_replicated', '1') ORDER BY key PARTITION BY key % 10;" -${CLICKHOUSE_CLIENT} --query "INSERT INTO table_for_freeze_replicated SELECT number, toString(number) from numbers(10);" +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query "INSERT INTO table_for_freeze_replicated SELECT number, toString(number) from numbers(10);" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE WITH NAME 'test_01417' FORMAT TSVWithNames SETTINGS alter_partition_verbose_result = 1;" \ | ${CLICKHOUSE_LOCAL} --structure "$ALTER_OUT_STRUCTURE, $FREEZE_OUT_STRUCTURE" \ @@ -25,7 +25,7 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE PAR --query "SELECT command_type, partition_id, part_name, backup_name FROM table" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated DETACH PARTITION '3';" -${CLICKHOUSE_CLIENT} --query "INSERT INTO table_for_freeze_replicated VALUES (3, '3');" +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query "INSERT INTO table_for_freeze_replicated VALUES (3, '3');" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated ATTACH PARTITION '3' FORMAT TSVWithNames SETTINGS alter_partition_verbose_result = 1;" \ | ${CLICKHOUSE_LOCAL} --structure "$ALTER_OUT_STRUCTURE, $ATTACH_OUT_STRUCTURE" \ @@ -38,4 +38,4 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE PAR --query "SELECT command_type, partition_id, part_name, backup_name, old_part_name FROM table" # teardown -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated;" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated SYNC;" diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql index 0155b83db31..eea231c9f58 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql @@ -1,5 +1,6 @@ -- Tags: long, replica, no-replicated-database +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET replication_alter_partitions_sync = 2; diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql index 9f9d1db78f1..bf7a471fa40 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql @@ -1,10 +1,11 @@ -- Tags: long, replica, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET replication_alter_partitions_sync = 2; -DROP TABLE IF EXISTS replica1; -DROP TABLE IF EXISTS replica2; +DROP TABLE IF EXISTS replica1 SYNC; +DROP TABLE IF EXISTS replica2 SYNC; CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/'||currentDatabase()||'test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/'||currentDatabase()||'test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; @@ -48,5 +49,5 @@ SELECT v FROM replica1 ORDER BY v; SELECT name FROM system.parts WHERE table = 'replica2' AND active AND database = currentDatabase(); -DROP TABLE replica1; -DROP TABLE replica2; +DROP TABLE replica1 SYNC; +DROP TABLE replica2 SYNC; diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh index 13c772e75d2..c05d813ca7f 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -11,7 +11,7 @@ NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r$i; + DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " done @@ -20,7 +20,11 @@ function thread { for x in {0..99}; do # sometimes we can try to commit obsolete part if fetches will be quite fast, # so supress warning messages like "Tried to commit obsolete part ... covered by ..." - $CLICKHOUSE_CLIENT --query "INSERT INTO r$1 SELECT $x % $NUM_REPLICAS = $1 ? $x - 1 : $x" 2>/dev/null # Replace some records as duplicates so they will be written by other replicas + # (2) keeper fault injection for inserts because + # it can be a cause of deduplicated parts be visible to SELECTs for sometime (until cleanup thread remove them), + # so the same SELECT on different replicas can return different results, i.e. test output will be non-deterministic + # (see #9712) + $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO r$1 SELECT $x % $NUM_REPLICAS = $1 ? $x - 1 : $x" 2>/dev/null # Replace some records as duplicates so they will be written by other replicas done } @@ -37,5 +41,5 @@ for i in $(seq 1 $NUM_REPLICAS); do done for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i;" + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i SYNC;" done diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh index 6eabc9ae1b5..209e18e3329 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh @@ -12,7 +12,7 @@ NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r$i; + DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " done @@ -22,7 +22,7 @@ valid_exceptions_to_retry='Quorum for previous write has not been satisfied yet| function thread { for x in {0..99}; do while true; do - $CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 0 --query "INSERT INTO r$1 SELECT $x" 2>&1 | grep -qE "$valid_exceptions_to_retry" || break + $CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 0 --insert_keeper_fault_injection_probability=0 --query "INSERT INTO r$1 SELECT $x" 2>&1 | grep -qE "$valid_exceptions_to_retry" || break done done } @@ -40,5 +40,5 @@ for i in $(seq 1 $NUM_REPLICAS); do done for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i;" + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i SYNC;" done diff --git a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh index 030ae017e71..a2ecef64bbb 100755 --- a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh +++ b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh @@ -12,7 +12,7 @@ NUM_REPLICAS=6 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r$i; + DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parallel_quorum_many', 'r$i') ORDER BY x; " done @@ -20,7 +20,7 @@ done function thread { i=0 retries=300 while [[ $i -lt $retries ]]; do # server can be dead - $CLICKHOUSE_CLIENT --insert_quorum 3 --insert_quorum_parallel 1 --query "INSERT INTO r$1 SELECT $2" && break + $CLICKHOUSE_CLIENT --insert_quorum 3 --insert_quorum_parallel 1 --insert_keeper_max_retries=100 --insert_keeper_retry_max_backoff_ms=10 --query "INSERT INTO r$1 SELECT $2" && break ((++i)) sleep 0.1 done diff --git a/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh b/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh index 9325cac0ae6..445706e35bf 100755 --- a/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh +++ b/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh @@ -9,8 +9,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1 SYNC" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2 SYNC" $CLICKHOUSE_CLIENT -q "CREATE TABLE parallel_q1 (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parallel_q', 'r1') ORDER BY tuple() SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0" @@ -19,9 +19,10 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE parallel_q2 (x UInt64) ENGINE=ReplicatedMerg $CLICKHOUSE_CLIENT -q "SYSTEM STOP REPLICATION QUEUES parallel_q2" -$CLICKHOUSE_CLIENT -q "INSERT INTO parallel_q1 VALUES (1)" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "INSERT INTO parallel_q1 VALUES (1)" -$CLICKHOUSE_CLIENT --insert_quorum 2 --insert_quorum_parallel 1 --query="INSERT INTO parallel_q1 VALUES (2)" & +# disable keeper fault injection during insert since test checks part names. Part names can differ in case of retries during insert +$CLICKHOUSE_CLIENT --insert_quorum 2 --insert_quorum_parallel 1 --insert_keeper_fault_injection_probability=0 --query="INSERT INTO parallel_q1 VALUES (2)" & part_count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT() FROM system.parts WHERE table='parallel_q1' and database='${CLICKHOUSE_DATABASE}'") @@ -66,5 +67,5 @@ $CLICKHOUSE_CLIENT --query="SELECT event_type FROM system.part_log WHERE table=' $CLICKHOUSE_CLIENT --query="SELECT COUNT() FROM parallel_q2" $CLICKHOUSE_CLIENT --query="SELECT COUNT() FROM parallel_q1" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1 SYNC" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2 SYNC" diff --git a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql index 2f0c59ab64b..86bef193f4d 100644 --- a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql +++ b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql @@ -16,6 +16,9 @@ CREATE TABLE r2 ( ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01509_parallel_quorum_insert_no_replicas', '2') ORDER BY tuple(); +SET insert_keeper_max_retries=100; +SET insert_keeper_retry_max_backoff_ms=10; + SET insert_quorum_parallel=1; SET insert_quorum=3; @@ -79,11 +82,11 @@ SYSTEM STOP FETCHES r2; SET insert_quorum_timeout=0; -INSERT INTO r1 VALUES (4, '4'); -- { serverError 319 } +INSERT INTO r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (4, '4'); -- { serverError 319 } -- retry should fail despite the insert_deduplicate enabled -INSERT INTO r1 VALUES (4, '4'); -- { serverError 319 } -INSERT INTO r1 VALUES (4, '4'); -- { serverError 319 } +INSERT INTO r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (4, '4'); -- { serverError 319 } +INSERT INTO r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (4, '4'); -- { serverError 319 } SELECT * FROM r2 WHERE key=4; SYSTEM START FETCHES r2; @@ -99,5 +102,5 @@ SELECT 'insert happened'; SELECT COUNT() FROM r1; SELECT COUNT() FROM r2; -DROP TABLE IF EXISTS r1; -DROP TABLE IF EXISTS r2; +DROP TABLE IF EXISTS r1 SYNC; +DROP TABLE IF EXISTS r2 SYNC; diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql index cf06af0113d..f217b6094b2 100644 --- a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql +++ b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql @@ -2,6 +2,8 @@ -- Tag no-replicated-database: Fails due to additional replicas or shards -- Tag no-parallel: static zk path +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + DROP TABLE IF EXISTS execute_on_single_replica_r1 NO DELAY; DROP TABLE IF EXISTS execute_on_single_replica_r2 NO DELAY; diff --git a/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql b/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql index af2a46cd77f..b55b4871363 100644 --- a/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql +++ b/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql @@ -1,6 +1,6 @@ -- Tags: zookeeper -DROP TABLE IF EXISTS merge_tree_pk; +DROP TABLE IF EXISTS merge_tree_pk SYNC; CREATE TABLE merge_tree_pk ( @@ -24,9 +24,9 @@ ATTACH TABLE merge_tree_pk; SELECT * FROM merge_tree_pk FINAL ORDER BY key, value; -DROP TABLE IF EXISTS merge_tree_pk; +DROP TABLE IF EXISTS merge_tree_pk SYNC; -DROP TABLE IF EXISTS merge_tree_pk_sql; +DROP TABLE IF EXISTS merge_tree_pk_sql SYNC; CREATE TABLE merge_tree_pk_sql ( @@ -60,9 +60,9 @@ SELECT * FROM merge_tree_pk_sql FINAL ORDER BY key, value; SHOW CREATE TABLE merge_tree_pk_sql; -DROP TABLE IF EXISTS merge_tree_pk_sql; +DROP TABLE IF EXISTS merge_tree_pk_sql SYNC; -DROP TABLE IF EXISTS replicated_merge_tree_pk_sql; +DROP TABLE IF EXISTS replicated_merge_tree_pk_sql SYNC; CREATE TABLE replicated_merge_tree_pk_sql ( @@ -99,4 +99,4 @@ ATTACH TABLE replicated_merge_tree_pk_sql; SHOW CREATE TABLE replicated_merge_tree_pk_sql; -DROP TABLE IF EXISTS replicated_merge_tree_pk_sql; +DROP TABLE IF EXISTS replicated_merge_tree_pk_sql SYNC; diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql index e52e0c94c3c..a5f301b1353 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql @@ -1,6 +1,6 @@ -- Tags: replica -DROP TABLE IF EXISTS replicated_mutations_empty_partitions; +DROP TABLE IF EXISTS replicated_mutations_empty_partitions SYNC; CREATE TABLE replicated_mutations_empty_partitions ( @@ -11,7 +11,8 @@ ENGINE = ReplicatedMergeTree('/clickhouse/test/'||currentDatabase()||'/01586_rep ORDER BY key PARTITION by key; -INSERT INTO replicated_mutations_empty_partitions SELECT number, toString(number) FROM numbers(10); +-- insert_keeper* settings are adjusted since several actual inserts are happening behind one statement due to partitioning i.e. inserts in different partitions +INSERT INTO replicated_mutations_empty_partitions SETTINGS insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=10 SELECT number, toString(number) FROM numbers(10); SELECT count(distinct value) FROM replicated_mutations_empty_partitions; @@ -31,4 +32,4 @@ SELECT sum(value) FROM replicated_mutations_empty_partitions; SHOW CREATE TABLE replicated_mutations_empty_partitions; -DROP TABLE IF EXISTS replicated_mutations_empty_partitions; +DROP TABLE IF EXISTS replicated_mutations_empty_partitions SYNC; diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh index acaa2cfcd25..d68f9bc1837 100755 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh @@ -5,11 +5,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill SYNC" $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_mutate_kill (key UInt64, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_mutate_kill', '1') ORDER BY key PARTITION BY key % 100 SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000" -$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_mutate_kill SELECT number, toString(number) FROM numbers(1000000)" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO concurrent_mutate_kill SELECT number, toString(number) FROM numbers(1000000)" function alter_thread { @@ -67,4 +67,4 @@ done $CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE concurrent_mutate_kill" $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE concurrent_mutate_kill FINAL" $CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM concurrent_mutate_kill" -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill SYNC" diff --git a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql index c77f29d89c2..e2926d9a8ac 100644 --- a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql +++ b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql @@ -1,7 +1,8 @@ -- Tags: long, zookeeper, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards -DROP TABLE IF EXISTS partitioned_table; +SET insert_keeper_fault_injection_probability=0; +DROP TABLE IF EXISTS partitioned_table SYNC; CREATE TABLE partitioned_table ( key UInt64, @@ -47,4 +48,4 @@ SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AN SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; -DROP TABLE IF EXISTS partitioned_table; +DROP TABLE IF EXISTS partitioned_table SYNC; diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql index dad5cdbf1f3..87e1a039488 100644 --- a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql +++ b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql @@ -1,7 +1,7 @@ -- Tags: long, zookeeper -DROP TABLE IF EXISTS i20203_1; -DROP TABLE IF EXISTS i20203_2; +DROP TABLE IF EXISTS i20203_1 SYNC; +DROP TABLE IF EXISTS i20203_2 SYNC; CREATE TABLE i20203_1 (a Int8) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01715_background_checker_i20203', 'r1') @@ -26,5 +26,5 @@ WHERE table = 'i20203_2' AND database = currentDatabase(); ATTACH TABLE i20203_1; -DROP TABLE IF EXISTS i20203_1; -DROP TABLE IF EXISTS i20203_2; +DROP TABLE i20203_1 SYNC; +DROP TABLE i20203_2 SYNC; diff --git a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh index 85662438f33..4e4255fe9c2 100755 --- a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh +++ b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh @@ -51,6 +51,9 @@ $CLICKHOUSE_CLIENT -nm -q """ partition by key%100 settings max_part_removal_threads=10, concurrent_part_removal_threshold=99, min_bytes_for_wide_part=0; + SET insert_keeper_max_retries=1000; + SET insert_keeper_retry_max_backoff_ms=10; + insert into rep_data_01810 select * from numbers(100); drop table rep_data_01810 settings log_queries=1; system flush logs; diff --git a/tests/queries/0_stateless/01825_type_json_3.sql.j2 b/tests/queries/0_stateless/01825_type_json_3.sql.j2 index 62d86c3efd4..0fbf7a936d8 100644 --- a/tests/queries/0_stateless/01825_type_json_3.sql.j2 +++ b/tests/queries/0_stateless/01825_type_json_3.sql.j2 @@ -2,6 +2,8 @@ {% for engine in ["ReplicatedMergeTree('/clickhouse/tables/{database}/test_01825_3/t_json_3', 'r1') ORDER BY tuple()", "Memory"] -%} +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + SET allow_experimental_object_type = 1; DROP TABLE IF EXISTS t_json_3; diff --git a/tests/queries/0_stateless/01870_modulo_partition_key.sql b/tests/queries/0_stateless/01870_modulo_partition_key.sql index 1de23c13a65..46cd09622b1 100644 --- a/tests/queries/0_stateless/01870_modulo_partition_key.sql +++ b/tests/queries/0_stateless/01870_modulo_partition_key.sql @@ -1,3 +1,6 @@ +SET insert_keeper_max_retries=100; +SET insert_keeper_retry_max_backoff_ms=10; + SELECT 'simple partition key:'; DROP TABLE IF EXISTS table1 SYNC; CREATE TABLE table1 (id Int64, v UInt64) @@ -15,7 +18,7 @@ select 'where id % 200 < 0:'; select id from table1 where id % 200 < 0 order by id; SELECT 'tuple as partition key:'; -DROP TABLE IF EXISTS table2; +DROP TABLE IF EXISTS table2 SYNC; CREATE TABLE table2 (id Int64, v UInt64) ENGINE = MergeTree() PARTITION BY (toInt32(id / 2) % 3, id % 200) ORDER BY id; @@ -24,7 +27,7 @@ INSERT INTO table2 SELECT number-205, number FROM numbers(400, 10); SELECT partition as p FROM system.parts WHERE table='table2' and database=currentDatabase() ORDER BY p; SELECT 'recursive modulo partition key:'; -DROP TABLE IF EXISTS table3; +DROP TABLE IF EXISTS table3 SYNC; CREATE TABLE table3 (id Int64, v UInt64) ENGINE = MergeTree() PARTITION BY (id % 200, (id % 200) % 10, toInt32(round((id % 200) / 2, 0))) ORDER BY id; @@ -38,7 +41,7 @@ SELECT 'After detach:'; SELECT partition as p FROM system.parts WHERE table='table3' and database=currentDatabase() ORDER BY p; SELECT 'Indexes:'; -DROP TABLE IF EXISTS table4; +DROP TABLE IF EXISTS table4 SYNC; CREATE TABLE table4 (id Int64, v UInt64, s String, INDEX a (id * 2, s) TYPE minmax GRANULARITY 3 ) ENGINE = MergeTree() PARTITION BY id % 10 ORDER BY v; diff --git a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql index 635da491aa0..8924627a717 100644 --- a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql +++ b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql @@ -1,4 +1,5 @@ -- Tags: long, no-parallel +SET insert_keeper_fault_injection_probability=0; -- to succeed this test can require too many retries due to 1024 partitions, so disable fault injections -- regression for MEMORY_LIMIT_EXCEEDED error because of deferred final part flush @@ -8,8 +9,8 @@ insert into data_02228 select number, 1, number from numbers_mt(100e3) settings insert into data_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=10000000; -- { serverError MEMORY_LIMIT_EXCEEDED } drop table data_02228; -drop table if exists data_rep_02228; +drop table if exists data_rep_02228 SYNC; create table data_rep_02228 (key1 UInt32, sign Int8, s UInt64) engine = ReplicatedCollapsingMergeTree('/clickhouse/{database}', 'r1', sign) order by (key1) partition by key1 % 1024; insert into data_rep_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=0; insert into data_rep_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=10000000; -- { serverError MEMORY_LIMIT_EXCEEDED } -drop table data_rep_02228; +drop table data_rep_02228 SYNC; diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index e8111bf7a03..9ea924377b2 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -10,7 +10,7 @@ function check_refcnt_for_table() local table=$1 && shift $CLICKHOUSE_CLIENT -q "system stop merges $table" - $CLICKHOUSE_CLIENT -q "insert into $table select number, number%4 from numbers(200)" + $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into $table select number, number%4 from numbers(200)" local query_id query_id="$table-$(random_str 10)" @@ -52,7 +52,7 @@ $CLICKHOUSE_CLIENT -nmq " check_refcnt_for_table data_02340 $CLICKHOUSE_CLIENT -nmq " - drop table if exists data_02340_rep; + drop table if exists data_02340_rep sync; create table data_02340_rep (key Int, part Int) engine=ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') partition by part order by key settings index_granularity=1; " || exit 1 check_refcnt_for_table data_02340_rep diff --git a/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh b/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh index 11caf1e45de..88ff1f5b7c6 100755 --- a/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh +++ b/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh @@ -1,18 +1,18 @@ #!/usr/bin/env bash -# Tags: zookeeper +# Tags: long, zookeeper CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "drop table if exists rmt1;" -$CLICKHOUSE_CLIENT -q "drop table if exists rmt2;" +$CLICKHOUSE_CLIENT -q "drop table if exists rmt1 sync;" +$CLICKHOUSE_CLIENT -q "drop table if exists rmt2 sync;" $CLICKHOUSE_CLIENT -q "create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02369/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{database}', '1') order by n;" $CLICKHOUSE_CLIENT -q "create table rmt2 (n int) engine=ReplicatedMergeTree('/test/02369/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{database}', '2') order by n;" -$CLICKHOUSE_CLIENT -q "insert into rmt1 values (1);" -$CLICKHOUSE_CLIENT -q "insert into rmt1 values (2);" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (1);" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (2);" $CLICKHOUSE_CLIENT -q "system sync replica rmt1;" $CLICKHOUSE_CLIENT -q "system sync replica rmt2;" @@ -32,7 +32,7 @@ $CLICKHOUSE_CLIENT -q "select * from rmt1;" 2>/dev/null $CLICKHOUSE_CLIENT -q "detach table rmt1;" $CLICKHOUSE_CLIENT -q "attach table rmt1;" -$CLICKHOUSE_CLIENT -q "insert into rmt1 values (3);" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (3);" $CLICKHOUSE_CLIENT -q "system start merges rmt2;" $CLICKHOUSE_CLIENT -q "system sync replica rmt1;" $CLICKHOUSE_CLIENT -q "optimize table rmt1 final;" @@ -42,5 +42,5 @@ $CLICKHOUSE_CLIENT -q "system sync replica rmt2;" $CLICKHOUSE_CLIENT -q "select 3, *, _part from rmt1 order by n;" $CLICKHOUSE_CLIENT -q "select 4, *, _part from rmt2 order by n;" -$CLICKHOUSE_CLIENT -q "drop table rmt1;" -$CLICKHOUSE_CLIENT -q "drop table rmt2;" +$CLICKHOUSE_CLIENT -q "drop table rmt1 sync;" +$CLICKHOUSE_CLIENT -q "drop table rmt2 sync;" diff --git a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql index 371f7389837..0db816332a1 100644 --- a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql +++ b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql @@ -1,5 +1,7 @@ -- Tags: long +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + drop table if exists rmt1; drop table if exists rmt2; create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '1') order by tuple() diff --git a/tests/queries/0_stateless/02456_keeper_retries_during_insert.reference b/tests/queries/0_stateless/02456_keeper_retries_during_insert.reference new file mode 100644 index 00000000000..f5f1cf8ac64 --- /dev/null +++ b/tests/queries/0_stateless/02456_keeper_retries_during_insert.reference @@ -0,0 +1,2 @@ +1 +11 diff --git a/tests/queries/0_stateless/02456_keeper_retries_during_insert.sql b/tests/queries/0_stateless/02456_keeper_retries_during_insert.sql new file mode 100644 index 00000000000..774370bc132 --- /dev/null +++ b/tests/queries/0_stateless/02456_keeper_retries_during_insert.sql @@ -0,0 +1,26 @@ +-- Tags: replica + +DROP TABLE IF EXISTS keeper_retries_r1 SYNC; +DROP TABLE IF EXISTS keeper_retries_r2 SYNC; + +CREATE TABLE keeper_retries_r1(a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/02456_keeper_retries_during_insert', 'r1') ORDER BY tuple (); +CREATE TABLE keeper_retries_r2(a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/02456_keeper_retries_during_insert', 'r2') ORDER BY tuple(); + +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (1); +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_max_retries=0 VALUES (2); -- { serverError KEEPER_EXCEPTION } +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_retry_max_backoff_ms=10 VALUES (3); -- { serverError KEEPER_EXCEPTION } + +SET insert_quorum=2; +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (11); +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_max_retries=0 VALUES (12); -- { serverError KEEPER_EXCEPTION } +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_retry_max_backoff_ms=1 VALUES (13); -- { serverError KEEPER_EXCEPTION } + +-- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.05, insert_keeper_fault_injection_seed=1 VALUES (21); +-- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.2, insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=1, insert_keeper_fault_injection_seed=2 VALUES (22); +-- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.3, insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=1, insert_keeper_fault_injection_seed=3 VALUES (23); +-- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.4, insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=1, insert_keeper_fault_injection_seed=4 VALUES (24); + +SELECT * FROM keeper_retries_r1 order by a; + +DROP TABLE keeper_retries_r1 SYNC; +DROP TABLE keeper_retries_r2 SYNC; From c00f71327a3eb7f90a58b42adaaa980a6b731f73 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 10 Nov 2022 12:29:24 +0000 Subject: [PATCH 31/82] Fix: backward-compatility check - remove unused config for now --- tests/config/users.d/insert_keeper_retries.xml | 8 -------- 1 file changed, 8 deletions(-) delete mode 100644 tests/config/users.d/insert_keeper_retries.xml diff --git a/tests/config/users.d/insert_keeper_retries.xml b/tests/config/users.d/insert_keeper_retries.xml deleted file mode 100644 index f7b652bf2ef..00000000000 --- a/tests/config/users.d/insert_keeper_retries.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - - 0 - 0.0 - - - From c4f522b41467aa8514cd7f57d1331419e0447d45 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Nov 2022 16:32:15 +0100 Subject: [PATCH 32/82] Support arbintrary structure collections --- src/Common/ErrorCodes.cpp | 1 + src/Storages/NamedCollections.cpp | 432 ++++++++++++++---- src/Storages/NamedCollections.h | 52 +-- .../tests/gtest_named_collections.cpp | 79 +++- 4 files changed, 434 insertions(+), 130 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 26f837f9fe5..ce96b1a9917 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -638,6 +638,7 @@ M(667, NOT_INITIALIZED) \ M(668, INVALID_STATE) \ M(669, UNKNOWN_NAMED_COLLECTION) \ + M(669, NAMED_COLLECTION_ALREADY_EXISTS) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/NamedCollections.cpp b/src/Storages/NamedCollections.cpp index 94870507b59..f598a302516 100644 --- a/src/Storages/NamedCollections.cpp +++ b/src/Storages/NamedCollections.cpp @@ -1,12 +1,15 @@ #include "NamedCollections.h" +#include #include +#include #include #include #include #include #include #include +#include #include #include #include @@ -15,18 +18,63 @@ namespace DB { -static constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections."; +static constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections"; namespace ErrorCodes { extern const int UNKNOWN_NAMED_COLLECTION; + extern const int NAMED_COLLECTION_ALREADY_EXISTS; } namespace { std::string getCollectionPrefix(const std::string & collection_name) { - return NAMED_COLLECTIONS_CONFIG_PREFIX + collection_name; + return fmt::format("{}.{}", NAMED_COLLECTIONS_CONFIG_PREFIX, collection_name); + } + + /// Enumerate keys paths of the config recursively. + /// E.g. if `enumerate_paths` = {"root.key1"} and config like + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// the `result` will contain two strings: "root.key1.key2" and "root.key1.key3.key4" + void collectKeys( + const Poco::Util::AbstractConfiguration & config, + std::queue & enumerate_paths, + std::set & result) + { + if (enumerate_paths.empty()) + return; + + auto initial_paths = std::move(enumerate_paths); + enumerate_paths = {}; + while (!initial_paths.empty()) + { + auto path = initial_paths.front(); + initial_paths.pop(); + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(path, keys); + + if (keys.empty()) + { + result.insert(path); + } + else + { + for (const auto & key : keys) + enumerate_paths.emplace(path + '.' + key); + } + } + + collectKeys(config, enumerate_paths, result); } } @@ -36,35 +84,48 @@ NamedCollectionFactory & NamedCollectionFactory::instance() return instance; } -void NamedCollectionFactory::initialize(const Poco::Util::AbstractConfiguration & server_config) +void NamedCollectionFactory::initialize( + const Poco::Util::AbstractConfiguration & server_config) { std::lock_guard lock(mutex); if (is_initialized) { throw Exception( ErrorCodes::LOGICAL_ERROR, - "Named collection factory is already initialzied"); + "Named collection factory already initialized"); } config = &server_config; is_initialized = true; } -void NamedCollectionFactory::assertInitialized(std::lock_guard & /* lock */) const +void NamedCollectionFactory::assertInitialized( + std::lock_guard & /* lock */) const { if (!is_initialized) { throw Exception( ErrorCodes::LOGICAL_ERROR, - "Named collection factory must be initialized before used"); + "Named collection factory must be initialized before being used"); } } bool NamedCollectionFactory::exists(const std::string & collection_name) const { std::lock_guard lock(mutex); + return existsUnlocked(collection_name, lock); +} + +bool NamedCollectionFactory::existsUnlocked( + const std::string & collection_name, + std::lock_guard & lock) const +{ assertInitialized(lock); - return config->has(getCollectionPrefix(collection_name)); + /// Named collections can be added via SQL command or via config. + /// Named collections from config are loaded on first access, + /// therefore it might not be in `named_collections` map yet. + return named_collections.contains(collection_name) + || config->has(getCollectionPrefix(collection_name)); } NamedCollectionPtr NamedCollectionFactory::get( @@ -74,11 +135,13 @@ NamedCollectionPtr NamedCollectionFactory::get( std::lock_guard lock(mutex); assertInitialized(lock); - if (!exists(collection_name)) + if (!existsUnlocked(collection_name, lock)) + { throw Exception( ErrorCodes::UNKNOWN_NAMED_COLLECTION, - "There is no named collection `{}` in config", + "There is no named collection `{}`", collection_name); + } return getImpl(collection_name, collection_info, lock); } @@ -90,7 +153,7 @@ NamedCollectionPtr NamedCollectionFactory::tryGet( std::lock_guard lock(mutex); assertInitialized(lock); - if (!exists(collection_name)) + if (!existsUnlocked(collection_name, lock)) return nullptr; return getImpl(collection_name, collection_info, lock); @@ -104,87 +167,192 @@ NamedCollectionPtr NamedCollectionFactory::getImpl( auto it = named_collections.find(collection_name); if (it == named_collections.end()) { - const auto collection_prefix = getCollectionPrefix(collection_name); - const auto collection_view = config->createView(collection_prefix); - - auto collection = std::make_unique(); - collection->initialize(*collection_view, collection_info); - it = named_collections.emplace(collection_name, std::move(collection)).first; + it = named_collections.emplace( + collection_name, + std::make_unique( + *config, collection_name, collection_info)).first; } return it->second; } +void NamedCollectionFactory::add( + const std::string & collection_name, + NamedCollectionPtr collection) +{ + std::lock_guard lock(mutex); + auto [it, inserted] = named_collections.emplace(collection_name, collection); + if (!inserted) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "A named collection `{}` already exists", + collection_name); + } +} + +void NamedCollectionFactory::remove(const std::string & collection_name) +{ + std::lock_guard lock(mutex); + assertInitialized(lock); + + if (!existsUnlocked(collection_name, lock)) + { + throw Exception( + ErrorCodes::UNKNOWN_NAMED_COLLECTION, + "There is no named collection `{}`", + collection_name); + } + + if (config->has(collection_name)) + { + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Collection {} is defined in config and cannot be removed", + collection_name); + } + + [[maybe_unused]] auto removed = named_collections.erase(collection_name); + assert(removed); +} + +NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const +{ + std::lock_guard lock(mutex); + assertInitialized(lock); + + NamedCollections result(named_collections); + + Poco::Util::AbstractConfiguration::Keys config_collections_names; + config->keys(NAMED_COLLECTIONS_CONFIG_PREFIX, config_collections_names); + + for (const auto & name : config_collections_names) + { + if (result.contains(name)) + continue; + + const auto collection_prefix = getCollectionPrefix(name); + std::queue enumerate_input; + std::set enumerate_result; + + enumerate_input.push(collection_prefix); + collectKeys(*config, enumerate_input, enumerate_result); + + NamedCollectionInfo collection_info; + + /// Collection does not have any keys. + /// (`enumerate_result` == ). + const bool collection_is_empty = enumerate_result.size() == 1; + if (!collection_is_empty) + { + for (const auto & path : enumerate_result) + { + collection_info.emplace( + /// Skip collection prefix and add +1 to avoid '.' in the beginning. + path.substr(std::strlen(collection_prefix.data()) + 1), + NamedCollectionValueInfo{}); + } + } + + result.emplace( + name, std::make_unique(*config, name, collection_info)); + } + + return result; +} + struct NamedCollection::Impl { - std::unordered_map collection; +private: + using IConfigurationPtr = Poco::AutoPtr; + using ConfigurationPtr = Poco::AutoPtr; - ImplPtr copy() const + /// Named collection configuration + /// + /// ... + /// + ConfigurationPtr config; + /// Information about the values of keys. Key is a path to the + /// value represented as a dot concatenated list of keys. + const CollectionInfo collection_info; + +public: + Impl(const Poco::Util::AbstractConfiguration & config_, + const std::string & collection_name_, + const NamedCollectionInfo & collection_info_) + : config(createEmptyConfiguration(collection_name_)) + , collection_info(collection_info_) { - auto impl = std::make_unique(); - impl->collection = collection; - return impl; + auto collection_path = getCollectionPrefix(collection_name_); + for (const auto & [key, value_info] : collection_info) + copyConfigValue( + config_, collection_path + '.' + key, *config, key, value_info.type); } Value get(const Key & key) const { - auto it = collection.find(key); - if (it == collection.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no key: {}", key); - return it->second; + auto value_info = collection_info.at(key); + return getConfigValue(*config, key, value_info.type, value_info.is_required); } - void replace(const Key & key, const Value & value) + void set(const Key & key, const Value & value) { - auto it = collection.find(key); - if (it == collection.end()) - collection.emplace(key, value); - else - it->second = value; + setConfigValue(*config, key, value); } - void initialize( - const Poco::Util::AbstractConfiguration & config, - const NamedCollectionInfo & collection_info) + /// Get a string representation of the collection structure. + /// Used for debugging and tests. + std::string toString() const { - for (const auto & [key, key_info] : collection_info) + /// Convert a collection config like + /// + /// value0 + /// + /// value2 + /// + /// value3 + /// + /// + /// + /// to a string: + /// "key0: value0 + /// key1: + /// key2: value2 + /// key3: + /// key4: value3" + WriteBufferFromOwnString wb; + for (const auto & [key, value_info] : collection_info) { - const auto & default_value = key_info.default_value; - const bool has_value = config.has(key); + Strings key_parts; + splitInto<'.'>(key_parts, key); + size_t tab_cnt = 0; - if (!default_value && !has_value) - continue; - - Field value; - switch (key_info.type) + for (auto it = key_parts.begin(); it != key_parts.end(); ++it) { - case Field::Types::Which::String: - value = has_value ? config.getString(key) : default_value->get(); - break; - case Field::Types::Which::UInt64: - value = has_value ? config.getUInt64(key) : default_value->get(); - break; - case Field::Types::Which::Int64: - value = has_value ? config.getInt64(key) : default_value->get(); - break; - case Field::Types::Which::Float64: - value = has_value ? config.getDouble(key) : default_value->get(); - break; - default: - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Unsupported type: {}", toString(key_info.type)); + if (it != key_parts.begin()) + wb << '\n' << std::string(tab_cnt++, '\t'); + wb << *it << ':'; } - - collection.emplace(key, value); + wb << '\t' << convertFieldToString(get(key)) << '\n'; } + return wb.str(); } +private: static void validate( const Poco::Util::AbstractConfiguration & config, - const NamedCollectionInfo & collection_info) + const std::string & collection_path, + const NamedCollectionInfo & collection_info_) { Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys("", config_keys); + config.keys(collection_path, config_keys); + checkKeys(config_keys, collection_info_); + } + static void checkKeys( + const Poco::Util::AbstractConfiguration::Keys & config_keys, + const NamedCollectionInfo & collection_info) + + { auto get_suggestion = [&](bool only_required_keys) { std::string suggestion; @@ -228,16 +396,121 @@ struct NamedCollection::Impl fmt::join(required_keys, ", "), get_suggestion(true)); } } + + static ConfigurationPtr createEmptyConfiguration(const std::string & root_name) + { + using DocumentPtr = Poco::AutoPtr; + DocumentPtr xml_document(new Poco::XML::Document()); + xml_document->appendChild(xml_document->createElement(root_name)); + ConfigurationPtr config(new Poco::Util::XMLConfiguration(xml_document)); + return config; + } + + using ConfigValueType = Field::Types::Which; + static void copyConfigValue( + const Poco::Util::AbstractConfiguration & from_config, + const std::string & from_path, + Poco::Util::AbstractConfiguration & to_config, + const std::string & to_path, + ConfigValueType type) + { + using Type = Field::Types::Which; + switch (type) + { + case Type::String: + to_config.setString(to_path, from_config.getString(from_path)); + break; + case Type::UInt64: + to_config.setUInt64(to_path, from_config.getUInt64(from_path)); + break; + case Type::Int64: + to_config.setInt64(to_path, from_config.getInt64(from_path)); + break; + case Type::Float64: + to_config.setDouble(to_path, from_config.getDouble(from_path)); + break; + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported type"); + } + } + + static void setConfigValue( + Poco::Util::AbstractConfiguration & config, + const std::string & path, + const Field & value) + { + using Type = Field::Types::Which; + switch (value.getType()) + { + case Type::String: + config.setString(path, value.safeGet()); + break; + case Type::UInt64: + config.setUInt64(path, value.safeGet()); + break; + case Type::Int64: + config.setInt64(path, value.safeGet()); + break; + case Type::Float64: + config.setDouble(path, value.safeGet()); + break; + default: + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Unsupported type"); + } + } + + static Field getConfigValue( + const Poco::Util::AbstractConfiguration & config, + const std::string & path, + ConfigValueType type, + bool throw_not_found, + std::optional default_value = std::nullopt) + { + const bool has_value = config.has(path); + if (!has_value) + { + if (throw_not_found) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Failed to find key `{}` in config, but this key is required", + path); + } + else if (!default_value) + return Null{}; + } + + Field value; + + using Type = Field::Types::Which; + switch (type) + { + case Type::String: + value = has_value ? config.getString(path) : default_value->get(); + break; + case Type::UInt64: + value = has_value ? config.getUInt64(path) : default_value->get(); + break; + case Type::Int64: + value = has_value ? config.getInt64(path) : default_value->get(); + break; + case Type::Float64: + value = has_value ? config.getDouble(path) : default_value->get(); + break; + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported type"); + } + return value; + } }; -NamedCollection::NamedCollection() +NamedCollection::NamedCollection( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_path, + const CollectionInfo & collection_info) + : pimpl(std::make_unique(config, collection_path, collection_info)) { - pimpl = std::make_unique(); -} - -NamedCollection::NamedCollection(ImplPtr pimpl_) -{ - pimpl = std::move(pimpl_); } NamedCollection::Value NamedCollection::get(const Key & key) const @@ -245,28 +518,9 @@ NamedCollection::Value NamedCollection::get(const Key & key) const return pimpl->get(key); } -std::shared_ptr NamedCollection::copy() const +std::string NamedCollection::toString() const { - return std::make_shared(pimpl->copy()); -} - -void NamedCollection::validate( - const Poco::Util::AbstractConfiguration & config, - const NamedCollectionInfo & collection_info) const -{ - pimpl->validate(config, collection_info); -} - -void NamedCollection::initialize( - const Poco::Util::AbstractConfiguration & config, - const NamedCollectionInfo & collection_info) -{ - pimpl->initialize(config, collection_info); -} - -void NamedCollection::replace(const Key & key, const Value & value) -{ - pimpl->replace(key, value); + return pimpl->toString(); } } diff --git a/src/Storages/NamedCollections.h b/src/Storages/NamedCollections.h index 870d167889d..cfc7ef5ef2f 100644 --- a/src/Storages/NamedCollections.h +++ b/src/Storages/NamedCollections.h @@ -10,7 +10,7 @@ namespace DB class NamedCollection; using NamedCollectionPtr = std::shared_ptr; struct NamedCollectionValueInfo; -using NamedCollectionInfo = std::unordered_map; +using NamedCollectionInfo = std::map; /** * A factory of immutable named collections. @@ -28,6 +28,8 @@ using NamedCollectionInfo = std::unordered_map; + NamedCollections getAll() const; private: NamedCollectionPtr getImpl( @@ -48,10 +57,14 @@ private: const NamedCollectionInfo & collection_info, std::lock_guard & lock) const; - using NamedCollections = std::unordered_map; + bool existsUnlocked( + const std::string & collection_name, + std::lock_guard & lock) const; + mutable NamedCollections named_collections; private: + /// FIXME: this will be invalid when config is reloaded const Poco::Util::AbstractConfiguration * config; void assertInitialized(std::lock_guard & lock) const; @@ -74,31 +87,17 @@ private: public: using Key = std::string; using Value = Field; + using ValueInfo = NamedCollectionValueInfo; + using CollectionInfo = NamedCollectionInfo; + + NamedCollection( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_path, + const CollectionInfo & collection_info); Value get(const Key & key) const; - void replace(const Key & key, const Value & value); - - /// Copy current named collection to allow modification as - /// NamedConnectionFactory returns immutable collections. - std::shared_ptr copy() const; - - NamedCollection(); - explicit NamedCollection(ImplPtr pimpl_); - -protected: - /// Initialize from config. `config` must be a view to the required collection, - /// e.g. the root of `config` is the root of collection. - void initialize( - const Poco::Util::AbstractConfiguration & config, - const NamedCollectionInfo & collection_info); - - /// Validate named collection in config. - /// Throws exception if named collection keys in config are not the same as - /// expected (contains unknown keys or misses required keys) - void validate( - const Poco::Util::AbstractConfiguration & config, - const NamedCollectionInfo & collection_info) const; + std::string toString() const; }; @@ -109,7 +108,8 @@ protected: struct NamedCollectionValueInfo { /// Type of the value. One of: String, UInt64, Int64, Double. - Field::Types::Which type; + using Type = Field::Types::Which; + Type type = Type::String; /// Optional default value for the case if there is no such key in config. std::optional default_value; /// Is this value required or optional? Throw exception if the value is diff --git a/src/Storages/tests/gtest_named_collections.cpp b/src/Storages/tests/gtest_named_collections.cpp index 669e7bae25b..34dfe5c1087 100644 --- a/src/Storages/tests/gtest_named_collections.cpp +++ b/src/Storages/tests/gtest_named_collections.cpp @@ -6,15 +6,15 @@ using namespace DB; -TEST(NamedCollections, Simple) +TEST(NamedCollections, SimpleConfig) { std::string xml(R"CONFIG( value1 2 - -3 - 4.4 + 3.3 + -4 value4 @@ -34,6 +34,18 @@ TEST(NamedCollections, Simple) ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection2")); ASSERT_TRUE(NamedCollectionFactory::instance().tryGet("collection3", {}) == nullptr); + auto collections = NamedCollectionFactory::instance().getAll(); + ASSERT_EQ(collections.size(), 2); + ASSERT_TRUE(collections.contains("collection1")); + ASSERT_TRUE(collections.contains("collection2")); + + ASSERT_EQ(collections["collection1"]->toString(), + R"CONFIG(key1: value1 +key2: 2 +key3: 3.3 +key4: -4 +)CONFIG"); + using ValueInfo = NamedCollectionValueInfo; ValueInfo string_def{Field::Types::Which::String, std::nullopt, true}; ValueInfo uint_def{Field::Types::Which::UInt64, std::nullopt, true}; @@ -43,16 +55,17 @@ TEST(NamedCollections, Simple) NamedCollectionInfo collection1_info; collection1_info.emplace("key1", string_def); collection1_info.emplace("key2", uint_def); - collection1_info.emplace("key3", int_def); - collection1_info.emplace("key4", double_def); + collection1_info.emplace("key3", double_def); + collection1_info.emplace("key4", int_def); auto collection1 = NamedCollectionFactory::instance().get("collection1", collection1_info); ASSERT_TRUE(collection1 != nullptr); + ASSERT_TRUE(collection1->get("key1").safeGet() == "value1"); ASSERT_TRUE(collection1->get("key2").safeGet() == 2); - ASSERT_TRUE(collection1->get("key3").safeGet() == -3); - ASSERT_TRUE(collection1->get("key4").safeGet() == 4.4); + ASSERT_TRUE(collection1->get("key3").safeGet() == 3.3); + ASSERT_TRUE(collection1->get("key4").safeGet() == -4); NamedCollectionInfo collection2_info; collection2_info.emplace("key4", string_def); @@ -64,12 +77,48 @@ TEST(NamedCollections, Simple) ASSERT_TRUE(collection2->get("key4").safeGet() == "value4"); ASSERT_TRUE(collection2->get("key5").safeGet() == 5); - ASSERT_TRUE(collection2->get("key6").safeGet() == 6.6); - - auto mutable_collection2 = collection2->copy(); - mutable_collection2->replace("key4", UInt64(4)); - mutable_collection2->replace("key7", "value7"); - - ASSERT_TRUE(mutable_collection2->get("key4").safeGet() == 4); - ASSERT_TRUE(mutable_collection2->get("key7").safeGet() == "value7"); + ASSERT_TRUE(collection2->get("key6").safeGet() == 6.6); } + +// TEST(NamedCollections, NestedConfig) +// { +// std::string xml(R"CONFIG( +// +// +// +// value1 +// +// +// value2_1 +// +// +// value2_4 +// value2_5 +// +// +// +// +// +// )CONFIG"); +// +// Poco::XML::DOMParser dom_parser; +// Poco::AutoPtr document = dom_parser.parseString(xml); +// Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); +// +// ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1")); +// +// using ValueInfo = NamedCollectionValueInfo; +// ValueInfo string_def{Field::Types::Which::String, std::nullopt, true}; +// +// NamedCollectionInfo collection1_info; +// collection1_info.emplace("key1.key1_1", string_def); +// collection1_info.emplace("key2.key2_1", string_def); +// collection1_info.emplace("key2.key2_2.key2_3.key2_4", string_def); +// collection1_info.emplace("key2.key2_2.key2_3.key2_5", string_def); +// +// auto collection1 = NamedCollectionFactory::instance().get("collection1", collection1_info); +// ASSERT_TRUE(collection1 != nullptr); +// +// ASSERT_TRUE(collection1->get("key1.key1_1").safeGet() == "value1"); +// +// } From b1e942e14c6f7bcc5494f5150c5684cbd87877f2 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 10 Nov 2022 15:59:11 +0000 Subject: [PATCH 33/82] Do not run global test with sanitizers --- .../configs/global_overcommit_tracker.xml | 0 .../test_global_overcommit_tracker/test.py | 61 +++++++++++++++++++ .../test_overcommit_tracker/test.py | 37 +---------- 3 files changed, 62 insertions(+), 36 deletions(-) rename tests/integration/{test_overcommit_tracker => test_global_overcommit_tracker}/configs/global_overcommit_tracker.xml (100%) create mode 100644 tests/integration/test_global_overcommit_tracker/test.py diff --git a/tests/integration/test_overcommit_tracker/configs/global_overcommit_tracker.xml b/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml similarity index 100% rename from tests/integration/test_overcommit_tracker/configs/global_overcommit_tracker.xml rename to tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml diff --git a/tests/integration/test_global_overcommit_tracker/test.py b/tests/integration/test_global_overcommit_tracker/test.py new file mode 100644 index 00000000000..654f11be45a --- /dev/null +++ b/tests/integration/test_global_overcommit_tracker/test.py @@ -0,0 +1,61 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", main_configs=["configs/global_overcommit_tracker.xml"] +) + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +GLOBAL_TEST_QUERY_A = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS memory_overcommit_ratio_denominator_for_user=1" +GLOBAL_TEST_QUERY_B = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS memory_overcommit_ratio_denominator_for_user=80000000" + + +def test_global_overcommit(): + # NOTE: another option is to increase waiting time. + if ( + node.is_built_with_thread_sanitizer() + or node.is_built_with_address_sanitizer() + or node.is_built_with_memory_sanitizer() + ): + pytest.skip("doesn't fit in memory limits") + + node.query("CREATE USER IF NOT EXISTS A") + node.query("GRANT ALL ON *.* TO A") + node.query("CREATE USER IF NOT EXISTS B") + node.query("GRANT ALL ON *.* TO B") + + responses_A = list() + responses_B = list() + for i in range(100): + if i % 2 == 0: + responses_A.append(node.get_query_request(GLOBAL_TEST_QUERY_A, user="A")) + else: + responses_B.append(node.get_query_request(GLOBAL_TEST_QUERY_B, user="B")) + + overcommited_killed = False + for response in responses_A: + _, err = response.get_answer_and_error() + if "MEMORY_LIMIT_EXCEEDED" in err: + overcommited_killed = True + finished = False + for response in responses_B: + _, err = response.get_answer_and_error() + if err == "": + finished = True + + assert overcommited_killed, "no overcommited task was killed" + assert finished, "all tasks are killed" + + node.query("DROP USER IF EXISTS A") + node.query("DROP USER IF EXISTS B") diff --git a/tests/integration/test_overcommit_tracker/test.py b/tests/integration/test_overcommit_tracker/test.py index 5c18ee950d5..50af8525b99 100644 --- a/tests/integration/test_overcommit_tracker/test.py +++ b/tests/integration/test_overcommit_tracker/test.py @@ -5,7 +5,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance( - "node", main_configs=["configs/global_overcommit_tracker.xml"] + "node" ) @@ -18,9 +18,6 @@ def start_cluster(): cluster.shutdown() -GLOBAL_TEST_QUERY_A = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS memory_overcommit_ratio_denominator_for_user=1" -GLOBAL_TEST_QUERY_B = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS memory_overcommit_ratio_denominator_for_user=80000000" - USER_TEST_QUERY_A = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS max_memory_usage_for_user=2000000000,memory_overcommit_ratio_denominator=1" USER_TEST_QUERY_B = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS max_memory_usage_for_user=2000000000,memory_overcommit_ratio_denominator=80000000" @@ -52,35 +49,3 @@ def test_user_overcommit(): assert finished, "all tasks are killed" node.query("DROP USER IF EXISTS A") - - -def test_global_overcommit(): - node.query("CREATE USER IF NOT EXISTS A") - node.query("GRANT ALL ON *.* TO A") - node.query("CREATE USER IF NOT EXISTS B") - node.query("GRANT ALL ON *.* TO B") - - responses_A = list() - responses_B = list() - for i in range(100): - if i % 2 == 0: - responses_A.append(node.get_query_request(GLOBAL_TEST_QUERY_A, user="A")) - else: - responses_B.append(node.get_query_request(GLOBAL_TEST_QUERY_B, user="B")) - - overcommited_killed = False - for response in responses_A: - _, err = response.get_answer_and_error() - if "MEMORY_LIMIT_EXCEEDED" in err: - overcommited_killed = True - finished = False - for response in responses_B: - _, err = response.get_answer_and_error() - if err == "": - finished = True - - assert overcommited_killed, "no overcommited task was killed" - assert finished, "all tasks are killed" - - node.query("DROP USER IF EXISTS A") - node.query("DROP USER IF EXISTS B") From 4e15d648e7028c220ebacf823b249b380b472419 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 10 Nov 2022 16:02:45 +0000 Subject: [PATCH 34/82] Add test_global_overcommit_tracker/__init__.py --- tests/integration/test_global_overcommit_tracker/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_global_overcommit_tracker/__init__.py diff --git a/tests/integration/test_global_overcommit_tracker/__init__.py b/tests/integration/test_global_overcommit_tracker/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From bed7ac9442d90e1abc29a5fc0c5630b0a0b0119a Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Nov 2022 17:05:04 +0100 Subject: [PATCH 35/82] Support arbitrary structure collections --- programs/local/LocalServer.cpp | 3 ++ programs/server/Server.cpp | 3 ++ src/Access/Common/AccessType.h | 1 + .../System/StorageSystemNamedCollections.cpp | 38 +++++++++++++++++++ .../System/StorageSystemNamedCollections.h | 21 ++++++++++ src/Storages/System/attachSystemTables.cpp | 2 + 6 files changed, 68 insertions(+) create mode 100644 src/Storages/System/StorageSystemNamedCollections.cpp create mode 100644 src/Storages/System/StorageSystemNamedCollections.h diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 3ac9c1e7c37..8514e0f11af 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include #include @@ -118,6 +119,8 @@ void LocalServer::initialize(Poco::Util::Application & self) config().getUInt("max_io_thread_pool_size", 100), config().getUInt("max_io_thread_pool_free_size", 0), config().getUInt("io_thread_pool_queue_size", 10000)); + + NamedCollectionFactory::instance().initialize(config()); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b412b579539..079dae52f15 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -60,6 +60,7 @@ #include #include #include +#include #include #include #include @@ -732,6 +733,8 @@ int Server::main(const std::vector & /*args*/) config().getUInt("max_io_thread_pool_free_size", 0), config().getUInt("io_thread_pool_queue_size", 10000)); + NamedCollectionFactory::instance().initialize(config()); + /// Initialize global local cache for remote filesystem. if (config().has("local_cache_for_remote_fs")) { diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 5c85c93c98f..8263f50d1b0 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -130,6 +130,7 @@ enum class AccessType M(SHOW_ROW_POLICIES, "SHOW POLICIES, SHOW CREATE ROW POLICY, SHOW CREATE POLICY", TABLE, SHOW_ACCESS) \ M(SHOW_QUOTAS, "SHOW CREATE QUOTA", GLOBAL, SHOW_ACCESS) \ M(SHOW_SETTINGS_PROFILES, "SHOW PROFILES, SHOW CREATE SETTINGS PROFILE, SHOW CREATE PROFILE", GLOBAL, SHOW_ACCESS) \ + M(SHOW_NAMED_COLLECTIONS, "SHOW NAMED COLLECTIONS", GLOBAL, SHOW_ACCESS) \ M(SHOW_ACCESS, "", GROUP, ACCESS_MANAGEMENT) \ M(ACCESS_MANAGEMENT, "", GROUP, ALL) \ \ diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp new file mode 100644 index 00000000000..a53b6dc9bd3 --- /dev/null +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -0,0 +1,38 @@ +#include "StorageSystemNamedCollections.h" +#include +#include +#include +#include +#include + + +namespace DB +{ + +NamesAndTypesList StorageSystemNamedCollections::getNamesAndTypes() +{ + return { + {"name", std::make_shared()}, + {"collection", std::make_shared()}, + }; +} + +StorageSystemNamedCollections::StorageSystemNamedCollections(const StorageID & table_id_) + : IStorageSystemOneBlock(table_id_) +{ +} + +void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +{ + context->checkAccess(AccessType::SHOW_NAMED_COLLECTIONS); + + auto collections = NamedCollectionFactory::instance().getAll(); + + for (const auto & [name, collection] : collections) + { + res_columns[0]->insert(name); + res_columns[1]->insert(collection->toString()); + } +} + +} diff --git a/src/Storages/System/StorageSystemNamedCollections.h b/src/Storages/System/StorageSystemNamedCollections.h new file mode 100644 index 00000000000..d20fa62d30b --- /dev/null +++ b/src/Storages/System/StorageSystemNamedCollections.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +class StorageSystemNamedCollections final : public IStorageSystemOneBlock +{ +public: + explicit StorageSystemNamedCollections(const StorageID & table_id_); + + std::string getName() const override { return "SystemNamedCollections"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index d3b81f4d1f9..068f7ddce46 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -72,6 +72,7 @@ #include #include #include +#include #include #include #include @@ -174,6 +175,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "filesystem_cache"); attach(context, system_database, "remote_data_paths"); attach(context, system_database, "certificates"); + attach(context, system_database, "named_collections"); if (has_zookeeper) attach(context, system_database, "zookeeper"); From a47aac3e2bec561364cdea6318c833bfa3b4d265 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Nov 2022 17:07:17 +0100 Subject: [PATCH 36/82] Better system table structure --- src/Storages/NamedCollections.cpp | 15 +++++++++-- src/Storages/NamedCollections.h | 4 +++ .../System/StorageSystemNamedCollections.cpp | 25 +++++++++++++++++-- 3 files changed, 40 insertions(+), 4 deletions(-) diff --git a/src/Storages/NamedCollections.cpp b/src/Storages/NamedCollections.cpp index f598a302516..e7c86f0e549 100644 --- a/src/Storages/NamedCollections.cpp +++ b/src/Storages/NamedCollections.cpp @@ -299,8 +299,14 @@ public: setConfigValue(*config, key, value); } - /// Get a string representation of the collection structure. - /// Used for debugging and tests. + std::map dumpStructure() + { + std::map result; + for (const auto & [key, _] : collection_info) + result.emplace(key, get(key)); + return result; + } + std::string toString() const { /// Convert a collection config like @@ -518,6 +524,11 @@ NamedCollection::Value NamedCollection::get(const Key & key) const return pimpl->get(key); } +std::map NamedCollection::dumpStructure() const +{ + return pimpl->dumpStructure(); +} + std::string NamedCollection::toString() const { return pimpl->toString(); diff --git a/src/Storages/NamedCollections.h b/src/Storages/NamedCollections.h index cfc7ef5ef2f..75884d96ce9 100644 --- a/src/Storages/NamedCollections.h +++ b/src/Storages/NamedCollections.h @@ -97,6 +97,10 @@ public: Value get(const Key & key) const; + std::map dumpStructure() const; + + /// Get a string representation of the collection structure. + /// Used for debugging and tests. std::string toString() const; }; diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index a53b6dc9bd3..3fc9c5c8313 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -1,8 +1,13 @@ #include "StorageSystemNamedCollections.h" + +#include #include +#include #include +#include #include #include +#include #include @@ -13,7 +18,7 @@ NamesAndTypesList StorageSystemNamedCollections::getNamesAndTypes() { return { {"name", std::make_shared()}, - {"collection", std::make_shared()}, + {"collection", std::make_shared(std::make_shared(), std::make_shared())}, }; } @@ -31,7 +36,23 @@ void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, Conte for (const auto & [name, collection] : collections) { res_columns[0]->insert(name); - res_columns[1]->insert(collection->toString()); + + auto * column_map = typeid_cast(res_columns[1].get()); + + auto & offsets = column_map->getNestedColumn().getOffsets(); + auto & tuple_column = column_map->getNestedData(); + auto & key_column = tuple_column.getColumn(0); + auto & value_column = tuple_column.getColumn(1); + + size_t size = 0; + for (const auto & [key, value] : collection->dumpStructure()) + { + key_column.insertData(key.data(), key.size()); + value_column.insert(convertFieldToString(value)); + size++; + } + + offsets.push_back(offsets.back() + size); } } From fe283af82bf7d168710440d69d28b4b6d5dbd194 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 10 Nov 2022 16:08:05 +0000 Subject: [PATCH 37/82] Automatic style fix --- tests/integration/test_global_overcommit_tracker/test.py | 1 + tests/integration/test_overcommit_tracker/test.py | 4 +--- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_global_overcommit_tracker/test.py b/tests/integration/test_global_overcommit_tracker/test.py index 654f11be45a..6bedc03a30e 100644 --- a/tests/integration/test_global_overcommit_tracker/test.py +++ b/tests/integration/test_global_overcommit_tracker/test.py @@ -8,6 +8,7 @@ node = cluster.add_instance( "node", main_configs=["configs/global_overcommit_tracker.xml"] ) + @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: diff --git a/tests/integration/test_overcommit_tracker/test.py b/tests/integration/test_overcommit_tracker/test.py index 50af8525b99..50979526e6a 100644 --- a/tests/integration/test_overcommit_tracker/test.py +++ b/tests/integration/test_overcommit_tracker/test.py @@ -4,9 +4,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance( - "node" -) +node = cluster.add_instance("node") @pytest.fixture(scope="module", autouse=True) From a5d06cd90967185b35a55af426169a7b7cdd09f3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Nov 2022 19:58:07 +0100 Subject: [PATCH 38/82] Better Impl --- src/Storages/NamedCollections.cpp | 389 ++++++++---------- src/Storages/NamedCollections.h | 110 ++--- .../System/StorageSystemNamedCollections.cpp | 4 +- .../tests/gtest_named_collections.cpp | 45 +- 4 files changed, 239 insertions(+), 309 deletions(-) diff --git a/src/Storages/NamedCollections.cpp b/src/Storages/NamedCollections.cpp index e7c86f0e549..3dc34b699d7 100644 --- a/src/Storages/NamedCollections.cpp +++ b/src/Storages/NamedCollections.cpp @@ -128,9 +128,7 @@ bool NamedCollectionFactory::existsUnlocked( || config->has(getCollectionPrefix(collection_name)); } -NamedCollectionPtr NamedCollectionFactory::get( - const std::string & collection_name, - const NamedCollectionInfo & collection_info) const +NamedCollectionPtr NamedCollectionFactory::get(const std::string & collection_name) const { std::lock_guard lock(mutex); assertInitialized(lock); @@ -143,12 +141,10 @@ NamedCollectionPtr NamedCollectionFactory::get( collection_name); } - return getImpl(collection_name, collection_info, lock); + return getImpl(collection_name, lock); } -NamedCollectionPtr NamedCollectionFactory::tryGet( - const std::string & collection_name, - const NamedCollectionInfo & collection_info) const +NamedCollectionPtr NamedCollectionFactory::tryGet(const std::string & collection_name) const { std::lock_guard lock(mutex); assertInitialized(lock); @@ -156,12 +152,11 @@ NamedCollectionPtr NamedCollectionFactory::tryGet( if (!existsUnlocked(collection_name, lock)) return nullptr; - return getImpl(collection_name, collection_info, lock); + return getImpl(collection_name, lock); } NamedCollectionPtr NamedCollectionFactory::getImpl( const std::string & collection_name, - const NamedCollectionInfo & collection_info, std::lock_guard & /* lock */) const { auto it = named_collections.find(collection_name); @@ -169,8 +164,7 @@ NamedCollectionPtr NamedCollectionFactory::getImpl( { it = named_collections.emplace( collection_name, - std::make_unique( - *config, collection_name, collection_info)).first; + NamedCollection::create(*config, collection_name)).first; } return it->second; } @@ -225,36 +219,12 @@ NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const Poco::Util::AbstractConfiguration::Keys config_collections_names; config->keys(NAMED_COLLECTIONS_CONFIG_PREFIX, config_collections_names); - for (const auto & name : config_collections_names) + for (const auto & collection_name : config_collections_names) { - if (result.contains(name)) + if (result.contains(collection_name)) continue; - const auto collection_prefix = getCollectionPrefix(name); - std::queue enumerate_input; - std::set enumerate_result; - - enumerate_input.push(collection_prefix); - collectKeys(*config, enumerate_input, enumerate_result); - - NamedCollectionInfo collection_info; - - /// Collection does not have any keys. - /// (`enumerate_result` == ). - const bool collection_is_empty = enumerate_result.size() == 1; - if (!collection_is_empty) - { - for (const auto & path : enumerate_result) - { - collection_info.emplace( - /// Skip collection prefix and add +1 to avoid '.' in the beginning. - path.substr(std::strlen(collection_prefix.data()) + 1), - NamedCollectionValueInfo{}); - } - } - - result.emplace( - name, std::make_unique(*config, name, collection_info)); + result.emplace(collection_name, NamedCollection::create(*config, collection_name)); } return result; @@ -271,43 +241,46 @@ private: /// ... /// ConfigurationPtr config; - /// Information about the values of keys. Key is a path to the - /// value represented as a dot concatenated list of keys. - const CollectionInfo collection_info; + Keys keys; public: Impl(const Poco::Util::AbstractConfiguration & config_, const std::string & collection_name_, - const NamedCollectionInfo & collection_info_) + const Keys & keys_) : config(createEmptyConfiguration(collection_name_)) - , collection_info(collection_info_) + , keys(keys_) { auto collection_path = getCollectionPrefix(collection_name_); - for (const auto & [key, value_info] : collection_info) - copyConfigValue( - config_, collection_path + '.' + key, *config, key, value_info.type); + for (const auto & key : keys) + copyConfigValue(config_, collection_path + '.' + key, *config, key); } - Value get(const Key & key) const + ImplPtr copy() const { - auto value_info = collection_info.at(key); - return getConfigValue(*config, key, value_info.type, value_info.is_required); + return std::make_unique(*this); } - void set(const Key & key, const Value & value) + template T get(const Key & key) const { - setConfigValue(*config, key, value); + return getConfigValue(*config, key); } - std::map dumpStructure() + template T getOrDefault(const Key & key, const T & default_value) const { - std::map result; - for (const auto & [key, _] : collection_info) - result.emplace(key, get(key)); - return result; + return getConfigValueOrDefault(*config, key, default_value); } - std::string toString() const + template void set(const Key & key, const T & value) + { + setConfigValue(*config, key, value); + } + + Keys getKeys() const + { + return keys; + } + + std::string dumpStructure() const { /// Convert a collection config like /// @@ -326,7 +299,7 @@ public: /// key3: /// key4: value3" WriteBufferFromOwnString wb; - for (const auto & [key, value_info] : collection_info) + for (const auto & key : keys) { Strings key_parts; splitInto<'.'>(key_parts, key); @@ -338,69 +311,98 @@ public: wb << '\n' << std::string(tab_cnt++, '\t'); wb << *it << ':'; } - wb << '\t' << convertFieldToString(get(key)) << '\n'; + wb << '\t' << get(key) << '\n'; } return wb.str(); } private: - static void validate( + template static T getConfigValue( const Poco::Util::AbstractConfiguration & config, - const std::string & collection_path, - const NamedCollectionInfo & collection_info_) + const std::string & path) { - Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys(collection_path, config_keys); - checkKeys(config_keys, collection_info_); + return getConfigValueOrDefault(config, path); } - static void checkKeys( - const Poco::Util::AbstractConfiguration::Keys & config_keys, - const NamedCollectionInfo & collection_info) - + template static T getConfigValueOrDefault( + const Poco::Util::AbstractConfiguration & config, + const std::string & path, + const std::optional & default_value = std::nullopt) { - auto get_suggestion = [&](bool only_required_keys) + const bool has_value = config.has(path); + if (!has_value) { - std::string suggestion; - for (const auto & [key, info] : collection_info) - { - if (only_required_keys && info.is_required) - continue; - - if (!suggestion.empty()) - suggestion += ", "; - - suggestion += key; - } - return suggestion; - }; - - std::set required_keys; - for (const auto & [key, info] : collection_info) - { - if (info.is_required) - required_keys.insert(key); + if (!default_value) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}`", path); + return *default_value; } - for (const auto & key : config_keys) - { - if (!collection_info.contains(key)) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Unknown key `{}`, expected one of: {}", - key, get_suggestion(false)); - } - required_keys.erase(key); - } - - if (!required_keys.empty()) - { + if constexpr (std::is_same_v) + return config.getString(path); + else if constexpr (std::is_same_v) + return config.getUInt64(path); + else if constexpr (std::is_same_v) + return config.getInt64(path); + else if constexpr (std::is_same_v) + return config.getDouble(path); + else throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Keys `{}` are required, but was not found in config. List of required keys: {}", - fmt::join(required_keys, ", "), get_suggestion(true)); - } + "Unsupported type in getConfigValueOrDefault(). " + "Supported types are String, UInt64, Int64, Float64"); + } + + template static void setConfigValue( + Poco::Util::AbstractConfiguration & config, + const std::string & path, + const T & value) + { + const bool has_value = config.has(path); + if (has_value) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Key `{}` already exists", path); + + if constexpr (std::is_same_v) + config.setString(path, value); + else if constexpr (std::is_same_v) + config.setUInt64(path, value); + else if constexpr (std::is_same_v) + config.setInt64(path, value); + else if constexpr (std::is_same_v) + config.setDouble(path, value); + else + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Unsupported type in setConfigValue(). " + "Supported types are String, UInt64, Int64, Float64"); + } + + template static void copyConfigValue( + const Poco::Util::AbstractConfiguration & from_config, + const std::string & from_path, + Poco::Util::AbstractConfiguration & to_config, + const std::string & to_path) + { + if (!from_config.has(from_path)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}`", from_path); + + if (to_config.has(to_path)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Key `{}` already exists", to_path); + + if constexpr (std::is_same_v) + to_config.setString(to_path, from_config.getString(from_path)); + else if constexpr (std::is_same_v) + to_config.setString(to_path, from_config.getString(from_path)); + else if constexpr (std::is_same_v) + to_config.setUInt64(to_path, from_config.getUInt64(from_path)); + else if constexpr (std::is_same_v) + to_config.setInt64(to_path, from_config.getInt64(from_path)); + else if constexpr (std::is_same_v) + to_config.setDouble(to_path, from_config.getDouble(from_path)); + else + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Unsupported type in copyConfigValue(). " + "Supported types are String, UInt64, Int64, Float64"); } static ConfigurationPtr createEmptyConfiguration(const std::string & root_name) @@ -411,127 +413,88 @@ private: ConfigurationPtr config(new Poco::Util::XMLConfiguration(xml_document)); return config; } - - using ConfigValueType = Field::Types::Which; - static void copyConfigValue( - const Poco::Util::AbstractConfiguration & from_config, - const std::string & from_path, - Poco::Util::AbstractConfiguration & to_config, - const std::string & to_path, - ConfigValueType type) - { - using Type = Field::Types::Which; - switch (type) - { - case Type::String: - to_config.setString(to_path, from_config.getString(from_path)); - break; - case Type::UInt64: - to_config.setUInt64(to_path, from_config.getUInt64(from_path)); - break; - case Type::Int64: - to_config.setInt64(to_path, from_config.getInt64(from_path)); - break; - case Type::Float64: - to_config.setDouble(to_path, from_config.getDouble(from_path)); - break; - default: - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported type"); - } - } - - static void setConfigValue( - Poco::Util::AbstractConfiguration & config, - const std::string & path, - const Field & value) - { - using Type = Field::Types::Which; - switch (value.getType()) - { - case Type::String: - config.setString(path, value.safeGet()); - break; - case Type::UInt64: - config.setUInt64(path, value.safeGet()); - break; - case Type::Int64: - config.setInt64(path, value.safeGet()); - break; - case Type::Float64: - config.setDouble(path, value.safeGet()); - break; - default: - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Unsupported type"); - } - } - - static Field getConfigValue( - const Poco::Util::AbstractConfiguration & config, - const std::string & path, - ConfigValueType type, - bool throw_not_found, - std::optional default_value = std::nullopt) - { - const bool has_value = config.has(path); - if (!has_value) - { - if (throw_not_found) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Failed to find key `{}` in config, but this key is required", - path); - } - else if (!default_value) - return Null{}; - } - - Field value; - - using Type = Field::Types::Which; - switch (type) - { - case Type::String: - value = has_value ? config.getString(path) : default_value->get(); - break; - case Type::UInt64: - value = has_value ? config.getUInt64(path) : default_value->get(); - break; - case Type::Int64: - value = has_value ? config.getInt64(path) : default_value->get(); - break; - case Type::Float64: - value = has_value ? config.getDouble(path) : default_value->get(); - break; - default: - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported type"); - } - return value; - } }; NamedCollection::NamedCollection( const Poco::Util::AbstractConfiguration & config, const std::string & collection_path, - const CollectionInfo & collection_info) - : pimpl(std::make_unique(config, collection_path, collection_info)) + const Keys & keys) + : pimpl(std::make_unique(config, collection_path, keys)) { } -NamedCollection::Value NamedCollection::get(const Key & key) const +NamedCollection::NamedCollection(ImplPtr pimpl_) + : pimpl(std::move(pimpl_)) { - return pimpl->get(key); } -std::map NamedCollection::dumpStructure() const +NamedCollectionPtr NamedCollection::create( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_name) +{ + const auto collection_prefix = getCollectionPrefix(collection_name); + std::queue enumerate_input; + std::set enumerate_result; + + enumerate_input.push(collection_prefix); + collectKeys(config, enumerate_input, enumerate_result); + + /// Collection does not have any keys. + /// (`enumerate_result` == ). + const bool collection_is_empty = enumerate_result.size() == 1; + std::set keys; + if (!collection_is_empty) + { + /// Skip collection prefix and add +1 to avoid '.' in the beginning. + for (const auto & path : enumerate_result) + keys.emplace(path.substr(std::strlen(collection_prefix.data()) + 1)); + } + return std::make_unique(config, collection_name, keys); +} + +template T NamedCollection::get(const Key & key) const +{ + return pimpl->get(key); +} + +template T NamedCollection::getOrDefault(const Key & key, const T & default_value) const +{ + return pimpl->getOrDefault(key, default_value); +} + +template void NamedCollection::set(const Key & key, const T & value) +{ + pimpl->set(key, value); +} + +NamedCollectionPtr NamedCollection::duplicate() const +{ + return std::make_shared(pimpl->copy()); +} + +NamedCollection::Keys NamedCollection::getKeys() const +{ + return pimpl->getKeys(); +} + +std::string NamedCollection::dumpStructure() const { return pimpl->dumpStructure(); } -std::string NamedCollection::toString() const -{ - return pimpl->toString(); -} +template String NamedCollection::get(const NamedCollection::Key & key) const; +template UInt64 NamedCollection::get(const NamedCollection::Key & key) const; +template Int64 NamedCollection::get(const NamedCollection::Key & key) const; +template Float64 NamedCollection::get(const NamedCollection::Key & key) const; + +template String NamedCollection::getOrDefault(const NamedCollection::Key & key, const String & default_value) const; +template UInt64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const UInt64 & default_value) const; +template Int64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const Int64 & default_value) const; +template Float64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const Float64 & default_value) const; + +template void NamedCollection::set(const NamedCollection::Key & key, const String & value); +template void NamedCollection::set(const NamedCollection::Key & key, const UInt64 & value); +template void NamedCollection::set(const NamedCollection::Key & key, const Int64 & value); +template void NamedCollection::set(const NamedCollection::Key & key, const Float64 & value); } diff --git a/src/Storages/NamedCollections.h b/src/Storages/NamedCollections.h index 75884d96ce9..0056b255025 100644 --- a/src/Storages/NamedCollections.h +++ b/src/Storages/NamedCollections.h @@ -9,21 +9,55 @@ namespace DB class NamedCollection; using NamedCollectionPtr = std::shared_ptr; -struct NamedCollectionValueInfo; -using NamedCollectionInfo = std::map; /** - * A factory of immutable named collections. - * Named collections are defined in server config as arbitrary - * structure configurations: + * Class to represent arbitrary-structured named collection object. + * It can be defined via config or via SQL command. * * * ... * * ... * - * In order to get a named collection, you need to know it's name - * and expected structure of the collection defined as NamedCollectionInfo. + */ +class NamedCollection +{ +private: + struct Impl; + using ImplPtr = std::unique_ptr; + + ImplPtr pimpl; + +public: + using Key = std::string; + using Keys = std::set; + + static NamedCollectionPtr create( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_name); + + NamedCollection( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_path, + const Keys & keys); + + explicit NamedCollection(ImplPtr pimpl_); + + template T get(const Key & key) const; + + template T getOrDefault(const Key & key, const T & default_value) const; + + template void set(const Key & key, const T & value); + + NamedCollectionPtr duplicate() const; + + Keys getKeys() const; + + std::string dumpStructure() const; +}; + +/** + * A factory of immutable named collections. */ class NamedCollectionFactory : boost::noncopyable { @@ -34,13 +68,9 @@ public: bool exists(const std::string & collection_name) const; - NamedCollectionPtr get( - const std::string & collection_name, - const NamedCollectionInfo & collection_info) const; + NamedCollectionPtr get(const std::string & collection_name) const; - NamedCollectionPtr tryGet( - const std::string & collection_name, - const NamedCollectionInfo & collection_info) const; + NamedCollectionPtr tryGet(const std::string & collection_name) const; void add( const std::string & collection_name, @@ -52,9 +82,10 @@ public: NamedCollections getAll() const; private: + void assertInitialized(std::lock_guard & lock) const; + NamedCollectionPtr getImpl( const std::string & collection_name, - const NamedCollectionInfo & collection_info, std::lock_guard & lock) const; bool existsUnlocked( @@ -63,62 +94,11 @@ private: mutable NamedCollections named_collections; -private: /// FIXME: this will be invalid when config is reloaded const Poco::Util::AbstractConfiguration * config; - void assertInitialized(std::lock_guard & lock) const; - bool is_initialized = false; mutable std::mutex mutex; }; - -class NamedCollection -{ -friend class NamedCollectionFactory; - -private: - struct Impl; - using ImplPtr = std::unique_ptr; - - ImplPtr pimpl; - -public: - using Key = std::string; - using Value = Field; - using ValueInfo = NamedCollectionValueInfo; - using CollectionInfo = NamedCollectionInfo; - - NamedCollection( - const Poco::Util::AbstractConfiguration & config, - const std::string & collection_path, - const CollectionInfo & collection_info); - - Value get(const Key & key) const; - - std::map dumpStructure() const; - - /// Get a string representation of the collection structure. - /// Used for debugging and tests. - std::string toString() const; -}; - - -/** - * Named collection info which allows to parse config. - * Contains a mapping key_path -> value_info. - */ -struct NamedCollectionValueInfo -{ - /// Type of the value. One of: String, UInt64, Int64, Double. - using Type = Field::Types::Which; - Type type = Type::String; - /// Optional default value for the case if there is no such key in config. - std::optional default_value; - /// Is this value required or optional? Throw exception if the value is - /// required, but is not specified in config. - bool is_required = true; -}; - } diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index 3fc9c5c8313..8c3092b948e 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -45,10 +45,10 @@ void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, Conte auto & value_column = tuple_column.getColumn(1); size_t size = 0; - for (const auto & [key, value] : collection->dumpStructure()) + for (const auto & key : collection->getKeys()) { key_column.insertData(key.data(), key.size()); - value_column.insert(convertFieldToString(value)); + value_column.insert(collection->get(key)); size++; } diff --git a/src/Storages/tests/gtest_named_collections.cpp b/src/Storages/tests/gtest_named_collections.cpp index 34dfe5c1087..6a3057df329 100644 --- a/src/Storages/tests/gtest_named_collections.cpp +++ b/src/Storages/tests/gtest_named_collections.cpp @@ -32,52 +32,39 @@ TEST(NamedCollections, SimpleConfig) ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1")); ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection2")); - ASSERT_TRUE(NamedCollectionFactory::instance().tryGet("collection3", {}) == nullptr); + ASSERT_TRUE(NamedCollectionFactory::instance().tryGet("collection3") == nullptr); auto collections = NamedCollectionFactory::instance().getAll(); ASSERT_EQ(collections.size(), 2); ASSERT_TRUE(collections.contains("collection1")); ASSERT_TRUE(collections.contains("collection2")); - ASSERT_EQ(collections["collection1"]->toString(), + ASSERT_EQ(collections["collection1"]->dumpStructure(), R"CONFIG(key1: value1 key2: 2 key3: 3.3 key4: -4 +)CONFIG"); + ASSERT_EQ(collections["collection2"]->dumpStructure(), + R"CONFIG(key4: value4 +key5: 5 +key6: 6.6 )CONFIG"); - using ValueInfo = NamedCollectionValueInfo; - ValueInfo string_def{Field::Types::Which::String, std::nullopt, true}; - ValueInfo uint_def{Field::Types::Which::UInt64, std::nullopt, true}; - ValueInfo int_def{Field::Types::Which::Int64, std::nullopt, true}; - ValueInfo double_def{Field::Types::Which::Float64, std::nullopt, true}; - - NamedCollectionInfo collection1_info; - collection1_info.emplace("key1", string_def); - collection1_info.emplace("key2", uint_def); - collection1_info.emplace("key3", double_def); - collection1_info.emplace("key4", int_def); - - auto collection1 = NamedCollectionFactory::instance().get("collection1", collection1_info); + auto collection1 = NamedCollectionFactory::instance().get("collection1"); ASSERT_TRUE(collection1 != nullptr); + ASSERT_TRUE(collection1->get("key1") == "value1"); + ASSERT_TRUE(collection1->get("key2") == 2); + ASSERT_TRUE(collection1->get("key3") == 3.3); + ASSERT_TRUE(collection1->get("key4") == -4); - ASSERT_TRUE(collection1->get("key1").safeGet() == "value1"); - ASSERT_TRUE(collection1->get("key2").safeGet() == 2); - ASSERT_TRUE(collection1->get("key3").safeGet() == 3.3); - ASSERT_TRUE(collection1->get("key4").safeGet() == -4); - - NamedCollectionInfo collection2_info; - collection2_info.emplace("key4", string_def); - collection2_info.emplace("key5", uint_def); - collection2_info.emplace("key6", double_def); - - auto collection2 = NamedCollectionFactory::instance().get("collection2", collection2_info); + auto collection2 = NamedCollectionFactory::instance().get("collection2"); ASSERT_TRUE(collection2 != nullptr); - ASSERT_TRUE(collection2->get("key4").safeGet() == "value4"); - ASSERT_TRUE(collection2->get("key5").safeGet() == 5); - ASSERT_TRUE(collection2->get("key6").safeGet() == 6.6); + ASSERT_TRUE(collection2->get("key4") == "value4"); + ASSERT_TRUE(collection2->get("key5") == 5); + ASSERT_TRUE(collection2->get("key6") == 6.6); } // TEST(NamedCollections, NestedConfig) From bb00a246f10ec993695cce4e0a8fadf50a0b2960 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Nov 2022 20:53:05 +0100 Subject: [PATCH 39/82] priority for DROP_RANGEs, assertion for others --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 28 +++++++++++++++---- 1 file changed, 22 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index e60403149b5..fec312fae2f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1032,6 +1032,14 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( [[maybe_unused]] bool called_for_broken_part = !covering_entry; assert(currently_executing_drop_replace_ranges.contains(part_info) || called_from_alter_query_directly || called_for_broken_part || !fetch_entry_znode.empty()); + auto is_simple_part_producing_op = [](const ReplicatedMergeTreeLogEntryData & data) + { + return data.type == LogEntry::GET_PART || + data.type == LogEntry::ATTACH_PART || + data.type == LogEntry::MERGE_PARTS || + data.type == LogEntry::MUTATE_PART; + }; + for (Queue::iterator it = queue.begin(); it != queue.end();) { /// Skipping currently processing entry @@ -1041,18 +1049,21 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( continue; } - auto type = (*it)->type; - bool is_simple_producing_op = type == LogEntry::GET_PART || - type == LogEntry::ATTACH_PART || - type == LogEntry::MERGE_PARTS || - type == LogEntry::MUTATE_PART; + bool is_simple_producing_op = is_simple_part_producing_op(**it); bool simple_op_covered = is_simple_producing_op && part_info.contains(MergeTreePartInfo::fromPartName((*it)->new_part_name, format_version)); bool replace_range_covered = covering_entry && checkReplaceRangeCanBeRemoved(part_info, *it, *covering_entry); if (simple_op_covered || replace_range_covered) { if ((*it)->currently_executing) + { + bool is_covered_by_simple_op = covering_entry && is_simple_part_producing_op(*covering_entry); + bool is_fetching_covering_part = !fetch_entry_znode.empty(); + if (is_covered_by_simple_op || is_fetching_covering_part) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove covered entry {} producing parts {}, it's a bug", + (*it)->znode_name, fmt::join((*it)->getVirtualPartNames(format_version), ", ")); to_wait.push_back(*it); + } auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / (*it)->znode_name); if (code != Coordination::Error::ZOK) LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / (*it)->znode_name).string(), Coordination::errorMessage(code)); @@ -1118,7 +1129,12 @@ bool ReplicatedMergeTreeQueue::isCoveredByFuturePartsImpl(const LogEntry & entry /// Parts are not disjoint. They can be even intersecting and it's not a problem, /// because we may have two queue entries producing intersecting parts if there's DROP_RANGE between them (so virtual_parts are ok). - /// We cannot execute `entry` (or upgrade its actual_part_name to `new_part_name`) + /// Give priority to DROP_RANGEs and allow processing them even if covered entries are currently executing. + /// DROP_RANGE will cancel covered operations and will wait for them in removePartProducingOpsInRange. + if (result_part.isFakeDropRangePart() && result_part.contains(future_part)) + continue; + + /// In other cases we cannot execute `entry` (or upgrade its actual_part_name to `new_part_name`) /// while any covered or covering parts are processed. /// But we also cannot simply return true and postpone entry processing, because it may lead to kind of livelock. /// Since queue is processed in multiple threads, it's likely that there will be at least one thread From 63de577172ee024a08e76db69f5000568673db48 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Nov 2022 22:44:26 +0100 Subject: [PATCH 40/82] Better, add tests --- programs/server/Server.cpp | 1 + src/Storages/NamedCollections.cpp | 127 ++++++++++++------ src/Storages/NamedCollections.h | 15 ++- .../System/StorageSystemNamedCollections.cpp | 1 - .../tests/gtest_named_collections.cpp | 126 ++++++++++------- 5 files changed, 175 insertions(+), 95 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 079dae52f15..d4e623b49b3 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1282,6 +1282,7 @@ int Server::main(const std::vector & /*args*/) #if USE_SSL CertificateReloader::instance().tryLoad(*config); #endif + NamedCollectionFactory::instance().reload(*config); ProfileEvents::increment(ProfileEvents::MainConfigLoads); /// Must be the last. diff --git a/src/Storages/NamedCollections.cpp b/src/Storages/NamedCollections.cpp index 3dc34b699d7..60c11c90b0a 100644 --- a/src/Storages/NamedCollections.cpp +++ b/src/Storages/NamedCollections.cpp @@ -24,6 +24,8 @@ namespace ErrorCodes { extern const int UNKNOWN_NAMED_COLLECTION; extern const int NAMED_COLLECTION_ALREADY_EXISTS; + extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; } namespace @@ -84,8 +86,7 @@ NamedCollectionFactory & NamedCollectionFactory::instance() return instance; } -void NamedCollectionFactory::initialize( - const Poco::Util::AbstractConfiguration & server_config) +void NamedCollectionFactory::initialize(const Poco::Util::AbstractConfiguration & config_) { std::lock_guard lock(mutex); if (is_initialized) @@ -95,10 +96,17 @@ void NamedCollectionFactory::initialize( "Named collection factory already initialized"); } - config = &server_config; + config = &config_; is_initialized = true; } +void NamedCollectionFactory::reload(const Poco::Util::AbstractConfiguration & config_) +{ + std::lock_guard lock(mutex); + config = &config_; + loaded_named_collections.clear(); +} + void NamedCollectionFactory::assertInitialized( std::lock_guard & /* lock */) const { @@ -124,7 +132,7 @@ bool NamedCollectionFactory::existsUnlocked( /// Named collections can be added via SQL command or via config. /// Named collections from config are loaded on first access, /// therefore it might not be in `named_collections` map yet. - return named_collections.contains(collection_name) + return loaded_named_collections.contains(collection_name) || config->has(getCollectionPrefix(collection_name)); } @@ -159,10 +167,10 @@ NamedCollectionPtr NamedCollectionFactory::getImpl( const std::string & collection_name, std::lock_guard & /* lock */) const { - auto it = named_collections.find(collection_name); - if (it == named_collections.end()) + auto it = loaded_named_collections.find(collection_name); + if (it == loaded_named_collections.end()) { - it = named_collections.emplace( + it = loaded_named_collections.emplace( collection_name, NamedCollection::create(*config, collection_name)).first; } @@ -174,7 +182,7 @@ void NamedCollectionFactory::add( NamedCollectionPtr collection) { std::lock_guard lock(mutex); - auto [it, inserted] = named_collections.emplace(collection_name, collection); + auto [it, inserted] = loaded_named_collections.emplace(collection_name, collection); if (!inserted) { throw Exception( @@ -205,7 +213,7 @@ void NamedCollectionFactory::remove(const std::string & collection_name) collection_name); } - [[maybe_unused]] auto removed = named_collections.erase(collection_name); + [[maybe_unused]] auto removed = loaded_named_collections.erase(collection_name); assert(removed); } @@ -214,7 +222,7 @@ NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const std::lock_guard lock(mutex); assertInitialized(lock); - NamedCollections result(named_collections); + NamedCollections result(loaded_named_collections); Poco::Util::AbstractConfiguration::Keys config_collections_names; config->keys(NAMED_COLLECTIONS_CONFIG_PREFIX, config_collections_names); @@ -230,7 +238,7 @@ NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const return result; } -struct NamedCollection::Impl +class NamedCollection::Impl { private: using IConfigurationPtr = Poco::AutoPtr; @@ -255,11 +263,6 @@ public: copyConfigValue(config_, collection_path + '.' + key, *config, key); } - ImplPtr copy() const - { - return std::make_unique(*this); - } - template T get(const Key & key) const { return getConfigValue(*config, key); @@ -270,9 +273,18 @@ public: return getConfigValueOrDefault(*config, key, default_value); } - template void set(const Key & key, const T & value) + template void set(const Key & key, const T & value, bool update_if_exists) { - setConfigValue(*config, key, value); + setConfigValue(*config, key, value, update_if_exists); + if (!keys.contains(key)) + keys.insert(key); + } + + void remove(const Key & key) + { + removeConfigValue(*config, key); + [[maybe_unused]] auto removed = keys.erase(key); + assert(removed); } Keys getKeys() const @@ -280,6 +292,11 @@ public: return keys; } + ImplPtr copy() const + { + return std::make_unique(*this); + } + std::string dumpStructure() const { /// Convert a collection config like @@ -299,19 +316,34 @@ public: /// key3: /// key4: value3" WriteBufferFromOwnString wb; + Strings prev_key_parts; for (const auto & key : keys) { Strings key_parts; splitInto<'.'>(key_parts, key); size_t tab_cnt = 0; - for (auto it = key_parts.begin(); it != key_parts.end(); ++it) + auto it = key_parts.begin(); + auto prev_key_parts_it = prev_key_parts.begin(); + while (it != key_parts.end() + && prev_key_parts_it != prev_key_parts.end() + && *it == *prev_key_parts_it) { - if (it != key_parts.begin()) - wb << '\n' << std::string(tab_cnt++, '\t'); + ++it; + ++prev_key_parts_it; + ++tab_cnt; + } + + auto start_it = it; + for (; it != key_parts.end(); ++it) + { + if (it != start_it) + wb << '\n'; + wb << std::string(tab_cnt++, '\t'); wb << *it << ':'; } wb << '\t' << get(key) << '\n'; + prev_key_parts = key_parts; } return wb.str(); } @@ -329,11 +361,10 @@ private: const std::string & path, const std::optional & default_value = std::nullopt) { - const bool has_value = config.has(path); - if (!has_value) + if (!config.has(path)) { if (!default_value) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}`", path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path); return *default_value; } @@ -347,7 +378,7 @@ private: return config.getDouble(path); else throw Exception( - ErrorCodes::BAD_ARGUMENTS, + ErrorCodes::NOT_IMPLEMENTED, "Unsupported type in getConfigValueOrDefault(). " "Supported types are String, UInt64, Int64, Float64"); } @@ -355,11 +386,11 @@ private: template static void setConfigValue( Poco::Util::AbstractConfiguration & config, const std::string & path, - const T & value) + const T & value, + bool update = false) { - const bool has_value = config.has(path); - if (has_value) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Key `{}` already exists", path); + if (!update && config.has(path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", path); if constexpr (std::is_same_v) config.setString(path, value); @@ -371,7 +402,7 @@ private: config.setDouble(path, value); else throw Exception( - ErrorCodes::BAD_ARGUMENTS, + ErrorCodes::NOT_IMPLEMENTED, "Unsupported type in setConfigValue(). " "Supported types are String, UInt64, Int64, Float64"); } @@ -383,10 +414,10 @@ private: const std::string & to_path) { if (!from_config.has(from_path)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}`", from_path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", from_path); if (to_config.has(to_path)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Key `{}` already exists", to_path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", to_path); if constexpr (std::is_same_v) to_config.setString(to_path, from_config.getString(from_path)); @@ -400,11 +431,20 @@ private: to_config.setDouble(to_path, from_config.getDouble(from_path)); else throw Exception( - ErrorCodes::BAD_ARGUMENTS, + ErrorCodes::NOT_IMPLEMENTED, "Unsupported type in copyConfigValue(). " "Supported types are String, UInt64, Int64, Float64"); } + static void removeConfigValue( + Poco::Util::AbstractConfiguration & config, + const std::string & path) + { + if (!config.has(path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path); + config.remove(path); + } + static ConfigurationPtr createEmptyConfiguration(const std::string & root_name) { using DocumentPtr = Poco::AutoPtr; @@ -419,7 +459,7 @@ NamedCollection::NamedCollection( const Poco::Util::AbstractConfiguration & config, const std::string & collection_path, const Keys & keys) - : pimpl(std::make_unique(config, collection_path, keys)) + : NamedCollection(std::make_unique(config, collection_path, keys)) { } @@ -462,12 +502,17 @@ template T NamedCollection::getOrDefault(const Key & key, const T & return pimpl->getOrDefault(key, default_value); } -template void NamedCollection::set(const Key & key, const T & value) +template void NamedCollection::set(const Key & key, const T & value, bool update_if_exists) { - pimpl->set(key, value); + pimpl->set(key, value, update_if_exists); } -NamedCollectionPtr NamedCollection::duplicate() const +void NamedCollection::remove(const Key & key) +{ + pimpl->remove(key); +} + +std::shared_ptr NamedCollection::duplicate() const { return std::make_shared(pimpl->copy()); } @@ -492,9 +537,9 @@ template UInt64 NamedCollection::getOrDefault(const NamedCollection::Key template Int64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const Int64 & default_value) const; template Float64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const Float64 & default_value) const; -template void NamedCollection::set(const NamedCollection::Key & key, const String & value); -template void NamedCollection::set(const NamedCollection::Key & key, const UInt64 & value); -template void NamedCollection::set(const NamedCollection::Key & key, const Int64 & value); -template void NamedCollection::set(const NamedCollection::Key & key, const Float64 & value); +template void NamedCollection::set(const NamedCollection::Key & key, const String & value, bool update_if_exists); +template void NamedCollection::set(const NamedCollection::Key & key, const UInt64 & value, bool update_if_exists); +template void NamedCollection::set(const NamedCollection::Key & key, const Int64 & value, bool update_if_exists); +template void NamedCollection::set(const NamedCollection::Key & key, const Float64 & value, bool update_if_exists); } diff --git a/src/Storages/NamedCollections.h b/src/Storages/NamedCollections.h index 0056b255025..83bb1dd964e 100644 --- a/src/Storages/NamedCollections.h +++ b/src/Storages/NamedCollections.h @@ -23,7 +23,7 @@ using NamedCollectionPtr = std::shared_ptr; class NamedCollection { private: - struct Impl; + class Impl; using ImplPtr = std::unique_ptr; ImplPtr pimpl; @@ -47,9 +47,11 @@ public: template T getOrDefault(const Key & key, const T & default_value) const; - template void set(const Key & key, const T & value); + template void set(const Key & key, const T & value, bool update_if_exists = false); - NamedCollectionPtr duplicate() const; + void remove(const Key & key); + + std::shared_ptr duplicate() const; Keys getKeys() const; @@ -64,7 +66,9 @@ class NamedCollectionFactory : boost::noncopyable public: static NamedCollectionFactory & instance(); - void initialize(const Poco::Util::AbstractConfiguration & server_config); + void initialize(const Poco::Util::AbstractConfiguration & config_); + + void reload(const Poco::Util::AbstractConfiguration & config_); bool exists(const std::string & collection_name) const; @@ -92,9 +96,8 @@ private: const std::string & collection_name, std::lock_guard & lock) const; - mutable NamedCollections named_collections; + mutable NamedCollections loaded_named_collections; - /// FIXME: this will be invalid when config is reloaded const Poco::Util::AbstractConfiguration * config; bool is_initialized = false; diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index 8c3092b948e..6f4078369d2 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -32,7 +32,6 @@ void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, Conte context->checkAccess(AccessType::SHOW_NAMED_COLLECTIONS); auto collections = NamedCollectionFactory::instance().getAll(); - for (const auto & [name, collection] : collections) { res_columns[0]->insert(name); diff --git a/src/Storages/tests/gtest_named_collections.cpp b/src/Storages/tests/gtest_named_collections.cpp index 6a3057df329..5ba9156bcd9 100644 --- a/src/Storages/tests/gtest_named_collections.cpp +++ b/src/Storages/tests/gtest_named_collections.cpp @@ -44,11 +44,6 @@ TEST(NamedCollections, SimpleConfig) key2: 2 key3: 3.3 key4: -4 -)CONFIG"); - ASSERT_EQ(collections["collection2"]->dumpStructure(), - R"CONFIG(key4: value4 -key5: 5 -key6: 6.6 )CONFIG"); auto collection1 = NamedCollectionFactory::instance().get("collection1"); @@ -59,53 +54,90 @@ key6: 6.6 ASSERT_TRUE(collection1->get("key3") == 3.3); ASSERT_TRUE(collection1->get("key4") == -4); + ASSERT_EQ(collections["collection2"]->dumpStructure(), + R"CONFIG(key4: value4 +key5: 5 +key6: 6.6 +)CONFIG"); + auto collection2 = NamedCollectionFactory::instance().get("collection2"); ASSERT_TRUE(collection2 != nullptr); ASSERT_TRUE(collection2->get("key4") == "value4"); ASSERT_TRUE(collection2->get("key5") == 5); ASSERT_TRUE(collection2->get("key6") == 6.6); + + auto collection2_copy = collections["collection2"]->duplicate(); + NamedCollectionFactory::instance().add("collection2_copy", collection2_copy); + ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection2_copy")); + ASSERT_EQ(NamedCollectionFactory::instance().get("collection2_copy")->dumpStructure(), + R"CONFIG(key4: value4 +key5: 5 +key6: 6.6 +)CONFIG"); + + collection2_copy->set("key4", "value44", true); + ASSERT_TRUE(collection2_copy->get("key4") == "value44"); + ASSERT_TRUE(collection2->get("key4") == "value4"); + + collection2_copy->remove("key4"); + ASSERT_TRUE(collection2_copy->getOrDefault("key4", "N") == "N"); + ASSERT_TRUE(collection2->getOrDefault("key4", "N") == "value4"); + + collection2_copy->set("key4", "value45"); + ASSERT_TRUE(collection2_copy->getOrDefault("key4", "N") == "value45"); + + NamedCollectionFactory::instance().remove("collection2_copy"); + ASSERT_FALSE(NamedCollectionFactory::instance().exists("collection2_copy")); + + config.reset(); } -// TEST(NamedCollections, NestedConfig) -// { -// std::string xml(R"CONFIG( -// -// -// -// value1 -// -// -// value2_1 -// -// -// value2_4 -// value2_5 -// -// -// -// -// -// )CONFIG"); -// -// Poco::XML::DOMParser dom_parser; -// Poco::AutoPtr document = dom_parser.parseString(xml); -// Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); -// -// ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1")); -// -// using ValueInfo = NamedCollectionValueInfo; -// ValueInfo string_def{Field::Types::Which::String, std::nullopt, true}; -// -// NamedCollectionInfo collection1_info; -// collection1_info.emplace("key1.key1_1", string_def); -// collection1_info.emplace("key2.key2_1", string_def); -// collection1_info.emplace("key2.key2_2.key2_3.key2_4", string_def); -// collection1_info.emplace("key2.key2_2.key2_3.key2_5", string_def); -// -// auto collection1 = NamedCollectionFactory::instance().get("collection1", collection1_info); -// ASSERT_TRUE(collection1 != nullptr); -// -// ASSERT_TRUE(collection1->get("key1.key1_1").safeGet() == "value1"); -// -// } +TEST(NamedCollections, NestedConfig) +{ + std::string xml(R"CONFIG( + + + + value1 + + + value2_1 + + + 4 + 5 + + + + + +)CONFIG"); + + Poco::XML::DOMParser dom_parser; + Poco::AutoPtr document = dom_parser.parseString(xml); + Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); + NamedCollectionFactory::instance().reload(*config); + + ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1")); + + auto collection1 = NamedCollectionFactory::instance().get("collection1"); + ASSERT_TRUE(collection1 != nullptr); + + ASSERT_EQ(collection1->dumpStructure(), + R"CONFIG(key1: + key1_1: value1 +key2: + key2_1: value2_1 + key2_2: + key2_3: + key2_4: 4 + key2_5: 5 +)CONFIG"); + + ASSERT_EQ(collection1->get("key1.key1_1"), "value1"); + ASSERT_EQ(collection1->get("key2.key2_1"), "value2_1"); + ASSERT_EQ(collection1->get("key2.key2_2.key2_3.key2_4"), 4); + ASSERT_EQ(collection1->get("key2.key2_2.key2_3.key2_5"), 5); + +} From d4e270b522bd4d869a6af329ac96938b24c1be59 Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 11 Nov 2022 10:44:16 +0800 Subject: [PATCH 41/82] to MaterializedMySQL_improvement_bug_fix-fix codestyle --- src/Core/MySQL/MySQLReplication.cpp | 6 +++--- src/Core/MySQL/MySQLReplication.h | 2 +- .../materialize_with_ddl.py | 12 +++++++++--- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 45437ea5843..f4785875be0 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -119,7 +119,7 @@ namespace MySQLReplication else if (query.starts_with("SAVEPOINT") || query.starts_with("ROLLBACK") || query.starts_with("RELEASE SAVEPOINT")) { - typ = QUERY_EVENT_OTHER; + typ = QUERY_SAVEPOINT; } } @@ -942,8 +942,8 @@ namespace MySQLReplication { case QUERY_EVENT_MULTI_TXN_FLAG: case QUERY_EVENT_XA: - /// Ignore queries that have no impact on the data - case QUERY_EVENT_OTHER: + /// Ignore queries that have no impact on the data. + case QUERY_SAVEPOINT: { event = std::make_shared(std::move(query->header)); break; diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index d4bb3fb86f9..e603ecb42bf 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -369,7 +369,7 @@ namespace MySQLReplication QUERY_EVENT_DDL = 0, QUERY_EVENT_MULTI_TXN_FLAG = 1, QUERY_EVENT_XA = 2, - QUERY_EVENT_OTHER = 3 + QUERY_SAVEPOINT = 3 }; class QueryEvent : public EventBase diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index bfb6d2bc6dd..8bc1b13c7a4 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -875,9 +875,15 @@ def alter_rename_table_with_materialized_mysql_database( "1\n2\n3\n4\n5\n", ) - mysql_node.query("ALTER TABLE test_database_rename_table.test_table_4 RENAME test_database_rename_table.test_table_5") - mysql_node.query("ALTER TABLE test_database_rename_table.test_table_5 RENAME TO test_database_rename_table.test_table_6") - mysql_node.query("ALTER TABLE test_database_rename_table.test_table_6 RENAME AS test_database_rename_table.test_table_7") + mysql_node.query( ++ "ALTER TABLE test_database_rename_table.test_table_4 RENAME test_database_rename_table.test_table_5" ++ ) + mysql_node.query( ++ "ALTER TABLE test_database_rename_table.test_table_5 RENAME TO test_database_rename_table.test_table_6" ++ ) + mysql_node.query( ++ "ALTER TABLE test_database_rename_table.test_table_6 RENAME AS test_database_rename_table.test_table_7" ++ ) check_query( clickhouse_node, From 0e614aec122d048dde4f53052fad198003dbcbed Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 11 Nov 2022 11:24:29 +0800 Subject: [PATCH 42/82] to MaterializedMySQL_improvement_bug_fix-fix code sytle --- .../materialize_with_ddl.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index 8bc1b13c7a4..a994863417e 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -877,13 +877,13 @@ def alter_rename_table_with_materialized_mysql_database( mysql_node.query( + "ALTER TABLE test_database_rename_table.test_table_4 RENAME test_database_rename_table.test_table_5" -+ ) + ) mysql_node.query( + "ALTER TABLE test_database_rename_table.test_table_5 RENAME TO test_database_rename_table.test_table_6" -+ ) + ) mysql_node.query( + "ALTER TABLE test_database_rename_table.test_table_6 RENAME AS test_database_rename_table.test_table_7" -+ ) + ) check_query( clickhouse_node, From a9a208c3cdc9ccc370a529429158e047e42c238a Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 11 Nov 2022 11:46:22 +0800 Subject: [PATCH 43/82] to MaterializedMySQL_improvement_bug_fix-fix code style --- .../materialize_with_ddl.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index a994863417e..cdb8ba464af 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -876,13 +876,13 @@ def alter_rename_table_with_materialized_mysql_database( ) mysql_node.query( -+ "ALTER TABLE test_database_rename_table.test_table_4 RENAME test_database_rename_table.test_table_5" + "ALTER TABLE test_database_rename_table.test_table_4 RENAME test_database_rename_table.test_table_5" ) mysql_node.query( -+ "ALTER TABLE test_database_rename_table.test_table_5 RENAME TO test_database_rename_table.test_table_6" + "ALTER TABLE test_database_rename_table.test_table_5 RENAME TO test_database_rename_table.test_table_6" ) mysql_node.query( -+ "ALTER TABLE test_database_rename_table.test_table_6 RENAME AS test_database_rename_table.test_table_7" + "ALTER TABLE test_database_rename_table.test_table_6 RENAME AS test_database_rename_table.test_table_7" ) check_query( From 3835373644c851b646c0b770f2bfa74f953ccacd Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 11 Nov 2022 16:40:10 +0800 Subject: [PATCH 44/82] to add_oss_function_and_StorageOSS --- .../maskSensitiveInfoInQueryForLogging.cpp | 4 ++-- src/Storages/StorageS3.cpp | 5 +++++ src/Storages/registerStorages.cpp | 2 ++ src/TableFunctions/TableFunctionS3.cpp | 5 +++++ src/TableFunctions/TableFunctionS3.h | 12 ++++++++++++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + ...14_all_new_table_functions_must_be_documented.sql | 2 +- 8 files changed, 29 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp b/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp index c69f91394b9..fe05283eef5 100644 --- a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp +++ b/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp @@ -164,7 +164,7 @@ namespace /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) wipePasswordFromArgument(*storage.engine, data, 4); } - else if (engine_name == "S3" || engine_name == "COSN") + else if (engine_name == "S3" || engine_name == "COSN" || engine_name == "OSS") { /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) wipePasswordFromS3TableEngineArguments(*storage.engine, data); @@ -222,7 +222,7 @@ namespace /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) wipePasswordFromArgument(function, data, 4); } - else if (function.name == "s3" || function.name == "cosn") + else if (function.name == "s3" || function.name == "cosn" || function.name == "oss") { /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) wipePasswordFromS3FunctionArguments(function, data, /* is_cluster_function= */ false); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index d759c339dea..ab9b71f5ff3 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1314,6 +1314,11 @@ void registerStorageCOS(StorageFactory & factory) return registerStorageS3Impl("COSN", factory); } +void registerStorageOSS(StorageFactory & factory) +{ + return registerStorageS3Impl("OSS", factory); +} + NamesAndTypesList StorageS3::getVirtuals() const { return virtual_columns; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index fd81b55ef61..200b8e637da 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -32,6 +32,7 @@ void registerStorageMeiliSearch(StorageFactory& factory); #if USE_AWS_S3 void registerStorageS3(StorageFactory & factory); void registerStorageCOS(StorageFactory & factory); +void registerStorageOSS(StorageFactory & factory); void registerStorageHudi(StorageFactory & factory); void registerStorageDelta(StorageFactory & factory); #endif @@ -120,6 +121,7 @@ void registerStorages() #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); + registerStorageOSS(factory); registerStorageHudi(factory); registerStorageDelta(factory); #endif diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 4c0b5352545..7f07dc16025 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -183,6 +183,11 @@ void registerTableFunctionCOS(TableFunctionFactory & factory) factory.registerFunction(); } +void registerTableFunctionOSS(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} + } #endif diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index 5c12c2a3975..e13fb7f88d9 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -62,6 +62,18 @@ private: const char * getStorageTypeName() const override { return "COSN"; } }; +class TableFunctionOSS : public TableFunctionS3 +{ +public: + static constexpr auto name = "oss"; + std::string getName() const override + { + return name; + } +private: + const char * getStorageTypeName() const override { return "OSS"; } +}; + } #endif diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 9328c12c122..89701f3bf70 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -27,6 +27,7 @@ void registerTableFunctions() registerTableFunctionS3(factory); registerTableFunctionS3Cluster(factory); registerTableFunctionCOS(factory); + registerTableFunctionOSS(factory); #endif #if USE_HDFS diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 3bc9e3a85da..afbda491e9a 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -24,6 +24,7 @@ void registerTableFunctionMeiliSearch(TableFunctionFactory & factory); void registerTableFunctionS3(TableFunctionFactory & factory); void registerTableFunctionS3Cluster(TableFunctionFactory & factory); void registerTableFunctionCOS(TableFunctionFactory & factory); +void registerTableFunctionOSS(TableFunctionFactory & factory); #endif #if USE_HDFS diff --git a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql index 5d43ec6f0c2..a73993f6a5a 100644 --- a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql +++ b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql @@ -2,5 +2,5 @@ -- Please help shorten this list down to zero elements. SELECT name FROM system.table_functions WHERE length(description) < 10 AND name NOT IN ( - 'cosn', 'hdfs', 'hdfsCluster', 'hive', 'mysql', 'postgresql', 's3', 's3Cluster', 'sqlite' -- these functions are not enabled in fast test + 'cosn', 'oss', 'hdfs', 'hdfsCluster', 'hive', 'mysql', 'postgresql', 's3', 's3Cluster', 'sqlite' -- these functions are not enabled in fast test ) ORDER BY name; From 4ce261dfd77da5ca5fbd249c9aa961ddba7e4143 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 10 Nov 2022 19:17:10 +0100 Subject: [PATCH 45/82] Analyzer improve JOIN with constants --- src/Planner/PlannerJoinTree.cpp | 6 ++- src/Planner/PlannerJoins.cpp | 40 +++++++++++++++++-- ...479_analyzer_join_with_constants.reference | 15 +++++++ .../02479_analyzer_join_with_constants.sql | 27 +++++++++++++ 4 files changed, 84 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02479_analyzer_join_with_constants.reference create mode 100644 tests/queries/0_stateless/02479_analyzer_join_with_constants.sql diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 205c6c5e740..2dc6e7fa678 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -227,7 +227,11 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, JoinClausesAndActions join_clauses_and_actions; JoinKind join_kind = join_node.getKind(); - auto join_constant = tryExtractConstantFromJoinNode(join_tree_node); + std::optional join_constant; + + if (join_node.getStrictness() == JoinStrictness::All) + join_constant = tryExtractConstantFromJoinNode(join_tree_node); + if (join_constant) { /** If there is JOIN with always true constant, we transform it to cross. diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index f62517eaaad..53b9cfc5d99 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -20,6 +20,7 @@ #include #include +#include #include #include #include @@ -76,6 +77,23 @@ void JoinClause::dump(WriteBuffer & buffer) const if (!right_filter_condition_nodes.empty()) buffer << " right_condition_nodes: " + dump_dag_nodes(right_filter_condition_nodes); + + if (!asof_conditions.empty()) + { + buffer << " asof_conditions: "; + size_t asof_conditions_size = asof_conditions.size(); + + for (size_t i = 0; i < asof_conditions_size; ++i) + { + const auto & asof_condition = asof_conditions[i]; + + buffer << "key_index: " << asof_condition.key_index; + buffer << "inequality: " << toString(asof_condition.asof_inequality); + + if (i + 1 != asof_conditions_size) + buffer << ','; + } + } } String JoinClause::dump() const @@ -249,9 +267,7 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, join_node); if (!expression_side_optional) - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} with constants is not supported", - join_node.formatASTForErrorMessage()); + expression_side_optional = JoinTableSide::Right; auto expression_side = *expression_side_optional; join_clause.addCondition(expression_side, join_expressions_actions_node); @@ -277,6 +293,22 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & for (const auto & node : join_expression_actions_nodes) join_expression_dag_input_nodes.insert(&node); + auto * function_node = join_node.getJoinExpression()->as(); + if (!function_node) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} join expression expected function", + join_node.formatASTForErrorMessage()); + + /** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction. + * If we do not ignore it, this function will be replaced by underlying constant. + * For example ASOF JOIN does not support JOIN with constants, and we should process it like ordinary JOIN. + * + * Example: SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id, 1 AS value) AS t2 + * ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t1.value); + */ + auto constant_value = function_node->getConstantValueOrNull(); + function_node->performConstantFolding({}); + PlannerActionsVisitor join_expression_visitor(planner_context); auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_node.getJoinExpression()); if (join_expression_dag_node_raw_pointers.size() != 1) @@ -284,6 +316,8 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & "JOIN {} ON clause contains multiple expressions", join_node.formatASTForErrorMessage()); + function_node->performConstantFolding(std::move(constant_value)); + const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0]; if (!join_expressions_actions_root_node->function) throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, diff --git a/tests/queries/0_stateless/02479_analyzer_join_with_constants.reference b/tests/queries/0_stateless/02479_analyzer_join_with_constants.reference new file mode 100644 index 00000000000..2a428d5d927 --- /dev/null +++ b/tests/queries/0_stateless/02479_analyzer_join_with_constants.reference @@ -0,0 +1,15 @@ +1 1 +-- +-- +-- +1 2 +-- +1 1 1 1 +-- +1 1 0 0 +-- +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql b/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql new file mode 100644 index 00000000000..0cc3ff3ab00 --- /dev/null +++ b/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql @@ -0,0 +1,27 @@ +SET allow_experimental_analyzer = 1; + +SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 1 AS id) AS t2 ON t1.id = t2.id AND 1; + +SELECT '--'; + +SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 2 AS id) AS t2 ON t1.id = t2.id AND 1; + +SELECT '--'; + +SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 1 AS id) AS t2 ON t1.id = t2.id AND 0; + +SELECT '--'; + +SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 2 AS id) AS t2 ON t1.id = t2.id OR 1; + +SELECT '--'; + +SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id, 1 AS value) AS t2 ON (t1.id = t2.id) AND 1 == 1 AND (t1.value >= t2.value); + +SELECT '--'; + +SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id, 1 AS value) AS t2 ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t2.value); + +SELECT '--'; + +SELECT b.dt FROM (SELECT NULL > NULL AS pk, 1 AS dt FROM numbers(5)) AS a ASOF LEFT JOIN (SELECT NULL AS pk, 1 AS dt) AS b ON (a.pk = b.pk) AND 1 != 1 AND (a.dt >= b.dt); From a7f0a2255fed546d15302e3f06f81ade83ce892b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 10 Nov 2022 12:50:56 +0100 Subject: [PATCH 46/82] Analyzer aggregation crash fix --- .../Passes/NormalizeCountVariantsPass.cpp | 19 ++++++++++++++----- ...02479_analyzer_aggregation_crash.reference | 2 ++ .../02479_analyzer_aggregation_crash.sql | 13 +++++++++++++ 3 files changed, 29 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02479_analyzer_aggregation_crash.reference create mode 100644 tests/queries/0_stateless/02479_analyzer_aggregation_crash.sql diff --git a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp index 8c92ecc3900..b91fc94e6cd 100644 --- a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp @@ -33,18 +33,27 @@ public: if (function_node->getFunctionName() == "count" && !first_argument_constant_literal.isNull()) { + resolveAsCountAggregateFunction(*function_node); function_node->getArguments().getNodes().clear(); } - else if (function_node->getFunctionName() == "sum" && first_argument_constant_literal.getType() == Field::Types::UInt64 && + else if (function_node->getFunctionName() == "sum" && + first_argument_constant_literal.getType() == Field::Types::UInt64 && first_argument_constant_literal.get() == 1) { - auto result_type = function_node->getResultType(); - AggregateFunctionProperties properties; - auto aggregate_function = AggregateFunctionFactory::instance().get("count", {}, {}, properties); - function_node->resolveAsAggregateFunction(std::move(aggregate_function), std::move(result_type)); + resolveAsCountAggregateFunction(*function_node); function_node->getArguments().getNodes().clear(); } } +private: + static inline void resolveAsCountAggregateFunction(FunctionNode & function_node) + { + auto function_result_type = function_node.getResultType(); + + AggregateFunctionProperties properties; + auto aggregate_function = AggregateFunctionFactory::instance().get("count", {}, {}, properties); + + function_node.resolveAsAggregateFunction(std::move(aggregate_function), std::move(function_result_type)); + } }; } diff --git a/tests/queries/0_stateless/02479_analyzer_aggregation_crash.reference b/tests/queries/0_stateless/02479_analyzer_aggregation_crash.reference new file mode 100644 index 00000000000..73d811f75f3 --- /dev/null +++ b/tests/queries/0_stateless/02479_analyzer_aggregation_crash.reference @@ -0,0 +1,2 @@ +10 123456789 +10 123456789 diff --git a/tests/queries/0_stateless/02479_analyzer_aggregation_crash.sql b/tests/queries/0_stateless/02479_analyzer_aggregation_crash.sql new file mode 100644 index 00000000000..c931a3ab634 --- /dev/null +++ b/tests/queries/0_stateless/02479_analyzer_aggregation_crash.sql @@ -0,0 +1,13 @@ +SET allow_experimental_analyzer = 1; +SET compile_aggregate_expressions = 1; +SET min_count_to_compile_aggregate_expression = 0; + +DROP TABLE IF EXISTS lc_00906__fuzz_46; +CREATE TABLE lc_00906__fuzz_46 (`b` Int64) ENGINE = MergeTree ORDER BY b; +INSERT INTO lc_00906__fuzz_46 SELECT '0123456789' FROM numbers(10); + +SELECT count(3.4028234663852886e38), b FROM lc_00906__fuzz_46 GROUP BY b; + +SELECT count(1), b FROM lc_00906__fuzz_46 GROUP BY b; + +DROP TABLE lc_00906__fuzz_46; From 66c0af1a1c8e7a5c0c7c2344d643ee8331b39928 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 11 Nov 2022 12:46:57 +0000 Subject: [PATCH 47/82] Move test --- .../0_stateless/02456_datetime_schema_inference.reference | 1 + tests/queries/0_stateless/02456_datetime_schema_inference.sql | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02456_datetime_schema_inference.reference b/tests/queries/0_stateless/02456_datetime_schema_inference.reference index 2b88f2783dc..0719f0eb44e 100644 --- a/tests/queries/0_stateless/02456_datetime_schema_inference.reference +++ b/tests/queries/0_stateless/02456_datetime_schema_inference.reference @@ -9,3 +9,4 @@ 2022-04-22 03:45:06.381000000 2022-04-22 03:45:06.381000000 1925-01-12 00:00:00.000000000 + diff --git a/tests/queries/0_stateless/02456_datetime_schema_inference.sql b/tests/queries/0_stateless/02456_datetime_schema_inference.sql index 34749dbd412..123bb324f87 100644 --- a/tests/queries/0_stateless/02456_datetime_schema_inference.sql +++ b/tests/queries/0_stateless/02456_datetime_schema_inference.sql @@ -12,4 +12,4 @@ set date_time_input_format = 'best_effort_us'; select * from format('TSV', '2022-04-22T03:45:06.381'); select toTimeZone(c1, 'UTC') from format('TSV', '2022-04-22T03:45:06.381Z'); select * from format('TSV', '01/12/1925'); - +select * from format(CSV, '""'); From dee865ce78aee73912370a12131a9e4cb2d16d8f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 10 Nov 2022 11:05:35 +0000 Subject: [PATCH 48/82] Fix trailing \n from SQLancer status --- tests/ci/sqlancer_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/sqlancer_check.py b/tests/ci/sqlancer_check.py index 51c95e50746..63c7d18fe46 100644 --- a/tests/ci/sqlancer_check.py +++ b/tests/ci/sqlancer_check.py @@ -150,7 +150,7 @@ if __name__ == "__main__": os.path.join(workspace_path, "summary.tsv"), "r", encoding="utf-8" ) as summary_f: for line in summary_f: - l = line.split("\t") + l = line.rstrip("\n").split("\t") test_results.append((l[0], l[1])) with open( From e83878493c9d0ea7492288b5fdf401532539bd20 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 11 Nov 2022 12:50:17 +0000 Subject: [PATCH 49/82] Update settings history --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b78b812da86..7635e121f8e 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -78,6 +78,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, From 5ec748c664b1fdae5acca024753e92fe7e610f99 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 11 Nov 2022 14:49:49 +0000 Subject: [PATCH 50/82] Fix function if in case of NULL and const Nullable arguments --- src/Functions/if.cpp | 2 ++ .../02479_if_with_null_and_cullable_const.reference | 4 ++++ .../0_stateless/02479_if_with_null_and_cullable_const.sql | 3 +++ 3 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/02479_if_with_null_and_cullable_const.reference create mode 100644 tests/queries/0_stateless/02479_if_with_null_and_cullable_const.sql diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 86707fc62d6..f3401713834 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -904,6 +904,7 @@ private: if (cond_col) { + arg_else_column = arg_else_column->convertToFullColumnIfConst(); auto result_column = IColumn::mutate(std::move(arg_else_column)); if (else_is_short) result_column->expand(cond_col->getData(), true); @@ -941,6 +942,7 @@ private: if (cond_col) { + arg_then_column = arg_then_column->convertToFullColumnIfConst(); auto result_column = IColumn::mutate(std::move(arg_then_column)); if (then_is_short) result_column->expand(cond_col->getData(), false); diff --git a/tests/queries/0_stateless/02479_if_with_null_and_cullable_const.reference b/tests/queries/0_stateless/02479_if_with_null_and_cullable_const.reference new file mode 100644 index 00000000000..376364af7b4 --- /dev/null +++ b/tests/queries/0_stateless/02479_if_with_null_and_cullable_const.reference @@ -0,0 +1,4 @@ +1 +\N +\N +1 diff --git a/tests/queries/0_stateless/02479_if_with_null_and_cullable_const.sql b/tests/queries/0_stateless/02479_if_with_null_and_cullable_const.sql new file mode 100644 index 00000000000..b684de88cb2 --- /dev/null +++ b/tests/queries/0_stateless/02479_if_with_null_and_cullable_const.sql @@ -0,0 +1,3 @@ +SELECT if(number % 2, NULL, toNullable(1)) FROM numbers(2); +SELECT if(number % 2, toNullable(1), NULL) FROM numbers(2); + From 28c090f7a553d7d5161c3d2c8b8bbf23498d82d4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 11 Nov 2022 15:42:53 +0000 Subject: [PATCH 51/82] Fix decimal math overflow in parsing datetime with 'best effort' algorithm --- src/IO/ReadHelpers.h | 5 +++-- src/IO/parseDateTimeBestEffort.cpp | 3 +++ ...02480_parse_date_time_best_effort_math_overflow.reference | 3 +++ .../02480_parse_date_time_best_effort_math_overflow.sql | 3 +++ 4 files changed, 12 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.reference create mode 100644 tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.sql diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 27a24eef804..7156ac0ceb3 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -964,15 +964,16 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re components.whole = components.whole / common::exp10_i32(scale); } + bool is_ok = true; if constexpr (std::is_same_v) datetime64 = DecimalUtils::decimalFromComponents(components, scale); else - DecimalUtils::tryGetDecimalFromComponents(components, scale, datetime64); + is_ok = DecimalUtils::tryGetDecimalFromComponents(components, scale, datetime64); datetime64 *= negative_multiplier; - return ReturnType(true); + return ReturnType(is_ok); } inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 1dd06401bf1..e0cba169e81 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -659,6 +659,9 @@ ReturnType parseDateTime64BestEffortImpl(DateTime64 & res, UInt32 scale, ReadBuf fractional *= common::exp10_i64(scale - subsecond.digits); } + if constexpr (std::is_same_v) + return DecimalUtils::tryGetDecimalFromComponents(whole, fractional, scale, res); + res = DecimalUtils::decimalFromComponents(whole, fractional, scale); return ReturnType(true); } diff --git a/tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.reference b/tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.reference new file mode 100644 index 00000000000..1515932af18 --- /dev/null +++ b/tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.reference @@ -0,0 +1,3 @@ +9279104479c7da1114861274de32208ead91b60e +\N +\N diff --git a/tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.sql b/tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.sql new file mode 100644 index 00000000000..5102fb47204 --- /dev/null +++ b/tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.sql @@ -0,0 +1,3 @@ +select * from format(TSV, '9279104479c7da1114861274de32208ead91b60e') settings date_time_input_format='best_effort'; +select parseDateTime64BestEffortOrNull('9279104477', 9); +select toDateTime64OrNull('9279104477', 9); From 616da1f69553a21e101fd8297d9b2ff451547a6d Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 11 Nov 2022 13:04:35 +0100 Subject: [PATCH 52/82] Review fixed, style check fix --- src/Common/ErrorCodes.cpp | 2 +- src/Storages/NamedCollections.cpp | 24 +++++++++---------- .../01271_show_privileges.reference | 1 + .../02117_show_create_table_system.reference | 6 ++--- utils/check-style/check-style | 2 +- 5 files changed, 18 insertions(+), 17 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index ce96b1a9917..2bc5d70421a 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -638,7 +638,7 @@ M(667, NOT_INITIALIZED) \ M(668, INVALID_STATE) \ M(669, UNKNOWN_NAMED_COLLECTION) \ - M(669, NAMED_COLLECTION_ALREADY_EXISTS) \ + M(670, NAMED_COLLECTION_ALREADY_EXISTS) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/NamedCollections.cpp b/src/Storages/NamedCollections.cpp index 60c11c90b0a..67847635f3f 100644 --- a/src/Storages/NamedCollections.cpp +++ b/src/Storages/NamedCollections.cpp @@ -18,18 +18,19 @@ namespace DB { -static constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections"; - namespace ErrorCodes { extern const int UNKNOWN_NAMED_COLLECTION; extern const int NAMED_COLLECTION_ALREADY_EXISTS; extern const int BAD_ARGUMENTS; extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; } namespace { + constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections"; + std::string getCollectionPrefix(const std::string & collection_name) { return fmt::format("{}.{}", NAMED_COLLECTIONS_CONFIG_PREFIX, collection_name); @@ -49,7 +50,7 @@ namespace /// the `result` will contain two strings: "root.key1.key2" and "root.key1.key3.key4" void collectKeys( const Poco::Util::AbstractConfiguration & config, - std::queue & enumerate_paths, + std::queue enumerate_paths, std::set & result) { if (enumerate_paths.empty()) @@ -241,8 +242,7 @@ NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const class NamedCollection::Impl { private: - using IConfigurationPtr = Poco::AutoPtr; - using ConfigurationPtr = Poco::AutoPtr; + using ConfigurationPtr = Poco::AutoPtr; /// Named collection configuration /// @@ -270,7 +270,7 @@ public: template T getOrDefault(const Key & key, const T & default_value) const { - return getConfigValueOrDefault(*config, key, default_value); + return getConfigValueOrDefault(*config, key, &default_value); } template void set(const Key & key, const T & value, bool update_if_exists) @@ -312,9 +312,9 @@ public: /// to a string: /// "key0: value0 /// key1: - /// key2: value2 - /// key3: - /// key4: value3" + /// key2: value2 + /// key3: + /// key4: value3" WriteBufferFromOwnString wb; Strings prev_key_parts; for (const auto & key : keys) @@ -359,7 +359,7 @@ private: template static T getConfigValueOrDefault( const Poco::Util::AbstractConfiguration & config, const std::string & path, - const std::optional & default_value = std::nullopt) + const T * default_value = nullptr) { if (!config.has(path)) { @@ -477,7 +477,7 @@ NamedCollectionPtr NamedCollection::create( std::set enumerate_result; enumerate_input.push(collection_prefix); - collectKeys(config, enumerate_input, enumerate_result); + collectKeys(config, std::move(enumerate_input), enumerate_result); /// Collection does not have any keys. /// (`enumerate_result` == ). @@ -487,7 +487,7 @@ NamedCollectionPtr NamedCollection::create( { /// Skip collection prefix and add +1 to avoid '.' in the beginning. for (const auto & path : enumerate_result) - keys.emplace(path.substr(std::strlen(collection_prefix.data()) + 1)); + keys.emplace(path.substr(collection_prefix.size() + 1)); } return std::make_unique(config, collection_name, keys); } diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index b332a7d71f7..59c7d978493 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -85,6 +85,7 @@ SHOW ROLES ['SHOW CREATE ROLE'] GLOBAL SHOW ACCESS SHOW ROW POLICIES ['SHOW POLICIES','SHOW CREATE ROW POLICY','SHOW CREATE POLICY'] TABLE SHOW ACCESS SHOW QUOTAS ['SHOW CREATE QUOTA'] GLOBAL SHOW ACCESS SHOW SETTINGS PROFILES ['SHOW PROFILES','SHOW CREATE SETTINGS PROFILE','SHOW CREATE PROFILE'] GLOBAL SHOW ACCESS +SHOW NAMED COLLECTIONS ['SHOW NAMED COLLECTIONS'] GLOBAL SHOW ACCESS SHOW ACCESS [] \N ACCESS MANAGEMENT ACCESS MANAGEMENT [] \N ALL SYSTEM SHUTDOWN ['SYSTEM KILL','SHUTDOWN'] GLOBAL SYSTEM diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 4d705e4b0d1..093e72a092a 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -281,7 +281,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW ACCESS' = 87, 'ACCESS MANAGEMENT' = 88, 'SYSTEM SHUTDOWN' = 89, 'SYSTEM DROP DNS CACHE' = 90, 'SYSTEM DROP MARK CACHE' = 91, 'SYSTEM DROP UNCOMPRESSED CACHE' = 92, 'SYSTEM DROP MMAP CACHE' = 93, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 94, 'SYSTEM DROP FILESYSTEM CACHE' = 95, 'SYSTEM DROP SCHEMA CACHE' = 96, 'SYSTEM DROP CACHE' = 97, 'SYSTEM RELOAD CONFIG' = 98, 'SYSTEM RELOAD USERS' = 99, 'SYSTEM RELOAD SYMBOLS' = 100, 'SYSTEM RELOAD DICTIONARY' = 101, 'SYSTEM RELOAD MODEL' = 102, 'SYSTEM RELOAD FUNCTION' = 103, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 104, 'SYSTEM RELOAD' = 105, 'SYSTEM RESTART DISK' = 106, 'SYSTEM MERGES' = 107, 'SYSTEM TTL MERGES' = 108, 'SYSTEM FETCHES' = 109, 'SYSTEM MOVES' = 110, 'SYSTEM DISTRIBUTED SENDS' = 111, 'SYSTEM REPLICATED SENDS' = 112, 'SYSTEM SENDS' = 113, 'SYSTEM REPLICATION QUEUES' = 114, 'SYSTEM DROP REPLICA' = 115, 'SYSTEM SYNC REPLICA' = 116, 'SYSTEM RESTART REPLICA' = 117, 'SYSTEM RESTORE REPLICA' = 118, 'SYSTEM SYNC DATABASE REPLICA' = 119, 'SYSTEM SYNC TRANSACTION LOG' = 120, 'SYSTEM FLUSH DISTRIBUTED' = 121, 'SYSTEM FLUSH LOGS' = 122, 'SYSTEM FLUSH' = 123, 'SYSTEM THREAD FUZZER' = 124, 'SYSTEM UNFREEZE' = 125, 'SYSTEM' = 126, 'dictGet' = 127, 'addressToLine' = 128, 'addressToLineWithInlines' = 129, 'addressToSymbol' = 130, 'demangle' = 131, 'INTROSPECTION' = 132, 'FILE' = 133, 'URL' = 134, 'REMOTE' = 135, 'MONGO' = 136, 'MEILISEARCH' = 137, 'MYSQL' = 138, 'POSTGRES' = 139, 'SQLITE' = 140, 'ODBC' = 141, 'JDBC' = 142, 'HDFS' = 143, 'S3' = 144, 'HIVE' = 145, 'SOURCES' = 146, 'CLUSTER' = 147, 'ALL' = 148, 'NONE' = 149), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW NAMED COLLECTIONS' = 87, 'SHOW ACCESS' = 88, 'ACCESS MANAGEMENT' = 89, 'SYSTEM SHUTDOWN' = 90, 'SYSTEM DROP DNS CACHE' = 91, 'SYSTEM DROP MARK CACHE' = 92, 'SYSTEM DROP UNCOMPRESSED CACHE' = 93, 'SYSTEM DROP MMAP CACHE' = 94, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 95, 'SYSTEM DROP FILESYSTEM CACHE' = 96, 'SYSTEM DROP SCHEMA CACHE' = 97, 'SYSTEM DROP CACHE' = 98, 'SYSTEM RELOAD CONFIG' = 99, 'SYSTEM RELOAD USERS' = 100, 'SYSTEM RELOAD SYMBOLS' = 101, 'SYSTEM RELOAD DICTIONARY' = 102, 'SYSTEM RELOAD MODEL' = 103, 'SYSTEM RELOAD FUNCTION' = 104, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 105, 'SYSTEM RELOAD' = 106, 'SYSTEM RESTART DISK' = 107, 'SYSTEM MERGES' = 108, 'SYSTEM TTL MERGES' = 109, 'SYSTEM FETCHES' = 110, 'SYSTEM MOVES' = 111, 'SYSTEM DISTRIBUTED SENDS' = 112, 'SYSTEM REPLICATED SENDS' = 113, 'SYSTEM SENDS' = 114, 'SYSTEM REPLICATION QUEUES' = 115, 'SYSTEM DROP REPLICA' = 116, 'SYSTEM SYNC REPLICA' = 117, 'SYSTEM RESTART REPLICA' = 118, 'SYSTEM RESTORE REPLICA' = 119, 'SYSTEM SYNC DATABASE REPLICA' = 120, 'SYSTEM SYNC TRANSACTION LOG' = 121, 'SYSTEM FLUSH DISTRIBUTED' = 122, 'SYSTEM FLUSH LOGS' = 123, 'SYSTEM FLUSH' = 124, 'SYSTEM THREAD FUZZER' = 125, 'SYSTEM UNFREEZE' = 126, 'SYSTEM' = 127, 'dictGet' = 128, 'addressToLine' = 129, 'addressToLineWithInlines' = 130, 'addressToSymbol' = 131, 'demangle' = 132, 'INTROSPECTION' = 133, 'FILE' = 134, 'URL' = 135, 'REMOTE' = 136, 'MONGO' = 137, 'MEILISEARCH' = 138, 'MYSQL' = 139, 'POSTGRES' = 140, 'SQLITE' = 141, 'ODBC' = 142, 'JDBC' = 143, 'HDFS' = 144, 'S3' = 145, 'HIVE' = 146, 'SOURCES' = 147, 'CLUSTER' = 148, 'ALL' = 149, 'NONE' = 150), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -544,10 +544,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW ACCESS' = 87, 'ACCESS MANAGEMENT' = 88, 'SYSTEM SHUTDOWN' = 89, 'SYSTEM DROP DNS CACHE' = 90, 'SYSTEM DROP MARK CACHE' = 91, 'SYSTEM DROP UNCOMPRESSED CACHE' = 92, 'SYSTEM DROP MMAP CACHE' = 93, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 94, 'SYSTEM DROP FILESYSTEM CACHE' = 95, 'SYSTEM DROP SCHEMA CACHE' = 96, 'SYSTEM DROP CACHE' = 97, 'SYSTEM RELOAD CONFIG' = 98, 'SYSTEM RELOAD USERS' = 99, 'SYSTEM RELOAD SYMBOLS' = 100, 'SYSTEM RELOAD DICTIONARY' = 101, 'SYSTEM RELOAD MODEL' = 102, 'SYSTEM RELOAD FUNCTION' = 103, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 104, 'SYSTEM RELOAD' = 105, 'SYSTEM RESTART DISK' = 106, 'SYSTEM MERGES' = 107, 'SYSTEM TTL MERGES' = 108, 'SYSTEM FETCHES' = 109, 'SYSTEM MOVES' = 110, 'SYSTEM DISTRIBUTED SENDS' = 111, 'SYSTEM REPLICATED SENDS' = 112, 'SYSTEM SENDS' = 113, 'SYSTEM REPLICATION QUEUES' = 114, 'SYSTEM DROP REPLICA' = 115, 'SYSTEM SYNC REPLICA' = 116, 'SYSTEM RESTART REPLICA' = 117, 'SYSTEM RESTORE REPLICA' = 118, 'SYSTEM SYNC DATABASE REPLICA' = 119, 'SYSTEM SYNC TRANSACTION LOG' = 120, 'SYSTEM FLUSH DISTRIBUTED' = 121, 'SYSTEM FLUSH LOGS' = 122, 'SYSTEM FLUSH' = 123, 'SYSTEM THREAD FUZZER' = 124, 'SYSTEM UNFREEZE' = 125, 'SYSTEM' = 126, 'dictGet' = 127, 'addressToLine' = 128, 'addressToLineWithInlines' = 129, 'addressToSymbol' = 130, 'demangle' = 131, 'INTROSPECTION' = 132, 'FILE' = 133, 'URL' = 134, 'REMOTE' = 135, 'MONGO' = 136, 'MEILISEARCH' = 137, 'MYSQL' = 138, 'POSTGRES' = 139, 'SQLITE' = 140, 'ODBC' = 141, 'JDBC' = 142, 'HDFS' = 143, 'S3' = 144, 'HIVE' = 145, 'SOURCES' = 146, 'CLUSTER' = 147, 'ALL' = 148, 'NONE' = 149), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW NAMED COLLECTIONS' = 87, 'SHOW ACCESS' = 88, 'ACCESS MANAGEMENT' = 89, 'SYSTEM SHUTDOWN' = 90, 'SYSTEM DROP DNS CACHE' = 91, 'SYSTEM DROP MARK CACHE' = 92, 'SYSTEM DROP UNCOMPRESSED CACHE' = 93, 'SYSTEM DROP MMAP CACHE' = 94, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 95, 'SYSTEM DROP FILESYSTEM CACHE' = 96, 'SYSTEM DROP SCHEMA CACHE' = 97, 'SYSTEM DROP CACHE' = 98, 'SYSTEM RELOAD CONFIG' = 99, 'SYSTEM RELOAD USERS' = 100, 'SYSTEM RELOAD SYMBOLS' = 101, 'SYSTEM RELOAD DICTIONARY' = 102, 'SYSTEM RELOAD MODEL' = 103, 'SYSTEM RELOAD FUNCTION' = 104, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 105, 'SYSTEM RELOAD' = 106, 'SYSTEM RESTART DISK' = 107, 'SYSTEM MERGES' = 108, 'SYSTEM TTL MERGES' = 109, 'SYSTEM FETCHES' = 110, 'SYSTEM MOVES' = 111, 'SYSTEM DISTRIBUTED SENDS' = 112, 'SYSTEM REPLICATED SENDS' = 113, 'SYSTEM SENDS' = 114, 'SYSTEM REPLICATION QUEUES' = 115, 'SYSTEM DROP REPLICA' = 116, 'SYSTEM SYNC REPLICA' = 117, 'SYSTEM RESTART REPLICA' = 118, 'SYSTEM RESTORE REPLICA' = 119, 'SYSTEM SYNC DATABASE REPLICA' = 120, 'SYSTEM SYNC TRANSACTION LOG' = 121, 'SYSTEM FLUSH DISTRIBUTED' = 122, 'SYSTEM FLUSH LOGS' = 123, 'SYSTEM FLUSH' = 124, 'SYSTEM THREAD FUZZER' = 125, 'SYSTEM UNFREEZE' = 126, 'SYSTEM' = 127, 'dictGet' = 128, 'addressToLine' = 129, 'addressToLineWithInlines' = 130, 'addressToSymbol' = 131, 'demangle' = 132, 'INTROSPECTION' = 133, 'FILE' = 134, 'URL' = 135, 'REMOTE' = 136, 'MONGO' = 137, 'MEILISEARCH' = 138, 'MYSQL' = 139, 'POSTGRES' = 140, 'SQLITE' = 141, 'ODBC' = 142, 'JDBC' = 143, 'HDFS' = 144, 'S3' = 145, 'HIVE' = 146, 'SOURCES' = 147, 'CLUSTER' = 148, 'ALL' = 149, 'NONE' = 150), `aliases` Array(String), `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW ACCESS' = 87, 'ACCESS MANAGEMENT' = 88, 'SYSTEM SHUTDOWN' = 89, 'SYSTEM DROP DNS CACHE' = 90, 'SYSTEM DROP MARK CACHE' = 91, 'SYSTEM DROP UNCOMPRESSED CACHE' = 92, 'SYSTEM DROP MMAP CACHE' = 93, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 94, 'SYSTEM DROP FILESYSTEM CACHE' = 95, 'SYSTEM DROP SCHEMA CACHE' = 96, 'SYSTEM DROP CACHE' = 97, 'SYSTEM RELOAD CONFIG' = 98, 'SYSTEM RELOAD USERS' = 99, 'SYSTEM RELOAD SYMBOLS' = 100, 'SYSTEM RELOAD DICTIONARY' = 101, 'SYSTEM RELOAD MODEL' = 102, 'SYSTEM RELOAD FUNCTION' = 103, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 104, 'SYSTEM RELOAD' = 105, 'SYSTEM RESTART DISK' = 106, 'SYSTEM MERGES' = 107, 'SYSTEM TTL MERGES' = 108, 'SYSTEM FETCHES' = 109, 'SYSTEM MOVES' = 110, 'SYSTEM DISTRIBUTED SENDS' = 111, 'SYSTEM REPLICATED SENDS' = 112, 'SYSTEM SENDS' = 113, 'SYSTEM REPLICATION QUEUES' = 114, 'SYSTEM DROP REPLICA' = 115, 'SYSTEM SYNC REPLICA' = 116, 'SYSTEM RESTART REPLICA' = 117, 'SYSTEM RESTORE REPLICA' = 118, 'SYSTEM SYNC DATABASE REPLICA' = 119, 'SYSTEM SYNC TRANSACTION LOG' = 120, 'SYSTEM FLUSH DISTRIBUTED' = 121, 'SYSTEM FLUSH LOGS' = 122, 'SYSTEM FLUSH' = 123, 'SYSTEM THREAD FUZZER' = 124, 'SYSTEM UNFREEZE' = 125, 'SYSTEM' = 126, 'dictGet' = 127, 'addressToLine' = 128, 'addressToLineWithInlines' = 129, 'addressToSymbol' = 130, 'demangle' = 131, 'INTROSPECTION' = 132, 'FILE' = 133, 'URL' = 134, 'REMOTE' = 135, 'MONGO' = 136, 'MEILISEARCH' = 137, 'MYSQL' = 138, 'POSTGRES' = 139, 'SQLITE' = 140, 'ODBC' = 141, 'JDBC' = 142, 'HDFS' = 143, 'S3' = 144, 'HIVE' = 145, 'SOURCES' = 146, 'CLUSTER' = 147, 'ALL' = 148, 'NONE' = 149)) + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW NAMED COLLECTIONS' = 87, 'SHOW ACCESS' = 88, 'ACCESS MANAGEMENT' = 89, 'SYSTEM SHUTDOWN' = 90, 'SYSTEM DROP DNS CACHE' = 91, 'SYSTEM DROP MARK CACHE' = 92, 'SYSTEM DROP UNCOMPRESSED CACHE' = 93, 'SYSTEM DROP MMAP CACHE' = 94, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 95, 'SYSTEM DROP FILESYSTEM CACHE' = 96, 'SYSTEM DROP SCHEMA CACHE' = 97, 'SYSTEM DROP CACHE' = 98, 'SYSTEM RELOAD CONFIG' = 99, 'SYSTEM RELOAD USERS' = 100, 'SYSTEM RELOAD SYMBOLS' = 101, 'SYSTEM RELOAD DICTIONARY' = 102, 'SYSTEM RELOAD MODEL' = 103, 'SYSTEM RELOAD FUNCTION' = 104, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 105, 'SYSTEM RELOAD' = 106, 'SYSTEM RESTART DISK' = 107, 'SYSTEM MERGES' = 108, 'SYSTEM TTL MERGES' = 109, 'SYSTEM FETCHES' = 110, 'SYSTEM MOVES' = 111, 'SYSTEM DISTRIBUTED SENDS' = 112, 'SYSTEM REPLICATED SENDS' = 113, 'SYSTEM SENDS' = 114, 'SYSTEM REPLICATION QUEUES' = 115, 'SYSTEM DROP REPLICA' = 116, 'SYSTEM SYNC REPLICA' = 117, 'SYSTEM RESTART REPLICA' = 118, 'SYSTEM RESTORE REPLICA' = 119, 'SYSTEM SYNC DATABASE REPLICA' = 120, 'SYSTEM SYNC TRANSACTION LOG' = 121, 'SYSTEM FLUSH DISTRIBUTED' = 122, 'SYSTEM FLUSH LOGS' = 123, 'SYSTEM FLUSH' = 124, 'SYSTEM THREAD FUZZER' = 125, 'SYSTEM UNFREEZE' = 126, 'SYSTEM' = 127, 'dictGet' = 128, 'addressToLine' = 129, 'addressToLineWithInlines' = 130, 'addressToSymbol' = 131, 'demangle' = 132, 'INTROSPECTION' = 133, 'FILE' = 134, 'URL' = 135, 'REMOTE' = 136, 'MONGO' = 137, 'MEILISEARCH' = 138, 'MYSQL' = 139, 'POSTGRES' = 140, 'SQLITE' = 141, 'ODBC' = 142, 'JDBC' = 143, 'HDFS' = 144, 'S3' = 145, 'HIVE' = 146, 'SOURCES' = 147, 'CLUSTER' = 148, 'ALL' = 149, 'NONE' = 150)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' diff --git a/utils/check-style/check-style b/utils/check-style/check-style index a0556d971e8..ed397a4a162 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -13,7 +13,7 @@ # and then to run formatter only for the specified files. ROOT_PATH=$(git rev-parse --show-toplevel) -EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing|benchmark' +EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing|benchmark|tests/' # From [1]: # But since array_to_string_internal() in array.c still loops over array From 5504f3af9b7624beadce5c4a97b099066a10dfcb Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Sat, 12 Nov 2022 00:03:36 +0800 Subject: [PATCH 53/82] fix skip_unavailable_shards does not work using s3Cluster table function --- src/Storages/StorageS3Cluster.cpp | 38 ++++++++----------- .../test_s3_cluster/configs/cluster.xml | 15 ++++++++ tests/integration/test_s3_cluster/test.py | 29 ++++++++++++++ 3 files changed, 59 insertions(+), 23 deletions(-) diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 3b8c8b1cb92..350e942f972 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -117,32 +117,24 @@ Pipe StorageS3Cluster::read( addColumnsStructureToQueryWithClusterEngine( query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 5, getName()); - for (const auto & replicas : cluster->getShardsAddresses()) + const auto & current_settings = context->getSettingsRef(); + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); + for (const auto & shard_info : cluster->getShardsInfo()) { - /// There will be only one replica, because we consider each replica as a shard - for (const auto & node : replicas) + auto try_results = shard_info.pool->getMany(timeouts, ¤t_settings, PoolMode::GET_MANY); + for (auto & try_result : try_results) { - auto connection = std::make_shared( - node.host_name, node.port, context->getGlobalContext()->getCurrentDatabase(), - node.user, node.password, node.quota_key, node.cluster, node.cluster_secret, - "S3ClusterInititiator", - node.compression, - node.secure - ); - - - /// For unknown reason global context is passed to IStorage::read() method - /// So, task_identifier is passed as constructor argument. It is more obvious. auto remote_query_executor = std::make_shared( - connection, - queryToString(query_to_send), - header, - context, - /*throttler=*/nullptr, - scalars, - Tables(), - processed_stage, - RemoteQueryExecutor::Extension{.task_iterator = callback}); + shard_info.pool, + std::vector{try_result}, + queryToString(query_to_send), + header, + context, + /*throttler=*/nullptr, + scalars, + Tables(), + processed_stage, + RemoteQueryExecutor::Extension{.task_iterator = callback}); pipes.emplace_back(std::make_shared(remote_query_executor, add_agg_info, false)); } diff --git a/tests/integration/test_s3_cluster/configs/cluster.xml b/tests/integration/test_s3_cluster/configs/cluster.xml index 18f15763633..3059340cfe4 100644 --- a/tests/integration/test_s3_cluster/configs/cluster.xml +++ b/tests/integration/test_s3_cluster/configs/cluster.xml @@ -20,6 +20,21 @@ + + + + s0_0_0 + 9000 + + + + + s0_0_0 + 19000 + + + + cluster_simple diff --git a/tests/integration/test_s3_cluster/test.py b/tests/integration/test_s3_cluster/test.py index 2cbb36fcf06..8e082f7d86a 100644 --- a/tests/integration/test_s3_cluster/test.py +++ b/tests/integration/test_s3_cluster/test.py @@ -195,3 +195,32 @@ def test_ambiguous_join(started_cluster): """ ) assert "AMBIGUOUS_COLUMN_NAME" not in result + + +def test_skip_unavailable_shards(started_cluster): + node = started_cluster.instances["s0_0_0"] + result = node.query( + """ + SELECT count(*) from s3Cluster( + 'cluster_non_existent_port', + 'http://minio1:9001/root/data/clickhouse/part1.csv', + 'minio', 'minio123', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') + SETTINGS skip_unavailable_shards = 1 + """ + ) + + assert result == "10\n" + + +def test_unskip_unavailable_shards(started_cluster): + node = started_cluster.instances["s0_0_0"] + error = node.query_and_get_error( + """ + SELECT count(*) from s3Cluster( + 'cluster_non_existent_port', + 'http://minio1:9001/root/data/clickhouse/part1.csv', + 'minio', 'minio123', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') + """ + ) + + assert "NETWORK_ERROR" in error From dbf64e27125bcd4e4882d22957695d466dde7152 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 11 Nov 2022 17:23:40 +0100 Subject: [PATCH 54/82] fix --- tests/queries/0_stateless/02448_clone_replica_lost_part.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql index 371f7389837..7029d6bf4ea 100644 --- a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql +++ b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql @@ -3,9 +3,9 @@ drop table if exists rmt1; drop table if exists rmt2; create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '1') order by tuple() - settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, old_parts_lifetime=0, max_parts_to_merge_at_once=5; + settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, old_parts_lifetime=0, max_parts_to_merge_at_once=4; create table rmt2 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '2') order by tuple() - settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, old_parts_lifetime=0, max_parts_to_merge_at_once=5; + settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, old_parts_lifetime=0, max_parts_to_merge_at_once=4; -- insert part only on one replica system stop replicated sends rmt1; From 0208fff321bd4090f3f873411a2ee5cbc3e31e99 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 11 Nov 2022 18:43:33 +0000 Subject: [PATCH 55/82] Randomize keeper fault injection settings in stress tests --- tests/clickhouse-test | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 20e63412d91..c054e66898e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -456,6 +456,8 @@ class SettingsRandomizer: "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), "optimize_distinct_in_order": lambda: random.randint(0, 1), "optimize_sorting_by_input_stream_properties": lambda: random.randint(0, 1), + "insert_keeper_max_retries": lambda: random.randint(20, 20), + "insert_keeper_fault_injection_probability": lambda: random.uniform(0, 0.01), } @staticmethod From 60d116729cf645cae40cab76c938e89c5cbf4e46 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 12 Nov 2022 02:55:26 +0000 Subject: [PATCH 56/82] adjustments for progress option --- programs/client/Client.cpp | 3 +- programs/local/LocalServer.cpp | 4 +- src/Client/ClientBase.cpp | 119 ++++++++++++++++++--------------- src/Client/ClientBase.h | 5 +- 4 files changed, 71 insertions(+), 60 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index b262c5a0db1..6e289b57845 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -243,6 +243,7 @@ try registerAggregateFunctions(); processConfig(); + initTtyBuffer(toProgressOption(config().getString("progress", "default"))); /// Includes delayed_interactive. if (is_interactive) @@ -1088,8 +1089,6 @@ void Client::processConfig() } else { - std::string progress = config().getString("progress", "off"); - need_render_progress = (Poco::icompare(progress, "off") && Poco::icompare(progress, "no") && Poco::icompare(progress, "false") && Poco::icompare(progress, "0")); echo_queries = config().getBool("echo", false); ignore_error = config().getBool("ignore-error", false); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index fea82b456cd..1d8b7edb20e 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -414,6 +414,8 @@ try registerFormats(); processConfig(); + initTtyBuffer(toProgressOption(config().getString("progress", "default"))); + applyCmdSettings(global_context); if (is_interactive) @@ -489,8 +491,6 @@ void LocalServer::processConfig() } else { - std::string progress = config().getString("progress", "off"); - need_render_progress = (Poco::icompare(progress, "off") && Poco::icompare(progress, "no") && Poco::icompare(progress, "false") && Poco::icompare(progress, "0")); echo_queries = config().hasOption("echo") || config().hasOption("verbose"); ignore_error = config().getBool("ignore-error", false); is_multiquery = true; diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5bd65ac2334..c69a3ff81ca 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -119,22 +119,27 @@ namespace ProfileEvents namespace DB { +ProgressOption toProgressOption(std::string progress) +{ + boost::to_upper(progress); + + if (progress == "OFF" || progress == "FALSE" || progress == "0" || progress == "NO") + return ProgressOption::OFF; + if (progress == "TTY" || progress == "ON" || progress == "TRUE" || progress == "1" || progress == "YES") + return ProgressOption::TTY; + if (progress == "ERR") + return ProgressOption::ERR; + if (progress == "DEFAULT") + return ProgressOption::DEFAULT; + + throw boost::program_options::validation_error(boost::program_options::validation_error::invalid_option_value); +} + std::istream& operator>> (std::istream & in, ProgressOption & progress) { std::string token; in >> token; - - boost::to_upper(token); - - if (token == "OFF" || token == "FALSE" || token == "0" || token == "NO") - progress = ProgressOption::OFF; - else if (token == "TTY" || token == "ON" || token == "TRUE" || token == "1" || token == "YES") - progress = ProgressOption::TTY; - else if (token == "ERR") - progress = ProgressOption::ERR; - else - throw boost::program_options::validation_error(boost::program_options::validation_error::invalid_option_value); - + progress = toProgressOption(token); return in; } @@ -662,56 +667,62 @@ void ClientBase::initLogsOutputStream() } } -void ClientBase::initTtyBuffer(bool to_err) +void ClientBase::initTtyBuffer(ProgressOption progress) { - if (!tty_buf) + if (tty_buf) + return; + + if (progress == ProgressOption::OFF || (!is_interactive && progress == ProgressOption::DEFAULT)) { - static constexpr auto tty_file_name = "/dev/tty"; + need_render_progress = false; + return; + } - /// Output all progress bar commands to terminal at once to avoid flicker. - /// This size is usually greater than the window size. - static constexpr size_t buf_size = 1024; + static constexpr auto tty_file_name = "/dev/tty"; - if (!to_err) + /// Output all progress bar commands to terminal at once to avoid flicker. + /// This size is usually greater than the window size. + static constexpr size_t buf_size = 1024; + + if (is_interactive || progress == ProgressOption::TTY) + { + std::error_code ec; + std::filesystem::file_status tty = std::filesystem::status(tty_file_name, ec); + + if (!ec && exists(tty) && is_character_file(tty) + && (tty.permissions() & std::filesystem::perms::others_write) != std::filesystem::perms::none) { - std::error_code ec; - std::filesystem::file_status tty = std::filesystem::status(tty_file_name, ec); - - if (!ec && exists(tty) && is_character_file(tty) - && (tty.permissions() & std::filesystem::perms::others_write) != std::filesystem::perms::none) + try { - try - { - tty_buf = std::make_unique(tty_file_name, buf_size); + tty_buf = std::make_unique(tty_file_name, buf_size); - /// It is possible that the terminal file has writeable permissions - /// but we cannot write anything there. Check it with invisible character. - tty_buf->write('\0'); - tty_buf->next(); + /// It is possible that the terminal file has writeable permissions + /// but we cannot write anything there. Check it with invisible character. + tty_buf->write('\0'); + tty_buf->next(); - return; - } - catch (const Exception & e) - { - if (tty_buf) - tty_buf.reset(); + return; + } + catch (const Exception & e) + { + if (tty_buf) + tty_buf.reset(); - if (e.code() != ErrorCodes::CANNOT_OPEN_FILE) - throw; + if (e.code() != ErrorCodes::CANNOT_OPEN_FILE) + throw; - /// It is normal if file exists, indicated as writeable but still cannot be opened. - /// Fallback to other options. - } + /// It is normal if file exists, indicated as writeable but still cannot be opened. + /// Fallback to other options. } } - - if (stderr_is_a_tty) - { - tty_buf = std::make_unique(STDERR_FILENO, buf_size); - } - else - need_render_progress = false; } + + if (stderr_is_a_tty || progress == ProgressOption::ERR) + { + tty_buf = std::make_unique(STDERR_FILENO, buf_size); + } + else + need_render_progress = false; } void ClientBase::updateSuggest(const ASTPtr & ast) @@ -2324,7 +2335,7 @@ void ClientBase::init(int argc, char ** argv) ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") ("query_id", po::value(), "query_id") - ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::OFF, "off"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR: err; OFF (default): off|0|false|no") + ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::DEFAULT, "default"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR non-interactive mode: err; OFF: off|0|false|no; DEFAULT - interactive to TTY, non-interactive is off") ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") @@ -2379,11 +2390,6 @@ void ClientBase::init(int argc, char ** argv) parseAndCheckOptions(options_description, options, common_arguments); po::notify(options); - if (options["progress"].as() == ProgressOption::OFF) - need_render_progress = false; - else - initTtyBuffer(options["progress"].as() == ProgressOption::ERR); - if (options.count("version") || options.count("V")) { showClientVersion(); @@ -2437,6 +2443,9 @@ void ClientBase::init(int argc, char ** argv) { switch (options["progress"].as()) { + case DEFAULT: + config().setString("progress", "default"); + break; case OFF: config().setString("progress", "off"); break; diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 9ea66881cf6..6c85d6a5f2b 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -38,10 +38,12 @@ enum MultiQueryProcessingStage enum ProgressOption { + DEFAULT, OFF, TTY, ERR, }; +ProgressOption toProgressOption(std::string progress); std::istream& operator>> (std::istream & in, ProgressOption & progress); void interruptSignalHandler(int signum); @@ -153,7 +155,6 @@ private: void initOutputFormat(const Block & block, ASTPtr parsed_query); void initLogsOutputStream(); - void initTtyBuffer(bool to_err = false); String prompt() const; @@ -168,6 +169,8 @@ protected: static bool isSyncInsertWithData(const ASTInsertQuery & insert_query, const ContextPtr & context); bool processMultiQueryFromFile(const String & file_name); + void initTtyBuffer(ProgressOption progress); + bool is_interactive = false; /// Use either interactive line editing interface or batch mode. bool is_multiquery = false; bool delayed_interactive = false; From b8c877db2988f89ccf9da848a53adf5f7eb22a86 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 12 Nov 2022 19:07:28 +0100 Subject: [PATCH 57/82] Fix annoying messages in AsynchronousMetrics --- src/Common/Exception.h | 1 + src/Interpreters/AsynchronousMetrics.cpp | 30 +++++++++++++++++------- 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index c5259d157b2..84687581e52 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -12,6 +12,7 @@ #include + namespace Poco { class Logger; } diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 488ac77e956..a6cb6ee1f68 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -24,9 +24,9 @@ #include #include #include +#include #include - #include "config.h" #if USE_JEMALLOC @@ -123,9 +123,9 @@ void AsynchronousMetrics::openSensors() { LOG_WARNING( &Poco::Logger::get("AsynchronousMetrics"), - "Thermal monitor '{}' exists but could not be read, error {}.", + "Thermal monitor '{}' exists but could not be read: {}.", thermal_device_index, - e.getErrno()); + errnoToString(e.getErrno())); continue; } @@ -252,10 +252,10 @@ void AsynchronousMetrics::openSensorsChips() { LOG_WARNING( &Poco::Logger::get("AsynchronousMetrics"), - "Hardware monitor '{}', sensor '{}' exists but could not be read, error {}.", + "Hardware monitor '{}', sensor '{}' exists but could not be read: {}.", hwmon_name, sensor_name, - e.getErrno()); + errnoToString(e.getErrno())); continue; } @@ -1083,7 +1083,17 @@ void AsynchronousMetrics::update(TimePoint update_time) BlockDeviceStatValues current_values{}; BlockDeviceStatValues & prev_values = block_device_stats[name]; - current_values.read(*device); + + try + { + current_values.read(*device); + } + catch (const ErrnoException & e) + { + LOG_DEBUG(log, "Cannot read statistics about the block device '{}': {}.", + name, errnoToString(e.getErrno())); + continue; + } BlockDeviceStatValues delta_values = current_values - prev_values; prev_values = current_values; @@ -1129,10 +1139,10 @@ void AsynchronousMetrics::update(TimePoint update_time) } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + LOG_DEBUG(log, getCurrentExceptionMessage(false)); /// Try to reopen block devices in case of error - /// (i.e. ENOENT means that some disk had been replaced, and it may apperas with a new name) + /// (i.e. ENOENT or ENODEV means that some disk had been replaced, and it may appear with a new name) try { openBlockDevices(); @@ -1271,7 +1281,9 @@ void AsynchronousMetrics::update(TimePoint update_time) } catch (const ErrnoException & e) { - LOG_DEBUG(&Poco::Logger::get("AsynchronousMetrics"), "Hardware monitor '{}', sensor '{}' exists but could not be read, error {}.", hwmon_name, sensor_name, e.getErrno()); + LOG_DEBUG(log, "Hardware monitor '{}', sensor '{}' exists but could not be read: {}.", + hwmon_name, sensor_name, errnoToString(e.getErrno())); + continue; } if (sensor_name.empty()) From c9d4598962309b9f26eacf08f74695189e19780e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 12 Nov 2022 19:09:37 +0100 Subject: [PATCH 58/82] Fix annoying messages in AsynchronousMetrics --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index a6cb6ee1f68..f11684e1f4a 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1139,7 +1139,7 @@ void AsynchronousMetrics::update(TimePoint update_time) } catch (...) { - LOG_DEBUG(log, getCurrentExceptionMessage(false)); + LOG_DEBUG(log, "Cannot read the statistics from block devices: {}", getCurrentExceptionMessage(false)); /// Try to reopen block devices in case of error /// (i.e. ENOENT or ENODEV means that some disk had been replaced, and it may appear with a new name) From 6599016aec6998be11decfb69c5c64802298028c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 12 Nov 2022 19:09:45 +0100 Subject: [PATCH 59/82] Fix annoying messages in AsynchronousMetrics --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index f11684e1f4a..574d67b3ef4 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1139,7 +1139,7 @@ void AsynchronousMetrics::update(TimePoint update_time) } catch (...) { - LOG_DEBUG(log, "Cannot read the statistics from block devices: {}", getCurrentExceptionMessage(false)); + LOG_DEBUG(log, "Cannot read statistics from block devices: {}", getCurrentExceptionMessage(false)); /// Try to reopen block devices in case of error /// (i.e. ENOENT or ENODEV means that some disk had been replaced, and it may appear with a new name) From 42de4686651774c97ef4bd701fcddde0ee5626f0 Mon Sep 17 00:00:00 2001 From: Joanna Hulboj Date: Sat, 12 Nov 2022 19:47:06 +0000 Subject: [PATCH 60/82] Fixed unexpected behaviour of Interval types with subquery and casting --- src/Functions/FunctionsConversion.h | 27 +++++++++++++++++++ ...80_interval_casting_and_subquery.reference | 25 +++++++++++++++++ .../02480_interval_casting_and_subquery.sql | 25 +++++++++++++++++ 3 files changed, 77 insertions(+) create mode 100644 tests/queries/0_stateless/02480_interval_casting_and_subquery.reference create mode 100644 tests/queries/0_stateless/02480_interval_casting_and_subquery.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index dd494d821bf..57c191e7e6c 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2828,6 +2828,31 @@ private: }; } +#define GENERATE_INTERVAL_CASE(INTEVAL_KIND) \ + case IntervalKind::INTEVAL_KIND: \ + return createFunctionAdaptor(FunctionConvert::create(), from_type); + + static WrapperType createIntervalWrapper(const DataTypePtr & from_type, IntervalKind kind) + { + switch (kind) + { + GENERATE_INTERVAL_CASE(Nanosecond) + GENERATE_INTERVAL_CASE(Microsecond) + GENERATE_INTERVAL_CASE(Millisecond) + GENERATE_INTERVAL_CASE(Second) + GENERATE_INTERVAL_CASE(Minute) + GENERATE_INTERVAL_CASE(Hour) + GENERATE_INTERVAL_CASE(Day) + GENERATE_INTERVAL_CASE(Week) + GENERATE_INTERVAL_CASE(Month) + GENERATE_INTERVAL_CASE(Quarter) + GENERATE_INTERVAL_CASE(Year) + } + throw Exception{ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion to unexpected IntervalKind: {}", kind.toString()}; + } + +#undef GENERATE_INTERVAL_CASE + template requires IsDataTypeDecimal WrapperType createDecimalWrapper(const DataTypePtr & from_type, const ToDataType * to_type, bool requested_result_is_nullable) const @@ -3853,6 +3878,8 @@ private: return createObjectWrapper(from_type, checkAndGetDataType(to_type.get())); case TypeIndex::AggregateFunction: return createAggregateFunctionWrapper(from_type, checkAndGetDataType(to_type.get())); + case TypeIndex::Interval: + return createIntervalWrapper(from_type, checkAndGetDataType(to_type.get())->getKind()); default: break; } diff --git a/tests/queries/0_stateless/02480_interval_casting_and_subquery.reference b/tests/queries/0_stateless/02480_interval_casting_and_subquery.reference new file mode 100644 index 00000000000..74df309720b --- /dev/null +++ b/tests/queries/0_stateless/02480_interval_casting_and_subquery.reference @@ -0,0 +1,25 @@ +5 2017-01-01 00:00:05 +5 2017-01-01 00:05:00 +5 2017-01-01 05:00:00 +5 2017-01-06 00:00:00 +5 2017-06-01 00:00:00 +5 2018-04-01 00:00:00 +5 2022-01-01 00:00:00 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 2017-01-01 00:00:05 +5 2017-01-01 00:05:00 +5 2017-01-01 05:00:00 +5 2017-01-06 00:00:00 +5 2017-06-01 00:00:00 +5 2018-04-01 00:00:00 +5 2022-01-01 00:00:00 diff --git a/tests/queries/0_stateless/02480_interval_casting_and_subquery.sql b/tests/queries/0_stateless/02480_interval_casting_and_subquery.sql new file mode 100644 index 00000000000..cb6eccb06c0 --- /dev/null +++ b/tests/queries/0_stateless/02480_interval_casting_and_subquery.sql @@ -0,0 +1,25 @@ +SELECT toIntervalSecond(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT toIntervalMinute(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT toIntervalHour(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT toIntervalDay(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT toIntervalMonth(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT toIntervalQuarter(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT toIntervalYear(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT CAST(5 AS IntervalNanosecond); +SELECT CAST(5 AS IntervalMicrosecond); +SELECT CAST(5 AS IntervalMillisecond); +SELECT CAST(5 AS IntervalSecond); +SELECT CAST(5 AS IntervalMinute); +SELECT CAST(5 AS IntervalHour); +SELECT CAST(5 AS IntervalDay); +SELECT CAST(5 AS IntervalWeek); +SELECT CAST(5 AS IntervalMonth); +SELECT CAST(5 AS IntervalQuarter); +SELECT CAST(5 AS IntervalYear); +SELECT (SELECT toIntervalSecond(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT (SELECT toIntervalMinute(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT (SELECT toIntervalHour(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT (SELECT toIntervalDay(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT (SELECT toIntervalMonth(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT (SELECT toIntervalQuarter(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT (SELECT toIntervalYear(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; From 1b7b87852089d997c82d400771ba6db254c879a7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 12 Nov 2022 21:04:01 +0000 Subject: [PATCH 61/82] don't randomize keeper fault injection setting for inserts --- tests/clickhouse-test | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c054e66898e..a07a39600d8 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -456,8 +456,8 @@ class SettingsRandomizer: "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), "optimize_distinct_in_order": lambda: random.randint(0, 1), "optimize_sorting_by_input_stream_properties": lambda: random.randint(0, 1), - "insert_keeper_max_retries": lambda: random.randint(20, 20), - "insert_keeper_fault_injection_probability": lambda: random.uniform(0, 0.01), + "insert_keeper_max_retries": lambda: 20, + "insert_keeper_fault_injection_probability": lambda: 0.01, } @staticmethod From 661c5cd0ae2b1fa65dd24b90ffaea2b3b3693e51 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sat, 12 Nov 2022 23:58:09 +0100 Subject: [PATCH 62/82] fixed type --- src/Functions/FunctionsConversion.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 57c191e7e6c..208da8a78fe 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2828,9 +2828,9 @@ private: }; } -#define GENERATE_INTERVAL_CASE(INTEVAL_KIND) \ - case IntervalKind::INTEVAL_KIND: \ - return createFunctionAdaptor(FunctionConvert::create(), from_type); +#define GENERATE_INTERVAL_CASE(INTERVAL_KIND) \ + case IntervalKind::INTERVAL_KIND: \ + return createFunctionAdaptor(FunctionConvert::create(), from_type); static WrapperType createIntervalWrapper(const DataTypePtr & from_type, IntervalKind kind) { From 5a0b7b294767afffcb65ca696d11ec35d173c9a2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 02:59:01 +0100 Subject: [PATCH 63/82] Add documentation for AsynchronousMetrics --- src/Interpreters/AsynchronousMetrics.cpp | 532 +++++++++++++++-------- src/Interpreters/AsynchronousMetrics.h | 16 +- src/Server/ProtocolServerAdapter.h | 2 + 3 files changed, 362 insertions(+), 188 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 574d67b3ef4..f861fbb8426 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -33,6 +32,7 @@ # include #endif + namespace DB { @@ -386,14 +386,15 @@ uint64_t updateJemallocEpoch() } template -static Value saveJemallocMetricImpl(AsynchronousMetricValues & values, +static Value saveJemallocMetricImpl( + AsynchronousMetricValues & values, const std::string & jemalloc_full_name, const std::string & clickhouse_full_name) { Value value{}; size_t size = sizeof(value); mallctl(jemalloc_full_name.c_str(), &value, &size, nullptr, 0); - values[clickhouse_full_name] = value; + values[clickhouse_full_name] = { value, "An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html" }; return value; } @@ -570,85 +571,93 @@ void AsynchronousMetrics::update(TimePoint update_time) previous_update_time = update_time; /// This is also a good indicator of system responsiveness. - new_values["Jitter"] = std::chrono::duration_cast(current_time - update_time).count() / 1e9; + new_values["Jitter"] = { std::chrono::duration_cast(current_time - update_time).count() / 1e9, + "The difference in time the thread for calculation of the asynchronous metrics was scheduled to wake up and the time it was in fact, woken up." + " A proxy-indicator of overall system latency and responsiveness." }; + if (auto mark_cache = getContext()->getMarkCache()) { - if (auto mark_cache = getContext()->getMarkCache()) - { - new_values["MarkCacheBytes"] = mark_cache->weight(); - new_values["MarkCacheFiles"] = mark_cache->count(); - } + new_values["MarkCacheBytes"] = { mark_cache->weight(), "Total size of mark cache in bytes" }; + new_values["MarkCacheFiles"] = { mark_cache->count(), "Total number of mark files cached in the mark cache" }; } + if (auto uncompressed_cache = getContext()->getUncompressedCache()) { - if (auto uncompressed_cache = getContext()->getUncompressedCache()) - { - new_values["UncompressedCacheBytes"] = uncompressed_cache->weight(); - new_values["UncompressedCacheCells"] = uncompressed_cache->count(); - } + new_values["UncompressedCacheBytes"] = { uncompressed_cache->weight(), + "Total size of uncompressed cache in bytes. Uncompressed cache does not usually improve the performance and should be mostly avoided." }; + new_values["UncompressedCacheCells"] = { uncompressed_cache->count(), + "Total number of entries in the uncompressed cache. Each entry represents a decompressed block of data. Uncompressed cache does not usually improve performance and should be mostly avoided." }; } + if (auto index_mark_cache = getContext()->getIndexMarkCache()) { - if (auto index_mark_cache = getContext()->getIndexMarkCache()) - { - new_values["IndexMarkCacheBytes"] = index_mark_cache->weight(); - new_values["IndexMarkCacheFiles"] = index_mark_cache->count(); - } + new_values["IndexMarkCacheBytes"] = { index_mark_cache->weight(), "Total size of mark cache for secondary indices in bytes." }; + new_values["IndexMarkCacheFiles"] = { index_mark_cache->count(), "Total number of mark files cached in the mark cache for secondary indices." }; } + if (auto index_uncompressed_cache = getContext()->getIndexUncompressedCache()) { - if (auto index_uncompressed_cache = getContext()->getIndexUncompressedCache()) - { - new_values["IndexUncompressedCacheBytes"] = index_uncompressed_cache->weight(); - new_values["IndexUncompressedCacheCells"] = index_uncompressed_cache->count(); - } + new_values["IndexUncompressedCacheBytes"] = { index_uncompressed_cache->weight(), + "Total size of uncompressed cache in bytes for secondary indices. Uncompressed cache does not usually improve the performance and should be mostly avoided." }; + new_values["IndexUncompressedCacheCells"] = { index_uncompressed_cache->count(), + "Total number of entries in the uncompressed cache for secondary indices. Each entry represents a decompressed block of data. Uncompressed cache does not usually improve performance and should be mostly avoided." }; } + if (auto mmap_cache = getContext()->getMMappedFileCache()) { - if (auto mmap_cache = getContext()->getMMappedFileCache()) - { - new_values["MMapCacheCells"] = mmap_cache->count(); - } + new_values["MMapCacheCells"] = { mmap_cache->count(), + "The number of files opened with `mmap` (mapped in memory)." + " This is used for queries with the setting `local_filesystem_read_method` set to `mmap`." + " The files opened with `mmap` are kept in the cache to avoid costly TLB flushes."}; } { auto caches = FileCacheFactory::instance().getAll(); + size_t total_bytes = 0; + size_t total_files = 0; + for (const auto & [_, cache_data] : caches) { - new_values["FilesystemCacheBytes"] = cache_data->cache->getUsedCacheSize(); - new_values["FilesystemCacheFiles"] = cache_data->cache->getFileSegmentsNum(); + total_bytes += cache_data->cache->getUsedCacheSize(); + total_files += cache_data->cache->getFileSegmentsNum(); } + + new_values["FilesystemCacheBytes"] = { total_bytes, + "Total bytes in the `cache` virtual filesystem. This cache is hold on disk." }; + new_values["FilesystemCacheFiles"] = { total_files, + "Total number of cached file segments in the `cache` virtual filesystem. This cache is hold on disk." }; } #if USE_ROCKSDB + if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache()) { - if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache()) - { - new_values["MergeTreeMetadataCacheSize"] = metadata_cache->getEstimateNumKeys(); - } + new_values["MergeTreeMetadataCacheSize"] = { metadata_cache->getEstimateNumKeys(), + "The size of the metadata cache for tables. This cache is experimental and not used in production." }; } #endif #if USE_EMBEDDED_COMPILER + if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) { - if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) - { - new_values["CompiledExpressionCacheBytes"] = compiled_expression_cache->weight(); - new_values["CompiledExpressionCacheCount"] = compiled_expression_cache->count(); - } + new_values["CompiledExpressionCacheBytes"] = { compiled_expression_cache->weight(), + "Total bytes used for the cache of JIT-compiled code." }; + new_values["CompiledExpressionCacheCount"] = { compiled_expression_cache->count(), + "Total entries in the cache of JIT-compiled code." }; } #endif + new_values["Uptime"] = { getContext()->getUptimeSeconds(), + "The server uptime in seconds. It includes the time spent for server initialization before accepting connections." }; - new_values["Uptime"] = getContext()->getUptimeSeconds(); - + if (const auto stats = getHashTablesCacheStatistics()) { - if (const auto stats = getHashTablesCacheStatistics()) - { - new_values["HashTableStatsCacheEntries"] = stats->entries; - new_values["HashTableStatsCacheHits"] = stats->hits; - new_values["HashTableStatsCacheMisses"] = stats->misses; - } + new_values["HashTableStatsCacheEntries"] = { stats->entries, + "The number of entries in the cache of hash table sizes." + " The cache for hash table sizes is used for predictive optimization of GROUP BY." }; + new_values["HashTableStatsCacheHits"] = { stats->hits, + "The number of times the prediction of a hash table size was correct." }; + new_values["HashTableStatsCacheMisses"] = { stats->misses, + "The number of times the prediction of a hash table size was incorrect." }; } #if defined(OS_LINUX) || defined(OS_FREEBSD) @@ -660,7 +669,7 @@ void AsynchronousMetrics::update(TimePoint update_time) // the following calls will return stale values. It increments and returns // the current epoch number, which might be useful to log as a sanity check. auto epoch = updateJemallocEpoch(); - new_values["jemalloc.epoch"] = epoch; + new_values["jemalloc.epoch"] = { epoch, "An internal incremental update number of the statistics of JeMalloc (a low-level memory allocator), used in all other `jemalloc` metrics." }; // Collect the statistics themselves. saveJemallocMetric(new_values, "allocated"); @@ -685,13 +694,24 @@ void AsynchronousMetrics::update(TimePoint update_time) { MemoryStatisticsOS::Data & data = memory_statistics_data; - new_values["MemoryVirtual"] = data.virt; - new_values["MemoryResident"] = data.resident; + new_values["MemoryVirtual"] = { data.virt, + "The size of the virtual address space allocated by the server process, in bytes." + " The size of the virtual address space is usually much greater than the physical memory consumption, and should not be used as an estimate for the memory consumption." + " The large values of this metric are totally normal, and makes only technical sense."}; + new_values["MemoryResident"] = { data.resident, + "The amount of physical memory used by the server process, in bytes." }; #if !defined(OS_FREEBSD) - new_values["MemoryShared"] = data.shared; + new_values["MemoryShared"] = { data.shared, + "The amount of memory used by the server process, that is also shared by another processes, in bytes." + " ClickHouse does not use shared memory, and the only case for this metric to be higher than zero is the usage of the system's C library (`libc`)." + " This metric does not make a lot of sense to watch, and it exists only for completeness reasons."}; #endif - new_values["MemoryCode"] = data.code; - new_values["MemoryDataAndStack"] = data.data_and_stack; + new_values["MemoryCode"] = { data.code, + "The amount of virtual memory mapped for the pages of machine code of the server process, in bytes." }; + new_values["MemoryDataAndStack"] = { data.data_and_stack, + "The amount of virtual memory mapped for the use of stack and for the allocated memory, in bytes." + " It is unspecified whether it includes the per-thread stacks and most of the allocated memory, that is allocated with the 'mmap' system call." + " This metric exists only for completeness reasons. I recommend to use the `MemoryResident` metric for monitoring."}; /// We must update the value of total_memory_tracker periodically. /// Otherwise it might be calculated incorrectly - it can include a "drift" of memory amount. @@ -754,11 +774,22 @@ void AsynchronousMetrics::update(TimePoint update_time) assertChar('/', *loadavg); readText(threads_total, *loadavg); - new_values["LoadAverage1"] = loadavg1; - new_values["LoadAverage5"] = loadavg5; - new_values["LoadAverage15"] = loadavg15; - new_values["OSThreadsRunnable"] = threads_runnable; - new_values["OSThreadsTotal"] = threads_total; +#define LOAD_AVERAGE_DOCUMENTATION \ + " The load represents the number of threads across all the processes (the scheduling entities of the OS kernel)," \ + " that are currently running by CPU or waiting for IO, or ready to run but not being scheduled at this point of time." \ + " This number includes all the processes, not only clickhouse-server. The number can be greater than the number of CPU cores," \ + " if the system is overloaded, and many processes are ready to run but waiting for CPU or IO." + + new_values["LoadAverage1"] = { loadavg1, + "The whole system load, averaged with exponential smoothing over 1 minute." LOAD_AVERAGE_DOCUMENTATION }; + new_values["LoadAverage5"] = { loadavg5, + "The whole system load, averaged with exponential smoothing over 5 minutes." LOAD_AVERAGE_DOCUMENTATION }; + new_values["LoadAverage15"] = { loadavg15, + "The whole system load, averaged with exponential smoothing over 15 minutes." LOAD_AVERAGE_DOCUMENTATION }; + new_values["OSThreadsRunnable"] = { threads_runnable, + "The total number of 'runnable' threads, as the OS kernel scheduler seeing it." }; + new_values["OSThreadsTotal"] = { threads_total, + "The total number of threads, as the OS kernel scheduler seeing it." }; } catch (...) { @@ -775,7 +806,7 @@ void AsynchronousMetrics::update(TimePoint update_time) Float64 uptime_seconds = 0; readText(uptime_seconds, *uptime); - new_values["OSUptime"] = uptime_seconds; + new_values["OSUptime"] = { uptime_seconds, "The uptime of the host server (the machine where ClickHouse is running), in seconds." }; } catch (...) { @@ -838,16 +869,43 @@ void AsynchronousMetrics::update(TimePoint update_time) else delta_values_all_cpus = delta_values; - new_values["OSUserTime" + cpu_suffix] = delta_values.user * multiplier; - new_values["OSNiceTime" + cpu_suffix] = delta_values.nice * multiplier; - new_values["OSSystemTime" + cpu_suffix] = delta_values.system * multiplier; - new_values["OSIdleTime" + cpu_suffix] = delta_values.idle * multiplier; - new_values["OSIOWaitTime" + cpu_suffix] = delta_values.iowait * multiplier; - new_values["OSIrqTime" + cpu_suffix] = delta_values.irq * multiplier; - new_values["OSSoftIrqTime" + cpu_suffix] = delta_values.softirq * multiplier; - new_values["OSStealTime" + cpu_suffix] = delta_values.steal * multiplier; - new_values["OSGuestTime" + cpu_suffix] = delta_values.guest * multiplier; - new_values["OSGuestNiceTime" + cpu_suffix] = delta_values.guest_nice * multiplier; + new_values["OSUserTime" + cpu_suffix] = { delta_values.user * multiplier, + "The ratio of time the CPU core was running userspace code. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " This includes also the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline stalls, branch mispredictions, running another SMT core)." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSNiceTime" + cpu_suffix] = { delta_values.nice * multiplier, + "The ratio of time the CPU core was running userspace code with higher priority. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSSystemTime" + cpu_suffix] = { delta_values.system * multiplier, + "The ratio of time the CPU core was running OS kernel (system) code. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSIdleTime" + cpu_suffix] = { delta_values.idle * multiplier, + "The ratio of time the CPU core was idle (not even ready to run a process waiting for IO) from the OS kernel standpoint. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " This does not include the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline stalls, branch mispredictions, running another SMT core)." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSIOWaitTime" + cpu_suffix] = { delta_values.iowait * multiplier, + "The ratio of time the CPU core was not running the code but when the OS kernel did not run any other process on this CPU as the processes were waiting for IO. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSIrqTime" + cpu_suffix] = { delta_values.irq * multiplier, + "The ratio of time spent for running hardware interrupt requests on the CPU. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " A high number of this metric may indicate hardware misconfiguration or a very high network load." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSSoftIrqTime" + cpu_suffix] = { delta_values.softirq * multiplier, + "The ratio of time spent for running software interrupt requests on the CPU. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " A high number of this metric may indicate inefficient software running on the system." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSStealTime" + cpu_suffix] = { delta_values.steal * multiplier, + "The ratio of time spent in other operating systems by the CPU when running in a virtualized environment. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " Not every virtualized environments present this metric, and most of them don't." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSGuestTime" + cpu_suffix] = { delta_values.guest * multiplier, + "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel (See `man procfs`). This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " This metric is irrelevant for ClickHouse, but still exists for completeness." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSGuestNiceTime" + cpu_suffix] = { delta_values.guest_nice * multiplier, + "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel, when a guest was set to a higher priority (See `man procfs`). This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " This metric is irrelevant for ClickHouse, but still exists for completeness." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; } prev_values = current_values; @@ -872,14 +930,18 @@ void AsynchronousMetrics::update(TimePoint update_time) UInt64 processes_running = 0; readText(processes_running, *proc_stat); skipToNextLineOrEOF(*proc_stat); - new_values["OSProcessesRunning"] = processes_running; + new_values["OSProcessesRunning"] = { processes_running, + "The number of runnable (running or ready to run) threads by the operating system." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "procs_blocked") { UInt64 processes_blocked = 0; readText(processes_blocked, *proc_stat); skipToNextLineOrEOF(*proc_stat); - new_values["OSProcessesBlocked"] = processes_blocked; + new_values["OSProcessesBlocked"] = { processes_blocked, + "Number of threads blocked waiting for I/O to complete (`man procfs`)." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else skipToNextLineOrEOF(*proc_stat); @@ -889,25 +951,45 @@ void AsynchronousMetrics::update(TimePoint update_time) { ProcStatValuesOther delta_values = current_other_values - proc_stat_values_other; - new_values["OSInterrupts"] = delta_values.interrupts; - new_values["OSContextSwitches"] = delta_values.context_switches; - new_values["OSProcessesCreated"] = delta_values.processes_created; + new_values["OSInterrupts"] = { delta_values.interrupts, "The number of interrupts on the host machine. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["OSContextSwitches"] = { delta_values.context_switches, "The number of context switches that the system underwent on the host machine. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["OSProcessesCreated"] = { delta_values.processes_created, "The number of processes created. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; /// Also write values normalized to 0..1 by diving to the number of CPUs. /// These values are good to be averaged across the cluster of non-uniform servers. if (num_cpus) { - new_values["OSUserTimeNormalized"] = delta_values_all_cpus.user * multiplier / num_cpus; - new_values["OSNiceTimeNormalized"] = delta_values_all_cpus.nice * multiplier / num_cpus; - new_values["OSSystemTimeNormalized"] = delta_values_all_cpus.system * multiplier / num_cpus; - new_values["OSIdleTimeNormalized"] = delta_values_all_cpus.idle * multiplier / num_cpus; - new_values["OSIOWaitTimeNormalized"] = delta_values_all_cpus.iowait * multiplier / num_cpus; - new_values["OSIrqTimeNormalized"] = delta_values_all_cpus.irq * multiplier / num_cpus; - new_values["OSSoftIrqTimeNormalized"] = delta_values_all_cpus.softirq * multiplier / num_cpus; - new_values["OSStealTimeNormalized"] = delta_values_all_cpus.steal * multiplier / num_cpus; - new_values["OSGuestTimeNormalized"] = delta_values_all_cpus.guest * multiplier / num_cpus; - new_values["OSGuestNiceTimeNormalized"] = delta_values_all_cpus.guest_nice * multiplier / num_cpus; + new_values["OSUserTimeNormalized"] = { delta_values_all_cpus.user * multiplier / num_cpus, + "The value is similar to `OSUserTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSNiceTimeNormalized"] = { delta_values_all_cpus.nice * multiplier / num_cpus, + "The value is similar to `OSNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSSystemTimeNormalized"] = { delta_values_all_cpus.system * multiplier / num_cpus, + "The value is similar to `OSSystemTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSIdleTimeNormalized"] = { delta_values_all_cpus.idle * multiplier / num_cpus, + "The value is similar to `OSIdleTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSIOWaitTimeNormalized"] = { delta_values_all_cpus.iowait * multiplier / num_cpus, + "The value is similar to `OSIOWaitTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSIrqTimeNormalized"] = { delta_values_all_cpus.irq * multiplier / num_cpus, + "The value is similar to `OSIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSSoftIrqTimeNormalized"] = { delta_values_all_cpus.softirq * multiplier / num_cpus, + "The value is similar to `OSSoftIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSStealTimeNormalized"] = { delta_values_all_cpus.steal * multiplier / num_cpus, + "The value is similar to `OSStealTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSGuestTimeNormalized"] = { delta_values_all_cpus.guest * multiplier / num_cpus, + "The value is similar to `OSGuestTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSGuestNiceTimeNormalized"] = { delta_values_all_cpus.guest_nice * multiplier / num_cpus, + "The value is similar to `OSGuestNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; } } @@ -962,39 +1044,47 @@ void AsynchronousMetrics::update(TimePoint update_time) if (name == "MemTotal:") { - new_values["OSMemoryTotal"] = bytes; + new_values["OSMemoryTotal"] = { bytes, "The total amount of memory on the host system, in bytes." }; } else if (name == "MemFree:") { - /// We cannot simply name this metric "Free", because it confuses users. - /// See https://www.linuxatemyram.com/ - /// For convenience we also provide OSMemoryFreePlusCached, that should be somewhat similar to OSMemoryAvailable. - free_plus_cached_bytes += bytes; - new_values["OSMemoryFreeWithoutCached"] = bytes; + new_values["OSMemoryFreeWithoutCached"] = { bytes, + "The amount of free memory on the host system, in bytes." + " This does not include the memory used by the OS page cache memory, in bytes." + " The page cache memory is also available for usage by programs, so the value of this metric can be confusing." + " See the `OSMemoryAvailable` metric instead." + " For convenience we also provide OSMemoryFreePlusCached, that should be somewhat similar to OSMemoryAvailable." + " See also https://www.linuxatemyram.com/." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "MemAvailable:") { - new_values["OSMemoryAvailable"] = bytes; + new_values["OSMemoryAvailable"] = { bytes, "The amount of memory available to be used by programs, in bytes. This is very similar to the `OSMemoryFreePlusCached` metric." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "Buffers:") { - new_values["OSMemoryBuffers"] = bytes; + new_values["OSMemoryBuffers"] = { bytes, "The amount of memory used by OS kernel buffers, in bytes. This should be typically small, and large values may indicate a misconfiguration of the OS." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "Cached:") { free_plus_cached_bytes += bytes; - new_values["OSMemoryCached"] = bytes; + new_values["OSMemoryCached"] = { bytes, "The amount of memory used by the OS page cache, in bytes. Typically, almost all available memory is used by the OS page cache - high values of this metric are normal and expected." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "SwapCached:") { - new_values["OSMemorySwapCached"] = bytes; + new_values["OSMemorySwapCached"] = { bytes, "The amount of memory in swap that was also loaded in RAM. Swap should be disabled on production systems. If the value of this metric is large, it indicates a misconfiguration." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } skipToNextLineOrEOF(*meminfo); } - new_values["OSMemoryFreePlusCached"] = free_plus_cached_bytes; + new_values["OSMemoryFreePlusCached"] = { free_plus_cached_bytes, "The amount of free memory or OS page cache memory on the host system, in bytes. This memory is available to be used by programs. The value should be very similar to `OSMemoryAvailable`." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } catch (...) { @@ -1043,7 +1133,7 @@ void AsynchronousMetrics::update(TimePoint update_time) if (auto colon = s.find_first_of(':')) { auto mhz = std::stod(s.substr(colon + 2)); - new_values[fmt::format("CPUFrequencyMHz_{}", core_id)] = mhz; + new_values[fmt::format("CPUFrequencyMHz_{}", core_id)] = { mhz, "The current frequency of the CPU, in MHz. Most of the modern CPUs adjust the frequency dynamically for power saving and Turbo Boosting." }; } } } @@ -1062,7 +1152,8 @@ void AsynchronousMetrics::update(TimePoint update_time) uint64_t open_files = 0; readText(open_files, *file_nr); - new_values["OSOpenFiles"] = open_files; + new_values["OSOpenFiles"] = { open_files, "The total number of opened files on the host machine." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } catch (...) { @@ -1107,33 +1198,80 @@ void AsynchronousMetrics::update(TimePoint update_time) /// Always in milliseconds according to the docs. static constexpr double time_multiplier = 1e-6; - new_values["BlockReadOps_" + name] = delta_values.read_ios; - new_values["BlockWriteOps_" + name] = delta_values.write_ios; - new_values["BlockDiscardOps_" + name] = delta_values.discard_ops; +#define BLOCK_DEVICE_EXPLANATION \ + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." \ + " See https://www.kernel.org/doc/Documentation/block/stat.txt" - new_values["BlockReadMerges_" + name] = delta_values.read_merges; - new_values["BlockWriteMerges_" + name] = delta_values.write_merges; - new_values["BlockDiscardMerges_" + name] = delta_values.discard_merges; + new_values["BlockReadOps_" + name] = { delta_values.read_ios, + "Number of read operations requested from the block device." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockWriteOps_" + name] = { delta_values.write_ios, + "Number of write operations requested from the block device." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockDiscardOps_" + name] = { delta_values.discard_ops, + "Number of discard operations requested from the block device. These operations are relevant for SSD." + " They are not used by ClickHouse, but can be used by other processes on the system." + BLOCK_DEVICE_EXPLANATION }; - new_values["BlockReadBytes_" + name] = delta_values.read_sectors * sector_size; - new_values["BlockWriteBytes_" + name] = delta_values.write_sectors * sector_size; - new_values["BlockDiscardBytes_" + name] = delta_values.discard_sectors * sector_size; + new_values["BlockReadMerges_" + name] = { delta_values.read_merges, + "Number of read operations requested from the block device and merged together by the OS IO scheduler." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockWriteMerges_" + name] = { delta_values.write_merges, + "Number of write operations requested from the block device and merged together by the OS IO scheduler." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockDiscardMerges_" + name] = { delta_values.discard_merges, + "Number of discard operations requested from the block device and merged together by the OS IO scheduler." + " These operations are relevant for SSD. They are not used by ClickHouse, but can be used by other processes on the system." + BLOCK_DEVICE_EXPLANATION }; - new_values["BlockReadTime_" + name] = delta_values.read_ticks * time_multiplier; - new_values["BlockWriteTime_" + name] = delta_values.write_ticks * time_multiplier; - new_values["BlockDiscardTime_" + name] = delta_values.discard_ticks * time_multiplier; + new_values["BlockReadBytes_" + name] = { delta_values.read_sectors * sector_size, + "Number of bytes read from the block device." + " It can be lower than the number of bytes read from the filesystem due to the usage of the OS page cache, that saves IO." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockWriteBytes_" + name] = { delta_values.write_sectors * sector_size, + "Number of bytes written to the block device." + " It can be lower than the number of bytes written to the filesystem due to the usage of the OS page cache, that saves IO." + " A write to the block device may happen later than the corresponding write to the filesystem due to write-through caching." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockDiscardBytes_" + name] = { delta_values.discard_sectors * sector_size, + "Number of discarded bytes on the block device." + " These operations are relevant for SSD. They are not used by ClickHouse, but can be used by other processes on the system." + BLOCK_DEVICE_EXPLANATION }; - new_values["BlockInFlightOps_" + name] = delta_values.in_flight_ios; + new_values["BlockReadTime_" + name] = { delta_values.read_ticks * time_multiplier, + "Time in seconds spend in read operations requested from the block device, summed across all the operations." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockWriteTime_" + name] = { delta_values.write_ticks * time_multiplier, + "Time in seconds spend in write operations requested from the block device, summed across all the operations." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockDiscardTime_" + name] = { delta_values.discard_ticks * time_multiplier, + "Time in seconds spend in discard operations requested from the block device, summed across all the operations." + " These operations are relevant for SSD. They are not used by ClickHouse, but can be used by other processes on the system." + BLOCK_DEVICE_EXPLANATION }; - new_values["BlockActiveTime_" + name] = delta_values.io_ticks * time_multiplier; - new_values["BlockQueueTime_" + name] = delta_values.time_in_queue * time_multiplier; + new_values["BlockInFlightOps_" + name] = { delta_values.in_flight_ios, + "This value counts the number of I/O requests that have been issued to" + " the device driver but have not yet completed. It does not include IO" + " requests that are in the queue but not yet issued to the device driver." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockActiveTime_" + name] = { delta_values.io_ticks * time_multiplier, + "Time in seconds the block device had the IO requests queued." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockQueueTime_" + name] = { delta_values.time_in_queue * time_multiplier, + "This value counts the number of milliseconds that IO requests have waited" + " on this block device. If there are multiple IO requests waiting, this" + " value will increase as the product of the number of milliseconds times the" + " number of requests waiting." + BLOCK_DEVICE_EXPLANATION }; if (delta_values.in_flight_ios) { /// TODO Check if these values are meaningful. - new_values["BlockActiveTimePerOp_" + name] = delta_values.io_ticks * time_multiplier / delta_values.in_flight_ios; - new_values["BlockQueueTimePerOp_" + name] = delta_values.time_in_queue * time_multiplier / delta_values.in_flight_ios; + new_values["BlockActiveTimePerOp_" + name] = { delta_values.io_ticks * time_multiplier / delta_values.in_flight_ios, + "Similar to the `BlockActiveTime` metrics, but the value is divided to the number of IO operations to count the per-operation time." }; + new_values["BlockQueueTimePerOp_" + name] = { delta_values.time_in_queue * time_multiplier / delta_values.in_flight_ios, + "Similar to the `BlockQueueTime` metrics, but the value is divided to the number of IO operations to count the per-operation time." }; } } } @@ -1221,15 +1359,31 @@ void AsynchronousMetrics::update(TimePoint update_time) if (!first_run) { - new_values["NetworkReceiveBytes_" + interface_name] = delta_values.recv_bytes; - new_values["NetworkReceivePackets_" + interface_name] = delta_values.recv_packets; - new_values["NetworkReceiveErrors_" + interface_name] = delta_values.recv_errors; - new_values["NetworkReceiveDrop_" + interface_name] = delta_values.recv_drop; + new_values["NetworkReceiveBytes_" + interface_name] = { delta_values.recv_bytes, + " Number of bytes received via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["NetworkReceivePackets_" + interface_name] = { delta_values.recv_packets, + " Number of network packets received via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["NetworkReceiveErrors_" + interface_name] = { delta_values.recv_errors, + " Number of times error happened receiving via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["NetworkReceiveDrop_" + interface_name] = { delta_values.recv_drop, + " Number of bytes a packet was dropped while received via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; - new_values["NetworkSendBytes_" + interface_name] = delta_values.send_bytes; - new_values["NetworkSendPackets_" + interface_name] = delta_values.send_packets; - new_values["NetworkSendErrors_" + interface_name] = delta_values.send_errors; - new_values["NetworkSendDrop_" + interface_name] = delta_values.send_drop; + new_values["NetworkSendBytes_" + interface_name] = { delta_values.send_bytes, + " Number of bytes sent via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["NetworkSendPackets_" + interface_name] = { delta_values.send_packets, + " Number of network packets sent via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["NetworkSendErrors_" + interface_name] = { delta_values.send_errors, + " Number of times error (e.g. TCP retransmit) happened while sending via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["NetworkSendDrop_" + interface_name] = { delta_values.send_drop, + " Number of times a packed was dropped while sending via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } } } @@ -1248,7 +1402,8 @@ void AsynchronousMetrics::update(TimePoint update_time) in.rewind(); Int64 temperature = 0; readText(temperature, in); - new_values[fmt::format("Temperature{}", i)] = temperature * 0.001; + new_values[fmt::format("Temperature{}", i)] = { temperature * 0.001, + "The temperature of the corresponding device in ℃. A sensor can return an unrealistic value." }; } } catch (...) @@ -1287,9 +1442,11 @@ void AsynchronousMetrics::update(TimePoint update_time) } if (sensor_name.empty()) - new_values[fmt::format("Temperature_{}", hwmon_name)] = temperature * 0.001; + new_values[fmt::format("Temperature_{}", hwmon_name)] = { temperature * 0.001, + "The temperature reported by the corresponding hardware monitor in ℃. A sensor can return an unrealistic value." }; else - new_values[fmt::format("Temperature_{}_{}", hwmon_name, sensor_name)] = temperature * 0.001; + new_values[fmt::format("Temperature_{}_{}", hwmon_name, sensor_name)] = { temperature * 0.001, + "The temperature reported by the corresponding hardware monitor and the corresponding sensor in ℃. A sensor can return an unrealistic value." }; } } } @@ -1325,7 +1482,9 @@ void AsynchronousMetrics::update(TimePoint update_time) in.rewind(); uint64_t errors = 0; readText(errors, in); - new_values[fmt::format("EDAC{}_Correctable", i)] = errors; + new_values[fmt::format("EDAC{}_Correctable", i)] = { errors, "The number of correctable ECC memory errors", + " A high number of this value indicates bad RAM which has to be immediately replaced," + " because in presence of a high number of corrected errors, a number of silent errors may happen as well, leading to data corruption." }; } if (edac[i].second) @@ -1334,7 +1493,9 @@ void AsynchronousMetrics::update(TimePoint update_time) in.rewind(); uint64_t errors = 0; readText(errors, in); - new_values[fmt::format("EDAC{}_Uncorrectable", i)] = errors; + new_values[fmt::format("EDAC{}_Uncorrectable", i)] = { errors, "The number of uncorrectable ECC memory errors", + " A non-zero number of this value indicates bad RAM which has to be immediately replaced," + " because it indicates potential data corruption." }; } } } @@ -1475,44 +1636,46 @@ void AsynchronousMetrics::update(TimePoint update_time) } } - new_values["ReplicasMaxQueueSize"] = max_queue_size; - new_values["ReplicasMaxInsertsInQueue"] = max_inserts_in_queue; - new_values["ReplicasMaxMergesInQueue"] = max_merges_in_queue; + new_values["ReplicasMaxQueueSize"] = { max_queue_size, "Maximum queue size (in the number of operations like get, merge) across Replicated tables." }; + new_values["ReplicasMaxInsertsInQueue"] = { max_inserts_in_queue, "Maximum number of INSERT operations in the queue (still to be replicated) across Replicated tables." }; + new_values["ReplicasMaxMergesInQueue"] = { max_merges_in_queue, "Maximum number of merge operations in the queue (still to be applied) across Replicated tables." }; - new_values["ReplicasSumQueueSize"] = sum_queue_size; - new_values["ReplicasSumInsertsInQueue"] = sum_inserts_in_queue; - new_values["ReplicasSumMergesInQueue"] = sum_merges_in_queue; + new_values["ReplicasSumQueueSize"] = { sum_queue_size, "Sum queue size (in the number of operations like get, merge) across Replicated tables." }; + new_values["ReplicasSumInsertsInQueue"] = { sum_inserts_in_queue, "Sum of INSERT operations in the queue (still to be replicated) across Replicated tables." }; + new_values["ReplicasSumMergesInQueue"] = { sum_merges_in_queue, "Sum of merge operations in the queue (still to be applied) across Replicated tables." }; - new_values["ReplicasMaxAbsoluteDelay"] = max_absolute_delay; - new_values["ReplicasMaxRelativeDelay"] = max_relative_delay; + new_values["ReplicasMaxAbsoluteDelay"] = { max_absolute_delay, "Maximum difference in seconds between the most fresh replicated part and the most fresh data part still to be replicated, across Replicated tables. A very high value indicates a replica with no data." }; + new_values["ReplicasMaxRelativeDelay"] = { max_relative_delay, "Maximum difference between the replica delay and the delay of the most up-to-date replica of the same table, across Replicated tables." }; - new_values["MaxPartCountForPartition"] = max_part_count_for_partition; + new_values["MaxPartCountForPartition"] = { max_part_count_for_partition, "Maximum number of parts per partition across all partitions of all tables of MergeTree family. Values larger than 300 indicates misconfiguration, overload, or massive data loading." }; - new_values["NumberOfDatabases"] = number_of_databases; - new_values["NumberOfTables"] = total_number_of_tables; + new_values["NumberOfDatabases"] = { number_of_databases, "Total number of databases on the server." }; + new_values["NumberOfTables"] = { total_number_of_tables, "Total number of tables summed across the databases on the server, excluding the databases that cannot contain MergeTree tables." + "The excluded database engines are those who generate the set of tables on the fly, like `Lazy`, `MySQL`, `PostgreSQL`, `SQlite`."}; - new_values["TotalBytesOfMergeTreeTables"] = total_number_of_bytes; - new_values["TotalRowsOfMergeTreeTables"] = total_number_of_rows; - new_values["TotalPartsOfMergeTreeTables"] = total_number_of_parts; + new_values["TotalBytesOfMergeTreeTables"] = { total_number_of_bytes, "Total amount of bytes (compressed, including data and indices) stored in all tables of MergeTree family." }; + new_values["TotalRowsOfMergeTreeTables"] = { total_number_of_rows, "Total amount of rows (records) stored in all tables of MergeTree family." }; + new_values["TotalPartsOfMergeTreeTables"] = { total_number_of_parts, "Total amount of data parts in all tables of MergeTree family." + " Numbers larger than 10 000 will negatively affect the server startup time and it may indicate unreasonable choice of the partition key." }; - auto get_metric_name = [](const String & name) -> const char * + auto get_metric_name_doc = [](const String & name) -> std::pair { - static std::map metric_map = + static std::map> metric_map = { - {"tcp_port", "TCPThreads"}, - {"tcp_port_secure", "TCPSecureThreads"}, - {"http_port", "HTTPThreads"}, - {"https_port", "HTTPSecureThreads"}, - {"interserver_http_port", "InterserverThreads"}, - {"interserver_https_port", "InterserverSecureThreads"}, - {"mysql_port", "MySQLThreads"}, - {"postgresql_port", "PostgreSQLThreads"}, - {"grpc_port", "GRPCThreads"}, - {"prometheus.port", "PrometheusThreads"} + {"tcp_port", {"TCPThreads", "Number of threads in the server of the TCP protocol (without TLS)."}}, + {"tcp_port_secure", {"TCPSecureThreads", "Number of threads in the server of the TCP protocol (with TLS)."}}, + {"http_port", {"HTTPThreads", "Number of threads in the server of the HTTP interface (without TLS)."}}, + {"https_port", {"HTTPSecureThreads", "Number of threads in the server of the HTTPS interface."}}, + {"interserver_http_port", {"InterserverThreads", "Number of threads in the server of the replicas communication protocol (without TLS)."}}, + {"interserver_https_port", {"InterserverSecureThreads", "Number of threads in the server of the replicas communication protocol (with TLS)."}}, + {"mysql_port", {"MySQLThreads", "Number of threads in the server of the MySQL compatibility protocol."}}, + {"postgresql_port", {"PostgreSQLThreads", "Number of threads in the server of the PostgreSQL compatibility protocol."}}, + {"grpc_port", {"GRPCThreads", "Number of threads in the server of the GRPC protocol."}}, + {"prometheus.port", {"PrometheusThreads", "Number of threads in the server of the Prometheus endpoint. Note: prometheus endpoints can be also used via the usual HTTP/HTTPs ports."}} }; auto it = metric_map.find(name); if (it == metric_map.end()) - return nullptr; + return { nullptr, nullptr }; else return it->second; }; @@ -1520,8 +1683,8 @@ void AsynchronousMetrics::update(TimePoint update_time) const auto server_metrics = protocol_server_metrics_func(); for (const auto & server_metric : server_metrics) { - if (const auto * name = get_metric_name(server_metric.port_name)) - new_values[name] = server_metric.current_threads; + if (auto name_doc = get_metric_name_doc(server_metric.port_name); name_doc.first != nullptr) + new_values[name_doc.first] = { server_metric.current_threads, name_doc.second }; } } #if USE_NURAFT @@ -1534,14 +1697,14 @@ void AsynchronousMetrics::update(TimePoint update_time) size_t is_observer = 0; size_t is_standalone = 0; size_t znode_count = 0; - size_t watch_count =0; + size_t watch_count = 0; size_t ephemerals_count = 0; - size_t approximate_data_size =0; + size_t approximate_data_size = 0; size_t key_arena_size = 0; - size_t latest_snapshot_size =0; - size_t open_file_descriptor_count =0; - size_t max_file_descriptor_count =0; - size_t followers =0; + size_t latest_snapshot_size = 0; + size_t open_file_descriptor_count = 0; + size_t max_file_descriptor_count = 0; + size_t followers = 0; size_t synced_followers = 0; size_t zxid = 0; size_t session_with_watches = 0; @@ -1582,29 +1745,29 @@ void AsynchronousMetrics::update(TimePoint update_time) } } - new_values["KeeperIsLeader"] = is_leader; - new_values["KeeperIsFollower"] = is_follower; - new_values["KeeperIsObserver"] = is_observer; - new_values["KeeperIsStandalone"] = is_standalone; + new_values["KeeperIsLeader"] = { is_leader, "1 if ClickHouse Keeper is a leader, 0 otherwise." }; + new_values["KeeperIsFollower"] = { is_follower, "1 if ClickHouse Keeper is a follower, 0 otherwise." }; + new_values["KeeperIsObserver"] = { is_observer, "1 if ClickHouse Keeper is an observer, 0 otherwise." }; + new_values["KeeperIsStandalone"] = { is_standalone, "1 if ClickHouse Keeper is in a standalone mode, 0 otherwise." }; - new_values["KeeperZnodeCount"] = znode_count; - new_values["KeeperWatchCount"] = watch_count; - new_values["KeeperEphemeralsCount"] = ephemerals_count; + new_values["KeeperZnodeCount"] = { znode_count, "The number of nodes (data entries) in ClickHouse Keeper." }; + new_values["KeeperWatchCount"] = { watch_count, "The number of watches in ClickHouse Keeper." }; + new_values["KeeperEphemeralsCount"] = { ephemerals_count, "The number of ephemeral nodes in ClickHouse Keeper." }; - new_values["KeeperApproximateDataSize"] = approximate_data_size; - new_values["KeeperKeyArenaSize"] = key_arena_size; - new_values["KeeperLatestSnapshotSize"] = latest_snapshot_size; + new_values["KeeperApproximateDataSize"] = { approximate_data_size, "The approximate data size of ClickHouse Keeper, in bytes." }; + new_values["KeeperKeyArenaSize"] = { key_arena_size, "The size in bytes of the memory arena for keys in ClickHouse Keeper." }; + new_values["KeeperLatestSnapshotSize"] = { latest_snapshot_size, "The uncompressed size in bytes of the latest snapshot created by ClickHouse Keeper." }; - new_values["KeeperOpenFileDescriptorCount"] = open_file_descriptor_count; - new_values["KeeperMaxFileDescriptorCount"] = max_file_descriptor_count; + new_values["KeeperOpenFileDescriptorCount"] = { open_file_descriptor_count, "The number of open file descriptors in ClickHouse Keeper." }; + new_values["KeeperMaxFileDescriptorCount"] = { max_file_descriptor_count, "The maximum number of open file descriptors in ClickHouse Keeper." }; - new_values["KeeperFollowers"] = followers; - new_values["KeeperSyncedFollowers"] = synced_followers; - new_values["KeeperZxid"] = zxid; - new_values["KeeperSessionWithWatches"] = session_with_watches; - new_values["KeeperPathsWatched"] = paths_watched; - new_values["KeeperSnapshotDirSize"] = snapshot_dir_size; - new_values["KeeperLogDirSize"] = log_dir_size; + new_values["KeeperFollowers"] = { followers, "The number of followers of ClickHouse Keeper." }; + new_values["KeeperSyncedFollowers"] = { synced_followers, "The number of followers of ClickHouse Keeper who are also in-sync." }; + new_values["KeeperZxid"] = { zxid, "The current transaction id number (zxid) in ClickHouse Keeper." }; + new_values["KeeperSessionWithWatches"] = { session_with_watches, "The number of client sessions of ClickHouse Keeper having watches." }; + new_values["KeeperPathsWatched"] = { paths_watched, "The number of different paths watched by the clients of ClickHouse Keeper." }; + new_values["KeeperSnapshotDirSize"] = { snapshot_dir_size, "The size of the snapshots directory of ClickHouse Keeper, in bytes." }; + new_values["KeeperLogDirSize"] = { log_dir_size, "The size of the logs directory of ClickHouse Keeper, in bytes." }; } } #endif @@ -1613,7 +1776,7 @@ void AsynchronousMetrics::update(TimePoint update_time) /// Add more metrics as you wish. - new_values["AsynchronousMetricsCalculationTimeSpent"] = watch.elapsedSeconds(); + new_values["AsynchronousMetricsCalculationTimeSpent"] = { watch.elapsedSeconds(), "Time in seconds spent for calculation of asynchronous metrics (this is the overhead of asynchronous metrics)." }; /// Log the new metrics. if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog()) @@ -1693,11 +1856,10 @@ void AsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, Tim update_period.count(), heavy_metric_update_period.count(), watch.elapsedSeconds()); - } - new_values["NumberOfDetachedParts"] = detached_parts_stats.count; - new_values["NumberOfDetachedByUserParts"] = detached_parts_stats.detached_by_user; + new_values["NumberOfDetachedParts"] = { detached_parts_stats.count, "The total number of parts detached from MergeTree tables. A part can be detached by a user with the `ALTER TABLE DETACH` query or by the server itself it the part is broken, unexpected or unneeded. The server does not care about detached parts and they can be removed." }; + new_values["NumberOfDetachedByUserParts"] = { detached_parts_stats.detached_by_user, "The total number of parts detached from MergeTree tables by users with the `ALTER TABLE DETACH` query (as opposed to unexpected, broken or ignored parts). The server does not care about detached parts and they can be removed." }; } } diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index 6e32bdb43b8..32a82bc106e 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -18,16 +18,23 @@ namespace Poco { -class Logger; + class Logger; } namespace DB { -class ProtocolServerAdapter; class ReadBuffer; -using AsynchronousMetricValue = double; +struct AsynchronousMetricValue +{ + double value; + const char * documentation; + + AsynchronousMetricValue(double value_, const char * documentation_) : value(value_), documentation(documentation_) {} + AsynchronousMetricValue(size_t value_, const char * documentation_) : value(value_), documentation(documentation_) {} +}; + using AsynchronousMetricValues = std::unordered_map; struct ProtocolServerMetrics @@ -42,6 +49,9 @@ struct ProtocolServerMetrics * * This includes both ClickHouse-related metrics (like memory usage of ClickHouse process) * and common OS-related metrics (like total memory usage on the server). + * + * All the values are either gauge type (like the total number of tables, the current memory usage). + * Or delta-counters representing some accumulation during the interval of time. */ class AsynchronousMetrics : WithContext { diff --git a/src/Server/ProtocolServerAdapter.h b/src/Server/ProtocolServerAdapter.h index 850640ab70a..514354f9723 100644 --- a/src/Server/ProtocolServerAdapter.h +++ b/src/Server/ProtocolServerAdapter.h @@ -6,8 +6,10 @@ #include #include + namespace DB { + class GRPCServer; class TCPServer; From ba05b7dd2c2218b1e63d701c5b762e2ace26f1b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 03:36:20 +0100 Subject: [PATCH 64/82] Add documentation for AsynchronousMetrics --- programs/server/MetricsTransmitter.cpp | 2 +- src/Interpreters/AsynchronousMetricLog.cpp | 2 +- src/Interpreters/AsynchronousMetricLog.h | 5 +- src/Interpreters/AsynchronousMetrics.cpp | 56 ++++++++++++------- src/Interpreters/AsynchronousMetrics.h | 1 + src/Server/PrometheusMetricsWriter.cpp | 7 ++- src/Server/PrometheusMetricsWriter.h | 2 +- .../StorageSystemAsynchronousMetrics.cpp | 4 +- 8 files changed, 51 insertions(+), 28 deletions(-) diff --git a/programs/server/MetricsTransmitter.cpp b/programs/server/MetricsTransmitter.cpp index 8ad519ba5aa..f7829a49a39 100644 --- a/programs/server/MetricsTransmitter.cpp +++ b/programs/server/MetricsTransmitter.cpp @@ -123,7 +123,7 @@ void MetricsTransmitter::transmit(std::vector & prev_count { for (const auto & name_value : async_metrics_values) { - key_vals.emplace_back(asynchronous_metrics_path_prefix + name_value.first, name_value.second); + key_vals.emplace_back(asynchronous_metrics_path_prefix + name_value.first, name_value.second.value); } } diff --git a/src/Interpreters/AsynchronousMetricLog.cpp b/src/Interpreters/AsynchronousMetricLog.cpp index 228934d5f4d..6176bb781ab 100644 --- a/src/Interpreters/AsynchronousMetricLog.cpp +++ b/src/Interpreters/AsynchronousMetricLog.cpp @@ -47,7 +47,7 @@ void AsynchronousMetricLog::addValues(const AsynchronousMetricValues & values) for (const auto & [key, value] : values) { element.metric_name = key; - element.value = round(value * precision) / precision; + element.value = round(value.value * precision) / precision; add(element); } diff --git a/src/Interpreters/AsynchronousMetricLog.h b/src/Interpreters/AsynchronousMetricLog.h index 900d84868bd..8a19fae29e9 100644 --- a/src/Interpreters/AsynchronousMetricLog.h +++ b/src/Interpreters/AsynchronousMetricLog.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -14,12 +15,8 @@ namespace DB { -using AsynchronousMetricValue = double; -using AsynchronousMetricValues = std::unordered_map; - /** AsynchronousMetricLog is a log of metric values measured at regular time interval. */ - struct AsynchronousMetricLogElement { UInt16 event_date; diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index f861fbb8426..6e8d745ef0e 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -394,7 +394,7 @@ static Value saveJemallocMetricImpl( Value value{}; size_t size = sizeof(value); mallctl(jemalloc_full_name.c_str(), &value, &size, nullptr, 0); - values[clickhouse_full_name] = { value, "An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html" }; + values[clickhouse_full_name] = AsynchronousMetricValue(value, "An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html"); return value; } @@ -1482,7 +1482,8 @@ void AsynchronousMetrics::update(TimePoint update_time) in.rewind(); uint64_t errors = 0; readText(errors, in); - new_values[fmt::format("EDAC{}_Correctable", i)] = { errors, "The number of correctable ECC memory errors", + new_values[fmt::format("EDAC{}_Correctable", i)] = { errors, + "The number of correctable ECC memory errors." " A high number of this value indicates bad RAM which has to be immediately replaced," " because in presence of a high number of corrected errors, a number of silent errors may happen as well, leading to data corruption." }; } @@ -1493,7 +1494,8 @@ void AsynchronousMetrics::update(TimePoint update_time) in.rewind(); uint64_t errors = 0; readText(errors, in); - new_values[fmt::format("EDAC{}_Uncorrectable", i)] = { errors, "The number of uncorrectable ECC memory errors", + new_values[fmt::format("EDAC{}_Uncorrectable", i)] = { errors, + "The number of uncorrectable ECC memory errors." " A non-zero number of this value indicates bad RAM which has to be immediately replaced," " because it indicates potential data corruption." }; } @@ -1519,24 +1521,36 @@ void AsynchronousMetrics::update(TimePoint update_time) { auto stat = getStatVFS(getContext()->getPath()); - new_values["FilesystemMainPathTotalBytes"] = stat.f_blocks * stat.f_frsize; - new_values["FilesystemMainPathAvailableBytes"] = stat.f_bavail * stat.f_frsize; - new_values["FilesystemMainPathUsedBytes"] = (stat.f_blocks - stat.f_bavail) * stat.f_frsize; - new_values["FilesystemMainPathTotalINodes"] = stat.f_files; - new_values["FilesystemMainPathAvailableINodes"] = stat.f_favail; - new_values["FilesystemMainPathUsedINodes"] = stat.f_files - stat.f_favail; + new_values["FilesystemMainPathTotalBytes"] = { stat.f_blocks * stat.f_frsize, + "The size of the volume where the main ClickHouse path is mounted, in bytes." }; + new_values["FilesystemMainPathAvailableBytes"] = { stat.f_bavail * stat.f_frsize, + "Available bytes on the volume where the main ClickHouse path is mounted." }; + new_values["FilesystemMainPathUsedBytes"] = { (stat.f_blocks - stat.f_bavail) * stat.f_frsize, + "Used bytes on the volume where the main ClickHouse path is mounted." }; + new_values["FilesystemMainPathTotalINodes"] = { stat.f_files, + "The total number of inodes on the volume where the main ClickHouse path is mounted. If it is less than 25 million, it indicates a misconfiguration." }; + new_values["FilesystemMainPathAvailableINodes"] = { stat.f_favail, + "The number of available inodes on the volume where the main ClickHouse path is mounted. If it is close to zero, it indicates a misconfiguration, and you will get 'no space left on device' even when the disk is not full." }; + new_values["FilesystemMainPathUsedINodes"] = { stat.f_files - stat.f_favail, + "The number of used inodes on the volume where the main ClickHouse path is mounted. This value mostly corresponds to the number of files." }; } { /// Current working directory of the server is the directory with logs. auto stat = getStatVFS("."); - new_values["FilesystemLogsPathTotalBytes"] = stat.f_blocks * stat.f_frsize; - new_values["FilesystemLogsPathAvailableBytes"] = stat.f_bavail * stat.f_frsize; - new_values["FilesystemLogsPathUsedBytes"] = (stat.f_blocks - stat.f_bavail) * stat.f_frsize; - new_values["FilesystemLogsPathTotalINodes"] = stat.f_files; - new_values["FilesystemLogsPathAvailableINodes"] = stat.f_favail; - new_values["FilesystemLogsPathUsedINodes"] = stat.f_files - stat.f_favail; + new_values["FilesystemLogsPathTotalBytes"] = { stat.f_blocks * stat.f_frsize, + "The size of the volume where ClickHouse logs path is mounted, in bytes. It's recommended to have at least 10 GB for logs." }; + new_values["FilesystemLogsPathAvailableBytes"] = { stat.f_bavail * stat.f_frsize, + "Available bytes on the volume where ClickHouse logs path is mounted. If this value approaches zero, you should tune the log rotation in the configuration file." }; + new_values["FilesystemLogsPathUsedBytes"] = { (stat.f_blocks - stat.f_bavail) * stat.f_frsize, + "Used bytes on the volume where ClickHouse logs path is mounted." }; + new_values["FilesystemLogsPathTotalINodes"] = { stat.f_files, + "The total number of inodes on the volume where ClickHouse logs path is mounted." }; + new_values["FilesystemLogsPathAvailableINodes"] = { stat.f_favail, + "The number of available inodes on the volume where ClickHouse logs path is mounted." }; + new_values["FilesystemLogsPathUsedINodes"] = { stat.f_files - stat.f_favail, + "The number of used inodes on the volume where ClickHouse logs path is mounted." }; } /// Free and total space on every configured disk. @@ -1553,10 +1567,14 @@ void AsynchronousMetrics::update(TimePoint update_time) auto available = disk->getAvailableSpace(); auto unreserved = disk->getUnreservedSpace(); - new_values[fmt::format("DiskTotal_{}", name)] = total; - new_values[fmt::format("DiskUsed_{}", name)] = total - available; - new_values[fmt::format("DiskAvailable_{}", name)] = available; - new_values[fmt::format("DiskUnreserved_{}", name)] = unreserved; + new_values[fmt::format("DiskTotal_{}", name)] = { total, + "The total size in bytes of the disk (virtual filesystem). Remote filesystems can show a large value like 16 EiB." }; + new_values[fmt::format("DiskUsed_{}", name)] = { total - available, + "Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." }; + new_values[fmt::format("DiskAvailable_{}", name)] = { available, + "Available bytes on the disk (virtual filesystem). Remote filesystems can show a large value like 16 EiB." }; + new_values[fmt::format("DiskUnreserved_{}", name)] = { unreserved, + "Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems can show a large value like 16 EiB." }; } } diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index 32a82bc106e..3529c162944 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -33,6 +33,7 @@ struct AsynchronousMetricValue AsynchronousMetricValue(double value_, const char * documentation_) : value(value_), documentation(documentation_) {} AsynchronousMetricValue(size_t value_, const char * documentation_) : value(value_), documentation(documentation_) {} + AsynchronousMetricValue() = default; /// For std::unordered_map::operator[]. }; using AsynchronousMetricValues = std::unordered_map; diff --git a/src/Server/PrometheusMetricsWriter.cpp b/src/Server/PrometheusMetricsWriter.cpp index 9168eb5f24d..843d1e64463 100644 --- a/src/Server/PrometheusMetricsWriter.cpp +++ b/src/Server/PrometheusMetricsWriter.cpp @@ -108,11 +108,16 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const if (!replaceInvalidChars(key)) continue; + auto value = name_value.second; + std::string metric_doc{value.documentation}; + convertHelpToSingleLine(metric_doc); + // TODO: add HELP section? asynchronous_metrics contains only key and value + writeOutLine(wb, "# HELP", key, metric_doc); writeOutLine(wb, "# TYPE", key, "gauge"); - writeOutLine(wb, key, value); + writeOutLine(wb, key, value.value); } } diff --git a/src/Server/PrometheusMetricsWriter.h b/src/Server/PrometheusMetricsWriter.h index 4422ced625e..0c2dde1f66f 100644 --- a/src/Server/PrometheusMetricsWriter.h +++ b/src/Server/PrometheusMetricsWriter.h @@ -3,11 +3,11 @@ #include #include - #include #include + namespace DB { diff --git a/src/Storages/System/StorageSystemAsynchronousMetrics.cpp b/src/Storages/System/StorageSystemAsynchronousMetrics.cpp index 70e12440678..e2f62b902b7 100644 --- a/src/Storages/System/StorageSystemAsynchronousMetrics.cpp +++ b/src/Storages/System/StorageSystemAsynchronousMetrics.cpp @@ -12,6 +12,7 @@ NamesAndTypesList StorageSystemAsynchronousMetrics::getNamesAndTypes() return { {"metric", std::make_shared()}, {"value", std::make_shared()}, + {"description", std::make_shared()}, }; } @@ -27,7 +28,8 @@ void StorageSystemAsynchronousMetrics::fillData(MutableColumns & res_columns, Co for (const auto & name_value : async_metrics_values) { res_columns[0]->insert(name_value.first); - res_columns[1]->insert(name_value.second); + res_columns[1]->insert(name_value.second.value); + res_columns[2]->insert(name_value.second.documentation); } } From 9e8fca3b4227beb1681e5cca20acddee47f07c31 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 03:59:10 +0100 Subject: [PATCH 65/82] Update test --- .../0_stateless/02117_show_create_table_system.reference | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 4ffa31a5618..f2726f17e88 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -23,7 +23,8 @@ COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.asynchronous_metrics ( `metric` String, - `value` Float64 + `value` Float64, + `description` String ) ENGINE = SystemAsynchronousMetrics COMMENT 'SYSTEM TABLE is built on the fly.' From 3096534d6a845cb06b930973768a20cffa7ef9a0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:07:29 +0100 Subject: [PATCH 66/82] Clarifiactions --- src/Interpreters/AsynchronousMetrics.cpp | 48 ++++++++++++------------ 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 6e8d745ef0e..9f657220999 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -703,7 +703,7 @@ void AsynchronousMetrics::update(TimePoint update_time) #if !defined(OS_FREEBSD) new_values["MemoryShared"] = { data.shared, "The amount of memory used by the server process, that is also shared by another processes, in bytes." - " ClickHouse does not use shared memory, and the only case for this metric to be higher than zero is the usage of the system's C library (`libc`)." + " ClickHouse does not use shared memory, but some memory can be labeled by OS as shared for its own reasons." " This metric does not make a lot of sense to watch, and it exists only for completeness reasons."}; #endif new_values["MemoryCode"] = { data.code, @@ -870,40 +870,40 @@ void AsynchronousMetrics::update(TimePoint update_time) delta_values_all_cpus = delta_values; new_values["OSUserTime" + cpu_suffix] = { delta_values.user * multiplier, - "The ratio of time the CPU core was running userspace code. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time the CPU core was running userspace code. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " This includes also the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline stalls, branch mispredictions, running another SMT core)." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSNiceTime" + cpu_suffix] = { delta_values.nice * multiplier, - "The ratio of time the CPU core was running userspace code with higher priority. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time the CPU core was running userspace code with higher priority. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSSystemTime" + cpu_suffix] = { delta_values.system * multiplier, - "The ratio of time the CPU core was running OS kernel (system) code. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time the CPU core was running OS kernel (system) code. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSIdleTime" + cpu_suffix] = { delta_values.idle * multiplier, - "The ratio of time the CPU core was idle (not even ready to run a process waiting for IO) from the OS kernel standpoint. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time the CPU core was idle (not even ready to run a process waiting for IO) from the OS kernel standpoint. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " This does not include the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline stalls, branch mispredictions, running another SMT core)." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSIOWaitTime" + cpu_suffix] = { delta_values.iowait * multiplier, - "The ratio of time the CPU core was not running the code but when the OS kernel did not run any other process on this CPU as the processes were waiting for IO. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time the CPU core was not running the code but when the OS kernel did not run any other process on this CPU as the processes were waiting for IO. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSIrqTime" + cpu_suffix] = { delta_values.irq * multiplier, - "The ratio of time spent for running hardware interrupt requests on the CPU. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time spent for running hardware interrupt requests on the CPU. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " A high number of this metric may indicate hardware misconfiguration or a very high network load." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSSoftIrqTime" + cpu_suffix] = { delta_values.softirq * multiplier, - "The ratio of time spent for running software interrupt requests on the CPU. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time spent for running software interrupt requests on the CPU. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " A high number of this metric may indicate inefficient software running on the system." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSStealTime" + cpu_suffix] = { delta_values.steal * multiplier, - "The ratio of time spent in other operating systems by the CPU when running in a virtualized environment. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time spent in other operating systems by the CPU when running in a virtualized environment. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " Not every virtualized environments present this metric, and most of them don't." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSGuestTime" + cpu_suffix] = { delta_values.guest * multiplier, - "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel (See `man procfs`). This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel (See `man procfs`). This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " This metric is irrelevant for ClickHouse, but still exists for completeness." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSGuestNiceTime" + cpu_suffix] = { delta_values.guest_nice * multiplier, - "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel, when a guest was set to a higher priority (See `man procfs`). This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel, when a guest was set to a higher priority (See `man procfs`). This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " This metric is irrelevant for ClickHouse, but still exists for completeness." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; } @@ -932,7 +932,7 @@ void AsynchronousMetrics::update(TimePoint update_time) skipToNextLineOrEOF(*proc_stat); new_values["OSProcessesRunning"] = { processes_running, "The number of runnable (running or ready to run) threads by the operating system." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "procs_blocked") { @@ -941,7 +941,7 @@ void AsynchronousMetrics::update(TimePoint update_time) skipToNextLineOrEOF(*proc_stat); new_values["OSProcessesBlocked"] = { processes_blocked, "Number of threads blocked waiting for I/O to complete (`man procfs`)." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else skipToNextLineOrEOF(*proc_stat); @@ -951,9 +951,9 @@ void AsynchronousMetrics::update(TimePoint update_time) { ProcStatValuesOther delta_values = current_other_values - proc_stat_values_other; - new_values["OSInterrupts"] = { delta_values.interrupts, "The number of interrupts on the host machine. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; - new_values["OSContextSwitches"] = { delta_values.context_switches, "The number of context switches that the system underwent on the host machine. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; - new_values["OSProcessesCreated"] = { delta_values.processes_created, "The number of processes created. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["OSInterrupts"] = { delta_values.interrupts, "The number of interrupts on the host machine. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["OSContextSwitches"] = { delta_values.context_switches, "The number of context switches that the system underwent on the host machine. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["OSProcessesCreated"] = { delta_values.processes_created, "The number of processes created. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; /// Also write values normalized to 0..1 by diving to the number of CPUs. /// These values are good to be averaged across the cluster of non-uniform servers. @@ -1056,35 +1056,35 @@ void AsynchronousMetrics::update(TimePoint update_time) " See the `OSMemoryAvailable` metric instead." " For convenience we also provide OSMemoryFreePlusCached, that should be somewhat similar to OSMemoryAvailable." " See also https://www.linuxatemyram.com/." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "MemAvailable:") { new_values["OSMemoryAvailable"] = { bytes, "The amount of memory available to be used by programs, in bytes. This is very similar to the `OSMemoryFreePlusCached` metric." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "Buffers:") { new_values["OSMemoryBuffers"] = { bytes, "The amount of memory used by OS kernel buffers, in bytes. This should be typically small, and large values may indicate a misconfiguration of the OS." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "Cached:") { free_plus_cached_bytes += bytes; new_values["OSMemoryCached"] = { bytes, "The amount of memory used by the OS page cache, in bytes. Typically, almost all available memory is used by the OS page cache - high values of this metric are normal and expected." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "SwapCached:") { new_values["OSMemorySwapCached"] = { bytes, "The amount of memory in swap that was also loaded in RAM. Swap should be disabled on production systems. If the value of this metric is large, it indicates a misconfiguration." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } skipToNextLineOrEOF(*meminfo); } - new_values["OSMemoryFreePlusCached"] = { free_plus_cached_bytes, "The amount of free memory or OS page cache memory on the host system, in bytes. This memory is available to be used by programs. The value should be very similar to `OSMemoryAvailable`." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["OSMemoryFreePlusCached"] = { free_plus_cached_bytes, "The amount of free memory plus OS page cache memory on the host system, in bytes. This memory is available to be used by programs. The value should be very similar to `OSMemoryAvailable`." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } catch (...) { @@ -1153,7 +1153,7 @@ void AsynchronousMetrics::update(TimePoint update_time) uint64_t open_files = 0; readText(open_files, *file_nr); new_values["OSOpenFiles"] = { open_files, "The total number of opened files on the host machine." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } catch (...) { From 884603cc5b7ce6bf1bcefea80fc0784a1f83ebbe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:08:17 +0100 Subject: [PATCH 67/82] Clarifiactions --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 9f657220999..8e84dec0dd9 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1054,7 +1054,7 @@ void AsynchronousMetrics::update(TimePoint update_time) " This does not include the memory used by the OS page cache memory, in bytes." " The page cache memory is also available for usage by programs, so the value of this metric can be confusing." " See the `OSMemoryAvailable` metric instead." - " For convenience we also provide OSMemoryFreePlusCached, that should be somewhat similar to OSMemoryAvailable." + " For convenience we also provide the `OSMemoryFreePlusCached` metric, that should be somewhat similar to OSMemoryAvailable." " See also https://www.linuxatemyram.com/." " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } From 90643aeba1b8d7b9918ec501b31de028778c6157 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:10:20 +0100 Subject: [PATCH 68/82] Clarifiactions --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 8e84dec0dd9..983eb36d042 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1669,7 +1669,7 @@ void AsynchronousMetrics::update(TimePoint update_time) new_values["NumberOfDatabases"] = { number_of_databases, "Total number of databases on the server." }; new_values["NumberOfTables"] = { total_number_of_tables, "Total number of tables summed across the databases on the server, excluding the databases that cannot contain MergeTree tables." - "The excluded database engines are those who generate the set of tables on the fly, like `Lazy`, `MySQL`, `PostgreSQL`, `SQlite`."}; + " The excluded database engines are those who generate the set of tables on the fly, like `Lazy`, `MySQL`, `PostgreSQL`, `SQlite`."}; new_values["TotalBytesOfMergeTreeTables"] = { total_number_of_bytes, "Total amount of bytes (compressed, including data and indices) stored in all tables of MergeTree family." }; new_values["TotalRowsOfMergeTreeTables"] = { total_number_of_rows, "Total amount of rows (records) stored in all tables of MergeTree family." }; From c06bff661514e24cab73c13e96aae471f01f6f40 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:12:58 +0100 Subject: [PATCH 69/82] More praise for heroes --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 983eb36d042..6fe4a1bc92e 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -669,7 +669,7 @@ void AsynchronousMetrics::update(TimePoint update_time) // the following calls will return stale values. It increments and returns // the current epoch number, which might be useful to log as a sanity check. auto epoch = updateJemallocEpoch(); - new_values["jemalloc.epoch"] = { epoch, "An internal incremental update number of the statistics of JeMalloc (a low-level memory allocator), used in all other `jemalloc` metrics." }; + new_values["jemalloc.epoch"] = { epoch, "An internal incremental update number of the statistics of jemalloc (Jason Evans' memory allocator), used in all other `jemalloc` metrics." }; // Collect the statistics themselves. saveJemallocMetric(new_values, "allocated"); From 7aec54b95a6d69984e2420f7d997f5dca4b5bbd9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:19:32 +0100 Subject: [PATCH 70/82] Clarifiactions --- src/Interpreters/AsynchronousMetrics.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 6fe4a1bc92e..d4018001a49 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1210,7 +1210,7 @@ void AsynchronousMetrics::update(TimePoint update_time) BLOCK_DEVICE_EXPLANATION }; new_values["BlockDiscardOps_" + name] = { delta_values.discard_ops, "Number of discard operations requested from the block device. These operations are relevant for SSD." - " They are not used by ClickHouse, but can be used by other processes on the system." + " Discard operations are not used by ClickHouse, but can be used by other processes on the system." BLOCK_DEVICE_EXPLANATION }; new_values["BlockReadMerges_" + name] = { delta_values.read_merges, @@ -1221,7 +1221,7 @@ void AsynchronousMetrics::update(TimePoint update_time) BLOCK_DEVICE_EXPLANATION }; new_values["BlockDiscardMerges_" + name] = { delta_values.discard_merges, "Number of discard operations requested from the block device and merged together by the OS IO scheduler." - " These operations are relevant for SSD. They are not used by ClickHouse, but can be used by other processes on the system." + " These operations are relevant for SSD. Discard operations are not used by ClickHouse, but can be used by other processes on the system." BLOCK_DEVICE_EXPLANATION }; new_values["BlockReadBytes_" + name] = { delta_values.read_sectors * sector_size, @@ -1235,7 +1235,7 @@ void AsynchronousMetrics::update(TimePoint update_time) BLOCK_DEVICE_EXPLANATION }; new_values["BlockDiscardBytes_" + name] = { delta_values.discard_sectors * sector_size, "Number of discarded bytes on the block device." - " These operations are relevant for SSD. They are not used by ClickHouse, but can be used by other processes on the system." + " These operations are relevant for SSD. Discard operations are not used by ClickHouse, but can be used by other processes on the system." BLOCK_DEVICE_EXPLANATION }; new_values["BlockReadTime_" + name] = { delta_values.read_ticks * time_multiplier, @@ -1246,7 +1246,7 @@ void AsynchronousMetrics::update(TimePoint update_time) BLOCK_DEVICE_EXPLANATION }; new_values["BlockDiscardTime_" + name] = { delta_values.discard_ticks * time_multiplier, "Time in seconds spend in discard operations requested from the block device, summed across all the operations." - " These operations are relevant for SSD. They are not used by ClickHouse, but can be used by other processes on the system." + " These operations are relevant for SSD. Discard operations are not used by ClickHouse, but can be used by other processes on the system." BLOCK_DEVICE_EXPLANATION }; new_values["BlockInFlightOps_" + name] = { delta_values.in_flight_ios, From ddc01fd05ee29a8568952f0b7b00d316730c01d0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:24:51 +0100 Subject: [PATCH 71/82] Add a test --- ...0_every_asynchronous_metric_must_have_documentation.reference | 0 .../02480_every_asynchronous_metric_must_have_documentation.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.reference create mode 100644 tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.sql diff --git a/tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.reference b/tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.sql b/tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.sql new file mode 100644 index 00000000000..3f0ab58cc43 --- /dev/null +++ b/tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.sql @@ -0,0 +1 @@ +SELECT metric FROM system.asynchronous_metrics WHERE length(description) < 10; From 045fd0651032cb8c7ec7bc7a99bd46f340d0f711 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:30:43 +0100 Subject: [PATCH 72/82] Clarifications --- src/Interpreters/AsynchronousMetrics.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index d4018001a49..291bca4277d 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1200,7 +1200,7 @@ void AsynchronousMetrics::update(TimePoint update_time) #define BLOCK_DEVICE_EXPLANATION \ " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." \ - " See https://www.kernel.org/doc/Documentation/block/stat.txt" + " Source: `/sys/block`. See https://www.kernel.org/doc/Documentation/block/stat.txt" new_values["BlockReadOps_" + name] = { delta_values.read_ios, "Number of read operations requested from the block device." @@ -1403,7 +1403,7 @@ void AsynchronousMetrics::update(TimePoint update_time) Int64 temperature = 0; readText(temperature, in); new_values[fmt::format("Temperature{}", i)] = { temperature * 0.001, - "The temperature of the corresponding device in ℃. A sensor can return an unrealistic value." }; + "The temperature of the corresponding device in ℃. A sensor can return an unrealistic value. Source: `/sys/class/thermal`" }; } } catch (...) @@ -1443,10 +1443,10 @@ void AsynchronousMetrics::update(TimePoint update_time) if (sensor_name.empty()) new_values[fmt::format("Temperature_{}", hwmon_name)] = { temperature * 0.001, - "The temperature reported by the corresponding hardware monitor in ℃. A sensor can return an unrealistic value." }; + "The temperature reported by the corresponding hardware monitor in ℃. A sensor can return an unrealistic value. Source: `/sys/class/hwmon`" }; else new_values[fmt::format("Temperature_{}_{}", hwmon_name, sensor_name)] = { temperature * 0.001, - "The temperature reported by the corresponding hardware monitor and the corresponding sensor in ℃. A sensor can return an unrealistic value." }; + "The temperature reported by the corresponding hardware monitor and the corresponding sensor in ℃. A sensor can return an unrealistic value. Source: `/sys/class/hwmon`" }; } } } @@ -1485,7 +1485,8 @@ void AsynchronousMetrics::update(TimePoint update_time) new_values[fmt::format("EDAC{}_Correctable", i)] = { errors, "The number of correctable ECC memory errors." " A high number of this value indicates bad RAM which has to be immediately replaced," - " because in presence of a high number of corrected errors, a number of silent errors may happen as well, leading to data corruption." }; + " because in presence of a high number of corrected errors, a number of silent errors may happen as well, leading to data corruption." + " Source: `/sys/devices/system/edac/mc/`" }; } if (edac[i].second) @@ -1497,7 +1498,8 @@ void AsynchronousMetrics::update(TimePoint update_time) new_values[fmt::format("EDAC{}_Uncorrectable", i)] = { errors, "The number of uncorrectable ECC memory errors." " A non-zero number of this value indicates bad RAM which has to be immediately replaced," - " because it indicates potential data corruption." }; + " because it indicates potential data corruption." + " Source: `/sys/devices/system/edac/mc/`" }; } } } From f5270a6ebeeac6221079524b4a602908e26d6f0c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:51:46 +0100 Subject: [PATCH 73/82] A workaround for LLVM bug, https://github.com/llvm/llvm-project/issues/58633 --- cmake/sanitize.cmake | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index f0cef54b0b8..3e3bb7ec2b2 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -16,7 +16,9 @@ endmacro() if (SANITIZE) if (SANITIZE STREQUAL "address") - set (ASAN_FLAGS "-fsanitize=address -fsanitize-address-use-after-scope") + # LLVM-15 has a bug in Address Sanitizer, preventing the usage of 'sanitize-address-use-after-scope', + # see https://github.com/llvm/llvm-project/issues/58633 + set (ASAN_FLAGS "-fsanitize=address -fno-sanitize-address-use-after-scope") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${ASAN_FLAGS}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${ASAN_FLAGS}") From c2ddef897f59fe7f60bf062eebfb9e510e0553d3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 07:12:25 +0300 Subject: [PATCH 74/82] Update Dockerfile.ubuntu --- docker/server/Dockerfile.ubuntu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index c552c26f731..f50160321e1 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -88,7 +88,7 @@ RUN arch=${TARGETARCH:-amd64} \ # Some vulnerability scanners are arguing about Ubuntu, which is not relevant to ClickHouse at all. # ClickHouse does not care when you report false vulnerabilities by running some Docker scanners. -RUN apt-get remove --purge -y libksba8 && apt-get autoremove +RUN apt-get remove --purge -y libksba8 && apt-get autoremove -y # we need to allow "others" access to clickhouse folder, because docker container # can be started with arbitrary uid (openshift usecase) From 38ee1a81a3efdf939df6e2891428d517fda128af Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 13 Nov 2022 19:17:41 +0000 Subject: [PATCH 75/82] Stop config reloader before anything else --- programs/server/Server.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b412b579539..573c3fbe1ba 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1486,11 +1486,6 @@ int Server::main(const std::vector & /*args*/) #endif SCOPE_EXIT({ - /// Stop reloading of the main config. This must be done before `global_context->shutdown()` because - /// otherwise the reloading may pass a changed config to some destroyed parts of ContextSharedPart. - main_config_reloader.reset(); - access_control.stopPeriodicReloading(); - async_metrics.stop(); /** Ask to cancel background jobs all table engines, @@ -1789,10 +1784,17 @@ int Server::main(const std::vector & /*args*/) SCOPE_EXIT_SAFE({ LOG_DEBUG(log, "Received termination signal."); - LOG_DEBUG(log, "Waiting for current connections to close."); + + /// Stop reloading of the main config. This must be done before everything else because it + /// can try to access/modify already deleted objects. + /// E.g. it can recreate new servers or it may pass a changed config to some destroyed parts of ContextSharedPart. + main_config_reloader.reset(); + access_control.stopPeriodicReloading(); is_cancelled = true; + LOG_DEBUG(log, "Waiting for current connections to close."); + size_t current_connections = 0; { std::lock_guard lock(servers_lock); From 4012b46a87daa5ea375055674920205cc52bd5b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 23:11:00 +0100 Subject: [PATCH 76/82] Adjustments for Mac OS --- src/Interpreters/AsynchronousMetrics.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index 3529c162944..22ed2e862ea 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -31,8 +31,9 @@ struct AsynchronousMetricValue double value; const char * documentation; - AsynchronousMetricValue(double value_, const char * documentation_) : value(value_), documentation(documentation_) {} - AsynchronousMetricValue(size_t value_, const char * documentation_) : value(value_), documentation(documentation_) {} + template + AsynchronousMetricValue(T value_, const char * documentation_) + : value(static_cast(value_)), documentation(documentation_) {} AsynchronousMetricValue() = default; /// For std::unordered_map::operator[]. }; From 1229f8fd09725227711c91799d0f4dbc18842ba9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Nov 2022 13:15:08 +0300 Subject: [PATCH 77/82] Update 02449_check_dependencies_and_table_shutdown.sql --- .../0_stateless/02449_check_dependencies_and_table_shutdown.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.sql b/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.sql index 49e1e36acc9..c586db447c0 100644 --- a/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.sql +++ b/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.sql @@ -8,7 +8,7 @@ INSERT INTO view VALUES (1, 'v'); CREATE DICTIONARY dict (id UInt32, value String) PRIMARY KEY id SOURCE(CLICKHOUSE(host 'localhost' port tcpPort() user 'default' db currentDatabase() table 'view')) -LAYOUT (HASHED()) LIFETIME (600); +LAYOUT (HASHED()) LIFETIME (MIN 600 MAX 600); SHOW CREATE dict; From db6d867bf2b9f3c5eb97c0f9296e4eee666c7359 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Nov 2022 13:16:23 +0300 Subject: [PATCH 78/82] Update 02449_check_dependencies_and_table_shutdown.reference --- .../02449_check_dependencies_and_table_shutdown.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.reference b/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.reference index 0ecea03c64e..37cd3d93e39 100644 --- a/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.reference +++ b/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.reference @@ -1,4 +1,4 @@ -CREATE DICTIONARY default.dict\n(\n `id` UInt32,\n `value` String\n)\nPRIMARY KEY id\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' DB \'default\' TABLE \'view\'))\nLIFETIME(MIN 0 MAX 600)\nLAYOUT(HASHED()) +CREATE DICTIONARY default.dict\n(\n `id` UInt32,\n `value` String\n)\nPRIMARY KEY id\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' DB \'default\' TABLE \'view\'))\nLIFETIME(MIN 600 MAX 600)\nLAYOUT(HASHED()) CREATE TABLE default.table\n(\n `col` String MATERIALIZED dictGet(\'default.dict\', \'value\', toUInt32(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1 v 1 v From 2dda059cbd414d401e333f04cc648f72775f339a Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 14 Nov 2022 14:09:47 +0100 Subject: [PATCH 79/82] Revert "Randomize keeper fault injection settings in stress tests" --- tests/clickhouse-test | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a07a39600d8..20e63412d91 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -456,8 +456,6 @@ class SettingsRandomizer: "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), "optimize_distinct_in_order": lambda: random.randint(0, 1), "optimize_sorting_by_input_stream_properties": lambda: random.randint(0, 1), - "insert_keeper_max_retries": lambda: 20, - "insert_keeper_fault_injection_probability": lambda: 0.01, } @staticmethod From 4a90f8d3f54360795c5876fc8aff742b325170a9 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Mon, 14 Nov 2022 11:34:06 -0500 Subject: [PATCH 80/82] Update permissions-for-queries.md closes https://github.com/ClickHouse/ClickHouse/issues/43068 --- docs/en/operations/settings/permissions-for-queries.md | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/settings/permissions-for-queries.md b/docs/en/operations/settings/permissions-for-queries.md index 3ba62b78cfe..fe803e7b1ee 100644 --- a/docs/en/operations/settings/permissions-for-queries.md +++ b/docs/en/operations/settings/permissions-for-queries.md @@ -16,14 +16,14 @@ Queries in ClickHouse can be divided into several types: The following settings regulate user permissions by the type of query: -- [readonly](#settings_readonly) — Restricts permissions for all types of queries except DDL queries. +- [readonly](#settings_readonly) — Restricts permissions for Write and DDL queries, and selectively on Change settings queries. - [allow_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. `KILL QUERY` can be performed with any settings. ## readonly {#settings_readonly} -Restricts permissions for reading data, write data and change settings queries. +Restricts permissions for read data, write data, and change settings queries. See how the queries are divided into types [above](#permissions_for_queries). @@ -37,7 +37,7 @@ After setting `readonly = 1`, the user can’t change `readonly` and `allow_ddl` When using the `GET` method in the [HTTP interface](../../interfaces/http.md), `readonly = 1` is set automatically. To modify data, use the `POST` method. -Setting `readonly = 1` prohibit the user from changing all the settings. There is a way to prohibit the user from changing only specific settings. Also there is a way to allow changing only specific settings under `readonly = 1` restrictions. For details see [constraints on settings](../../operations/settings/constraints-on-settings.md). +Setting `readonly = 1` prohibits the user from changing settings. There is a way to prohibit the user from changing only specific settings. Also there is a way to allow changing only specific settings under `readonly = 1` restrictions. For details see [constraints on settings](../../operations/settings/constraints-on-settings.md). Default value: 0 @@ -52,8 +52,6 @@ Possible values: - 0 — DDL queries are not allowed. - 1 — DDL queries are allowed. -You can’t execute `SET allow_ddl = 1` if `allow_ddl = 0` for the current session. +You can’t run `SET allow_ddl = 1` if `allow_ddl = 0` for the current session. Default value: 1 - -[Original article](https://clickhouse.com/docs/en/operations/settings/permissions_for_queries/) From 446c029810174155d5604450ba047b901908c9cf Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 14 Nov 2022 11:53:44 -0500 Subject: [PATCH 81/82] make read only clearer --- docs/en/operations/settings/permissions-for-queries.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/permissions-for-queries.md b/docs/en/operations/settings/permissions-for-queries.md index fe803e7b1ee..09797146773 100644 --- a/docs/en/operations/settings/permissions-for-queries.md +++ b/docs/en/operations/settings/permissions-for-queries.md @@ -29,9 +29,9 @@ See how the queries are divided into types [above](#permissions_for_queries). Possible values: -- 0 — All queries are allowed. -- 1 — Only read data queries are allowed. -- 2 — Read data and change settings queries are allowed. +- 0 — Read, Write, and Change settings queries are allowed. +- 1 — Only Read data queries are allowed. +- 2 — Read data and Change settings queries are allowed. After setting `readonly = 1`, the user can’t change `readonly` and `allow_ddl` settings in the current session. From 5d6c622e40aaa73cb1ff9c7ad08a81bcad411d1a Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 14 Nov 2022 14:20:55 -0500 Subject: [PATCH 82/82] apply suggestions and simplify --- .../settings/permissions-for-queries.md | 38 ++++++++++++------- 1 file changed, 25 insertions(+), 13 deletions(-) diff --git a/docs/en/operations/settings/permissions-for-queries.md b/docs/en/operations/settings/permissions-for-queries.md index 09797146773..c565de9b21a 100644 --- a/docs/en/operations/settings/permissions-for-queries.md +++ b/docs/en/operations/settings/permissions-for-queries.md @@ -16,16 +16,20 @@ Queries in ClickHouse can be divided into several types: The following settings regulate user permissions by the type of query: -- [readonly](#settings_readonly) — Restricts permissions for Write and DDL queries, and selectively on Change settings queries. -- [allow_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. - -`KILL QUERY` can be performed with any settings. - -## readonly {#settings_readonly} - +## readonly Restricts permissions for read data, write data, and change settings queries. -See how the queries are divided into types [above](#permissions_for_queries). +When set to 1, allows: + +- All types of read queries (like SELECT and equivalent queries). +- Queries that modify only session context (like USE). + +When set to 2, allows the above plus: +- SET and CREATE TEMPORARY TABLE + + :::tip + Queries like EXISTS, DESCRIBE, EXPLAIN, SHOW PROCESSLIST, etc are equivalent to SELECT, because they just do select from system tables. + ::: Possible values: @@ -33,25 +37,33 @@ Possible values: - 1 — Only Read data queries are allowed. - 2 — Read data and Change settings queries are allowed. +Default value: 0 + +:::note After setting `readonly = 1`, the user can’t change `readonly` and `allow_ddl` settings in the current session. When using the `GET` method in the [HTTP interface](../../interfaces/http.md), `readonly = 1` is set automatically. To modify data, use the `POST` method. Setting `readonly = 1` prohibits the user from changing settings. There is a way to prohibit the user from changing only specific settings. Also there is a way to allow changing only specific settings under `readonly = 1` restrictions. For details see [constraints on settings](../../operations/settings/constraints-on-settings.md). +::: -Default value: 0 ## allow_ddl {#settings_allow_ddl} Allows or denies [DDL](https://en.wikipedia.org/wiki/Data_definition_language) queries. -See how the queries are divided into types [above](#permissions_for_queries). - Possible values: - 0 — DDL queries are not allowed. - 1 — DDL queries are allowed. -You can’t run `SET allow_ddl = 1` if `allow_ddl = 0` for the current session. - Default value: 1 + +:::note +You cannot run `SET allow_ddl = 1` if `allow_ddl = 0` for the current session. +::: + + +:::note KILL QUERY +`KILL QUERY` can be performed with any combination of readonly and allow_ddl settings. +:::