From ffd3d2519eb51117f6c143122b2b13f32d1b2b8d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Oct 2020 03:13:17 +0300 Subject: [PATCH 01/68] Avoid stack overflow in materialized views, part 1 --- src/Storages/StorageMaterializedView.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 3e1df80ff42..8c7c6c7c9f1 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -21,6 +21,7 @@ #include #include +#include #include @@ -319,11 +320,13 @@ void StorageMaterializedView::shutdown() StoragePtr StorageMaterializedView::getTargetTable() const { + checkStackSize(); return DatabaseCatalog::instance().getTable(target_table_id, global_context); } StoragePtr StorageMaterializedView::tryGetTargetTable() const { + checkStackSize(); return DatabaseCatalog::instance().tryGetTable(target_table_id, global_context); } From 6c3aad34bec63250abce2b59def849ba18f31e1d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Oct 2020 03:19:45 +0300 Subject: [PATCH 02/68] Avoid stack overflow in materialized views, part 2: sanity check --- src/Storages/StorageMaterializedView.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 8c7c6c7c9f1..fd75807eb2e 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -30,6 +30,7 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int NOT_IMPLEMENTED; extern const int INCORRECT_QUERY; extern const int QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW; @@ -72,7 +73,9 @@ StorageMaterializedView::StorageMaterializedView( setInMemoryMetadata(storage_metadata); if (!has_inner_table) + { target_table_id = query.to_table_id; + } else if (attach_) { /// If there is an ATTACH request, then the internal table must already be created. @@ -98,6 +101,9 @@ StorageMaterializedView::StorageMaterializedView( target_table_id = DatabaseCatalog::instance().getTable({manual_create_query->database, manual_create_query->table}, global_context)->getStorageID(); } + if (target_table_id == getStorageID()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Materialized view {} cannot point to itself", getStorageID().getFullTableName()); + if (!select.select_table_id.empty()) DatabaseCatalog::instance().addDependency(select.select_table_id, getStorageID()); } From cc3feb36a65a993c7a2a0838236e5035a011683c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Oct 2020 03:21:50 +0300 Subject: [PATCH 03/68] Avoid stack overflow in materialized views, part 3: more checks --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 3baa2b30c3f..1252dd7f4de 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -29,6 +30,8 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( , context(context_) , query_ptr(query_ptr_) { + checkStackSize(); + /** TODO This is a very important line. At any insertion into the table one of streams should own lock. * Although now any insertion into the table is done via PushingToViewsBlockOutputStream, * but it's clear that here is not the best place for this functionality. From 625d03d8c244d57dcd687a3dfdd177ce51dbe5c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Oct 2020 03:37:53 +0300 Subject: [PATCH 04/68] Avoid stack overflow in materialized views, part 4: correct checks --- src/Interpreters/StorageID.cpp | 9 +++++++++ src/Interpreters/StorageID.h | 1 + 2 files changed, 10 insertions(+) diff --git a/src/Interpreters/StorageID.cpp b/src/Interpreters/StorageID.cpp index 2d6a4900dd3..a7d02601dbf 100644 --- a/src/Interpreters/StorageID.cpp +++ b/src/Interpreters/StorageID.cpp @@ -79,6 +79,15 @@ bool StorageID::operator<(const StorageID & rhs) const return !hasUUID(); } +bool StorageID::operator==(const StorageID & rhs) const +{ + assertNotEmpty(); + if (!hasUUID() && !rhs.hasUUID()) + return std::tie(database_name, table_name) == std::tie(rhs.database_name, rhs.table_name); + else + return hasUUID() && rhs.hasUUID() && uuid == rhs.uuid; +} + String StorageID::getFullTableName() const { return backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name); diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index 9343f67fe7a..d42dfda06fd 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -68,6 +68,7 @@ struct StorageID return uuid != UUIDHelpers::Nil; } + bool operator==(const StorageID & rhs) const; bool operator<(const StorageID & rhs) const; void assertNotEmpty() const From 622cfafcb4a7dc1d9aa2b73624b385c2bd77468f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Oct 2020 03:45:05 +0300 Subject: [PATCH 05/68] Avoid stack overflow in materialized views, part 5: add a test --- ...materialized_view_stack_overflow.reference | 0 ...01527_materialized_view_stack_overflow.sql | 28 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/01527_materialized_view_stack_overflow.reference create mode 100644 tests/queries/0_stateless/01527_materialized_view_stack_overflow.sql diff --git a/tests/queries/0_stateless/01527_materialized_view_stack_overflow.reference b/tests/queries/0_stateless/01527_materialized_view_stack_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01527_materialized_view_stack_overflow.sql b/tests/queries/0_stateless/01527_materialized_view_stack_overflow.sql new file mode 100644 index 00000000000..4a67ef4b2d8 --- /dev/null +++ b/tests/queries/0_stateless/01527_materialized_view_stack_overflow.sql @@ -0,0 +1,28 @@ +DROP TABLE IF EXISTS t; +DROP TABLE IF EXISTS v; + +CREATE TABLE t (c String) ENGINE = Memory; + +CREATE MATERIALIZED VIEW v to v AS SELECT c FROM t; -- { serverError 36 } +CREATE MATERIALIZED VIEW v to t AS SELECT * FROM v; -- { serverError 60 } + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS v1; +DROP TABLE IF EXISTS v2; + +CREATE TABLE t1 (c String) ENGINE = Memory; +CREATE TABLE t2 (c String) ENGINE = Memory; + +CREATE MATERIALIZED VIEW v1 to t1 AS SELECT * FROM t2; +CREATE MATERIALIZED VIEW v2 to t2 AS SELECT * FROM t1; + +INSERT INTO t1 VALUES ('Hello'); -- { serverError 306 } +INSERT INTO t2 VALUES ('World'); -- { serverError 306 } + +DROP TABLE IF EXISTS t; +DROP TABLE IF EXISTS v; +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS v1; +DROP TABLE IF EXISTS v2; From 66b852cf6990a4e336de271f6ff529a1ac708b7d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 2 Nov 2020 23:55:32 +0800 Subject: [PATCH 06/68] Revert "Revert "Try fix IfAggCombinator with NullAggCombinator"" This reverts commit 962a89843cc30df136458e3fe8d95efe3bd28537. --- .../AggregateFunctionCount.cpp | 2 +- .../AggregateFunctionCount.h | 2 +- .../AggregateFunctionIf.cpp | 160 ++++++++++++++++++ src/AggregateFunctions/AggregateFunctionIf.h | 4 + .../AggregateFunctionNull.cpp | 2 +- .../AggregateFunctionWindowFunnel.h | 3 +- src/AggregateFunctions/IAggregateFunction.h | 4 +- ...able_type_with_if_agg_combinator.reference | 3 + ...5_nullable_type_with_if_agg_combinator.sql | 6 + 9 files changed, 181 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.reference create mode 100644 tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql diff --git a/src/AggregateFunctions/AggregateFunctionCount.cpp b/src/AggregateFunctions/AggregateFunctionCount.cpp index 6ea63bedaf0..05824947b87 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.cpp +++ b/src/AggregateFunctions/AggregateFunctionCount.cpp @@ -8,7 +8,7 @@ namespace DB { AggregateFunctionPtr AggregateFunctionCount::getOwnNullAdapter( - const AggregateFunctionPtr &, const DataTypes & types, const Array & params) const + const AggregateFunctionPtr &, const DataTypes & types, const Array & params, const AggregateFunctionProperties & /*properties*/) const { return std::make_shared(types[0], params); } diff --git a/src/AggregateFunctions/AggregateFunctionCount.h b/src/AggregateFunctions/AggregateFunctionCount.h index 29c5de0021c..eb1583df92a 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.h +++ b/src/AggregateFunctions/AggregateFunctionCount.h @@ -69,7 +69,7 @@ public: } AggregateFunctionPtr getOwnNullAdapter( - const AggregateFunctionPtr &, const DataTypes & types, const Array & params) const override; + const AggregateFunctionPtr &, const DataTypes & types, const Array & params, const AggregateFunctionProperties & /*properties*/) const override; }; diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index 19a175de911..47afddaf7ff 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -1,6 +1,7 @@ #include #include #include "registerAggregateFunctions.h" +#include "AggregateFunctionNull.h" namespace DB @@ -8,6 +9,7 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -40,6 +42,164 @@ public: } }; +/** There are two cases: for single argument and variadic. + * Code for single argument is much more efficient. + */ +template +class AggregateFunctionIfNullUnary final + : public AggregateFunctionNullBase> +{ +private: + size_t num_arguments; + + using Base = AggregateFunctionNullBase>; +public: + + String getName() const override + { + return Base::getName() + "If"; + } + + AggregateFunctionIfNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : Base(std::move(nested_function_), arguments, params), num_arguments(arguments.size()) + { + if (num_arguments == 0) + throw Exception("Aggregate function " + getName() + " require at least one argument", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + static inline bool singleFilter(const IColumn ** columns, size_t row_num, size_t num_arguments) + { + const IColumn * filter_column = columns[num_arguments - 1]; + if (const ColumnNullable * nullable_column = typeid_cast(filter_column)) + filter_column = nullable_column->getNestedColumnPtr().get(); + + return assert_cast(*filter_column).getData()[row_num]; + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + const ColumnNullable * column = assert_cast(columns[0]); + const IColumn * nested_column = &column->getNestedColumn(); + if (!column->isNullAt(row_num) && singleFilter(columns, row_num, num_arguments)) + { + this->setFlag(place); + this->nested_function->add(this->nestedPlace(place), &nested_column, row_num, arena); + } + } +}; + +template +class AggregateFunctionIfNullVariadic final + : public AggregateFunctionNullBase> +{ +public: + + String getName() const override + { + return Base::getName() + "If"; + } + + AggregateFunctionIfNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : Base(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size()) + { + if (number_of_arguments == 1) + throw Exception("Logical error: single argument is passed to AggregateFunctionIfNullVariadic", ErrorCodes::LOGICAL_ERROR); + + if (number_of_arguments > MAX_ARGS) + throw Exception("Maximum number of arguments for aggregate function with Nullable types is " + toString(size_t(MAX_ARGS)), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (size_t i = 0; i < number_of_arguments; ++i) + is_nullable[i] = arguments[i]->isNullable(); + } + + static inline bool singleFilter(const IColumn ** columns, size_t row_num, size_t num_arguments) + { + return assert_cast(*columns[num_arguments - 1]).getData()[row_num]; + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + /// This container stores the columns we really pass to the nested function. + const IColumn * nested_columns[number_of_arguments]; + + for (size_t i = 0; i < number_of_arguments; ++i) + { + if (is_nullable[i]) + { + const ColumnNullable & nullable_col = assert_cast(*columns[i]); + if (null_is_skipped && nullable_col.isNullAt(row_num)) + { + /// If at least one column has a null value in the current row, + /// we don't process this row. + return; + } + nested_columns[i] = &nullable_col.getNestedColumn(); + } + else + nested_columns[i] = columns[i]; + } + + if (singleFilter(nested_columns, row_num, number_of_arguments)) + { + this->setFlag(place); + this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); + } + } + +private: + using Base = AggregateFunctionNullBase>; + + enum { MAX_ARGS = 8 }; + size_t number_of_arguments = 0; + std::array is_nullable; /// Plain array is better than std::vector due to one indirection less. +}; + + +AggregateFunctionPtr AggregateFunctionIf::getOwnNullAdapter( + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, + const Array & params, const AggregateFunctionProperties & properties) const +{ + bool return_type_is_nullable = !properties.returns_default_when_only_null && getReturnType()->canBeInsideNullable(); + size_t nullable_size = std::count_if(arguments.begin(), arguments.end(), [](const auto & element) { return element->isNullable(); }); + return_type_is_nullable &= nullable_size != 1 || !arguments.back()->isNullable(); /// If only condition is nullable. we should non-nullable type. + bool serialize_flag = return_type_is_nullable || properties.returns_default_when_only_null; + + if (arguments.size() <= 2 && arguments.front()->isNullable()) + { + if (return_type_is_nullable) + { + return std::make_shared>(nested_func, arguments, params); + } + else + { + if (serialize_flag) + return std::make_shared>(nested_func, arguments, params); + else + return std::make_shared>(nested_func, arguments, params); + } + } + else + { + if (return_type_is_nullable) + { + return std::make_shared>(nested_function, arguments, params); + } + else + { + if (serialize_flag) + return std::make_shared>(nested_function, arguments, params); + else + return std::make_shared>(nested_function, arguments, params); + } + } +} + void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory & factory) { factory.registerCombinator(std::make_shared()); diff --git a/src/AggregateFunctions/AggregateFunctionIf.h b/src/AggregateFunctions/AggregateFunctionIf.h index f04450c9142..d5d2b9be0dd 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.h +++ b/src/AggregateFunctions/AggregateFunctionIf.h @@ -109,6 +109,10 @@ public: { return nested_func->isState(); } + + AggregateFunctionPtr getOwnNullAdapter( + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, + const Array & params, const AggregateFunctionProperties & properties) const override; }; } diff --git a/src/AggregateFunctions/AggregateFunctionNull.cpp b/src/AggregateFunctions/AggregateFunctionNull.cpp index 5e0d6ee6e21..f584ae1f34c 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.cpp +++ b/src/AggregateFunctions/AggregateFunctionNull.cpp @@ -72,7 +72,7 @@ public: assert(nested_function); - if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params)) + if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params, properties)) return adapter; /// If applied to aggregate function with -State combinator, we apply -Null combinator to it's nested_function instead of itself. diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index 3297819a9ff..fe45fec4b76 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -241,7 +241,8 @@ public: } AggregateFunctionPtr getOwnNullAdapter( - const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params) const override + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, + const AggregateFunctionProperties & /*properties*/) const override { return std::make_shared>(nested_function, arguments, params); } diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 4f9552d2345..b5a15eb8cbe 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -33,6 +33,7 @@ using ConstAggregateDataPtr = const char *; class IAggregateFunction; using AggregateFunctionPtr = std::shared_ptr; +struct AggregateFunctionProperties; /** Aggregate functions interface. * Instances of classes with this interface do not contain the data itself for aggregation, @@ -185,7 +186,8 @@ public: * arguments and params are for nested_function. */ virtual AggregateFunctionPtr getOwnNullAdapter( - const AggregateFunctionPtr & /*nested_function*/, const DataTypes & /*arguments*/, const Array & /*params*/) const + const AggregateFunctionPtr & /*nested_function*/, const DataTypes & /*arguments*/, + const Array & /*params*/, const AggregateFunctionProperties & /*properties*/) const { return nullptr; } diff --git a/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.reference b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.reference new file mode 100644 index 00000000000..77f38b722ce --- /dev/null +++ b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.reference @@ -0,0 +1,3 @@ +\N Nullable(UInt8) +\N Nullable(UInt8) +0 UInt8 diff --git a/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql new file mode 100644 index 00000000000..852660117f5 --- /dev/null +++ b/tests/queries/0_stateless/01455_nullable_type_with_if_agg_combinator.sql @@ -0,0 +1,6 @@ +-- Value nullable +SELECT anyIf(CAST(number, 'Nullable(UInt8)'), number = 3) AS a, toTypeName(a) FROM numbers(2); +-- Value and condition nullable +SELECT anyIf(number, number = 3) AS a, toTypeName(a) FROM (SELECT CAST(number, 'Nullable(UInt8)') AS number FROM numbers(2)); +-- Condition nullable +SELECT anyIf(CAST(number, 'UInt8'), number = 3) AS a, toTypeName(a) FROM (SELECT CAST(number, 'Nullable(UInt8)') AS number FROM numbers(2)); From 869578f072157321cd05db1a9842b45be3d4dcec Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 2 Nov 2020 23:57:14 +0800 Subject: [PATCH 07/68] ISSUES-16574 try fix if suffix agg function with remote query --- src/AggregateFunctions/AggregateFunctionIf.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index 47afddaf7ff..d654387e5db 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -59,7 +59,7 @@ public: String getName() const override { - return Base::getName() + "If"; + return Base::getName(); } AggregateFunctionIfNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) @@ -100,7 +100,7 @@ public: String getName() const override { - return Base::getName() + "If"; + return Base::getName(); } AggregateFunctionIfNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) From 34990f7ee4a2f4eaa48fc7795f48d82c36169168 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 6 Nov 2020 21:27:10 +0800 Subject: [PATCH 08/68] ISSUES-16574 trigger CI From 1726fb0f4e43f6b99e191529689382af3125bcbc Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 9 Nov 2020 12:40:28 +0800 Subject: [PATCH 09/68] ISSUES-16574 try fix test failure --- src/AggregateFunctions/AggregateFunctionIf.cpp | 2 +- tests/queries/0_stateless/01556_if_null.reference | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index d654387e5db..276abb90920 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -195,7 +195,7 @@ AggregateFunctionPtr AggregateFunctionIf::getOwnNullAdapter( if (serialize_flag) return std::make_shared>(nested_function, arguments, params); else - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } } } diff --git a/tests/queries/0_stateless/01556_if_null.reference b/tests/queries/0_stateless/01556_if_null.reference index a0c5e7faf40..adc63ecf47b 100644 --- a/tests/queries/0_stateless/01556_if_null.reference +++ b/tests/queries/0_stateless/01556_if_null.reference @@ -1 +1 @@ -([1],[5]) 4 4 +([1],[4]) 4 4 From 9867387a5f7c1c9afd733913c81bb5b31e631c95 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 10 Nov 2020 10:16:46 +0800 Subject: [PATCH 10/68] Fix crash in agg empty arglist --- src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h | 2 +- .../0_stateless/01560_crash_in_agg_empty_arglist.reference | 0 tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.sql | 1 + 3 files changed, 2 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.reference create mode 100644 tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.sql diff --git a/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h b/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h index 6b903ec45cf..3a7efe3d6b9 100644 --- a/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h +++ b/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h @@ -104,7 +104,7 @@ public: KeepAggregateFunctionVisitor(keep_data).visit(function_node->arguments); /// Place argument of an aggregate function instead of function - if (!keep_aggregator) + if (!keep_aggregator && !function_node->arguments->children.empty()) { String alias = function_node->alias; ast = (function_node->arguments->children[0])->clone(); diff --git a/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.reference b/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.sql b/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.sql new file mode 100644 index 00000000000..951cb3ca81b --- /dev/null +++ b/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.sql @@ -0,0 +1 @@ +SELECT any() as t, substring(query, 1, 70) AS query, avg(memory_usage) usage, count() count FROM system.query_log WHERE event_date >= toDate(1604295323) AND event_time >= toDateTime(1604295323) AND type in (1,2,3,4) and initial_user in ('') and('all' = 'all' or(positionCaseInsensitive(query, 'all') = 1)) GROUP BY query ORDER BY usage desc LIMIT 5; -- { serverError 42 } From b3a061443b69f0e960e053f53c81dee0839fd796 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 10 Nov 2020 21:09:05 +0800 Subject: [PATCH 11/68] ISSUES-16574 trigger CI From 3f086deb91112f1fcea537db44341cd8b9356a81 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 11 Nov 2020 02:42:16 +0300 Subject: [PATCH 12/68] Update 01560_crash_in_agg_empty_arglist.sql --- .../queries/0_stateless/01560_crash_in_agg_empty_arglist.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.sql b/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.sql index 951cb3ca81b..57dec2253d6 100644 --- a/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.sql +++ b/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.sql @@ -1 +1,5 @@ +-- make sure the system.query_log table is created +SELECT 1; +SYSTEM FLUSH LOGS; + SELECT any() as t, substring(query, 1, 70) AS query, avg(memory_usage) usage, count() count FROM system.query_log WHERE event_date >= toDate(1604295323) AND event_time >= toDateTime(1604295323) AND type in (1,2,3,4) and initial_user in ('') and('all' = 'all' or(positionCaseInsensitive(query, 'all') = 1)) GROUP BY query ORDER BY usage desc LIMIT 5; -- { serverError 42 } From d907afa77e2967d5588bbe6f5544d2d8f97d076f Mon Sep 17 00:00:00 2001 From: taichong Date: Tue, 27 Oct 2020 19:17:35 +0800 Subject: [PATCH 13/68] throw exception when mysql sync user privs error. --- src/Common/ErrorCodes.cpp | 1 + src/Databases/MySQL/MaterializeMetadata.cpp | 45 ++++++++++++++++ .../MySQL/MaterializeMySQLSyncThread.cpp | 31 +++++++++-- .../MySQL/MaterializeMySQLSyncThread.h | 5 ++ .../materialize_with_ddl.py | 53 ++++++++++++++++++- .../test_materialize_mysql_database/test.py | 9 ++++ 6 files changed, 139 insertions(+), 5 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f069b27827e..7093ed773fa 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -519,6 +519,7 @@ M(550, CONDITIONAL_TREE_PARENT_NOT_FOUND) \ M(551, ILLEGAL_PROJECTION_MANIPULATOR) \ M(552, UNRECOGNIZED_ARGUMENTS) \ + M(553, SYNC_MYSQL_USER_ACCESS_ERROR)\ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 3c5bfdec594..f672e5fb24f 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -19,6 +19,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int SYNC_MYSQL_USER_ACCESS_ERR; } static std::unordered_map fetchTablesCreateQuery( @@ -64,6 +65,7 @@ static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entr return tables_in_db; } + void MaterializeMetadata::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection) { Block header{ @@ -105,6 +107,47 @@ static Block getShowMasterLogHeader(const String & mysql_version) }; } +static bool rightSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, std::ostream & out) +{ + Block sync_user_privs_header{ + {std::make_shared(), "current_user_grants"} + }; + + String grants_query, sub_privs; + MySQLBlockInputStream input(connection, "SHOW GRANTS FOR CURRENT_USER();", sync_user_privs_header, DEFAULT_BLOCK_SIZE); + while (Block block = input.read()) + { + for (size_t index = 0; index < block.rows(); ++index) + { + out << (*block.getByPosition(0).column)[index].safeGet() + "; "; + grants_query = (*block.getByPosition(0).column)[index].safeGet(); + sub_privs = grants_query.substr(0, grants_query.find(" ON ")); + if (sub_privs.find("ALL PRIVILEGES") == std::string::npos) + { + if ((sub_privs.find("RELOAD") != std::string::npos and + sub_privs.find("REPLICATION SLAVE") != std::string::npos and + sub_privs.find("REPLICATION CLIENT") != std::string::npos)) + return true; + } + else + { + return true; + } + } + } + return false; +} + +static void rightSyncUserPriv(mysqlxx::PoolWithFailover::Entry & connection) +{ + std::stringstream out; + if (!rightSyncUserPrivImpl(connection, out)) + throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " + "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " + "and SELECT PRIVILEGE on MySQL Database." + "But the SYNC USER grant query is: " + out.str(), ErrorCodes::SYNC_MYSQL_USER_ACCESS_ERR); +} + bool MaterializeMetadata::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection, const String & mysql_version) const { MySQLBlockInputStream input(connection, "SHOW MASTER LOGS", getShowMasterLogHeader(mysql_version), DEFAULT_BLOCK_SIZE); @@ -167,6 +210,8 @@ MaterializeMetadata::MaterializeMetadata( const String & database, bool & opened_transaction, const String & mysql_version) : persistent_path(path_) { + rightSyncUserPriv(connection); + if (Poco::File(persistent_path).exists()) { ReadBufferFromFile in(persistent_path, DBMS_DEFAULT_BUFFER_SIZE); diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 2a3de25c24f..d5d034eaa8a 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -5,7 +5,6 @@ #if USE_MYSQL #include - # include # include # include @@ -34,6 +33,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int ILLEGAL_MYSQL_VARIABLE; + extern const int SYNC_MYSQL_USER_ACCESS_ERR; + extern const int UNKNOWN_DATABASE; } static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; @@ -215,10 +216,32 @@ void MaterializeMySQLSyncThread::stopSynchronization() void MaterializeMySQLSyncThread::startSynchronization() { - const auto & mysql_server_version = checkVariableAndGetVersion(pool.get()); + try{ + const auto & mysql_server_version = checkVariableAndGetVersion(pool.get()); - background_thread_pool = std::make_unique( - [this, mysql_server_version = mysql_server_version]() { synchronization(mysql_server_version); }); + background_thread_pool = std::make_unique( + [this, mysql_server_version = mysql_server_version]() { synchronization(mysql_server_version); }); + } + catch (...) + { + try + { + throw; + } + catch (mysqlxx::ConnectionFailed & e) + { + if (e.errnum() == ER_ACCESS_DENIED_ERROR + || e.errnum() == ER_DBACCESS_DENIED_ERROR) + throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " + "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " + "and SELECT PRIVILEGE on Database " + mysql_database_name + , ErrorCodes::SYNC_MYSQL_USER_ACCESS_ERR); + else if (e.errnum() == ER_BAD_DB_ERROR) + throw Exception("Unknown database '" + mysql_database_name + "' on MySQL", ErrorCodes::UNKNOWN_DATABASE); + else + throw; + } + } } static inline void cleanOutdatedTables(const String & database_name, const Context & context) diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index 9a0df4823e5..c1f1fc5a391 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -20,6 +20,7 @@ # include # include + namespace DB { @@ -63,6 +64,10 @@ private: MaterializeMySQLSettings * settings; String query_prefix; + const int ER_ACCESS_DENIED_ERROR = 1045; + const int ER_DBACCESS_DENIED_ERROR = 1044; + const int ER_BAD_DB_ERROR = 1049; + struct Buffers { String database; diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index b97a1563212..534d7ff895e 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -2,6 +2,8 @@ import time import pymysql.cursors +import pytest +from helpers.client import QueryRuntimeException def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seconds=3): lastest_result = '' @@ -164,7 +166,6 @@ def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node, serv clickhouse_node.query("DROP DATABASE test_database") mysql_node.query("DROP DATABASE test_database") - def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created @@ -461,3 +462,53 @@ def select_without_columns(clickhouse_node, mysql_node, service_name): clickhouse_node.query("DROP VIEW v") clickhouse_node.query("DROP DATABASE db") mysql_node.query("DROP DATABASE db") + + +def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (4), (5), (6);") + + mysql_node.query("CREATE USER 'test'@'%' IDENTIFIED BY '123'") + + with pytest.raises(QueryRuntimeException) as exception: + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + + assert 'MySQL SYNC USER ACCESS ERR:' in str(exception.value) + assert "test_database" not in clickhouse_node.query("SHOW DATABASES") + + mysql_node.query("GRANT SELECT ON test_database.* TO 'test'@'%'") + + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") + clickhouse_node.query("DROP DATABASE test_database") + + mysql_node.query("GRANT REPLICATION CLIENT, RELOAD ON *.* TO 'test'@'%'") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") + clickhouse_node.query("DROP DATABASE test_database") + + mysql_node.query("GRANT REPLICATION SLAVE ON *.* TO 'test'@'%'") + + # wait mysql grant done + time.sleep(15) + + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + + check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 5, 5) + mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") + check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") + + clickhouse_node.query("DROP DATABASE test_database;") + mysql_node.query("DROP DATABASE test_database;") + mysql_node.query("DROP USER 'test'@'%';") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 6df831e1e7d..04053e09ee1 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -153,3 +153,12 @@ def test_select_without_columns_5_7(started_cluster, started_mysql_5_7): def test_select_without_columns_8_0(started_cluster, started_mysql_8_0): materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_8_0, "mysql8_0") + + +def test_materialize_database_err_sync_user_privs_5_7(started_cluster, started_mysql_5_7): + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + + +def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0): + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + From 2f6510d75ba539c0d20d0dd60b9c402ea09cf3b4 Mon Sep 17 00:00:00 2001 From: taichong Date: Thu, 29 Oct 2020 16:09:28 +0800 Subject: [PATCH 14/68] try to fix ci failed --- .../materialize_with_ddl.py | 59 +++++++++---------- .../test_materialize_mysql_database/test.py | 5 ++ 2 files changed, 34 insertions(+), 30 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 534d7ff895e..a4f1884c546 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -470,36 +470,10 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (4), (5), (6);") mysql_node.query("CREATE USER 'test'@'%' IDENTIFIED BY '123'") - - with pytest.raises(QueryRuntimeException) as exception: - clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( - service_name)) - - assert 'MySQL SYNC USER ACCESS ERR:' in str(exception.value) - assert "test_database" not in clickhouse_node.query("SHOW DATABASES") - + mysql_node.query("GRANT REPLICATION SLAVE, REPLICATION CLIENT, RELOAD ON *.* TO 'test'@'%'") mysql_node.query("GRANT SELECT ON test_database.* TO 'test'@'%'") - - clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( - service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") - clickhouse_node.query("DROP DATABASE test_database") - - mysql_node.query("GRANT REPLICATION CLIENT, RELOAD ON *.* TO 'test'@'%'") - clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( - service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") - clickhouse_node.query("DROP DATABASE test_database") - - mysql_node.query("GRANT REPLICATION SLAVE ON *.* TO 'test'@'%'") - - # wait mysql grant done - time.sleep(15) + print('Eason test') + mysql_node.result("SHOW GRANTS FOR 'test'@'%'") clickhouse_node.query( "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( @@ -508,7 +482,32 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 5, 5) mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") - clickhouse_node.query("DROP DATABASE test_database;") + + mysql_node.query("REVOKE REPLICATION SLAVE ON *.* FROM 'test'@'%'") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") + clickhouse_node.query("DROP DATABASE test_database") + + mysql_node.query("REVOKE REPLICATION CLIENT, RELOAD ON *.* FROM 'test'@'%'") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + assert "test_database" in clickhouse_node.query("SHOW DATABASES") + assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") + clickhouse_node.query("DROP DATABASE test_database") + + mysql_node.query("REVOKE SELECT ON test_database.* FROM 'test'@'%'") + with pytest.raises(QueryRuntimeException) as exception: + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + service_name)) + + assert 'MySQL SYNC USER ACCESS ERR:' in str(exception.value) + assert "test_database" not in clickhouse_node.query("SHOW DATABASES") + mysql_node.query("DROP DATABASE test_database;") mysql_node.query("DROP USER 'test'@'%';") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 04053e09ee1..267621812ad 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -41,6 +41,11 @@ class MySQLNodeInstance: with self.alloc_connection().cursor() as cursor: cursor.execute(execution_query) + def result(self, execution_query): + with self.alloc_connection().cursor() as cursor: + cursor.execute(execution_query) + print(cursor.fetchone()) + def close(self): if self.mysql_connection is not None: self.mysql_connection.close() From aecb9716e1c2175b593f7f3ff83ef7d370bffc97 Mon Sep 17 00:00:00 2001 From: taichong Date: Fri, 30 Oct 2020 09:31:45 +0800 Subject: [PATCH 15/68] print mysql result for test --- .../materialize_with_ddl.py | 10 +++---- .../test_materialize_mysql_database/test.py | 26 ++++++++++++++++--- 2 files changed, 27 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index a4f1884c546..6711b36ca13 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -469,16 +469,16 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (4), (5), (6);") - mysql_node.query("CREATE USER 'test'@'%' IDENTIFIED BY '123'") - mysql_node.query("GRANT REPLICATION SLAVE, REPLICATION CLIENT, RELOAD ON *.* TO 'test'@'%'") - mysql_node.query("GRANT SELECT ON test_database.* TO 'test'@'%'") - print('Eason test') - mysql_node.result("SHOW GRANTS FOR 'test'@'%'") + mysql_node.result("SHOW GRANTS FOR 'test'@'%';") clickhouse_node.query( "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( service_name)) + # wait MaterializeMySQL read binlog events + time.sleep(90) + + assert "test_table_1" in clickhouse_node.query("SHOW TABLES FROM test_database") check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 5, 5) mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 267621812ad..72310ea30cb 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -41,10 +41,16 @@ class MySQLNodeInstance: with self.alloc_connection().cursor() as cursor: cursor.execute(execution_query) + def create_min_priv_user(self): + self.query("CREATE USER 'test'@'%' IDENTIFIED BY '123'") + self.query("GRANT REPLICATION SLAVE, REPLICATION CLIENT, RELOAD ON *.* TO 'test'@'%'") + self.query("GRANT SELECT ON test_database.* TO 'test'@'%'") + def result(self, execution_query): with self.alloc_connection().cursor() as cursor: - cursor.execute(execution_query) - print(cursor.fetchone()) + result = cursor.execute(execution_query) + if result is not None: + print(cursor.fetchall()) def close(self): if self.mysql_connection is not None: @@ -56,6 +62,8 @@ class MySQLNodeInstance: try: self.alloc_connection() print("Mysql Started") + self.create_min_priv_user() + print("min priv user created") return except Exception as ex: print("Can't connect to MySQL " + str(ex)) @@ -161,9 +169,19 @@ def test_select_without_columns_8_0(started_cluster, started_mysql_8_0): def test_materialize_database_err_sync_user_privs_5_7(started_cluster, started_mysql_5_7): - materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + try: + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + except: + print((clickhouse_node.query( + "select '\n', thread_id, query_id, arrayStringConcat(arrayMap(x -> concat(demangle(addressToSymbol(x)), '\n ', addressToLine(x)), trace), '\n') AS sym from system.stack_trace format TSVRaw"))) + raise def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0): - materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + try: + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + except: + print((clickhouse_node.query( + "select '\n', thread_id, query_id, arrayStringConcat(arrayMap(x -> concat(demangle(addressToSymbol(x)), '\n ', addressToLine(x)), trace), '\n') AS sym from system.stack_trace format TSVRaw"))) + raise From 5c1a01897ac79e23e5c5224605a43270750cfd4d Mon Sep 17 00:00:00 2001 From: taichong Date: Tue, 3 Nov 2020 11:12:05 +0800 Subject: [PATCH 16/68] Revise the code according to the review --- src/Databases/MySQL/MaterializeMetadata.cpp | 11 ++++++----- src/Databases/MySQL/MaterializeMySQLSyncThread.cpp | 3 ++- src/Databases/MySQL/MaterializeMySQLSyncThread.h | 2 ++ .../materialize_with_ddl.py | 5 +---- 4 files changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index f672e5fb24f..01a1493fa22 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -107,9 +107,10 @@ static Block getShowMasterLogHeader(const String & mysql_version) }; } -static bool rightSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, std::ostream & out) +static bool checkSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, std::ostream & out) { - Block sync_user_privs_header{ + Block sync_user_privs_header + { {std::make_shared(), "current_user_grants"} }; @@ -138,10 +139,10 @@ static bool rightSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, return false; } -static void rightSyncUserPriv(mysqlxx::PoolWithFailover::Entry & connection) +static void checkSyncUserPriv(mysqlxx::PoolWithFailover::Entry & connection) { std::stringstream out; - if (!rightSyncUserPrivImpl(connection, out)) + if (!checkSyncUserPrivImpl(connection, out)) throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " "and SELECT PRIVILEGE on MySQL Database." @@ -210,7 +211,7 @@ MaterializeMetadata::MaterializeMetadata( const String & database, bool & opened_transaction, const String & mysql_version) : persistent_path(path_) { - rightSyncUserPriv(connection); + checkSyncUserPriv(connection); if (Poco::File(persistent_path).exists()) { diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index d5d034eaa8a..f76e6b77448 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -216,7 +216,8 @@ void MaterializeMySQLSyncThread::stopSynchronization() void MaterializeMySQLSyncThread::startSynchronization() { - try{ + try + { const auto & mysql_server_version = checkVariableAndGetVersion(pool.get()); background_thread_pool = std::make_unique( diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.h b/src/Databases/MySQL/MaterializeMySQLSyncThread.h index c1f1fc5a391..323ae5beb80 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.h @@ -64,6 +64,8 @@ private: MaterializeMySQLSettings * settings; String query_prefix; + // USE MySQL ERROR CODE: + // https://dev.mysql.com/doc/mysql-errors/5.7/en/server-error-reference.html const int ER_ACCESS_DENIED_ERROR = 1045; const int ER_DBACCESS_DENIED_ERROR = 1044; const int ER_BAD_DB_ERROR = 1049; diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 6711b36ca13..8c9256cb1c7 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -476,10 +476,7 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n service_name)) # wait MaterializeMySQL read binlog events - time.sleep(90) - - assert "test_table_1" in clickhouse_node.query("SHOW TABLES FROM test_database") - check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 5, 5) + check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 30, 5) mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") clickhouse_node.query("DROP DATABASE test_database;") From 64acfea79c0e662a842a01ccc5240a2b86c01956 Mon Sep 17 00:00:00 2001 From: taichong Date: Tue, 27 Oct 2020 19:17:35 +0800 Subject: [PATCH 17/68] throw exception when mysql sync user privs error. --- tests/integration/test_materialize_mysql_database/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 72310ea30cb..543ff7e1bdd 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -184,4 +184,3 @@ def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_m print((clickhouse_node.query( "select '\n', thread_id, query_id, arrayStringConcat(arrayMap(x -> concat(demangle(addressToSymbol(x)), '\n ', addressToLine(x)), trace), '\n') AS sym from system.stack_trace format TSVRaw"))) raise - From e5da2d4086d33505ad3b181baba744d01e1191bf Mon Sep 17 00:00:00 2001 From: taichong Date: Fri, 30 Oct 2020 09:31:45 +0800 Subject: [PATCH 18/68] print mysql result for test --- .../test_materialize_mysql_database/materialize_with_ddl.py | 1 + tests/integration/test_materialize_mysql_database/test.py | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 8c9256cb1c7..9ea4cfbda00 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -477,6 +477,7 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n # wait MaterializeMySQL read binlog events check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 30, 5) + print mysql result for test mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") clickhouse_node.query("DROP DATABASE test_database;") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 543ff7e1bdd..72310ea30cb 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -184,3 +184,4 @@ def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_m print((clickhouse_node.query( "select '\n', thread_id, query_id, arrayStringConcat(arrayMap(x -> concat(demangle(addressToSymbol(x)), '\n ', addressToLine(x)), trace), '\n') AS sym from system.stack_trace format TSVRaw"))) raise + From 6c8b5b573bf0d670145424c776404f242a8d84b4 Mon Sep 17 00:00:00 2001 From: taichong Date: Tue, 3 Nov 2020 13:22:39 +0800 Subject: [PATCH 19/68] modify test for Integration Tests Flaky Check --- .../test_materialize_mysql_database/materialize_with_ddl.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 9ea4cfbda00..dc9eca07b27 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -5,7 +5,7 @@ import pymysql.cursors import pytest from helpers.client import QueryRuntimeException -def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seconds=3): +def check_query(clickhouse_node, query, result_set, retry_count=5, interval_seconds=30): lastest_result = '' for index in range(retry_count): lastest_result = clickhouse_node.query(query) @@ -477,7 +477,6 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n # wait MaterializeMySQL read binlog events check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 30, 5) - print mysql result for test mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") clickhouse_node.query("DROP DATABASE test_database;") From f4f437916873b6c73be12e4f9fe2f7dff4e61c09 Mon Sep 17 00:00:00 2001 From: taichong Date: Tue, 3 Nov 2020 18:47:32 +0800 Subject: [PATCH 20/68] modify test case add precheck --- .../materialize_with_ddl.py | 28 ++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index dc9eca07b27..52119add336 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -5,7 +5,7 @@ import pymysql.cursors import pytest from helpers.client import QueryRuntimeException -def check_query(clickhouse_node, query, result_set, retry_count=5, interval_seconds=30): +def check_query(clickhouse_node, query, result_set, retry_count=60, interval_seconds=3): lastest_result = '' for index in range(retry_count): lastest_result = clickhouse_node.query(query) @@ -20,6 +20,8 @@ def check_query(clickhouse_node, query, result_set, retry_count=5, interval_seco def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created @@ -102,6 +104,8 @@ def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_nam def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") @@ -134,6 +138,8 @@ def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_ def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") @@ -167,6 +173,8 @@ def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node, serv mysql_node.query("DROP DATABASE test_database") def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") @@ -195,6 +203,8 @@ def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node, se def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") @@ -215,6 +225,8 @@ def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, se def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") @@ -256,6 +268,8 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") @@ -288,6 +302,8 @@ def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_nod def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # maybe should test rename primary key? @@ -323,6 +339,8 @@ def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_n def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") # maybe should test rename primary key? @@ -367,6 +385,8 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n # pass def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") @@ -402,6 +422,8 @@ def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_no def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database") mysql_node.query("RESET MASTER") @@ -434,6 +456,8 @@ def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name mysql_node.query("DROP DATABASE test_database") def select_without_columns(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS db") + clickhouse_node.query("DROP DATABASE IF EXISTS db") mysql_node.query("CREATE DATABASE db") mysql_node.query("CREATE TABLE db.t (a INT PRIMARY KEY, b INT)") clickhouse_node.query( @@ -465,6 +489,8 @@ def select_without_columns(clickhouse_node, mysql_node, service_name): def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (4), (5), (6);") From e2a9fe44b021d4b001fcd5ed6f744cc700fc315b Mon Sep 17 00:00:00 2001 From: taichong Date: Wed, 11 Nov 2020 23:37:44 +0800 Subject: [PATCH 21/68] modify test error --- src/Databases/MySQL/MaterializeMetadata.cpp | 4 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 4 +- .../materialize_with_ddl.py | 50 +++++++++---------- .../test_materialize_mysql_database/test.py | 13 +++-- 4 files changed, 37 insertions(+), 34 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 01a1493fa22..fdc9d7aa9ea 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -19,7 +19,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int SYNC_MYSQL_USER_ACCESS_ERR; + extern const int SYNC_MYSQL_USER_ACCESS_ERROR; } static std::unordered_map fetchTablesCreateQuery( @@ -146,7 +146,7 @@ static void checkSyncUserPriv(mysqlxx::PoolWithFailover::Entry & connection) throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " "and SELECT PRIVILEGE on MySQL Database." - "But the SYNC USER grant query is: " + out.str(), ErrorCodes::SYNC_MYSQL_USER_ACCESS_ERR); + "But the SYNC USER grant query is: " + out.str(), ErrorCodes::SYNC_MYSQL_USER_ACCESS_ERROR); } bool MaterializeMetadata::checkBinlogFileExists(mysqlxx::PoolWithFailover::Entry & connection, const String & mysql_version) const diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index f76e6b77448..f2a71bb16a5 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -33,7 +33,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int ILLEGAL_MYSQL_VARIABLE; - extern const int SYNC_MYSQL_USER_ACCESS_ERR; + extern const int SYNC_MYSQL_USER_ACCESS_ERROR; extern const int UNKNOWN_DATABASE; } @@ -236,7 +236,7 @@ void MaterializeMySQLSyncThread::startSynchronization() throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " "and SELECT PRIVILEGE on Database " + mysql_database_name - , ErrorCodes::SYNC_MYSQL_USER_ACCESS_ERR); + , ErrorCodes::SYNC_MYSQL_USER_ACCESS_ERROR); else if (e.errnum() == ER_BAD_DB_ERROR) throw Exception("Unknown database '" + mysql_database_name + "' on MySQL", ErrorCodes::UNKNOWN_DATABASE); else diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 52119add336..c248caf40a4 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -489,48 +489,48 @@ def select_without_columns(clickhouse_node, mysql_node, service_name): def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (4), (5), (6);") + mysql_node.query("DROP DATABASE IF EXISTS priv_err_db") + clickhouse_node.query("DROP DATABASE IF EXISTS priv_err_db") + mysql_node.query("CREATE DATABASE priv_err_db DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE priv_err_db.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO priv_err_db.test_table_1 VALUES(1);") mysql_node.result("SHOW GRANTS FOR 'test'@'%';") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + "CREATE DATABASE priv_err_db ENGINE = MaterializeMySQL('{}:3306', 'priv_err_db', 'test', '123')".format( service_name)) # wait MaterializeMySQL read binlog events - check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "6\n", 30, 5) - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(7);") - check_query(clickhouse_node, "SELECT count() FROM test_database.test_table_1 FORMAT TSV", "7\n") - clickhouse_node.query("DROP DATABASE test_database;") + check_query(clickhouse_node, "SELECT count() FROM priv_err_db.test_table_1 FORMAT TSV", "1\n", 30, 5) + mysql_node.query("INSERT INTO priv_err_db.test_table_1 VALUES(2);") + check_query(clickhouse_node, "SELECT count() FROM priv_err_db.test_table_1 FORMAT TSV", "2\n") + clickhouse_node.query("DROP DATABASE priv_err_db;") mysql_node.query("REVOKE REPLICATION SLAVE ON *.* FROM 'test'@'%'") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + "CREATE DATABASE priv_err_db ENGINE = MaterializeMySQL('{}:3306', 'priv_err_db', 'test', '123')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") - clickhouse_node.query("DROP DATABASE test_database") + assert "priv_err_db" in clickhouse_node.query("SHOW DATABASES") + assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM priv_err_db") + clickhouse_node.query("DROP DATABASE priv_err_db") mysql_node.query("REVOKE REPLICATION CLIENT, RELOAD ON *.* FROM 'test'@'%'") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( + "CREATE DATABASE priv_err_db ENGINE = MaterializeMySQL('{}:3306', 'priv_err_db', 'test', '123')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM test_database") - clickhouse_node.query("DROP DATABASE test_database") + assert "priv_err_db" in clickhouse_node.query("SHOW DATABASES") + assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM priv_err_db") + clickhouse_node.query("DETACH DATABASE priv_err_db") + + mysql_node.query("REVOKE SELECT ON priv_err_db.* FROM 'test'@'%'") + time.sleep(3) - mysql_node.query("REVOKE SELECT ON test_database.* FROM 'test'@'%'") with pytest.raises(QueryRuntimeException) as exception: - clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'test', '123')".format( - service_name)) + clickhouse_node.query("ATTACH DATABASE priv_err_db") assert 'MySQL SYNC USER ACCESS ERR:' in str(exception.value) - assert "test_database" not in clickhouse_node.query("SHOW DATABASES") + assert "priv_err_db" not in clickhouse_node.query("SHOW DATABASES") - mysql_node.query("DROP DATABASE test_database;") - mysql_node.query("DROP USER 'test'@'%';") + mysql_node.query("DROP DATABASE priv_err_db;") + mysql_node.grant_min_priv_for_user("test") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 72310ea30cb..3224b3fd19e 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -41,10 +41,13 @@ class MySQLNodeInstance: with self.alloc_connection().cursor() as cursor: cursor.execute(execution_query) - def create_min_priv_user(self): - self.query("CREATE USER 'test'@'%' IDENTIFIED BY '123'") - self.query("GRANT REPLICATION SLAVE, REPLICATION CLIENT, RELOAD ON *.* TO 'test'@'%'") - self.query("GRANT SELECT ON test_database.* TO 'test'@'%'") + def create_min_priv_user(self, user, password): + self.query("CREATE USER '" + user + "'@'%' IDENTIFIED BY '" + password + "'") + self.grant_min_priv_for_user(user) + + def grant_min_priv_for_user(self, user, db='test_database'): + self.query("GRANT REPLICATION SLAVE, REPLICATION CLIENT, RELOAD ON *.* TO '" + user + "'@'%'") + self.query("GRANT SELECT ON " + db + ".* TO '" + user + "'@'%'") def result(self, execution_query): with self.alloc_connection().cursor() as cursor: @@ -62,7 +65,7 @@ class MySQLNodeInstance: try: self.alloc_connection() print("Mysql Started") - self.create_min_priv_user() + self.create_min_priv_user("test", "123") print("min priv user created") return except Exception as ex: From ed88379646d12664415e08bb72d56de6a65edb6c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 12 Nov 2020 03:22:43 +0300 Subject: [PATCH 22/68] Update 01560_crash_in_agg_empty_arglist.reference --- .../0_stateless/01560_crash_in_agg_empty_arglist.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.reference b/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.reference index e69de29bb2d..f33dfa25aa1 100644 --- a/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.reference +++ b/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.reference @@ -0,0 +1 @@ +1 From 091df85cdc8f279376315a7587867fff40d49417 Mon Sep 17 00:00:00 2001 From: taichong Date: Thu, 12 Nov 2020 09:49:12 +0800 Subject: [PATCH 23/68] modify test db name --- tests/integration/test_materialize_mysql_database/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 3224b3fd19e..3befd708b2e 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -45,7 +45,7 @@ class MySQLNodeInstance: self.query("CREATE USER '" + user + "'@'%' IDENTIFIED BY '" + password + "'") self.grant_min_priv_for_user(user) - def grant_min_priv_for_user(self, user, db='test_database'): + def grant_min_priv_for_user(self, user, db='priv_err_db'): self.query("GRANT REPLICATION SLAVE, REPLICATION CLIENT, RELOAD ON *.* TO '" + user + "'@'%'") self.query("GRANT SELECT ON " + db + ".* TO '" + user + "'@'%'") From 010e12eb4b15f936c964358b9961284bf06cf40e Mon Sep 17 00:00:00 2001 From: taichong Date: Thu, 12 Nov 2020 13:49:56 +0800 Subject: [PATCH 24/68] Replace std::*stringstreams --- src/Databases/MySQL/MaterializeMetadata.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index fdc9d7aa9ea..eed4bddb588 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB { @@ -107,7 +108,7 @@ static Block getShowMasterLogHeader(const String & mysql_version) }; } -static bool checkSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, std::ostream & out) +static bool checkSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, WriteBuffer & out) { Block sync_user_privs_header { @@ -120,8 +121,8 @@ static bool checkSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, { for (size_t index = 0; index < block.rows(); ++index) { - out << (*block.getByPosition(0).column)[index].safeGet() + "; "; grants_query = (*block.getByPosition(0).column)[index].safeGet(); + out << grants_query << "; "; sub_privs = grants_query.substr(0, grants_query.find(" ON ")); if (sub_privs.find("ALL PRIVILEGES") == std::string::npos) { @@ -141,7 +142,8 @@ static bool checkSyncUserPrivImpl(mysqlxx::PoolWithFailover::Entry & connection, static void checkSyncUserPriv(mysqlxx::PoolWithFailover::Entry & connection) { - std::stringstream out; + WriteBufferFromOwnString out; + if (!checkSyncUserPrivImpl(connection, out)) throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " From bbcd10f415da4d5000698742a7a0eef082071bad Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Nov 2020 13:58:14 +0300 Subject: [PATCH 25/68] Don't visit functions with multiple arguments in MonotonicityCheckMatcher --- src/Interpreters/MonotonicityCheckVisitor.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index 405ec1cb479..0b3660eb85c 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -131,8 +131,11 @@ public: data.reject(); } - static bool needChildVisit(const ASTPtr &, const ASTPtr &) + static bool needChildVisit(const ASTPtr & parent, const ASTPtr &) { + if (const auto * func = typeid_cast(parent.get())) + return func->children.size() < 2; + return true; } }; From e27afbee2a5770ab0be2cec959aa6b1635b86cce Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Nov 2020 15:12:52 +0300 Subject: [PATCH 26/68] Don't visit functions with multiple arguments in MonotonicityCheckMatcher --- src/Interpreters/MonotonicityCheckVisitor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index 0b3660eb85c..1e29964f04b 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -134,7 +134,7 @@ public: static bool needChildVisit(const ASTPtr & parent, const ASTPtr &) { if (const auto * func = typeid_cast(parent.get())) - return func->children.size() < 2; + return func->arguments->children.size() < 2; return true; } From 1fec1de199398daa228840eac1cd7d805d2890da Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Nov 2020 15:39:35 +0300 Subject: [PATCH 27/68] Sdded test --- ...tonicity_check_multiple_args_bug.reference | 4 ++++ ...0_monotonicity_check_multiple_args_bug.sql | 19 +++++++++++++++++++ 2 files changed, 23 insertions(+) create mode 100644 tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference create mode 100644 tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql diff --git a/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference new file mode 100644 index 00000000000..b0a59aa58cc --- /dev/null +++ b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference @@ -0,0 +1,4 @@ +2020-11-12 20.12.1.1 +2020-11-13 20.12.1.1 +2020-11-12 20.12.1.1 +2020-11-13 20.12.1.1 diff --git a/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql new file mode 100644 index 00000000000..d2afd4b29be --- /dev/null +++ b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql @@ -0,0 +1,19 @@ +WITH arrayJoin(range(2)) AS delta +SELECT + toDate(time) + toIntervalDay(delta) AS dt, + version() +FROM +( + SELECT NOW() AS time +) +ORDER BY dt ASC; + +WITH arrayJoin([0, 1]) AS delta +SELECT + toDate(time) + toIntervalDay(delta) AS dt, + version() +FROM +( + SELECT NOW() AS time +) +ORDER BY dt ASC; From fefa2a25480bc54329b57f85eac48132af031198 Mon Sep 17 00:00:00 2001 From: taichong Date: Thu, 12 Nov 2020 22:24:22 +0800 Subject: [PATCH 28/68] add log std out --- .../test_materialize_mysql_database/materialize_with_ddl.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index c248caf40a4..6d69d536bea 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -500,7 +500,8 @@ def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_n clickhouse_node.query( "CREATE DATABASE priv_err_db ENGINE = MaterializeMySQL('{}:3306', 'priv_err_db', 'test', '123')".format( service_name)) - + print("\n=== print log for CI test err ===\n") + print(clickhouse_node.exec_in_container(["bash", "-c", 'cat /var/log/clickhouse-server/clickhouse-server.log'])) # wait MaterializeMySQL read binlog events check_query(clickhouse_node, "SELECT count() FROM priv_err_db.test_table_1 FORMAT TSV", "1\n", 30, 5) mysql_node.query("INSERT INTO priv_err_db.test_table_1 VALUES(2);") From e031e805c44ff2e3b05e105de8970193fc759b24 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Nov 2020 19:04:35 +0300 Subject: [PATCH 29/68] Fix test. --- ...1560_monotonicity_check_multiple_args_bug.reference | 8 ++++---- .../01560_monotonicity_check_multiple_args_bug.sql | 10 ++++------ 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference index b0a59aa58cc..5d632f2f5b8 100644 --- a/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference +++ b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.reference @@ -1,4 +1,4 @@ -2020-11-12 20.12.1.1 -2020-11-13 20.12.1.1 -2020-11-12 20.12.1.1 -2020-11-13 20.12.1.1 +2020-11-12 +2020-11-13 +2020-11-12 +2020-11-13 diff --git a/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql index d2afd4b29be..befc13be8eb 100644 --- a/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql +++ b/tests/queries/0_stateless/01560_monotonicity_check_multiple_args_bug.sql @@ -1,19 +1,17 @@ WITH arrayJoin(range(2)) AS delta SELECT - toDate(time) + toIntervalDay(delta) AS dt, - version() + toDate(time) + toIntervalDay(delta) AS dt FROM ( - SELECT NOW() AS time + SELECT toDateTime('2020.11.12 19:02:04') AS time ) ORDER BY dt ASC; WITH arrayJoin([0, 1]) AS delta SELECT - toDate(time) + toIntervalDay(delta) AS dt, - version() + toDate(time) + toIntervalDay(delta) AS dt FROM ( - SELECT NOW() AS time + SELECT toDateTime('2020.11.12 19:02:04') AS time ) ORDER BY dt ASC; From 26f6ce3fb528a73a0abac7541b6af3380a690b8a Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Thu, 12 Nov 2020 17:48:17 +0100 Subject: [PATCH 30/68] Update Install.cpp --- programs/install/Install.cpp | 66 ++++++++++++++++++++---------------- 1 file changed, 37 insertions(+), 29 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 8290118089c..4574b68b2f6 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -329,14 +329,20 @@ int mainEntryClickHouseInstall(int argc, char ** argv) bool has_password_for_default_user = false; - if (!fs::exists(main_config_file)) + if (!fs::exists(config_d)) { fmt::print("Creating config directory {} that is used for tweaks of main server configuration.\n", config_d.string()); fs::create_directory(config_d); + } + if (!fs::exists(users_d)) + { fmt::print("Creating config directory {} that is used for tweaks of users configuration.\n", users_d.string()); fs::create_directory(users_d); + } + if (!fs::exists(main_config_file)) + { std::string_view main_config_content = getResource("config.xml"); if (main_config_content.empty()) { @@ -349,7 +355,30 @@ int mainEntryClickHouseInstall(int argc, char ** argv) out.sync(); out.finalize(); } + } + else + { + fmt::print("Config file {} already exists, will keep it and extract path info from it.\n", main_config_file.string()); + ConfigProcessor processor(main_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); + ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(processor.processConfig())); + + if (configuration->has("path")) + { + data_path = configuration->getString("path"); + fmt::print("{} has {} as data path.\n", main_config_file.string(), data_path); + } + + if (configuration->has("logger.log")) + { + log_path = fs::path(configuration->getString("logger.log")).remove_filename(); + fmt::print("{} has {} as log path.\n", main_config_file.string(), log_path); + } + } + + + if (!fs::exists(users_config_file)) + { std::string_view users_config_content = getResource("users.xml"); if (users_config_content.empty()) { @@ -365,38 +394,17 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } else { - { - fmt::print("Config file {} already exists, will keep it and extract path info from it.\n", main_config_file.string()); - - ConfigProcessor processor(main_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); - ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(processor.processConfig())); - - if (configuration->has("path")) - { - data_path = configuration->getString("path"); - fmt::print("{} has {} as data path.\n", main_config_file.string(), data_path); - } - - if (configuration->has("logger.log")) - { - log_path = fs::path(configuration->getString("logger.log")).remove_filename(); - fmt::print("{} has {} as log path.\n", main_config_file.string(), log_path); - } - } + fmt::print("Users config file {} already exists, will keep it and extract path info from it.\n", main_config_file.string()); /// Check if password for default user already specified. + ConfigProcessor processor(users_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); + ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(processor.processConfig())); - if (fs::exists(users_config_file)) + if (!configuration->getString("users.default.password", "").empty() + || configuration->getString("users.default.password_sha256_hex", "").empty() + || configuration->getString("users.default.password_double_sha1_hex", "").empty()) { - ConfigProcessor processor(users_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); - ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(processor.processConfig())); - - if (!configuration->getString("users.default.password", "").empty() - || configuration->getString("users.default.password_sha256_hex", "").empty() - || configuration->getString("users.default.password_double_sha1_hex", "").empty()) - { - has_password_for_default_user = true; - } + has_password_for_default_user = true; } } From 385e8f5ee063532b47826a6fd63310cfa5196808 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 12 Nov 2020 19:52:58 +0300 Subject: [PATCH 31/68] Update Install.cpp --- programs/install/Install.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 4574b68b2f6..6cb37c36b41 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -394,7 +394,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } else { - fmt::print("Users config file {} already exists, will keep it and extract path info from it.\n", main_config_file.string()); + fmt::print("Users config file {} already exists, will keep it and extract users info from it.\n", main_config_file.string()); /// Check if password for default user already specified. ConfigProcessor processor(users_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); From 8eff47420bf07f44ac54bfe1a7e7caf18fda038c Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Thu, 12 Nov 2020 18:18:34 +0100 Subject: [PATCH 32/68] Update Connection.cpp --- src/Client/Connection.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 31e88b5d872..a16f35e060f 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -73,6 +73,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) { #if USE_SSL socket = std::make_unique(); + socket->setPeerHostName(host); #else throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; #endif From 8a20133373e1475cac3eb124d605ff91741f3755 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Thu, 12 Nov 2020 18:28:24 +0100 Subject: [PATCH 33/68] style --- programs/install/Install.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 6cb37c36b41..27f4ceec732 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -373,7 +373,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) { log_path = fs::path(configuration->getString("logger.log")).remove_filename(); fmt::print("{} has {} as log path.\n", main_config_file.string(), log_path); - } + } } From 178d8e9b75d154e5a835106dc3c20b8bed3f8f39 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Thu, 12 Nov 2020 22:07:19 +0100 Subject: [PATCH 34/68] Update Connection.cpp --- src/Client/Connection.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index a16f35e060f..2a130f4ee6f 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -73,7 +73,8 @@ void Connection::connect(const ConnectionTimeouts & timeouts) { #if USE_SSL socket = std::make_unique(); - socket->setPeerHostName(host); + auto secure_socket = static_cast(socket.get()); + secure_socket->setPeerHostName(host); #else throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; #endif From c0940f2ebd487f7f2001aa258a2912a88d6bd68a Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 13 Nov 2020 02:27:18 +0300 Subject: [PATCH 35/68] done --- src/Core/Defines.h | 2 +- src/Interpreters/ActionsVisitor.cpp | 2 +- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/executeQuery.cpp | 4 ++-- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 2 +- src/Storages/MergeTree/BackgroundJobsExecutor.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp | 2 +- 8 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 4d7d8e08ac3..9b6578092c9 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -70,7 +70,7 @@ /// Minimum revision supporting OpenTelemetry #define DBMS_MIN_REVISION_WITH_OPENTELEMETRY 54442 -/// Mininum revision supporting interserver secret. +/// Minimum revision supporting interserver secret. #define DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET 54441 /// Version of ClickHouse TCP protocol. Increment it manually when you change the protocol. diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 67ef37ba319..5d4da8c5c3e 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1051,7 +1051,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su * - this function shows the expression IN_data1. * * In case that we have HAVING with IN subquery, we have to force creating set for it. - * Also it doesn't make sence if it is GLOBAL IN or ordinary IN. + * Also it doesn't make sense if it is GLOBAL IN or ordinary IN. */ if (!subquery_for_set.source && data.create_source_for_in) { diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 9de938ae184..03ec4ccb1eb 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -167,7 +167,7 @@ void DatabaseCatalog::shutdownImpl() std::lock_guard lock(databases_mutex); assert(std::find_if(uuid_map.begin(), uuid_map.end(), [](const auto & elem) { - /// Ensure that all UUID mappings are emtpy (i.e. all mappings contain nullptr instead of a pointer to storage) + /// Ensure that all UUID mappings are empty (i.e. all mappings contain nullptr instead of a pointer to storage) const auto & not_empty_mapping = [] (const auto & mapping) { auto & table = mapping.second.second; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cdb3d9b7d7b..c41abd3c7f2 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -259,7 +259,7 @@ static void onExceptionBeforeStart(const String & query_for_logging, Context & c span.finish_time_us = current_time_us; span.duration_ns = 0; - // keep values synchonized to type enum in QueryLogElement::createBlock + /// Keep values synchronized to type enum in QueryLogElement::createBlock. span.attribute_names.push_back("clickhouse.query_status"); span.attribute_values.push_back("ExceptionBeforeStart"); @@ -697,7 +697,7 @@ static std::tuple executeQueryImpl( span.finish_time_us = time_in_microseconds(finish_time); span.duration_ns = elapsed_seconds * 1000000000; - // keep values synchonized to type enum in QueryLogElement::createBlock + /// Keep values synchronized to type enum in QueryLogElement::createBlock. span.attribute_names.push_back("clickhouse.query_status"); span.attribute_values.push_back("QueryFinish"); diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 5aba208a86e..0a23cfb1338 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -114,7 +114,7 @@ try } }); /// We've scheduled task in the background pool and when it will finish we will be triggered again. But this task can be - /// extremely long and we may have a lot of other small tasks to do, so we schedule ourselfs here. + /// extremely long and we may have a lot of other small tasks to do, so we schedule ourselves here. scheduleTask(true); } catch (...) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 537a6064c86..ab9fc118ec4 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -22,7 +22,7 @@ struct BackgroundTaskSchedulingSettings double task_sleep_seconds_when_no_work_random_part = 1.0; - /// deprected settings, don't affect background execution + /// Deprecated settings, don't affect background execution double thread_sleep_seconds = 10; double task_sleep_seconds_when_no_work_min = 10; }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 175bc72b08d..07d881d5fe9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2648,7 +2648,7 @@ void MergeTreeData::checkPartCanBeDropped(const ASTPtr & part_ast) String part_name = part_ast->as().value.safeGet(); auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Committed}); if (!part) - throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in commited state", part_name); + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in committed state", part_name); auto table_id = getStorageID(); global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, part->getBytesOnDisk()); diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index fa0aa03e820..a9915f01645 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -345,7 +345,7 @@ static bool indexOfCanUseBloomFilter(const ASTPtr & parent) if (function->arguments->children.size() != 2) return false; - /// We don't allow constant expressions like `indexOf(arr, x) = 1 + 0` but it's neglible. + /// We don't allow constant expressions like `indexOf(arr, x) = 1 + 0` but it's negligible. /// We should return true when the corresponding expression implies that the array contains the element. /// Example: when `indexOf(arr, x)` > 10 is written, it means that arr definitely should contain the element From fdf979d5ce37abfea4609e514daf9a2b900be512 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 13 Nov 2020 11:43:35 +0800 Subject: [PATCH 36/68] Fix missing conversion of unmutated parts --- .../MergeTree/MergeTreeRangeReader.cpp | 38 ++++++++++--------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 726e405b3e8..ecf0d59b42b 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -635,30 +635,32 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar merge_tree_reader->fillMissingColumns(columns, should_evaluate_missing_defaults, num_rows); } - if (!columns.empty() && should_evaluate_missing_defaults) + if (!columns.empty()) { - auto block = prev_reader->sample_block.cloneWithColumns(read_result.columns); - auto block_before_prewhere = read_result.block_before_prewhere; - for (auto & ctn : block) + if (should_evaluate_missing_defaults) { - if (block_before_prewhere.has(ctn.name)) - block_before_prewhere.erase(ctn.name); - } - - if (block_before_prewhere) - { - if (read_result.need_filter) + auto block = prev_reader->sample_block.cloneWithColumns(read_result.columns); + auto block_before_prewhere = read_result.block_before_prewhere; + for (auto & ctn : block) { - auto old_columns = block_before_prewhere.getColumns(); - filterColumns(old_columns, read_result.getFilterOriginal()->getData()); - block_before_prewhere.setColumns(std::move(old_columns)); + if (block_before_prewhere.has(ctn.name)) + block_before_prewhere.erase(ctn.name); } - for (auto && ctn : block_before_prewhere) - block.insert(std::move(ctn)); - } + if (block_before_prewhere) + { + if (read_result.need_filter) + { + auto old_columns = block_before_prewhere.getColumns(); + filterColumns(old_columns, read_result.getFilterOriginal()->getData()); + block_before_prewhere.setColumns(std::move(old_columns)); + } - merge_tree_reader->evaluateMissingDefaults(block, columns); + for (auto && ctn : block_before_prewhere) + block.insert(std::move(ctn)); + } + merge_tree_reader->evaluateMissingDefaults(block, columns); + } merge_tree_reader->performRequiredConversions(columns); } From 2175fea0acdd08a7187daadd5ed97b846ae9a11a Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 13 Nov 2020 09:28:36 +0300 Subject: [PATCH 37/68] Some changelogs --- CHANGELOG.md | 192 ++++++++++++++++++++++ utils/simple-backport/format-changelog.py | 5 +- 2 files changed, 196 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4474675e9ee..555c9f28534 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,12 @@ +## ClickHouse release 20.11 + +### ClickHouse release v20.11.3.3-stable, 2020-11-13 + +#### Bug Fix + +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). + + ### ClickHouse release v20.11.2.1, 2020-11-11 #### Backward Incompatible Change @@ -119,6 +128,24 @@ ## ClickHouse release 20.10 +### ClickHouse release v20.10.4.1-stable, 2020-11-13 + +#### Bug Fix + +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `IN` operator over several columns and tuples with enabled `transform_null_in` setting. Fixes [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). +* This will fix optimize_read_in_order/optimize_aggregation_in_order with max_threads>0 and expression in ORDER BY. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). +* Now when parsing AVRO from input the LowCardinality is removed from type. Fixes [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). +* Fix rapid growth of metadata when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine, and `slave_parallel_worker` enabled on MySQL Slave, by properly shrinking GTID sets. This fixes [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). +* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). +* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Improvement + +* Workaround for use S3 with nginx server as proxy. Nginx currenty does not accept urls with empty path like http://domain.com?delete, but vanilla aws-sdk-cpp produces this kind of urls. This commit uses patched aws-sdk-cpp version, which makes urls with "/" as path in this cases, like http://domain.com/?delete. [#16813](https://github.com/ClickHouse/ClickHouse/pull/16813) ([ianton-ru](https://github.com/ianton-ru)). + + ### ClickHouse release v20.10.3.30, 2020-10-28 #### Backward Incompatible Change @@ -331,6 +358,76 @@ ## ClickHouse release 20.9 +### ClickHouse release v20.9.4.76-stable (2020-10-29) + +#### Bug Fix + +* Fix double free in case of exception in function `dictGet`. It could have happened if dictionary was loaded with error. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix group by with totals/rollup/cube modifers and min/max functions over group by keys. Fixes [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). +* Fix async Distributed INSERT w/ prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). +* Fix a very wrong code in TwoLevelStringHashTable implementation, which might lead to memory leak. I'm suprised how this bug can lurk for so long.... [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). +* Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Not for changelog. [#16031](https://github.com/ClickHouse/ClickHouse/pull/16031) ([tavplubix](https://github.com/tavplubix)). +* Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). +* Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). +* Allow to use direct layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). +* Prevent replica hang for 5-10 mins when replication error happens after a period of inactivity. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). +* Fix rare segfaults when inserting into or selecting from MaterializedView and concurrently dropping target table (for Atomic database engine). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). +* Fix ambiguity in parsing of settings profiles: `CREATE USER ... SETTINGS profile readonly` is now considered as using a profile named `readonly`, not a setting named `profile` with the readonly constraint. This fixes https://github.com/ClickHouse/ClickHouse/issues/15628. [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix a crash when database creation fails. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `DROP TABLE IF EXISTS` failure with `Table ... doesn't exist` error when table is concurrently renamed (for Atomic database engine). Fixed rare deadlock when concurrently executing some DDL queries with multiple tables (like `DROP DATABASE` and `RENAME TABLE`) Fixed `DROP/DETACH DATABASE` failure with `Table ... doesn't exist` when concurrently executing `DROP/DETACH TABLE`. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible deadlocks in RBAC. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix exception `Block structure mismatch` in `SELECT ... ORDER BY DESC` queries which were executed after `ALTER MODIFY COLUMN` query. Fixes [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). +* Fix `select count()` inaccuracy for MaterializeMySQL. [#15767](https://github.com/ClickHouse/ClickHouse/pull/15767) ([tavplubix](https://github.com/tavplubix)). +* Fix some cases of queries, in which only virtual columns are selected. Previously `Not found column _nothing in block` exception may be thrown. Fixes [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed too low default value of `max_replicated_logs_to_keep` setting, which might cause replicas to become lost too often. Improve lost replica recovery process by choosing the most up-to-date replica to clone. Also do not remove old parts from lost replica, detach them instead. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). +* Fix error `Cannot add simple transform to empty Pipe` which happened while reading from `Buffer` table which has different structure than destination table. It was possible if destination table returned empty result for query. Fixes [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug with globs in S3 table function, region from URL was not applied to S3 client configuration. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Decrement the `ReadonlyReplica` metric when detaching read-only tables. This fixes https://github.com/ClickHouse/ClickHouse/issues/15598. [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). +* Throw an error when a single parameter is passed to ReplicatedMergeTree instead of ignoring it. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). + +#### Improvement + +* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). +* Unfold `{database}`, `{table}` and `{uuid}` macros in `zookeeper_path` on replicated table creation. Do not allow `RENAME TABLE` if it may break `zookeeper_path` after server restart. Fixes [#6917](https://github.com/ClickHouse/ClickHouse/issues/6917). [#15348](https://github.com/ClickHouse/ClickHouse/pull/15348) ([tavplubix](https://github.com/tavplubix)). + +#### Other + +* Unfold `{database}`, `{table}` and `{uuid}` macros in `ReplicatedMergeTree` arguments on table creation. [#16160](https://github.com/ClickHouse/ClickHouse/pull/16160) ([tavplubix](https://github.com/tavplubix)). + + +### ClickHouse release v20.9.3.45-stable (2020-10-09) + +#### Bug Fix + +* Fix error `Cannot find column` which may happen at insertion into `MATERIALIZED VIEW` in case if query for `MV` containes `ARRAY JOIN`. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix race condition in AMQP-CPP. [#15667](https://github.com/ClickHouse/ClickHouse/pull/15667) ([alesapin](https://github.com/alesapin)). +* Fix the order of destruction for resources in `ReadFromStorage` step of query plan. It might cause crashes in rare cases. Possibly connected with [#15610](https://github.com/ClickHouse/ClickHouse/issues/15610). [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed `Element ... is not a constant expression` error when using `JSON*` function result in `VALUES`, `LIMIT` or right side of `IN` operator. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). +* Prevent the possibility of error message `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call`. This fixes [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Significantly reduce memory usage in AggregatingInOrderTransform/optimize_aggregation_in_order. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). +* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ILIKE` operator stops being case insensitive if `LIKE` with the same pattern was executed. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). +* Fix `Missing columns` errors when selecting columns which absent in data, but depend on other columns which also absent in data. Fixes [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). +* Fix bug with event subscription in DDLWorker which rarely may lead to query hangs in `ON CLUSTER`. Introduced in [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). +* Report proper error when the second argument of `boundingRatio` aggregate function has a wrong type. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). +* Fix bug where queries like SELECT toStartOfDay(today()) fail complaining about empty time_zone argument. [#15319](https://github.com/ClickHouse/ClickHouse/pull/15319) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix race condition during MergeTree table rename and background cleanup. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). +* Fix rare race condition on server startup when system.logs are enabled. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). +* Fix MSan report in QueryLog. Uninitialized memory can be used for the field `memory_usage`. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix instance crash when using joinGet with LowCardinality types. This fixes https://github.com/ClickHouse/ClickHouse/issues/15214. [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). +* Fix bug in table engine `Buffer` which doesn't allow to insert data of new structure into `Buffer` after `ALTER` query. Fixes [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). +* Adjust decimals field size in mysql column definition packet. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). +* Fixed `Cannot rename ... errno: 22, strerror: Invalid argument` error on DDL query execution in Atomic database when running clickhouse-server in docker on Mac OS. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). +* Fix to make predicate push down work when subquery contains finalizeAggregation function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). +* Fix a problem where the server may get stuck on startup while talking to ZooKeeper, if the configuration files have to be fetched from ZK (using the `from_zk` include option). This fixes [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). + +#### Improvement + +* Now it's possible to change the type of version column for `VersionedCollapsingMergeTree` with `ALTER` query. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). + + ### ClickHouse release v20.9.2.20, 2020-09-22 #### New Feature @@ -405,6 +502,101 @@ ## ClickHouse release 20.8 +### ClickHouse release v20.8.5.45-lts, 2020-10-29 + +#### Bug Fix + +* Fix double free in case of exception in function `dictGet`. It could have happened if dictionary was loaded with error. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix group by with totals/rollup/cube modifers and min/max functions over group by keys. Fixes [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). +* Fix async Distributed INSERT w/ prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). +* Fix a possible memory leak during `GROUP BY` with string keys, caused by an error in `TwoLevelStringHashTable` implementation. [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). +* Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). +* Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). +* Allow to use direct layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). +* Prevent replica hang for 5-10 mins when replication error happens after a period of inactivity. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). +* Fix rare segfaults when inserting into or selecting from MaterializedView and concurrently dropping target table (for Atomic database engine). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). +* Fix ambiguity in parsing of settings profiles: `CREATE USER ... SETTINGS profile readonly` is now considered as using a profile named `readonly`, not a setting named `profile` with the readonly constraint. This fixes [#15628](https://github.com/ClickHouse/ClickHouse/issues/15628). [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix a crash when database creation fails. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `DROP TABLE IF EXISTS` failure with `Table ... doesn't exist` error when table is concurrently renamed (for Atomic database engine). Fixed rare deadlock when concurrently executing some DDL queries with multiple tables (like `DROP DATABASE` and `RENAME TABLE`) Fixed `DROP/DETACH DATABASE` failure with `Table ... doesn't exist` when concurrently executing `DROP/DETACH TABLE`. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible deadlocks in RBAC. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix exception `Block structure mismatch` in `SELECT ... ORDER BY DESC` queries which were executed after `ALTER MODIFY COLUMN` query. Fixes [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). +* Fix some cases of queries, in which only virtual columns are selected. Previously `Not found column _nothing in block` exception may be thrown. Fixes [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). +* Fix error `Cannot find column` which may happen at insertion into `MATERIALIZED VIEW` in case if query for `MV` containes `ARRAY JOIN`. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed too low default value of `max_replicated_logs_to_keep` setting, which might cause replicas to become lost too often. Improve lost replica recovery process by choosing the most up-to-date replica to clone. Also do not remove old parts from lost replica, detach them instead. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). +* Fix error `Cannot add simple transform to empty Pipe` which happened while reading from `Buffer` table which has different structure than destination table. It was possible if destination table returned empty result for query. Fixes [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug with globs in S3 table function, region from URL was not applied to S3 client configuration. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Decrement the `ReadonlyReplica` metric when detaching read-only tables. This fixes [#15598](https://github.com/ClickHouse/ClickHouse/issues/15598). [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). +* Throw an error when a single parameter is passed to ReplicatedMergeTree instead of ignoring it. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). + +#### Improvement + +* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). +* Unfold `{database}`, `{table}` and `{uuid}` macros in `zookeeper_path` on replicated table creation. Do not allow `RENAME TABLE` if it may break `zookeeper_path` after server restart. Fixes [#6917](https://github.com/ClickHouse/ClickHouse/issues/6917). [#15348](https://github.com/ClickHouse/ClickHouse/pull/15348) ([tavplubix](https://github.com/tavplubix)). + +#### Other + +* Unfold `{database}`, `{table}` and `{uuid}` macros in `ReplicatedMergeTree` arguments on table creation. [#16159](https://github.com/ClickHouse/ClickHouse/pull/16159) ([tavplubix](https://github.com/tavplubix)). + + +### ClickHouse release v20.8.4.11-lts, 2020-10-09 + +#### Bug Fix + +* Fix the order of destruction for resources in `ReadFromStorage` step of query plan. It might cause crashes in rare cases. Possibly connected with [#15610](https://github.com/ClickHouse/ClickHouse/issues/15610). [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed `Element ... is not a constant expression` error when using `JSON*` function result in `VALUES`, `LIMIT` or right side of `IN` operator. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). +* Prevent the possibility of error message `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call`. This fixes [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Significantly reduce memory usage in AggregatingInOrderTransform/optimize_aggregation_in_order. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). +* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ILIKE` operator stops being case insensitive if `LIKE` with the same pattern was executed. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). +* Fix `Missing columns` errors when selecting columns which absent in data, but depend on other columns which also absent in data. Fixes [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). +* Fix bug with event subscription in DDLWorker which rarely may lead to query hangs in `ON CLUSTER`. Introduced in [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). +* Report proper error when the second argument of `boundingRatio` aggregate function has a wrong type. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). +* Fix race condition during MergeTree table rename and background cleanup. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). +* Fix rare race condition on server startup when system.logs are enabled. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). +* Fix MSan report in QueryLog. Uninitialized memory can be used for the field `memory_usage`. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix instance crash when using joinGet with LowCardinality types. This fixes https://github.com/ClickHouse/ClickHouse/issues/15214. [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). +* Fix bug in table engine `Buffer` which doesn't allow to insert data of new structure into `Buffer` after `ALTER` query. Fixes [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). +* Adjust decimals field size in mysql column definition packet. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). +* We already use padded comparison between String and FixedString (https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/FunctionsComparison.h#L333). This PR applies the same logic to field comparison which corrects the usage of FixedString as primary keys. This fixes https://github.com/ClickHouse/ClickHouse/issues/14908. [#15033](https://github.com/ClickHouse/ClickHouse/pull/15033) ([Amos Bird](https://github.com/amosbird)). +* If function `bar` was called with specifically crafter arguments, buffer overflow was possible. This closes [#13926](https://github.com/ClickHouse/ClickHouse/issues/13926). [#15028](https://github.com/ClickHouse/ClickHouse/pull/15028) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `Cannot rename ... errno: 22, strerror: Invalid argument` error on DDL query execution in Atomic database when running clickhouse-server in docker on Mac OS. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). +* Now settings `number_of_free_entries_in_pool_to_execute_mutation` and `number_of_free_entries_in_pool_to_lower_max_size_of_merge` can be equal to `background_pool_size`. [#14975](https://github.com/ClickHouse/ClickHouse/pull/14975) ([alesapin](https://github.com/alesapin)). +* Fix to make predicate push down work when subquery contains finalizeAggregation function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). +* Publish CPU frequencies per logical core in `system.asynchronous_metrics`. This fixes https://github.com/ClickHouse/ClickHouse/issues/14923. [#14924](https://github.com/ClickHouse/ClickHouse/pull/14924) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fixed `.metadata.tmp File exists` error when using `MaterializeMySQL` database engine. [#14898](https://github.com/ClickHouse/ClickHouse/pull/14898) ([Winter Zhang](https://github.com/zhang2014)). +* Fix a problem where the server may get stuck on startup while talking to ZooKeeper, if the configuration files have to be fetched from ZK (using the `from_zk` include option). This fixes [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix wrong monotonicity detection for shrunk `Int -> Int` cast of signed types. It might lead to incorrect query result. This bug is unveiled in [#14513](https://github.com/ClickHouse/ClickHouse/issues/14513). [#14783](https://github.com/ClickHouse/ClickHouse/pull/14783) ([Amos Bird](https://github.com/amosbird)). +* Fixed the incorrect sorting order of `Nullable` column. This fixes [#14344](https://github.com/ClickHouse/ClickHouse/issues/14344). [#14495](https://github.com/ClickHouse/ClickHouse/pull/14495) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Improvement + +* Now it's possible to change the type of version column for `VersionedCollapsingMergeTree` with `ALTER` query. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). + + +### ClickHouse release v20.8.3.18-stable, 2020-09-18 + +#### Bug Fix + +* Fix the issue when some invocations of `extractAllGroups` function may trigger "Memory limit exceeded" error. This fixes [#13383](https://github.com/ClickHouse/ClickHouse/issues/13383). [#14889](https://github.com/ClickHouse/ClickHouse/pull/14889) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix SIGSEGV for an attempt to INSERT into StorageFile(fd). [#14887](https://github.com/ClickHouse/ClickHouse/pull/14887) ([Azat Khuzhin](https://github.com/azat)). +* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). +* Fixed missed default database name in metadata of materialized view when executing `ALTER ... MODIFY QUERY`. [#14664](https://github.com/ClickHouse/ClickHouse/pull/14664) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ALTER UPDATE` mutation with Nullable column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). +* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). +* Added the checker as neither calling `lc->isNullable()` nor calling `ls->getDictionaryPtr()->isNullable()` would return the correct result. [#14591](https://github.com/ClickHouse/ClickHouse/pull/14591) ([myrrc](https://github.com/myrrc)). +* Cleanup data directory after Zookeeper exceptions during CreateQuery for StorageReplicatedMergeTree Engine. [#14563](https://github.com/ClickHouse/ClickHouse/pull/14563) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix rare segfaults in functions with combinator -Resample, which could appear in result of overflow with very large parameters. [#14562](https://github.com/ClickHouse/ClickHouse/pull/14562) ([Anton Popov](https://github.com/CurtizJ)). + +#### Improvement + +* Speed up server shutdown process if there are ongoing S3 requests. [#14858](https://github.com/ClickHouse/ClickHouse/pull/14858) ([Pavel Kovalenko](https://github.com/Jokser)). +* Allow using multi-volume storage configuration in storage Distributed. [#14839](https://github.com/ClickHouse/ClickHouse/pull/14839) ([Pavel Kovalenko](https://github.com/Jokser)). +* Speed up server shutdown process if there are ongoing S3 requests. [#14496](https://github.com/ClickHouse/ClickHouse/pull/14496) ([Pavel Kovalenko](https://github.com/Jokser)). +* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). + + ### ClickHouse release v20.8.2.3-stable, 2020-09-08 #### Backward Incompatible Change diff --git a/utils/simple-backport/format-changelog.py b/utils/simple-backport/format-changelog.py index 91547befed4..861faafdcfd 100755 --- a/utils/simple-backport/format-changelog.py +++ b/utils/simple-backport/format-changelog.py @@ -108,8 +108,11 @@ def print_category(category): user = users[pr["user"]["id"]] user_name = user["name"] if user["name"] else user["login"] - # Substitute issue links + # Substitute issue links. + # 1) issue number w/o markdown link pr["entry"] = re.sub(r'([^[])#([0-9]{4,})', r'\1[#\2](https://github.com/ClickHouse/ClickHouse/issues/\2)', pr["entry"]) + # 2) issue URL w/o markdown link + pr["entry"] = re.sub(r'([^(])https://github.com/ClickHouse/ClickHouse/issues/([0-9]{4,})', r'\1[#\2](https://github.com/ClickHouse/ClickHouse/issues/\2)', pr["entry"]) print(f'* {pr["entry"]} [#{pr["number"]}]({pr["html_url"]}) ([{user_name}]({user["html_url"]})).') From 2092aed4159b9c5619657b150f7c50d24cf69a10 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 13 Nov 2020 09:39:14 +0300 Subject: [PATCH 38/68] more --- CHANGELOG.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 555c9f28534..fc5b2e19748 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -358,6 +358,19 @@ ## ClickHouse release 20.9 +### ClickHouse release v20.9.5.5-stable, 2020-11-13 + +#### Bug Fix + +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now when parsing AVRO from input the LowCardinality is removed from type. Fixes [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). +* Fix rapid growth of metadata when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine, and `slave_parallel_worker` enabled on MySQL Slave, by properly shrinking GTID sets. This fixes [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). +* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). +* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the inconsistent behaviour when a part of return data could be dropped because the set for its filtration wasn't created. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). + + ### ClickHouse release v20.9.4.76-stable (2020-10-29) #### Bug Fix From e590a341e56a967a191dbb6a0b2bfd222a207b43 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Nov 2020 11:45:51 +0300 Subject: [PATCH 39/68] Add test --- ...alter_low_cardinality_and_select.reference | 6 +++ .../01576_alter_low_cardinality_and_select.sh | 41 +++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 tests/queries/0_stateless/01576_alter_low_cardinality_and_select.reference create mode 100755 tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh diff --git a/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.reference b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.reference new file mode 100644 index 00000000000..532cba023b6 --- /dev/null +++ b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.reference @@ -0,0 +1,6 @@ +733 733 +CREATE TABLE default.alter_table\n(\n `key` UInt64,\n `value` LowCardinality(String)\n)\nENGINE = MergeTree\nORDER BY key\nSETTINGS index_granularity = 8192 +all_1_1_0 +all_2_2_0 +all_3_3_0 +701 701 diff --git a/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh new file mode 100755 index 00000000000..1055c23ccad --- /dev/null +++ b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + + +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS alter_table" + +${CLICKHOUSE_CLIENT} --query "CREATE TABLE alter_table (key UInt64, value String) ENGINE MergeTree ORDER BY key" + +# we don't need mutations and merges +${CLICKHOUSE_CLIENT} --query "SYSTEM STOP MERGES alter_table" + +${CLICKHOUSE_CLIENT} --query "INSERT INTO alter_table SELECT number, toString(number) FROM numbers(10000)" +${CLICKHOUSE_CLIENT} --query "INSERT INTO alter_table SELECT number, toString(number) FROM numbers(10000, 10000)" +${CLICKHOUSE_CLIENT} --query "INSERT INTO alter_table SELECT number, toString(number) FROM numbers(20000, 10000)" + +${CLICKHOUSE_CLIENT} --query "SELECT * FROM alter_table WHERE value == '733'" + +${CLICKHOUSE_CLIENT} --query "ALTER TABLE alter_table MODIFY COLUMN value LowCardinality(String)" & + +type_query="SELECT type FROM system.columns WHERE name = 'value' and table='alter_table' and database='${CLICKHOUSE_DATABASE}'" +value_type="" + +# waiting until schema will change (but not data) +while [[ "$value_type" != "LowCardinality(String)" ]] +do + sleep 0.1 + value_type=$($CLICKHOUSE_CLIENT --query "$type_query") +done + +# checking type is LowCardinalty +${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE alter_table" + +# checking no mutations happened +${CLICKHOUSE_CLIENT} --query "SELECT name FROM system.parts where table='alter_table' and active and database='${CLICKHOUSE_DATABASE}' ORDER BY name" + +# checking that conversions applied "on fly" works +${CLICKHOUSE_CLIENT} --query "SELECT * FROM alter_table PREWHERE key > 700 WHERE value = '701'" + +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS alter_table" From b6befc10f9b00236bea2ce040dfce140fb5df3c9 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 13 Nov 2020 13:43:37 +0300 Subject: [PATCH 40/68] Update CHANGELOG.md --- CHANGELOG.md | 9 --------- 1 file changed, 9 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index fc5b2e19748..4c35a5f0255 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -380,7 +380,6 @@ * Fix async Distributed INSERT w/ prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). * Fix a very wrong code in TwoLevelStringHashTable implementation, which might lead to memory leak. I'm suprised how this bug can lurk for so long.... [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). * Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Not for changelog. [#16031](https://github.com/ClickHouse/ClickHouse/pull/16031) ([tavplubix](https://github.com/tavplubix)). * Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). * Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). * Allow to use direct layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). @@ -403,10 +402,6 @@ #### Improvement * Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). -* Unfold `{database}`, `{table}` and `{uuid}` macros in `zookeeper_path` on replicated table creation. Do not allow `RENAME TABLE` if it may break `zookeeper_path` after server restart. Fixes [#6917](https://github.com/ClickHouse/ClickHouse/issues/6917). [#15348](https://github.com/ClickHouse/ClickHouse/pull/15348) ([tavplubix](https://github.com/tavplubix)). - -#### Other - * Unfold `{database}`, `{table}` and `{uuid}` macros in `ReplicatedMergeTree` arguments on table creation. [#16160](https://github.com/ClickHouse/ClickHouse/pull/16160) ([tavplubix](https://github.com/tavplubix)). @@ -546,10 +541,6 @@ #### Improvement * Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). -* Unfold `{database}`, `{table}` and `{uuid}` macros in `zookeeper_path` on replicated table creation. Do not allow `RENAME TABLE` if it may break `zookeeper_path` after server restart. Fixes [#6917](https://github.com/ClickHouse/ClickHouse/issues/6917). [#15348](https://github.com/ClickHouse/ClickHouse/pull/15348) ([tavplubix](https://github.com/tavplubix)). - -#### Other - * Unfold `{database}`, `{table}` and `{uuid}` macros in `ReplicatedMergeTree` arguments on table creation. [#16159](https://github.com/ClickHouse/ClickHouse/pull/16159) ([tavplubix](https://github.com/tavplubix)). From 276059394067041079d4b9aa006b7c0e1ced5d45 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Nov 2020 13:50:15 +0300 Subject: [PATCH 41/68] Better test --- .../01576_alter_low_cardinality_and_select.sh | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh index 1055c23ccad..94596da9258 100755 --- a/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh +++ b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh @@ -19,14 +19,13 @@ ${CLICKHOUSE_CLIENT} --query "SELECT * FROM alter_table WHERE value == '733'" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE alter_table MODIFY COLUMN value LowCardinality(String)" & -type_query="SELECT type FROM system.columns WHERE name = 'value' and table='alter_table' and database='${CLICKHOUSE_DATABASE}'" -value_type="" - # waiting until schema will change (but not data) -while [[ "$value_type" != "LowCardinality(String)" ]] +show_query="SHOW CREATE TABLE alter_table" +create_query="" +while [[ "$create_query" != *"LowCardinality"* ]] do sleep 0.1 - value_type=$($CLICKHOUSE_CLIENT --query "$type_query") + create_query=$($CLICKHOUSE_CLIENT --query "$show_query") done # checking type is LowCardinalty From 88789b5c146f9284bc77d41dd6a31a8a3ef0e0c5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 13 Nov 2020 13:57:03 +0300 Subject: [PATCH 42/68] 20.3 --- CHANGELOG.md | 68 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 68 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4c35a5f0255..f196d4421d8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1951,6 +1951,74 @@ No changes compared to v20.4.3.16-stable. ## ClickHouse release v20.3 + +### ClickHouse release v20.3.21.2-lts, 2020-11-02 + +#### Bug Fix + +* Fix dictGet in sharding_key (and similar places, i.e. when the function context is stored permanently). [#16205](https://github.com/ClickHouse/ClickHouse/pull/16205) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix missing or excessive headers in `TSV/CSVWithNames` formats. This fixes [#12504](https://github.com/ClickHouse/ClickHouse/issues/12504). [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). + + +### ClickHouse release v20.3.20.6-lts, 2020-10-09 + +#### Bug Fix + +* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15724](https://github.com/ClickHouse/ClickHouse/pull/15724), [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). +* Fix hang of queries with a lot of subqueries to same table of `MySQL` engine. Previously, if there were more than 16 subqueries to same `MySQL` table in query, it hang forever. [#15299](https://github.com/ClickHouse/ClickHouse/pull/15299) ([Anton Popov](https://github.com/CurtizJ)). +* Fix 'Unknown identifier' in GROUP BY when query has JOIN over Merge table. [#15242](https://github.com/ClickHouse/ClickHouse/pull/15242) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix to make predicate push down work when subquery contains finalizeAggregation function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). +* Concurrent `ALTER ... REPLACE/MOVE PARTITION ...` queries might cause deadlock. It's fixed. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). + + +### ClickHouse release v20.3.19.4-lts, 2020-09-18 + +#### Bug Fix + +* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). +* Fix bug when `ALTER UPDATE` mutation with Nullable column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). +* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). + +#### Improvement + +* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). + + +### ClickHouse release v20.3.18.10-lts, 2020-09-08 + +#### Bug Fix + +* Stop query execution if exception happened in `PipelineExecutor` itself. This could prevent rare possible query hung. Continuation of [#14334](https://github.com/ClickHouse/ClickHouse/issues/14334). [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed the behaviour when sometimes cache-dictionary returned default value instead of present value from source. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix parsing row policies from users.xml when names of databases or tables contain dots. This fixes [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779), [#12527](https://github.com/ClickHouse/ClickHouse/issues/12527). [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix CAST(Nullable(String), Enum()). [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). +* Fixed data race in `text_log`. It does not correspond to any real bug. [#9726](https://github.com/ClickHouse/ClickHouse/pull/9726) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Improvement + +* Fix wrong error for long queries. It was possible to get syntax error other than `Max query size exceeded` for correct query. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Return NULL/zero when value is not parsed completely in parseDateTimeBestEffortOrNull/Zero functions. This fixes [#7876](https://github.com/ClickHouse/ClickHouse/issues/7876). [#11653](https://github.com/ClickHouse/ClickHouse/pull/11653) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Performance Improvement + +* Slightly optimize very short queries with LowCardinality. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). + +#### Build/Testing/Packaging Improvement + +* Fix UBSan report (adding zero to nullptr) in HashTable that appeared after migration to clang-10. [#10638](https://github.com/ClickHouse/ClickHouse/pull/10638) ([alexey-milovidov](https://github.com/alexey-milovidov)). + + +### ClickHouse release v20.3.17.173-lts, 2020-08-15 + +#### Bug Fix + +* Fix crash in JOIN with StorageMerge and `set enable_optimize_predicate_expression=1`. [#13679](https://github.com/ClickHouse/ClickHouse/pull/13679) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix invalid return type for comparison of tuples with `NULL` elements. Fixes [#12461](https://github.com/ClickHouse/ClickHouse/issues/12461). [#13420](https://github.com/ClickHouse/ClickHouse/pull/13420) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix queries with constant columns and `ORDER BY` prefix of primary key. [#13396](https://github.com/ClickHouse/ClickHouse/pull/13396) ([Anton Popov](https://github.com/CurtizJ)). +* Return passed number for numbers with MSB set in roundUpToPowerOfTwoOrZero(). [#13234](https://github.com/ClickHouse/ClickHouse/pull/13234) ([Azat Khuzhin](https://github.com/azat)). + + ### ClickHouse release v20.3.16.165-lts 2020-08-10 #### Bug Fix From be16b4ef779e74c50712f3a4ee41345a9893d384 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Fri, 13 Nov 2020 12:04:56 +0100 Subject: [PATCH 43/68] Update Connection.cpp --- src/Client/Connection.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 2a130f4ee6f..b810d24b3d3 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -73,8 +73,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) { #if USE_SSL socket = std::make_unique(); - auto secure_socket = static_cast(socket.get()); - secure_socket->setPeerHostName(host); + static_cast(socket.get())->setPeerHostName(host); #else throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; #endif From 0e4cbf7508e47f7d9a7b29ca55887dc6f6101a5b Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Fri, 13 Nov 2020 12:31:37 +0100 Subject: [PATCH 44/68] Update programs/install/Install.cpp --- programs/install/Install.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 27f4ceec732..da22452819a 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -394,7 +394,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } else { - fmt::print("Users config file {} already exists, will keep it and extract users info from it.\n", main_config_file.string()); + fmt::print("Users config file {} already exists, will keep it and extract users info from it.\n", users_config_file.string()); /// Check if password for default user already specified. ConfigProcessor processor(users_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); From f7b60c17f592166c0c37acb0da9dd8e3821e6b84 Mon Sep 17 00:00:00 2001 From: ubuntu Date: Fri, 13 Nov 2020 07:09:50 +0000 Subject: [PATCH 45/68] make test stable --- .../test_materialize_mysql_database/materialize_with_ddl.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 37b0dbe9625..387064dd70a 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -127,6 +127,7 @@ def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_ mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") + check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY key FORMAT TSV", "1\t2020-01-01 01:02:03.999999\t2020-01-01 01:02:03.999\t" + ('9' * 35) + "." + ('9' * 30) + "\n" "2\t2020-01-01 01:02:03.000000\t2020-01-01 01:02:03.000\t0." + ('0' * 29) + "1\n" @@ -489,8 +490,8 @@ def select_without_columns(clickhouse_node, mysql_node, service_name): def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS priv_err_db") clickhouse_node.query("DROP DATABASE IF EXISTS priv_err_db") + mysql_node.query("DROP DATABASE IF EXISTS priv_err_db") mysql_node.query("CREATE DATABASE priv_err_db DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE priv_err_db.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") mysql_node.query("INSERT INTO priv_err_db.test_table_1 VALUES(1);") From b8e33bd35b5b9329025301b591166aec680f7f5d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Nov 2020 15:42:26 +0300 Subject: [PATCH 46/68] Add comments --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index ecf0d59b42b..66b75a15289 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -637,6 +637,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar if (!columns.empty()) { + /// If some columns absent in part, than evaulate default values if (should_evaluate_missing_defaults) { auto block = prev_reader->sample_block.cloneWithColumns(read_result.columns); @@ -661,6 +662,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar } merge_tree_reader->evaluateMissingDefaults(block, columns); } + /// If columns not empty, than apply on-fly alter conversions if any required merge_tree_reader->performRequiredConversions(columns); } @@ -679,9 +681,11 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar merge_tree_reader->fillMissingColumns(read_result.columns, should_evaluate_missing_defaults, read_result.num_rows); + /// If some columns absent in part, than evaulate default values if (should_evaluate_missing_defaults) merge_tree_reader->evaluateMissingDefaults({}, read_result.columns); + /// If result not empty, than apply on-fly alter conversions if any required merge_tree_reader->performRequiredConversions(read_result.columns); } else From b2a8b6135dda680c243fd7f413dd2ab63d03b266 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Nov 2020 16:24:14 +0300 Subject: [PATCH 47/68] then --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 66b75a15289..e22c6bb3d0c 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -637,7 +637,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar if (!columns.empty()) { - /// If some columns absent in part, than evaulate default values + /// If some columns absent in part, then evaulate default values if (should_evaluate_missing_defaults) { auto block = prev_reader->sample_block.cloneWithColumns(read_result.columns); @@ -662,7 +662,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar } merge_tree_reader->evaluateMissingDefaults(block, columns); } - /// If columns not empty, than apply on-fly alter conversions if any required + /// If columns not empty, then apply on-fly alter conversions if any required merge_tree_reader->performRequiredConversions(columns); } @@ -681,11 +681,11 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar merge_tree_reader->fillMissingColumns(read_result.columns, should_evaluate_missing_defaults, read_result.num_rows); - /// If some columns absent in part, than evaulate default values + /// If some columns absent in part, then evaulate default values if (should_evaluate_missing_defaults) merge_tree_reader->evaluateMissingDefaults({}, read_result.columns); - /// If result not empty, than apply on-fly alter conversions if any required + /// If result not empty, then apply on-fly alter conversions if any required merge_tree_reader->performRequiredConversions(read_result.columns); } else From 5ef2eaeeef32298559ca81deeca416015094ee17 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Nov 2020 16:25:18 +0300 Subject: [PATCH 48/68] Trying to fix some flaky tests --- .../00804_test_delta_codec_compression.reference | 2 +- .../00804_test_delta_codec_compression.sql | 12 ++++++------ ...40_dictionary_invalidate_query_switchover_long.sh | 10 ++++------ 3 files changed, 11 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.reference b/tests/queries/0_stateless/00804_test_delta_codec_compression.reference index 79302586e92..949d37ed27a 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.reference +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.reference @@ -1,4 +1,4 @@ -83 +84 1 46 1 diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql index 91bc45df63d..6da43298347 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql @@ -23,9 +23,9 @@ OPTIMIZE TABLE default_codec_synthetic FINAL; SELECT floor(big_size / small_size) AS ratio FROM - (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database == currentDatabase() and table == 'delta_codec_synthetic') + (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database == currentDatabase() and table == 'delta_codec_synthetic' and active) INNER JOIN - (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database == currentDatabase() and table == 'default_codec_synthetic') + (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database == currentDatabase() and table == 'default_codec_synthetic' and active) USING(key); SELECT @@ -61,9 +61,9 @@ OPTIMIZE TABLE default_codec_float FINAL; SELECT floor(big_size / small_size) as ratio FROM - (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database = currentDatabase() and table = 'delta_codec_float') + (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database = currentDatabase() and table = 'delta_codec_float' and active) INNER JOIN - (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database = currentDatabase() and table = 'default_codec_float') USING(key); + (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database = currentDatabase() and table = 'default_codec_float' and active) USING(key); SELECT small_hash == big_hash @@ -99,9 +99,9 @@ OPTIMIZE TABLE default_codec_string FINAL; SELECT floor(big_size / small_size) as ratio FROM - (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database = currentDatabase() and table = 'delta_codec_string') + (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database = currentDatabase() and table = 'delta_codec_string' and active) INNER JOIN - (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database = currentDatabase() and table = 'default_codec_string') USING(key); + (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database = currentDatabase() and table = 'default_codec_string' and active) USING(key); SELECT small_hash == big_hash diff --git a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh index 6b509ac7925..6879fedf978 100755 --- a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh +++ b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh @@ -30,12 +30,9 @@ LAYOUT(FLAT())" $CLICKHOUSE_CLIENT --query "SELECT dictGetUInt8('dictdb.invalidate', 'two', toUInt64(122))" +# No exception happened $CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" -# Bad solution, but it's quite complicated to detect, that invalidte_query stopped updates. -# In worst case we don't check anything, but fortunately it doesn't lead to false negatives. -sleep 5 - $CLICKHOUSE_CLIENT --query "DROP TABLE dictdb.dict_invalidate" function check_exception_detected() @@ -52,7 +49,7 @@ function check_exception_detected() export -f check_exception_detected; -timeout 10 bash -c check_exception_detected 2> /dev/null +timeout 30 bash -c check_exception_detected 2> /dev/null $CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" 2>&1 | grep -Eo "Table dictdb.dict_invalidate .* exist." @@ -76,7 +73,8 @@ function check_exception_fixed() } export -f check_exception_fixed; -timeout 10 bash -c check_exception_fixed 2> /dev/null +# it may take a long until dictionary reloads +timeout 60 bash -c check_exception_fixed 2> /dev/null $CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" 2>&1 $CLICKHOUSE_CLIENT --query "SELECT dictGetUInt8('dictdb.invalidate', 'two', toUInt64(133))" From 14485238b1b82375513f00920346d35293ca5b7e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Nov 2020 16:28:04 +0300 Subject: [PATCH 49/68] Update 01040_dictionary_invalidate_query_switchover_long.sh --- .../01040_dictionary_invalidate_query_switchover_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh index 6879fedf978..f9b4573bfb4 100755 --- a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh +++ b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh @@ -73,7 +73,7 @@ function check_exception_fixed() } export -f check_exception_fixed; -# it may take a long until dictionary reloads +# it may take a while until dictionary reloads timeout 60 bash -c check_exception_fixed 2> /dev/null $CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" 2>&1 From e78010c28cd0152f33ece50ec647b39b1100a3c9 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 13 Nov 2020 15:21:04 +0100 Subject: [PATCH 50/68] Attempt to fix #16926 --- .../Formats/Impl/AvroRowInputFormat.cpp | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 787059ddcc8..5d5f977a22f 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -644,11 +644,21 @@ private: request.setHost(url.getHost()); auto session = makePooledHTTPSession(url, timeouts, 1); - session->sendRequest(request); - - Poco::Net::HTTPResponse response; - auto * response_body = receiveResponse(*session, request, response, false); + std::istream * response_body; + try + { + session->sendRequest(request); + Poco::Net::HTTPResponse response; + response_body = receiveResponse(*session, request, response, false); + } + catch (const Poco::Exception & e) + { + /// We use session data storage as storage for exception text + /// Depend on it we can deduce to reconnect session or reresolve session host + session->attachSessionData(e.message()); + throw; + } Poco::JSON::Parser parser; auto json_body = parser.parse(*response_body).extract(); auto schema = json_body->getValue("schema"); From 257bbcafeadfdbbe3e565dc6c6c593b5b81bd927 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 13 Nov 2020 15:51:50 +0300 Subject: [PATCH 51/68] Added test from #16588 (cherry picked from commit b5a8ef3cebcc9a1471ec71c17a41b9ed78789945) --- .../01576_if_null_external_aggregation.reference | 0 .../0_stateless/01576_if_null_external_aggregation.sql | 7 +++++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/01576_if_null_external_aggregation.reference create mode 100644 tests/queries/0_stateless/01576_if_null_external_aggregation.sql diff --git a/tests/queries/0_stateless/01576_if_null_external_aggregation.reference b/tests/queries/0_stateless/01576_if_null_external_aggregation.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01576_if_null_external_aggregation.sql b/tests/queries/0_stateless/01576_if_null_external_aggregation.sql new file mode 100644 index 00000000000..b9c36a9cecc --- /dev/null +++ b/tests/queries/0_stateless/01576_if_null_external_aggregation.sql @@ -0,0 +1,7 @@ +SET max_bytes_before_external_group_by = 200000000; + +SET max_memory_usage = 1500000000; +SET max_threads = 12; + +SELECT bitAnd(number, pow(2, 20) - 1) as k, argMaxIf(k, number % 2 = 0 ? number : Null, number > 42), uniq(number) AS u FROM numbers(1000000) GROUP BY k format Null; + From 2430d9d20df72a2c81393b9829ef696f7636f5fe Mon Sep 17 00:00:00 2001 From: myrrc Date: Fri, 13 Nov 2020 18:13:15 +0300 Subject: [PATCH 52/68] fixed the UB when casting from double to int64_t Corresponding upstream fix: https://github.com/cerevra/int/pull/24 --- base/common/wide_integer_impl.h | 62 +++++++++++++----- src/Columns/ColumnDecimal.cpp | 87 +++++++++---------------- src/Columns/ColumnDecimal.h | 111 ++++++++------------------------ src/Core/DecimalComparison.h | 1 + src/Core/Types.h | 2 +- src/DataTypes/IDataType.h | 107 ++++++++++++++---------------- 6 files changed, 156 insertions(+), 214 deletions(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index d90bde30a43..2a889819c11 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -5,6 +5,9 @@ /// (See at http://www.boost.org/LICENSE_1_0.txt) #include "throwError.h" +#include +#include +#include namespace wide { @@ -192,7 +195,7 @@ struct integer::_impl } template - constexpr static auto to_Integral(T f) noexcept + __attribute__((no_sanitize("undefined"))) constexpr static auto to_Integral(T f) noexcept { if constexpr (std::is_same_v) return f; @@ -225,25 +228,54 @@ struct integer::_impl self.items[i] = 0; } - constexpr static void wide_integer_from_bultin(integer & self, double rhs) noexcept - { - if ((rhs > 0 && rhs < std::numeric_limits::max()) || (rhs < 0 && rhs > std::numeric_limits::min())) + /** + * N.B. t is constructed from double, so max(t) = max(double) ~ 2^310 + * the recursive call happens when t / 2^64 > 2^64, so there won't be more than 5 of them. + * + * t = a1 * max_int + b1, a1 > max_int, b1 < max_int + * a1 = a2 * max_int + b2, a2 > max_int, b2 < max_int + * a_(n - 1) = a_n * max_int + b2, a_n <= max_int <- base case. + */ + template + constexpr static void set_multiplier(integer & self, T t) noexcept { + constexpr uint64_t max_int = std::numeric_limits::max(); + const T alpha = t / max_int; + + if (alpha <= max_int) + self = static_cast(alpha); + else // max(double) / 2^64 will surely contain less than 52 precision bits, so speed up computations. + set_multiplier(self, alpha); + + self *= max_int; + self += static_cast(t - alpha * max_int); // += b_i + } + + constexpr static void wide_integer_from_bultin(integer& self, double rhs) noexcept { + constexpr int64_t max_int = std::numeric_limits::max(); + constexpr int64_t min_int = std::numeric_limits::min(); + + /// There are values in int64 that have more than 53 significant bits (in terms of double + /// representation). Such values, being promoted to double, are rounded up or down. If they are rounded up, + /// the result may not fit in 64 bits. + /// The example of such a number is 9.22337e+18. + /// As to_Integral does a static_cast to int64_t, it may result in UB. + /// The necessary check here is that long double has enough significant (mantissa) bits to store the + /// int64_t max value precisely. + static_assert(LDBL_MANT_DIG >= 64, + "On your system long double has less than 64 precision bits," + "which may result in UB when initializing double from int64_t"); + + if ((rhs > 0 && rhs < max_int) || (rhs < 0 && rhs > min_int)) { - self = to_Integral(rhs); + self = static_cast(rhs); return; } - long double r = rhs; - if (r < 0) - r = -r; + const long double rhs_long_double = (static_cast(rhs) < 0) + ? -static_cast(rhs) + : rhs; - size_t count = r / std::numeric_limits::max(); - self = count; - self *= std::numeric_limits::max(); - long double to_diff = count; - to_diff *= std::numeric_limits::max(); - - self += to_Integral(r - to_diff); + set_multiplier(self, rhs_long_double); if (rhs < 0) self = -self; diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 6a0afe55357..8b3ff214713 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -7,8 +7,10 @@ #include #include -#include #include +#if !defined(ARCADIA_BUILD) + #include // Y_IGNORE +#endif #include @@ -55,32 +57,16 @@ void ColumnDecimal::compareColumn(const IColumn & rhs, size_t rhs_row_num, template StringRef ColumnDecimal::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const { - if constexpr (is_POD) - { - auto * pos = arena.allocContinue(sizeof(T), begin); - memcpy(pos, &data[n], sizeof(T)); - return StringRef(pos, sizeof(T)); - } - else - { - char * pos = arena.allocContinue(BigInt::size, begin); - return BigInt::serialize(data[n], pos); - } + auto * pos = arena.allocContinue(sizeof(T), begin); + memcpy(pos, &data[n], sizeof(T)); + return StringRef(pos, sizeof(T)); } template const char * ColumnDecimal::deserializeAndInsertFromArena(const char * pos) { - if constexpr (is_POD) - { - data.push_back(unalignedLoad(pos)); - return pos + sizeof(T); - } - else - { - data.push_back(BigInt::deserialize(pos)); - return pos + BigInt::size; - } + data.push_back(unalignedLoad(pos)); + return pos + sizeof(T); } template @@ -195,11 +181,21 @@ void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColum /// Since then we are working inside the interval. if (reverse) - partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, +#if !defined(ARCADIA_BUILD) + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this](size_t a, size_t b) { return data[a] > data[b]; }); +#else + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, + [this](size_t a, size_t b) { return data[a] > data[b]; }); +#endif else - partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, +#if !defined(ARCADIA_BUILD) + miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this](size_t a, size_t b) { return data[a] < data[b]; }); +#else + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, + [this](size_t a, size_t b) { return data[a] > data[b]; }); +#endif auto new_first = first; for (auto j = first + 1; j < limit; ++j) { @@ -252,24 +248,13 @@ MutableColumnPtr ColumnDecimal::cloneResized(size_t size) const new_col.data.resize(size); size_t count = std::min(this->size(), size); - if constexpr (is_POD) - { - memcpy(new_col.data.data(), data.data(), count * sizeof(data[0])); - if (size > count) - { - void * tail = &new_col.data[count]; - memset(tail, 0, (size - count) * sizeof(T)); - } - } - else - { - for (size_t i = 0; i < count; i++) - new_col.data[i] = data[i]; + memcpy(new_col.data.data(), data.data(), count * sizeof(data[0])); - if (size > count) - for (size_t i = count; i < size; i++) - new_col.data[i] = T{}; + if (size > count) + { + void * tail = &new_col.data[count]; + memset(tail, 0, (size - count) * sizeof(T)); } } @@ -279,16 +264,9 @@ MutableColumnPtr ColumnDecimal::cloneResized(size_t size) const template void ColumnDecimal::insertData(const char * src, size_t /*length*/) { - if constexpr (is_POD) - { - T tmp; - memcpy(&tmp, src, sizeof(T)); - data.emplace_back(tmp); - } - else - { - data.push_back(BigInt::deserialize(src)); - } + T tmp; + memcpy(&tmp, src, sizeof(T)); + data.emplace_back(tmp); } template @@ -303,13 +281,8 @@ void ColumnDecimal::insertRangeFrom(const IColumn & src, size_t start, size_t size_t old_size = data.size(); data.resize(old_size + length); - if constexpr (is_POD) - memcpy(data.data() + old_size, &src_vec.data[start], length * sizeof(data[0])); - else - { - for (size_t i = 0; i < length; i++) - data[old_size + i] = src_vec.data[start + i]; - } + + memcpy(data.data() + old_size, &src_vec.data[start], length * sizeof(data[0])); } template diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index e35af1343b2..196fd2d627e 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -1,23 +1,20 @@ #pragma once -#include +#include + +#include +#include "Core/DecimalFunctions.h" #include #include +#include #include -#include -#include - -#include +#if !defined(ARCADIA_BUILD) + #include // Y_IGNORE +#endif namespace DB { - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - /// PaddedPODArray extended by Decimal scale template class DecimalPaddedPODArray : public PaddedPODArray @@ -55,43 +52,6 @@ private: UInt32 scale; }; -/// std::vector extended by Decimal scale -template -class DecimalVector : public std::vector -{ -public: - using Base = std::vector; - using Base::operator[]; - - DecimalVector(size_t size, UInt32 scale_) - : Base(size), - scale(scale_) - {} - - DecimalVector(const DecimalVector & other) - : Base(other.begin(), other.end()), - scale(other.scale) - {} - - DecimalVector(DecimalVector && other) - { - this->swap(other); - std::swap(scale, other.scale); - } - - DecimalVector & operator=(DecimalVector && other) - { - this->swap(other); - std::swap(scale, other.scale); - return *this; - } - - UInt32 getScale() const { return scale; } - -private: - UInt32 scale; -}; - /// A ColumnVector for Decimals template class ColumnDecimal final : public COWHelper> @@ -105,10 +65,7 @@ private: public: using ValueType = T; using NativeT = typename T::NativeType; - static constexpr bool is_POD = !is_big_int_v; - using Container = std::conditional_t, - DecimalVector>; + using Container = DecimalPaddedPODArray; private: ColumnDecimal(const size_t n, UInt32 scale_) @@ -127,23 +84,13 @@ public: bool isNumeric() const override { return false; } bool canBeInsideNullable() const override { return true; } - bool isFixedAndContiguous() const override { return true; } + bool isFixedAndContiguous() const final { return true; } size_t sizeOfValueIfFixed() const override { return sizeof(T); } size_t size() const override { return data.size(); } size_t byteSize() const override { return data.size() * sizeof(data[0]); } - size_t allocatedBytes() const override - { - if constexpr (is_POD) - return data.allocated_bytes(); - else - return data.capacity() * sizeof(data[0]); - } - void protect() override - { - if constexpr (is_POD) - data.protect(); - } + size_t allocatedBytes() const override { return data.allocated_bytes(); } + void protect() override { data.protect(); } void reserve(size_t n) override { data.reserve(n); } void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast(src).getData()[n]); } @@ -151,38 +98,28 @@ public: void insertDefault() override { data.push_back(T()); } virtual void insertManyDefaults(size_t length) override { - if constexpr (is_POD) - data.resize_fill(data.size() + length); - else - data.resize(data.size() + length); + data.resize_fill(data.size() + length); } void insert(const Field & x) override { data.push_back(DB::get>(x)); } void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; void popBack(size_t n) override { - if constexpr (is_POD) - data.resize_assume_reserved(data.size() - n); - else - data.resize(data.size() - n); + data.resize_assume_reserved(data.size() - n); } StringRef getRawData() const override { - if constexpr (is_POD) - return StringRef(reinterpret_cast(data.data()), byteSize()); - else - throw Exception("getRawData() is not implemented for big integers", ErrorCodes::NOT_IMPLEMENTED); + return StringRef(reinterpret_cast(data.data()), byteSize()); } StringRef getDataAt(size_t n) const override { - if constexpr (is_POD) - return StringRef(reinterpret_cast(&data[n]), sizeof(data[n])); - else - throw Exception("getDataAt() is not implemented for big integers", ErrorCodes::NOT_IMPLEMENTED); + return StringRef(reinterpret_cast(&data[n]), sizeof(data[n])); } + Float64 getFloat64(size_t n) const final { return DecimalUtils::convertTo(data[n], scale); } + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; const char * deserializeAndInsertFromArena(const char * pos) override; void updateHashWithValue(size_t n, SipHash & hash) const override; @@ -254,9 +191,17 @@ protected: sort_end = res.begin() + limit; if (reverse) - partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; }); +#if !defined(ARCADIA_BUILD) + miniselect::floyd_rivest_partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; }); +#else + std::partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; }); +#endif else - partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; }); +#if !defined(ARCADIA_BUILD) + miniselect::floyd_rivest_partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; }); +#else + std::partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; }); +#endif } }; diff --git a/src/Core/DecimalComparison.h b/src/Core/DecimalComparison.h index 674ed31683b..6da1fc00b7c 100644 --- a/src/Core/DecimalComparison.h +++ b/src/Core/DecimalComparison.h @@ -57,6 +57,7 @@ public: using Op = Operation; using ColVecA = std::conditional_t, ColumnDecimal, ColumnVector>; using ColVecB = std::conditional_t, ColumnDecimal, ColumnVector>; + using ArrayA = typename ColVecA::Container; using ArrayB = typename ColVecB::Container; diff --git a/src/Core/Types.h b/src/Core/Types.h index 3157598adc0..28ca7146aaf 100644 --- a/src/Core/Types.h +++ b/src/Core/Types.h @@ -145,7 +145,7 @@ struct Decimal operator T () const { return value; } template - U convertTo() + U convertTo() const { /// no IsDecimalNumber defined yet if constexpr (std::is_same_v> || diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 5e25d47534e..c32aba721f4 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -466,75 +466,66 @@ struct WhichDataType { TypeIndex idx; - WhichDataType(TypeIndex idx_ = TypeIndex::Nothing) - : idx(idx_) - {} + constexpr WhichDataType(TypeIndex idx_ = TypeIndex::Nothing) : idx(idx_) {} + constexpr WhichDataType(const IDataType & data_type) : idx(data_type.getTypeId()) {} + constexpr WhichDataType(const IDataType * data_type) : idx(data_type->getTypeId()) {} - WhichDataType(const IDataType & data_type) - : idx(data_type.getTypeId()) - {} + // shared ptr -> is non-constexpr in gcc + WhichDataType(const DataTypePtr & data_type) : idx(data_type->getTypeId()) {} - WhichDataType(const IDataType * data_type) - : idx(data_type->getTypeId()) - {} + constexpr bool isUInt8() const { return idx == TypeIndex::UInt8; } + constexpr bool isUInt16() const { return idx == TypeIndex::UInt16; } + constexpr bool isUInt32() const { return idx == TypeIndex::UInt32; } + constexpr bool isUInt64() const { return idx == TypeIndex::UInt64; } + constexpr bool isUInt128() const { return idx == TypeIndex::UInt128; } + constexpr bool isUInt256() const { return idx == TypeIndex::UInt256; } + constexpr bool isUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64() || isUInt128() || isUInt256(); } + constexpr bool isNativeUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64(); } - WhichDataType(const DataTypePtr & data_type) - : idx(data_type->getTypeId()) - {} + constexpr bool isInt8() const { return idx == TypeIndex::Int8; } + constexpr bool isInt16() const { return idx == TypeIndex::Int16; } + constexpr bool isInt32() const { return idx == TypeIndex::Int32; } + constexpr bool isInt64() const { return idx == TypeIndex::Int64; } + constexpr bool isInt128() const { return idx == TypeIndex::Int128; } + constexpr bool isInt256() const { return idx == TypeIndex::Int256; } + constexpr bool isInt() const { return isInt8() || isInt16() || isInt32() || isInt64() || isInt128() || isInt256(); } + constexpr bool isNativeInt() const { return isInt8() || isInt16() || isInt32() || isInt64(); } - bool isUInt8() const { return idx == TypeIndex::UInt8; } - bool isUInt16() const { return idx == TypeIndex::UInt16; } - bool isUInt32() const { return idx == TypeIndex::UInt32; } - bool isUInt64() const { return idx == TypeIndex::UInt64; } - bool isUInt128() const { return idx == TypeIndex::UInt128; } - bool isUInt256() const { return idx == TypeIndex::UInt256; } - bool isUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64() || isUInt128() || isUInt256(); } - bool isNativeUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64(); } + constexpr bool isDecimal32() const { return idx == TypeIndex::Decimal32; } + constexpr bool isDecimal64() const { return idx == TypeIndex::Decimal64; } + constexpr bool isDecimal128() const { return idx == TypeIndex::Decimal128; } + constexpr bool isDecimal256() const { return idx == TypeIndex::Decimal256; } + constexpr bool isDecimal() const { return isDecimal32() || isDecimal64() || isDecimal128() || isDecimal256(); } - bool isInt8() const { return idx == TypeIndex::Int8; } - bool isInt16() const { return idx == TypeIndex::Int16; } - bool isInt32() const { return idx == TypeIndex::Int32; } - bool isInt64() const { return idx == TypeIndex::Int64; } - bool isInt128() const { return idx == TypeIndex::Int128; } - bool isInt256() const { return idx == TypeIndex::Int256; } - bool isInt() const { return isInt8() || isInt16() || isInt32() || isInt64() || isInt128() || isInt256(); } - bool isNativeInt() const { return isInt8() || isInt16() || isInt32() || isInt64(); } + constexpr bool isFloat32() const { return idx == TypeIndex::Float32; } + constexpr bool isFloat64() const { return idx == TypeIndex::Float64; } + constexpr bool isFloat() const { return isFloat32() || isFloat64(); } - bool isDecimal32() const { return idx == TypeIndex::Decimal32; } - bool isDecimal64() const { return idx == TypeIndex::Decimal64; } - bool isDecimal128() const { return idx == TypeIndex::Decimal128; } - bool isDecimal256() const { return idx == TypeIndex::Decimal256; } - bool isDecimal() const { return isDecimal32() || isDecimal64() || isDecimal128() || isDecimal256(); } + constexpr bool isEnum8() const { return idx == TypeIndex::Enum8; } + constexpr bool isEnum16() const { return idx == TypeIndex::Enum16; } + constexpr bool isEnum() const { return isEnum8() || isEnum16(); } - bool isFloat32() const { return idx == TypeIndex::Float32; } - bool isFloat64() const { return idx == TypeIndex::Float64; } - bool isFloat() const { return isFloat32() || isFloat64(); } + constexpr bool isDate() const { return idx == TypeIndex::Date; } + constexpr bool isDateTime() const { return idx == TypeIndex::DateTime; } + constexpr bool isDateTime64() const { return idx == TypeIndex::DateTime64; } + constexpr bool isDateOrDateTime() const { return isDate() || isDateTime() || isDateTime64(); } - bool isEnum8() const { return idx == TypeIndex::Enum8; } - bool isEnum16() const { return idx == TypeIndex::Enum16; } - bool isEnum() const { return isEnum8() || isEnum16(); } + constexpr bool isString() const { return idx == TypeIndex::String; } + constexpr bool isFixedString() const { return idx == TypeIndex::FixedString; } + constexpr bool isStringOrFixedString() const { return isString() || isFixedString(); } - bool isDate() const { return idx == TypeIndex::Date; } - bool isDateTime() const { return idx == TypeIndex::DateTime; } - bool isDateTime64() const { return idx == TypeIndex::DateTime64; } - bool isDateOrDateTime() const { return isDate() || isDateTime() || isDateTime64(); } + constexpr bool isUUID() const { return idx == TypeIndex::UUID; } + constexpr bool isArray() const { return idx == TypeIndex::Array; } + constexpr bool isTuple() const { return idx == TypeIndex::Tuple; } + constexpr bool isSet() const { return idx == TypeIndex::Set; } + constexpr bool isInterval() const { return idx == TypeIndex::Interval; } - bool isString() const { return idx == TypeIndex::String; } - bool isFixedString() const { return idx == TypeIndex::FixedString; } - bool isStringOrFixedString() const { return isString() || isFixedString(); } + constexpr bool isNothing() const { return idx == TypeIndex::Nothing; } + constexpr bool isNullable() const { return idx == TypeIndex::Nullable; } + constexpr bool isFunction() const { return idx == TypeIndex::Function; } + constexpr bool isAggregateFunction() const { return idx == TypeIndex::AggregateFunction; } - bool isUUID() const { return idx == TypeIndex::UUID; } - bool isArray() const { return idx == TypeIndex::Array; } - bool isTuple() const { return idx == TypeIndex::Tuple; } - bool isSet() const { return idx == TypeIndex::Set; } - bool isInterval() const { return idx == TypeIndex::Interval; } - - bool isNothing() const { return idx == TypeIndex::Nothing; } - bool isNullable() const { return idx == TypeIndex::Nullable; } - bool isFunction() const { return idx == TypeIndex::Function; } - bool isAggregateFunction() const { return idx == TypeIndex::AggregateFunction; } - - bool IsBigIntOrDeimal() const { return isInt128() || isInt256() || isUInt256() || isDecimal256(); } + constexpr bool IsBigIntOrDeimal() const { return isInt128() || isInt256() || isUInt256() || isDecimal256(); } }; /// IDataType helpers (alternative for IDataType virtual methods with single point of truth) From 7f7f66a1f1c694daab2b1359e45b9eff7d255f46 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Fri, 13 Nov 2020 17:45:58 +0100 Subject: [PATCH 53/68] add comment & restart CI... --- src/Client/Connection.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index b810d24b3d3..0a499540c41 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -73,6 +73,10 @@ void Connection::connect(const ConnectionTimeouts & timeouts) { #if USE_SSL socket = std::make_unique(); + + /// we resolve the ip when we open SecureStreamSocket, so to make Server Name Indication (SNI) + /// work we need to pass host name separately. It will be send into TLS Hello packet to let + /// the server know which host we want to talk with (single IP can process requests for multiple hosts using SNI). static_cast(socket.get())->setPeerHostName(host); #else throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; From 33fcbf813f6d18106682c4a4243fbf9c078580fe Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 13 Nov 2020 20:43:05 +0300 Subject: [PATCH 54/68] 20.8 --- CHANGELOG.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index f196d4421d8..355c664664d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -510,6 +510,19 @@ ## ClickHouse release 20.8 +### ClickHouse release v20.8.6.6-lts, 2020-11-13 + +#### Bug Fix + +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now when parsing AVRO from input the LowCardinality is removed from type. Fixes [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). +* Fix rapid growth of metadata when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine, and `slave_parallel_worker` enabled on MySQL Slave, by properly shrinking GTID sets. This fixes [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). +* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). +* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the inconsistent behaviour when a part of return data could be dropped because the set for its filtration wasn't created. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). + + ### ClickHouse release v20.8.5.45-lts, 2020-10-29 #### Bug Fix From ea8f6fde994823821691dbf12ed2587850e2a7e6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 13 Nov 2020 22:30:38 +0300 Subject: [PATCH 55/68] Add test to skiplist. --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 265b1a8295d..6875151520a 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -169,3 +169,4 @@ 01557_max_parallel_replicas_no_sample.sql 01525_select_with_offset_fetch_clause 01560_timeseriesgroupsum_segfault +00976_ttl_with_old_parts From 4f4beab1ff8447b3ccb0341a71abccad2110f05d Mon Sep 17 00:00:00 2001 From: myrrc Date: Sat, 14 Nov 2020 16:50:25 +0300 Subject: [PATCH 56/68] re-merged with master --- src/Columns/ColumnDecimal.h | 26 ++++++++------------------ 1 file changed, 8 insertions(+), 18 deletions(-) diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 196fd2d627e..6610662c88f 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -1,16 +1,14 @@ #pragma once -#include - -#include -#include "Core/DecimalFunctions.h" +#include #include #include -#include #include -#if !defined(ARCADIA_BUILD) - #include // Y_IGNORE -#endif +#include +#include +#include + +#include namespace DB @@ -191,17 +189,9 @@ protected: sort_end = res.begin() + limit; if (reverse) -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; }); -#else - std::partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; }); -#endif + partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; }); else -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; }); -#else - std::partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; }); -#endif + partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; }); } }; From 2c9b77d5dd42865b0642b2de7f2811ce98bed4be Mon Sep 17 00:00:00 2001 From: myrrc Date: Sat, 14 Nov 2020 16:51:38 +0300 Subject: [PATCH 57/68] re-merged with master --- src/Columns/ColumnDecimal.cpp | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 8b3ff214713..a70c1a6ba6d 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -7,10 +7,8 @@ #include #include +#include #include -#if !defined(ARCADIA_BUILD) - #include // Y_IGNORE -#endif #include @@ -181,21 +179,11 @@ void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColum /// Since then we are working inside the interval. if (reverse) -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, + partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this](size_t a, size_t b) { return data[a] > data[b]; }); -#else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, - [this](size_t a, size_t b) { return data[a] > data[b]; }); -#endif else -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, + partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this](size_t a, size_t b) { return data[a] < data[b]; }); -#else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, - [this](size_t a, size_t b) { return data[a] > data[b]; }); -#endif auto new_first = first; for (auto j = first + 1; j < limit; ++j) { From 9ae14d8f190bdb32414bdde176762a824e2e429d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 14 Nov 2020 17:45:47 +0300 Subject: [PATCH 58/68] Update AvroRowInputFormat.cpp --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 5d5f977a22f..c41de7f4530 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -644,7 +644,7 @@ private: request.setHost(url.getHost()); auto session = makePooledHTTPSession(url, timeouts, 1); - std::istream * response_body; + std::istream * response_body{}; try { session->sendRequest(request); From d1504e420778c4333985b11816e16b71db84e888 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 14 Nov 2020 22:56:54 +0300 Subject: [PATCH 59/68] Update MonotonicityCheckVisitor.h --- src/Interpreters/MonotonicityCheckVisitor.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index 1e29964f04b..73c1e5497dd 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -133,6 +133,8 @@ public: static bool needChildVisit(const ASTPtr & parent, const ASTPtr &) { + /// Currently we check monotonicity only for single-argument functions. + /// Although, multi-argument functions with all but one constant arguments can also be monotonic. if (const auto * func = typeid_cast(parent.get())) return func->arguments->children.size() < 2; From df081e57cf38c4a36714b04488bf644fe7a6f5cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Nov 2020 23:22:15 +0300 Subject: [PATCH 60/68] Fix test --- .../0_stateless/01560_crash_in_agg_empty_arglist.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.reference b/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.reference index f33dfa25aa1..d00491fd7e5 100644 --- a/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.reference +++ b/tests/queries/0_stateless/01560_crash_in_agg_empty_arglist.reference @@ -1 +1 @@ -1 +1 From 30cc19efc6b15e707bdeb3c785afffd65171e3fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Nov 2020 23:37:32 +0300 Subject: [PATCH 61/68] Added results from Raspberry Pi --- website/benchmark/hardware/index.html | 3 +- .../hardware/results/do_xeon_6140_4.json | 56 +++++++++++++++++++ .../hardware/results/raspberry_pi_b.json | 54 ++++++++++++++++++ 3 files changed, 112 insertions(+), 1 deletion(-) create mode 100644 website/benchmark/hardware/results/do_xeon_6140_4.json create mode 100644 website/benchmark/hardware/results/raspberry_pi_b.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index 73503179bc0..41a87a4e49f 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -70,7 +70,8 @@ Results for AWS Lightsail is from Vamsi Krishna B.
Results for Dell XPS laptop and Google Pixel phone is from Alexander Kuzmenkov.
Results for Android phones for "cold cache" are done without cache flushing, so they are not "cold" and cannot be compared.
Results for Digital Ocean are from Zimin Aleksey.
-Results for 2x EPYC 7642 w/ 512 GB RAM (192 Cores) + 12X 1TB SSD (RAID6) are from Yiğit Konur and Metehan Çetinkaya of seo.do. +Results for 2x EPYC 7642 w/ 512 GB RAM (192 Cores) + 12X 1TB SSD (RAID6) are from Yiğit Konur and Metehan Çetinkaya of seo.do.
+Results for Raspberry Pi and Digital Ocean CPU-optimized are from Fritz Wijaya.

diff --git a/website/benchmark/hardware/results/do_xeon_6140_4.json b/website/benchmark/hardware/results/do_xeon_6140_4.json new file mode 100644 index 00000000000..284d2b5bacc --- /dev/null +++ b/website/benchmark/hardware/results/do_xeon_6140_4.json @@ -0,0 +1,56 @@ +[ + { + "system": "DigitalOcean CPU-opt 4", + "system_full": "DigitalOcean CPU-Optimized 4CPU/8GB RAM Intel(R) Xeon(R) Gold 6140", + "cpu_vendor": "Intel", + "cpu_model": "Xeon Gold 6140", + "time": "2020-11-14 00:00:00", + "kind": "cloud", + "result": + [ +[0.001, 0.002, 0.002], +[0.039, 0.046, 0.025], +[0.116, 0.087, 0.086], +[0.250, 0.120, 0.109], +[0.391, 0.313, 0.321], +[1.035, 0.946, 0.960], +[0.058, 0.047, 0.047], +[0.030, 0.026, 0.026], +[1.498, 1.368, 1.371], +[1.708, 1.568, 1.568], +[0.568, 0.480, 0.478], +[0.652, 0.568, 0.566], +[2.200, 1.968, 1.924], +[2.739, 2.561, 2.531], +[2.358, 2.208, 2.206], +[2.544, 2.407, 2.405], +[6.307, 5.914, 5.927], +[3.838, 3.608, 3.589], +[null, null, null], +[0.251, 0.121, 0.120], +[3.337, 2.447, 2.441], +[3.785, 2.669, 2.602], +[8.053, 6.082, 6.054], +[6.301, 2.976, 2.931], +[1.109, 0.816, 0.811], +[0.791, 0.693, 0.681], +[1.111, 0.821, 0.817], +[3.162, 2.162, 2.090], +[4.601, 3.854, 3.825], +[3.590, 3.560, 3.582], +[2.114, 1.847, 1.823], +[3.559, 2.851, 2.797], +[null, null, null], +[null, null, null], +[null, null, null], +[3.620, 3.446, 3.397], +[0.231, 0.196, 0.182], +[0.079, 0.066, 0.066], +[0.095, 0.059, 0.069], +[0.447, 0.382, 0.386], +[0.050, 0.034, 0.021], +[0.042, 0.016, 0.015], +[0.006, 0.008, 0.007] + ] + } +] diff --git a/website/benchmark/hardware/results/raspberry_pi_b.json b/website/benchmark/hardware/results/raspberry_pi_b.json new file mode 100644 index 00000000000..fff027f2bfe --- /dev/null +++ b/website/benchmark/hardware/results/raspberry_pi_b.json @@ -0,0 +1,54 @@ +[ + { + "system": "Raspberry Pi 4", + "system_full": "Raspberry Pi 4 Model B 8GB", + "time": "2020-11-14 00:00:00", + "kind": "desktop", + "result": + [ +[0.015, 0.005, 0.005], +[0.214, 0.176, 0.171], +[1.205, 0.535, 0.534], +[5.138, 1.320, 1.342], +[5.129, 1.489, 1.494], +[8.349, 3.230, 3.290], +[0.327, 0.234, 0.231], +[0.228, 0.183, 0.175], +[7.530, 5.440, 5.662], +[9.073, 6.912, 6.881], +[5.609, 2.995, 3.201], +[5.817, 3.245, 3.239], +[12.712, 11.231, 11.141], +[18.517, 14.798, 14.698], +[13.510, 11.171, 11.260], +[12.944, 11.576, 11.706], +[28.042, 23.958, 22.930], +[18.430, 13.992, 14.173], +[null, null, null], +[5.193, 1.342, 1.311], +[59.597, 19.483, 20.791], +[68.012, 24.377, 24.159], +[127.859, 49.266, 47.251], +[133.812, 25.078, 24.812], +[16.838, 5.128, 4.824], +[8.195, 4.025, 4.066], +[16.791, 4.911, 4.997], +[59.740, 24.009, 23.916], +[50.460, 25.922, 26.049], +[23.961, 23.536, 23.835], +[15.293, 8.960, 8.687], +[36.904, 14.905, 14.755], +[null, null, null], +[74.268, 74.887, 74.103], +[74.727, 59.369, 65.550], +[15.400, 14.807, 15.437], +[1.286, 0.836, 0.804], +[0.501, 0.341, 0.320], +[0.704, 0.299, 0.265], +[2.539, 1.756, 1.710], +[0.345, 0.085, 0.082], +[0.219, 0.070, 0.072], +[0.044, 0.021, 0.023] + ] + } +] From de5ead0c403c3413d54cd4ce428c72151143a774 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 15 Nov 2020 05:24:47 +0300 Subject: [PATCH 62/68] add setting remove_empty_parts --- src/Storages/MergeTree/MergeTreeData.cpp | 3 +++ src/Storages/MergeTree/MergeTreeSettings.h | 1 + tests/queries/0_stateless/00933_alter_ttl.reference | 2 +- tests/queries/0_stateless/00933_alter_ttl.sql | 5 ++--- tests/queries/0_stateless/00933_ttl_simple.sql | 12 +++++++++--- .../queries/0_stateless/00976_ttl_with_old_parts.sql | 2 +- .../01006_simpod_empty_part_single_column_write.sh | 2 +- .../01047_simple_aggregate_sizes_of_columns_bug.sql | 2 +- 8 files changed, 19 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a5d4bd87a7a..a11dd25d8e3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1217,6 +1217,9 @@ void MergeTreeData::clearOldWriteAheadLogs() void MergeTreeData::clearEmptyParts() { + if (!getSettings()->remove_empty_parts) + return; + auto parts = getDataPartsVector(); for (const auto & part : parts) { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index fe97298fe75..6f0401e061b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -105,6 +105,7 @@ struct Settings; M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ M(String, storage_policy, "default", "Name of storage disk policy", 0) \ M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ + M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm", 0) \ \ /** Settings for testing purposes */ \ M(Bool, randomize_part_type, false, "For testing purposes only. Randomizes part type between wide and compact", 0) \ diff --git a/tests/queries/0_stateless/00933_alter_ttl.reference b/tests/queries/0_stateless/00933_alter_ttl.reference index 545f5644e94..ca46a6e70ce 100644 --- a/tests/queries/0_stateless/00933_alter_ttl.reference +++ b/tests/queries/0_stateless/00933_alter_ttl.reference @@ -1,4 +1,4 @@ -CREATE TABLE default.ttl\n(\n `d` Date,\n `a` Int32\n)\nENGINE = MergeTree\nPARTITION BY toDayOfMonth(d)\nORDER BY a\nTTL d + toIntervalDay(1)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.ttl\n(\n `d` Date,\n `a` Int32\n)\nENGINE = MergeTree\nPARTITION BY toDayOfMonth(d)\nORDER BY a\nTTL d + toIntervalDay(1)\nSETTINGS remove_empty_parts = 0, index_granularity = 8192 2100-10-10 3 2100-10-10 4 d Date diff --git a/tests/queries/0_stateless/00933_alter_ttl.sql b/tests/queries/0_stateless/00933_alter_ttl.sql index 1aacb9a6694..f3f6858cbb4 100644 --- a/tests/queries/0_stateless/00933_alter_ttl.sql +++ b/tests/queries/0_stateless/00933_alter_ttl.sql @@ -2,14 +2,13 @@ set send_logs_level = 'fatal'; drop table if exists ttl; -create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d); +create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) settings remove_empty_parts = 0; alter table ttl modify ttl d + interval 1 day; show create table ttl; insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3); insert into ttl values (toDateTime('2100-10-10 00:00:00'), 4); -select sleep(1) format Null; -- wait if very fast merge happen optimize table ttl partition 10 final; select * from ttl order by d; @@ -18,7 +17,7 @@ alter table ttl modify ttl a; -- { serverError 450 } drop table if exists ttl; -create table ttl (d Date, a Int) engine = MergeTree order by tuple() partition by toDayOfMonth(d); +create table ttl (d Date, a Int) engine = MergeTree order by tuple() partition by toDayOfMonth(d) settings remove_empty_parts = 0; alter table ttl modify column a Int ttl d + interval 1 day; desc table ttl; alter table ttl modify column d Int ttl d + interval 1 day; -- { serverError 43 } diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index aa8b33b2999..2bf686822d5 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -11,7 +11,9 @@ select a, b from ttl_00933_1; drop table if exists ttl_00933_1; -create table ttl_00933_1 (d DateTime, a Int, b Int) engine = MergeTree order by toDate(d) partition by tuple() ttl d + interval 1 second; +create table ttl_00933_1 (d DateTime, a Int, b Int) + engine = MergeTree order by toDate(d) partition by tuple() ttl d + interval 1 second + settings remove_empty_parts = 0; insert into ttl_00933_1 values (now(), 1, 2); insert into ttl_00933_1 values (now(), 3, 4); insert into ttl_00933_1 values (now() + 1000, 5, 6); @@ -30,7 +32,9 @@ select * from ttl_00933_1 order by d; drop table if exists ttl_00933_1; -create table ttl_00933_1 (d DateTime, a Int) engine = MergeTree order by tuple() partition by tuple() ttl d + interval 1 day; +create table ttl_00933_1 (d DateTime, a Int) + engine = MergeTree order by tuple() partition by tuple() ttl d + interval 1 day + settings remove_empty_parts = 0; insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl_00933_1 values (toDateTime('2100-10-10 00:00:00'), 3); @@ -39,7 +43,9 @@ select * from ttl_00933_1 order by d; drop table if exists ttl_00933_1; -create table ttl_00933_1 (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) ttl d + interval 1 day; +create table ttl_00933_1 (d Date, a Int) + engine = MergeTree order by a partition by toDayOfMonth(d) ttl d + interval 1 day + settings remove_empty_parts = 0; insert into ttl_00933_1 values (toDate('2000-10-10'), 1); insert into ttl_00933_1 values (toDate('2100-10-10'), 2); optimize table ttl_00933_1 final; diff --git a/tests/queries/0_stateless/00976_ttl_with_old_parts.sql b/tests/queries/0_stateless/00976_ttl_with_old_parts.sql index 0ed69850cf6..60151595983 100644 --- a/tests/queries/0_stateless/00976_ttl_with_old_parts.sql +++ b/tests/queries/0_stateless/00976_ttl_with_old_parts.sql @@ -1,6 +1,6 @@ drop table if exists ttl; -create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d); +create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) settings remove_empty_parts = 0; insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3); diff --git a/tests/queries/0_stateless/01006_simpod_empty_part_single_column_write.sh b/tests/queries/0_stateless/01006_simpod_empty_part_single_column_write.sh index 3405b85089c..5473185af3d 100755 --- a/tests/queries/0_stateless/01006_simpod_empty_part_single_column_write.sh +++ b/tests/queries/0_stateless/01006_simpod_empty_part_single_column_write.sh @@ -16,7 +16,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE table_with_empty_part ENGINE = MergeTree() ORDER BY id PARTITION BY id -SETTINGS vertical_merge_algorithm_min_rows_to_activate=0, vertical_merge_algorithm_min_columns_to_activate=0 +SETTINGS vertical_merge_algorithm_min_rows_to_activate=0, vertical_merge_algorithm_min_columns_to_activate=0, remove_empty_parts = 0 " diff --git a/tests/queries/0_stateless/01047_simple_aggregate_sizes_of_columns_bug.sql b/tests/queries/0_stateless/01047_simple_aggregate_sizes_of_columns_bug.sql index 7213aaa16c7..a2ea13b26fc 100644 --- a/tests/queries/0_stateless/01047_simple_aggregate_sizes_of_columns_bug.sql +++ b/tests/queries/0_stateless/01047_simple_aggregate_sizes_of_columns_bug.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS column_size_bug; -CREATE TABLE column_size_bug (date_time DateTime, value SimpleAggregateFunction(sum,UInt64)) ENGINE = AggregatingMergeTree PARTITION BY toStartOfInterval(date_time, INTERVAL 1 DAY) ORDER BY (date_time); +CREATE TABLE column_size_bug (date_time DateTime, value SimpleAggregateFunction(sum,UInt64)) ENGINE = AggregatingMergeTree PARTITION BY toStartOfInterval(date_time, INTERVAL 1 DAY) ORDER BY (date_time) SETTINGS remove_empty_parts = 0; INSERT INTO column_size_bug VALUES(now(),1); INSERT INTO column_size_bug VALUES(now(),1); From 3c69dbe42e9b1082170660ad60c65b74106a9a70 Mon Sep 17 00:00:00 2001 From: Artur Date: Sun, 15 Nov 2020 12:52:41 +0300 Subject: [PATCH 63/68] Fix arrayAUC function header in documentation --- docs/ru/sql-reference/functions/array-functions.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index 62181eebb4b..16f21e3a80c 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -1157,6 +1157,7 @@ SELECT arrayCumSum([1, 1, 1, 1]) AS res ┌─res──────────┐ │ [1, 2, 3, 4] │ └──────────────┘ +``` ## arrayAUC {#arrayauc} From 5c801c42c17c5519e6970fa10d0a402f89388b7e Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sun, 15 Nov 2020 23:48:46 +0400 Subject: [PATCH 64/68] Fix ROCKSDB_ERROR value --- src/Common/ErrorCodes.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 405b8c60af8..e3411096fbc 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -519,9 +519,9 @@ M(550, CONDITIONAL_TREE_PARENT_NOT_FOUND) \ M(551, ILLEGAL_PROJECTION_MANIPULATOR) \ M(552, UNRECOGNIZED_ARGUMENTS) \ - M(553, ROCKSDB_ERROR) \ M(553, LZMA_STREAM_ENCODER_FAILED) \ M(554, LZMA_STREAM_DECODER_FAILED) \ + M(555, ROCKSDB_ERROR) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ From 01ec7aca6877fdef70015cd3f88cdf044db403e3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 15 Nov 2020 23:08:18 +0300 Subject: [PATCH 65/68] Add missing for RadixSort.h Fails compiling quantile-t-digest.cpp test on: - clang 11 - libstdc++ --- src/Common/RadixSort.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/RadixSort.h b/src/Common/RadixSort.h index 22e93a2c324..7ceb8569bd1 100644 --- a/src/Common/RadixSort.h +++ b/src/Common/RadixSort.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include From 5c6191f79827a7dc25871cff05f926ad5e805f58 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 16 Nov 2020 00:31:52 +0300 Subject: [PATCH 66/68] fix --- src/Interpreters/StorageID.cpp | 9 --------- src/Interpreters/StorageID.h | 1 - src/Storages/StorageMaterializedView.cpp | 5 ++--- 3 files changed, 2 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/StorageID.cpp b/src/Interpreters/StorageID.cpp index a7d02601dbf..2d6a4900dd3 100644 --- a/src/Interpreters/StorageID.cpp +++ b/src/Interpreters/StorageID.cpp @@ -79,15 +79,6 @@ bool StorageID::operator<(const StorageID & rhs) const return !hasUUID(); } -bool StorageID::operator==(const StorageID & rhs) const -{ - assertNotEmpty(); - if (!hasUUID() && !rhs.hasUUID()) - return std::tie(database_name, table_name) == std::tie(rhs.database_name, rhs.table_name); - else - return hasUUID() && rhs.hasUUID() && uuid == rhs.uuid; -} - String StorageID::getFullTableName() const { return backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name); diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index d42dfda06fd..9343f67fe7a 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -68,7 +68,6 @@ struct StorageID return uuid != UUIDHelpers::Nil; } - bool operator==(const StorageID & rhs) const; bool operator<(const StorageID & rhs) const; void assertNotEmpty() const diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index fd75807eb2e..dcf7105d728 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -74,6 +74,8 @@ StorageMaterializedView::StorageMaterializedView( if (!has_inner_table) { + if (query.to_table_id.database_name == table_id_.database_name && query.to_table_id.table_name == table_id_.table_name) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Materialized view {} cannot point to itself", table_id_.getFullTableName()); target_table_id = query.to_table_id; } else if (attach_) @@ -101,9 +103,6 @@ StorageMaterializedView::StorageMaterializedView( target_table_id = DatabaseCatalog::instance().getTable({manual_create_query->database, manual_create_query->table}, global_context)->getStorageID(); } - if (target_table_id == getStorageID()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Materialized view {} cannot point to itself", getStorageID().getFullTableName()); - if (!select.select_table_id.empty()) DatabaseCatalog::instance().addDependency(select.select_table_id, getStorageID()); } From faf0195d33904f98d09df56f4677aea79fb47059 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Mon, 16 Nov 2020 06:16:31 +0000 Subject: [PATCH 67/68] Bump protobuf from 3.13.0 to 3.14.0 in /docs/tools Bumps [protobuf](https://github.com/protocolbuffers/protobuf) from 3.13.0 to 3.14.0. - [Release notes](https://github.com/protocolbuffers/protobuf/releases) - [Changelog](https://github.com/protocolbuffers/protobuf/blob/master/generate_changelog.py) - [Commits](https://github.com/protocolbuffers/protobuf/compare/v3.13.0...v3.14.0) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 3560b6ad40a..4106100bfa3 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -21,7 +21,7 @@ mkdocs-htmlproofer-plugin==0.0.3 mkdocs-macros-plugin==0.4.20 nltk==3.5 nose==1.3.7 -protobuf==3.13.0 +protobuf==3.14.0 numpy==1.19.2 Pygments==2.5.2 pymdown-extensions==8.0 From 4d33e79cb402376e98edd4dfaaafe315b9c5e393 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 16 Nov 2020 14:31:52 +0300 Subject: [PATCH 68/68] Update ColumnDecimal.h --- src/Columns/ColumnDecimal.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 6610662c88f..46f7cfe581e 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -82,7 +82,7 @@ public: bool isNumeric() const override { return false; } bool canBeInsideNullable() const override { return true; } - bool isFixedAndContiguous() const final { return true; } + bool isFixedAndContiguous() const override { return true; } size_t sizeOfValueIfFixed() const override { return sizeof(T); } size_t size() const override { return data.size(); }