Fix data-race DatabaseReplicated::startupTables()/canExecuteReplicatedMetadataAlter() (#52490)

CI founds [1]:

    Exception: Sanitizer assert found for instance ==================
    WARNING: ThreadSanitizer: data race (pid=348)
      Write of size 8 at 0x7b58000044a0 by main thread:
        2 DB::DatabaseReplicated::startupTables(ThreadPoolImpl<ThreadFromGlobalPoolImpl<false>>&, DB::LoadingStrictnessLevel) build_docker/./src/Databases/DatabaseReplicated.cpp:526:16 (clickhouse+0x1ec45092)
        3 DB::TablesLoader::startupTables() build_docker/./src/Databases/TablesLoader.cpp:87:26 (clickhouse+0x1f9258ab) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63)
        4 DB::loadMetadata(std::__1::shared_ptr<DB::Context>, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&) build_docker/./src/Interpreters/loadMetadata.cpp:234:12 (clickhouse+0x1fff3834) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63)
        5 DB::Server::main() build_docker/./programs/server/Server.cpp:1615:9 (clickhouse+0x163e7f78) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63)
        6 Poco::Util::Application::run() build_docker/./base/poco/Util/src/Application.cpp:315:8 (clickhouse+0x257608fe) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63)
        7 DB::Server::run() build_docker/./programs/server/Server.cpp:391:25 (clickhouse+0x163d7d7c) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63)
        8 Poco::Util::ServerApplication::run(int, char**) build_docker/./base/poco/Util/src/ServerApplication.cpp:131:9 (clickhouse+0x25780114) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63)
        9 mainEntryClickHouseServer(int, char**) build_docker/./programs/server/Server.cpp:196:20 (clickhouse+0x163d4c23) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63)
        10 main build_docker/./programs/main.cpp:487:12 (clickhouse+0xdf8c877) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63)

      Previous read of size 8 at 0x7b58000044a0 by thread T27 (mutexes: write M0, write M1):
        1 DB::DatabaseReplicated::canExecuteReplicatedMetadataAlter() const build_docker/./src/Databases/DatabaseReplicated.cpp:1303:12 (clickhouse+0x1ec5c5bd)
        2 DB::ReplicatedMergeTreeQueue::shouldExecuteLogEntry() const build_docker/./src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp:1471:24 (clickhouse+0x2115fb56) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63)
        3 DB::ReplicatedMergeTreeQueue::selectEntryToProcess(DB::MergeTreeDataMergerMutator&, DB::MergeTreeData&) build_docker/./src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp:1676:13 (clickhouse+0x21163c58) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63)
        4 DB::StorageReplicatedMergeTree::selectQueueEntry() build_docker/./src/Storages/StorageReplicatedMergeTree.cpp:3240:26 (clickhouse+0x20823db2) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63)
        5 DB::StorageReplicatedMergeTree::scheduleDataProcessingJob(DB::BackgroundJobsAssignee&) build_docker/./src/Storages/StorageReplicatedMergeTree.cpp:3304:65 (clickhouse+0x208240fc) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63)

  [1]: https://s3.amazonaws.com/clickhouse-test-reports/52395/0b258dda4ee618a4d002e2b5246d68bbd2c77c7e/integration_tests__tsan__[5_6].html

Add ddl_worker_initialized flag to avoid this race.

Note, that it should be enough to check this flag only in
canExecuteReplicatedMetadataAlter() since only it can be run in parallel
with ctor before it had been finished.

v0: initialize ddl before startupTables()
v2: ddl_worker_initialized

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
Co-authored-by: Alexander Tokmakov <tavplubix@clickhouse.com>
This commit is contained in:
Azat Khuzhin 2023-07-25 16:35:01 +02:00 committed by GitHub
parent 79d0343bec
commit 85082ad8f8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 4 additions and 1 deletions

View File

@ -524,6 +524,7 @@ void DatabaseReplicated::startupTables(ThreadPool & thread_pool, LoadingStrictne
ddl_worker = std::make_unique<DatabaseReplicatedDDLWorker>(this, getContext()); ddl_worker = std::make_unique<DatabaseReplicatedDDLWorker>(this, getContext());
ddl_worker->startup(); ddl_worker->startup();
ddl_worker_initialized = true;
} }
bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool debug_check /* = true */) const bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool debug_check /* = true */) const
@ -1155,6 +1156,7 @@ void DatabaseReplicated::stopReplication()
void DatabaseReplicated::shutdown() void DatabaseReplicated::shutdown()
{ {
stopReplication(); stopReplication();
ddl_worker_initialized = false;
ddl_worker = nullptr; ddl_worker = nullptr;
DatabaseAtomic::shutdown(); DatabaseAtomic::shutdown();
} }
@ -1299,7 +1301,7 @@ bool DatabaseReplicated::canExecuteReplicatedMetadataAlter() const
/// It may update the metadata digest (both locally and in ZooKeeper) /// It may update the metadata digest (both locally and in ZooKeeper)
/// before DatabaseReplicatedDDLWorker::initializeReplication() has finished. /// before DatabaseReplicatedDDLWorker::initializeReplication() has finished.
/// We should not update metadata until the database is initialized. /// We should not update metadata until the database is initialized.
return ddl_worker && ddl_worker->isCurrentlyActive(); return ddl_worker_initialized && ddl_worker->isCurrentlyActive();
} }
void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const String & table_name) void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const String & table_name)

View File

@ -134,6 +134,7 @@ private:
std::atomic_bool is_readonly = true; std::atomic_bool is_readonly = true;
std::atomic_bool is_probably_dropped = false; std::atomic_bool is_probably_dropped = false;
std::atomic_bool is_recovering = false; std::atomic_bool is_recovering = false;
std::atomic_bool ddl_worker_initialized = false;
std::unique_ptr<DatabaseReplicatedDDLWorker> ddl_worker; std::unique_ptr<DatabaseReplicatedDDLWorker> ddl_worker;
UInt32 max_log_ptr_at_creation = 0; UInt32 max_log_ptr_at_creation = 0;