diff --git a/src/DataTypes/DataTypeDecimalBase.h b/src/DataTypes/DataTypeDecimalBase.h index 0be345ba879..adbe9c95b14 100644 --- a/src/DataTypes/DataTypeDecimalBase.h +++ b/src/DataTypes/DataTypeDecimalBase.h @@ -71,9 +71,12 @@ public: scale(scale_) { if (unlikely(precision < 1 || precision > maxPrecision())) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Precision {} is out of bounds", std::to_string(precision)); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Precision {} is out of bounds (precision range: [1, {}])", + std::to_string(precision), maxPrecision()); if (unlikely(scale > maxPrecision())) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds", std::to_string(scale)); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds (max scale: {})", + std::to_string(scale), maxPrecision()); } TypeIndex getTypeId() const override { return TypeToTypeIndex; } diff --git a/src/DataTypes/DataTypesDecimal.h b/src/DataTypes/DataTypesDecimal.h index 7a49238b5be..583f7ea804a 100644 --- a/src/DataTypes/DataTypesDecimal.h +++ b/src/DataTypes/DataTypesDecimal.h @@ -116,7 +116,8 @@ inline ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & v if (common::mulOverflow(static_cast(value.value), converted_value, converted_value)) { if constexpr (throw_exception) - throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow", std::string(ToDataType::family_name)); + throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow while multiplying {} by scale {}", + std::string(ToDataType::family_name), toString(value.value), toString(converted_value)); else return ReturnType(false); } @@ -136,7 +137,10 @@ inline ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & v converted_value > std::numeric_limits::max()) { if constexpr (throw_exception) - throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow", std::string(ToDataType::family_name)); + throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "{} convert overflow: {} is not in range ({}, {})", + std::string(ToDataType::family_name), toString(converted_value), + toString(std::numeric_limits::min()), + toString(std::numeric_limits::max())); else return ReturnType(false); } diff --git a/src/Functions/array/arrayAggregation.cpp b/src/Functions/array/arrayAggregation.cpp index 59991b7b313..8818ebde9f1 100644 --- a/src/Functions/array/arrayAggregation.cpp +++ b/src/Functions/array/arrayAggregation.cpp @@ -223,7 +223,8 @@ struct ArrayAggregateImpl auto result_scale = column_typed->getScale() * array_size; if (unlikely(result_scale > DecimalUtils::max_precision)) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds", result_scale); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds (max scale: {})", + result_scale, DecimalUtils::max_precision); res[i] = DecimalUtils::convertTo(product, static_cast(result_scale)); } @@ -332,7 +333,8 @@ struct ArrayAggregateImpl auto result_scale = column->getScale() * count; if (unlikely(result_scale > DecimalUtils::max_precision)) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds", result_scale); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds (max scale: {})", + result_scale, DecimalUtils::max_precision); res[i] = DecimalUtils::convertTo(aggregate_value, static_cast(result_scale)); } diff --git a/src/Functions/pointInEllipses.cpp b/src/Functions/pointInEllipses.cpp index 208f2ad2f82..2147428cee3 100644 --- a/src/Functions/pointInEllipses.cpp +++ b/src/Functions/pointInEllipses.cpp @@ -71,8 +71,9 @@ private: /// For array on stack, see below. if (arguments.size() > 10000) { - throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Number of arguments of function {} is too large.", - getName()); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, + "Number of arguments of function {} is too large (maximum: 10000).", + getName()); } for (const auto arg_idx : collections::range(0, arguments.size())) diff --git a/src/Functions/pointInPolygon.cpp b/src/Functions/pointInPolygon.cpp index 24ad1d20611..0e4467a8210 100644 --- a/src/Functions/pointInPolygon.cpp +++ b/src/Functions/pointInPolygon.cpp @@ -37,7 +37,7 @@ namespace DB { namespace ErrorCodes { - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; @@ -87,7 +87,7 @@ public: { if (arguments.size() < 2) { - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Too few arguments"); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least 2 arguments", getName()); } /** We allow function invocation in one of the following forms: diff --git a/src/Functions/svg.cpp b/src/Functions/svg.cpp index 69e619df901..f8f85216b3f 100644 --- a/src/Functions/svg.cpp +++ b/src/Functions/svg.cpp @@ -13,8 +13,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } class FunctionSvg : public IFunction @@ -48,13 +47,9 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments.size() > 2) + if (arguments.empty() || arguments.size() > 2) { - throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Too many arguments"); - } - else if (arguments.empty()) - { - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Too few arguments"); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Incorrect number of arguments: expected 1 or 2 arguments"); } else if (arguments.size() == 2 && checkAndGetDataType(arguments[1].get()) == nullptr) { diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index e16403bed67..0beb4492aef 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -355,19 +355,22 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, /// Flush should not be done if shouldBeEmptyOnDetach() == false, /// since in this case getTablesIterator() may do some additional work, /// see DatabaseMaterializedMySQL::getTablesIterator() - for (auto iterator = database->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) - { - iterator->table()->flush(); - } - auto table_context = Context::createCopy(getContext()); table_context->setInternalQuery(true); + /// Do not hold extra shared pointers to tables + std::vector> tables_to_drop; for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next()) { + iterator->table()->flush(); + tables_to_drop.push_back({iterator->name(), iterator->table()->isDictionary()}); + } + + for (const auto & table : tables_to_drop) + { + query_for_table.setTable(table.first); + query_for_table.is_dictionary = table.second; DatabasePtr db; UUID table_to_wait = UUIDHelpers::Nil; - query_for_table.setTable(iterator->name()); - query_for_table.is_dictionary = iterator->table()->isDictionary(); executeToTableImpl(table_context, query_for_table, db, table_to_wait); uuids_to_wait.push_back(table_to_wait); } @@ -428,7 +431,8 @@ AccessRightsElements InterpreterDropQuery::getRequiredAccessForDDLOnCluster() co return required_access; } -void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool sync) +void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, + const StorageID & target_table_id, bool sync, bool ignore_sync_setting) { if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context)) { @@ -445,6 +449,8 @@ void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr /// and not allowed to drop inner table explicitly. Allowing to drop inner table without explicit grant /// looks like expected behaviour and we have tests for it. auto drop_context = Context::createCopy(global_context); + if (ignore_sync_setting) + drop_context->setSetting("database_atomic_wait_for_drop_and_detach_synchronously", false); drop_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; if (auto txn = current_context->getZooKeeperMetadataTransaction()) { diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index afec26424ba..af7a4ddef25 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -24,7 +24,8 @@ public: /// Drop table or database. BlockIO execute() override; - static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool sync); + static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, + const StorageID & target_table_id, bool sync, bool ignore_sync_setting = false); bool supportsTransactions() const override; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index b45618be1f8..e9905821fd1 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -604,6 +604,7 @@ void InterpreterSystemQuery::restoreReplica() StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, ContextMutablePtr system_context, bool need_ddl_guard) { + LOG_TRACE(log, "Restarting replica {}", replica); auto table_ddl_guard = need_ddl_guard ? DatabaseCatalog::instance().getDDLGuard(replica.getDatabaseName(), replica.getTableName()) : nullptr; @@ -647,6 +648,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, database->attachTable(system_context, replica.table_name, table, data_path); table->startup(); + LOG_TRACE(log, "Restarted replica {}", replica); return table; } @@ -693,11 +695,11 @@ void InterpreterSystemQuery::restartReplicas(ContextMutablePtr system_context) guard.second = catalog.getDDLGuard(guard.first.database_name, guard.first.table_name); size_t threads = std::min(static_cast(getNumberOfPhysicalCPUCores()), replica_names.size()); + LOG_DEBUG(log, "Will restart {} replicas using {} threads", replica_names.size(), threads); ThreadPool pool(CurrentMetrics::RestartReplicaThreads, CurrentMetrics::RestartReplicaThreadsActive, threads); for (auto & replica : replica_names) { - LOG_TRACE(log, "Restarting replica on {}", replica.getNameForLogs()); pool.scheduleOrThrowOnError([&]() { tryRestartReplica(replica, system_context, false); }); } pool.wait(); diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index d194c8b8201..78e72564ab7 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -250,7 +250,7 @@ void StorageMaterializedPostgreSQL::dropInnerTableIfAny(bool sync, ContextPtr lo auto nested_table = tryGetNested() != nullptr; if (nested_table) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, getNestedStorageID(), sync); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, getNestedStorageID(), sync, /* ignore_sync_setting */ true); } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index ae3fa62b38c..b96c132d601 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -212,13 +212,26 @@ void StorageMaterializedView::drop() if (!select_query.select_table_id.empty()) DatabaseCatalog::instance().removeViewDependency(select_query.select_table_id, table_id); - dropInnerTableIfAny(true, getContext()); + /// Sync flag and the setting make sense for Atomic databases only. + /// However, with Atomic databases, IStorage::drop() can be called only from a background task in DatabaseCatalog. + /// Running synchronous DROP from that task leads to deadlock. + /// Usually dropInnerTableIfAny is no-op, because the inner table is dropped before enqueueing a drop task for the MV itself. + /// But there's a race condition with SYSTEM RESTART REPLICA: the inner table might be detached due to RESTART. + /// In this case, dropInnerTableIfAny will not find the inner table and will not drop it during executions of DROP query for the MV itself. + /// DDLGuard does not protect from that, because RESTART REPLICA acquires DDLGuard for the inner table name, + /// but DROP acquires DDLGuard for the name of MV. And we cannot acquire second DDLGuard for the inner name in DROP, + /// because it may lead to lock-order-inversion (DDLGuards must be acquired in lexicographical order). + dropInnerTableIfAny(/* sync */ false, getContext()); } void StorageMaterializedView::dropInnerTableIfAny(bool sync, ContextPtr local_context) { + /// We will use `sync` argument wneh this function is called from a DROP query + /// and will ignore database_atomic_wait_for_drop_and_detach_synchronously when it's called from drop task. + /// See the comment in StorageMaterializedView::drop if (has_inner_table && tryGetTargetTable()) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, sync); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, + sync, /* ignore_sync_setting */ true); } void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 3471e4ea6bf..bfa126c3525 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1609,7 +1609,7 @@ void StorageWindowView::drop() { /// Must be guaranteed at this point for database engine Atomic that has_inner_table == false, /// because otherwise will be a deadlock. - dropInnerTableIfAny(true, getContext()); + dropInnerTableIfAny(false, getContext()); } void StorageWindowView::dropInnerTableIfAny(bool sync, ContextPtr local_context) @@ -1623,7 +1623,7 @@ void StorageWindowView::dropInnerTableIfAny(bool sync, ContextPtr local_context) ASTDropQuery::Kind::Drop, getContext(), local_context, inner_table_id, sync); if (has_inner_target_table) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, sync); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, sync, /* ignore_sync_setting */ true); } catch (...) { diff --git a/tests/ci/stress_tests.lib b/tests/ci/stress_tests.lib index 75195baaeeb..04df50b3248 100644 --- a/tests/ci/stress_tests.lib +++ b/tests/ci/stress_tests.lib @@ -149,9 +149,14 @@ function stop() if [ $check_hang == true ] then # We failed to stop the server with SIGTERM. Maybe it hang, let's collect stacktraces. - echo -e "Possible deadlock on shutdown (see gdb.log)$FAIL" >> /test_output/test_results.tsv + # Add a special status just in case, so it will be possible to find in the CI DB + echo -e "Warning: server did not stop yet$OK" >> /test_output/test_results.tsv kill -TERM "$(pidof gdb)" ||: sleep 5 + + # The server could finally stop while we were terminating gdb, let's recheck if it's still running + kill -s 0 $pid || return + echo -e "Possible deadlock on shutdown (see gdb.log)$FAIL" >> /test_output/test_results.tsv echo "thread apply all backtrace (on stop)" >> /test_output/gdb.log timeout 30m gdb -batch -ex 'thread apply all backtrace' -p "$pid" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log clickhouse stop --force diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a355c2f8e73..d407f73d033 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -278,7 +278,7 @@ def need_retry(args, stdout, stderr, total_time): def get_processlist_with_stacktraces(args): try: if args.replicated_database: - return clickhouse_execute_json( + return clickhouse_execute( args, """ SELECT materialize(hostName() || '::' || tcpPort()::String) as host_port, * @@ -295,14 +295,14 @@ def get_processlist_with_stacktraces(args): WHERE query NOT LIKE '%system.processes%' GROUP BY p.* )) - ORDER BY elapsed DESC + ORDER BY elapsed DESC FORMAT Vertical """, settings={ "allow_introspection_functions": 1, }, ) else: - return clickhouse_execute_json( + return clickhouse_execute( args, """ SELECT @@ -315,7 +315,7 @@ def get_processlist_with_stacktraces(args): JOIN system.stack_trace s USING (query_id) WHERE query NOT LIKE '%system.processes%' GROUP BY p.* - ORDER BY elapsed DESC + ORDER BY elapsed DESC FORMAT Vertical """, settings={ "allow_introspection_functions": 1, @@ -2058,7 +2058,10 @@ def reportLogStats(args): 'Table {} is not replicated', '{} {}.{} already exists', 'Attempt to read after eof', 'Replica {} already exists', 'Convert overflow', 'key must be a tuple', 'Division by zero', 'No part {} in committed state', 'Files set to {}', 'Bytes set to {}', 'Sharding key {} is not used', - 'Cannot parse datetime', 'Bad get: has {}, requested {}', 'There is no {} in {}', 'Numeric overflow' + 'Cannot parse datetime', 'Bad get: has {}, requested {}', 'There is no {} in {}', 'Numeric overflow', + 'Polygon is not valid: {}', 'Decimal math overflow', '{} only accepts maps', 'Dictionary ({}) not found', + 'Unknown format {}', 'Invalid IPv4 value', 'Invalid IPv6 value', 'Unknown setting {}', + 'Unknown table function {}' ) AS known_short_messages SELECT count() AS c, message_format_string, substr(any(message), 1, 120) FROM system.text_log diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index 6b27c69462a..67f7ce47451 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -91,8 +91,8 @@ def test_mutate_and_upgrade(start_cluster): node2.query("OPTIMIZE TABLE mt FINAL") - assert node1.query("SELECT id FROM mt") == "1\n4\n" - assert node2.query("SELECT id FROM mt") == "1\n4\n" + assert node1.query("SELECT id FROM mt ORDER BY id") == "1\n4\n" + assert node2.query("SELECT id FROM mt ORDER BY id") == "1\n4\n" for node in [node1, node2]: node.query("DROP TABLE mt") diff --git a/tests/queries/0_stateless/02437_drop_mv_restart_replicas.reference b/tests/queries/0_stateless/02437_drop_mv_restart_replicas.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02437_drop_mv_restart_replicas.sh b/tests/queries/0_stateless/02437_drop_mv_restart_replicas.sh new file mode 100755 index 00000000000..e4f52b6e4ad --- /dev/null +++ b/tests/queries/0_stateless/02437_drop_mv_restart_replicas.sh @@ -0,0 +1,63 @@ +#!/usr/bin/env bash +# Tags: long, zookeeper, race, no-ordinary-database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "create user u_$CLICKHOUSE_DATABASE" +$CLICKHOUSE_CLIENT -q "grant all on db_$CLICKHOUSE_DATABASE.* to u_$CLICKHOUSE_DATABASE" + +# For tests with Replicated +ENGINE=$($CLICKHOUSE_CLIENT -q "select replace(engine_full, '$CLICKHOUSE_DATABASE', 'db_$CLICKHOUSE_DATABASE') from system.databases where name='$CLICKHOUSE_DATABASE' format TSVRaw") +export ENGINE + +function thread_ddl() +{ + while true; do + $CLICKHOUSE_CLIENT -q "create database if not exists db_$CLICKHOUSE_DATABASE engine=$ENGINE" + $CLICKHOUSE_CLIENT -q "CREATE TABLE if not exists db_$CLICKHOUSE_DATABASE.test (test String, A Int64, B Int64) ENGINE = ReplicatedMergeTree ('/clickhouse/tables/{database}/test_02124/{table}', '1') ORDER BY tuple();" + $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW if not exists db_$CLICKHOUSE_DATABASE.test_mv_a Engine=ReplicatedMergeTree ('/clickhouse/tables/{database}/test_02124/{table}', '1') order by tuple() AS SELECT test, A, count() c FROM db_$CLICKHOUSE_DATABASE.test group by test, A;" + $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW if not exists db_$CLICKHOUSE_DATABASE.test_mv_b Engine=ReplicatedMergeTree ('/clickhouse/tables/{database}/test_02124/{table}', '1') partition by A order by tuple() AS SELECT test, A, count() c FROM db_$CLICKHOUSE_DATABASE.test group by test, A;" + $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW if not exists db_$CLICKHOUSE_DATABASE.test_mv_c Engine=ReplicatedMergeTree ('/clickhouse/tables/{database}/test_02124/{table}', '1') order by tuple() AS SELECT test, A, count() c FROM db_$CLICKHOUSE_DATABASE.test group by test, A;" + sleep 0.$RANDOM; + + # A kind of backoff + timeout 5s $CLICKHOUSE_CLIENT -q "select sleepEachRow(0.1) from system.dropped_tables format Null" 2>/dev/null ||: + + $CLICKHOUSE_CLIENT -q "drop database if exists db_$CLICKHOUSE_DATABASE" + done +} + +function thread_insert() +{ + while true; do + $CLICKHOUSE_CLIENT -q "INSERT INTO db_$CLICKHOUSE_DATABASE.test SELECT 'case1', number%3, rand() FROM numbers(5)" + sleep 0.$RANDOM; + done +} + +function thread_restart() +{ + while true; do + # The simplest way to restart only replicas from a specific database is to use a special user + $CLICKHOUSE_CLIENT --user "u_$CLICKHOUSE_DATABASE" -q "system restart replicas" + sleep 0.$RANDOM; + done +} + +export -f thread_ddl; +export -f thread_insert; +export -f thread_restart; + +TIMEOUT=15 + +timeout $TIMEOUT bash -c thread_ddl 2>&1| grep -Fa "Exception: " | grep -Fv -e "TABLE_IS_DROPPED" -e "UNKNOWN_TABLE" -e "DATABASE_NOT_EMPTY" & +timeout $TIMEOUT bash -c thread_insert 2> /dev/null & +timeout $TIMEOUT bash -c thread_restart 2>&1| grep -Fa "Exception: " | grep -Fv -e "is currently dropped or renamed" & + +wait + +timeout 45s $CLICKHOUSE_CLIENT -q "select sleepEachRow(0.3) from system.dropped_tables format Null" 2>/dev/null ||: + +$CLICKHOUSE_CLIENT -q "drop database if exists db_$CLICKHOUSE_DATABASE" 2>&1| grep -Fa "Exception: " | grep -Fv -e "TABLE_IS_DROPPED" -e "UNKNOWN_TABLE" -e "DATABASE_NOT_EMPTY" ||: