fix tests

This commit is contained in:
Alexander Tokmakov 2022-06-29 16:27:21 +02:00
parent c6c22409a2
commit 894acf0c5c
6 changed files with 25 additions and 25 deletions

View File

@ -325,7 +325,7 @@ bool DatabaseReplicated::createDatabaseNodesInZooKeeper(const zkutil::ZooKeeperP
/// Other codes are unexpected, will throw
zkutil::KeeperMultiException::check(res, ops, responses);
assert(false);
chassert(false);
__builtin_unreachable();
}
@ -533,7 +533,7 @@ static UUID getTableUUIDIfReplicated(const String & metadata, ContextPtr context
return UUIDHelpers::Nil;
if (!startsWith(create.storage->engine->name, "Replicated") || !endsWith(create.storage->engine->name, "MergeTree"))
return UUIDHelpers::Nil;
assert(create.uuid != UUIDHelpers::Nil);
chassert(create.uuid != UUIDHelpers::Nil);
return create.uuid;
}
@ -767,8 +767,8 @@ std::map<String, String> DatabaseReplicated::tryGetConsistentMetadataSnapshot(co
}
else
{
assert(max_log_ptr == new_max_log_ptr);
assert(table_names.size() != table_name_to_metadata.size());
chassert(max_log_ptr == new_max_log_ptr);
chassert(table_names.size() != table_name_to_metadata.size());
LOG_DEBUG(log, "Cannot get metadata of some tables due to ZooKeeper error, will retry");
}
}

View File

@ -32,7 +32,7 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread()
{
try
{
assert(!database->is_probably_dropped);
chassert(!database->is_probably_dropped);
auto zookeeper = getAndSetZooKeeper();
if (database->is_readonly)
database->tryConnectToZooKeeperAndInitDatabase(/* force_attach */ false, /* is_create_query */ false);
@ -94,7 +94,7 @@ bool DatabaseReplicatedDDLWorker::waitForReplicaToProcessAllEntries(UInt64 timeo
const auto max_log_ptr_path = database->zookeeper_path + "/max_log_ptr";
UInt32 our_log_ptr = parse<UInt32>(zookeeper->get(our_log_ptr_path));
UInt32 max_log_ptr = parse<UInt32>(zookeeper->get(max_log_ptr_path));
assert(our_log_ptr <= max_log_ptr);
chassert(our_log_ptr <= max_log_ptr);
/// max_log_ptr is the number of the last successfully executed request on the initiator
/// The log could contain other entries which are not committed yet
@ -206,7 +206,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr
auto task = std::make_unique<DatabaseReplicatedTask>(entry_name, entry_path, database);
task->entry = entry;
task->parseQueryFromEntry(context);
assert(!task->entry.query.empty());
chassert(!task->entry.query.empty());
assert(!zookeeper->exists(task->getFinishedNodePath()));
task->is_initial_query = true;

View File

@ -361,7 +361,7 @@ void DatabaseReplicatedTask::parseQueryFromEntry(ContextPtr context)
if (auto * ddl_query = dynamic_cast<ASTQueryWithTableAndOutput *>(query.get()))
{
/// Update database name with actual name of local database
assert(!ddl_query->database);
chassert(!ddl_query->database);
ddl_query->setDatabase(database->getDatabaseName());
}
}

View File

@ -249,7 +249,7 @@ void DDLWorker::scheduleTasks(bool reinitialized)
auto & task = *task_it;
if (task->completely_processed)
{
assert(task->was_executed);
chassert(task->was_executed);
/// Status must be written (but finished/ node may not exist if entry was deleted).
/// If someone is deleting entry concurrently, then /active status dir must not exist.
assert(zookeeper->exists(task->getFinishedNodePath()) || !zookeeper->exists(fs::path(task->entry_path) / "active"));
@ -310,7 +310,7 @@ void DDLWorker::scheduleTasks(bool reinitialized)
if (first_failed_task_name)
{
/// If we had failed tasks, then we should start from the first failed task.
assert(reinitialized);
chassert(reinitialized);
begin_node = std::lower_bound(queue_nodes.begin(), queue_nodes.end(), first_failed_task_name);
}
else
@ -503,7 +503,7 @@ void DDLWorker::updateMaxDDLEntryID(const String & entry_name)
void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper)
{
LOG_DEBUG(log, "Processing task {} ({})", task.entry_name, task.entry.query);
assert(!task.completely_processed);
chassert(!task.completely_processed);
String active_node_path = task.getActiveNodePath();
String finished_node_path = task.getFinishedNodePath();
@ -521,14 +521,14 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper)
{
if (create_active_res != Coordination::Error::ZNONODE && create_active_res != Coordination::Error::ZNODEEXISTS)
{
assert(Coordination::isHardwareError(create_active_res));
chassert(Coordination::isHardwareError(create_active_res));
throw Coordination::Exception(create_active_res, active_node_path);
}
/// Status dirs were not created in enqueueQuery(...) or someone is removing entry
if (create_active_res == Coordination::Error::ZNONODE)
{
assert(dynamic_cast<DatabaseReplicatedTask *>(&task) == nullptr);
chassert(dynamic_cast<DatabaseReplicatedTask *>(&task) == nullptr);
if (task.was_executed)
{
/// Special case:
@ -804,7 +804,7 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica(
}
}
assert(!(executed_by_us && executed_by_other_leader));
chassert(!(executed_by_us && executed_by_other_leader));
/// Not executed by leader so was not executed at all
if (!executed_by_us && !executed_by_other_leader)
@ -895,9 +895,9 @@ void DDLWorker::cleanupQueue(Int64, const ZooKeeperPtr & zookeeper)
/// Possible rare case: initiator node has lost connection after enqueueing entry and failed to create status dirs.
/// No one has started to process the entry, so node_path/active and node_path/finished nodes were never created, node_path has no children.
/// Entry became outdated, but we cannot remove remove it in a transaction with node_path/finished.
assert(res[0]->error == Coordination::Error::ZOK && res[1]->error == Coordination::Error::ZNONODE);
chassert(res[0]->error == Coordination::Error::ZOK && res[1]->error == Coordination::Error::ZNONODE);
rm_entry_res = zookeeper->tryRemove(node_path);
assert(rm_entry_res != Coordination::Error::ZNOTEMPTY);
chassert(rm_entry_res != Coordination::Error::ZNOTEMPTY);
continue;
}
zkutil::KeeperMultiException::check(rm_entry_res, ops, res);
@ -998,7 +998,7 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry)
bool DDLWorker::initializeMainThread()
{
assert(!initialized);
chassert(!initialized);
setThreadName("DDLWorker");
LOG_DEBUG(log, "Initializing DDLWorker thread");
@ -1017,7 +1017,7 @@ bool DDLWorker::initializeMainThread()
{
/// A logical error.
LOG_ERROR(log, "ZooKeeper error: {}. Failed to start DDLWorker.", getCurrentExceptionMessage(true));
assert(false); /// Catch such failures in tests with debug build
chassert(false); /// Catch such failures in tests with debug build
}
tryLogCurrentException(__PRETTY_FUNCTION__);

View File

@ -417,7 +417,7 @@ def test_alters_from_different_replicas(started_cluster):
"distributed_ddl_task_timeout": 5,
"distributed_ddl_output_mode": "null_status_on_timeout",
}
assert "shard1|replica2\t\\N\t\\N" in main_node.query(
assert "shard1\treplica2\tQUEUED\t" in main_node.query(
"ALTER TABLE testdb.concurrent_test ADD COLUMN Added2 UInt32;",
settings=settings,
)
@ -425,7 +425,7 @@ def test_alters_from_different_replicas(started_cluster):
"distributed_ddl_task_timeout": 5,
"distributed_ddl_output_mode": "never_throw",
}
assert "shard1|replica2\t\\N\t\\N" in competing_node.query(
assert "shard1\treplica2\tQUEUED\t" in competing_node.query(
"ALTER TABLE testdb.concurrent_test ADD COLUMN Added1 UInt32 AFTER Added0;",
settings=settings,
)
@ -495,11 +495,11 @@ def test_alters_from_different_replicas(started_cluster):
)
res = main_node.query("ALTER TABLE testdb.concurrent_test DELETE WHERE UserID % 2")
assert (
"shard1|replica1" in res
and "shard1|replica2" in res
and "shard1|replica3" in res
"shard1\treplica1\tOK" in res
and "shard1\treplica2\tOK" in res
and "shard1\treplica3\tOK" in res
)
assert "shard2|replica1" in res and "shard2|replica2" in res
assert "shard2\treplica1\tOK" in res and "shard2\treplica2\tOK" in res
expected = (
"1\t1\tmain_node\n"

View File

@ -43,7 +43,7 @@ function create_table()
database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1")
if [ -z "$database" ]; then return; fi
$CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \
"create table $database.rmt_$RANDOM (n int) engine=ReplicatedMergeTree order by tuple()" \
"create table $database.rmt_$RANDOM (n int) engine=ReplicatedMergeTree order by tuple() -- suppress CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" \
2>&1| grep -Fa "Exception: " | grep -Fv "Macro 'uuid' and empty arguments" | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE
sleep 0.$RANDOM
}