This commit is contained in:
Alexander Tokmakov 2021-01-25 21:59:23 +03:00
parent 3bd4d97353
commit 3146a1a954
5 changed files with 21 additions and 9 deletions

View File

@ -22,12 +22,15 @@ def get_options(i):
if 0 < i:
options += " --order=random"
if i % 2 == 1:
if i % 3 == 1:
options += " --db-engine=Ordinary"
if i % 3 == 2:
options += ''' --db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i)
# If database name is not specified, new database is created for each functional test.
# Run some threads with one database for all tests.
if i % 3 == 1:
if i % 2 == 1:
options += " --database=test_{}".format(i)
if i == 13:

View File

@ -140,7 +140,7 @@ bool DDLTask::findCurrentHostID(const Context & global_context, Poco::Logger * l
void DDLTask::setClusterInfo(const Context & context, Poco::Logger * log)
{
auto query_on_cluster = dynamic_cast<ASTQueryWithOnCluster *>(query.get());
auto * query_on_cluster = dynamic_cast<ASTQueryWithOnCluster *>(query.get());
if (!query_on_cluster)
throw Exception("Received unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY);

View File

@ -201,11 +201,7 @@ void DDLWorker::shutdown()
stop_flag = true;
queue_updated_event->set();
cleanup_event->set();
}
DDLWorker::~DDLWorker()
{
shutdown();
worker_pool.reset();
if (main_thread.joinable())
main_thread.join();
@ -213,6 +209,11 @@ DDLWorker::~DDLWorker()
cleanup_thread.join();
}
DDLWorker::~DDLWorker()
{
shutdown();
}
ZooKeeperPtr DDLWorker::tryGetZooKeeper() const
{
@ -490,9 +491,14 @@ void DDLWorker::processTask(DDLTaskBase & task)
}
if (task.execute_on_leader)
{
tryExecuteQueryOnLeaderReplica(task, storage, rewritten_query, task.entry_path, zookeeper);
}
else
{
storage.reset();
tryExecuteQuery(rewritten_query, task);
}
}
catch (const Coordination::Exception &)
{
@ -892,6 +898,7 @@ void DDLWorker::initializeMainThread()
{
tryLogCurrentException(log, "Cannot initialize DDL queue.");
reset_state(false);
sleepForSeconds(5);
}
}
while (!initialized && !stop_flag);
@ -949,11 +956,13 @@ void DDLWorker::runMainThread()
LOG_ERROR(log, "Unexpected ZooKeeper error: {}", getCurrentExceptionMessage(true));
reset_state();
}
sleepForSeconds(5);
}
catch (...)
{
tryLogCurrentException(log, "Unexpected error, will try to restart main thread:");
reset_state();
sleepForSeconds(5);
}
}
}

View File

@ -718,7 +718,7 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
const auto * kind = create.is_dictionary ? "Dictionary" : "Table";
const auto * kind_upper = create.is_dictionary ? "DICTIONARY" : "TABLE";
if (database->getEngineName() == "Replicated" && context.getClientInfo().query_kind == ClientInfo::QueryKind::REPLICATED_LOG_QUERY)
if (database->getEngineName() == "Replicated" && context.getClientInfo().query_kind == ClientInfo::QueryKind::REPLICATED_LOG_QUERY && !internal)
{
if (create.uuid == UUIDHelpers::Nil)
throw Exception("Table UUID is not specified in DDL log", ErrorCodes::LOGICAL_ERROR);

View File

@ -14,7 +14,7 @@ DOCKER_COMPOSE_PATH = get_docker_compose_path()
cluster = ClickHouseCluster(__file__)
node_db_ordinary = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=False, stay_alive=True)
node_db_ordinary = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=False, stay_alive=True, with_zookeeper=True) #FIXME
node_db_atomic = cluster.add_instance('node2', user_configs=["configs/users_db_atomic.xml"], with_mysql=False, stay_alive=True)