From 8b5a05947c227f68843aacc82e07890af7958bb1 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Wed, 28 Nov 2018 18:50:52 +0300 Subject: [PATCH] Remove redundant information from RWLock. And fix broken exception in some race conditions --- dbms/src/Common/RWLock.cpp | 29 ++++++++----------- dbms/src/Common/RWLock.h | 23 ++------------- ...rw_lock_fifo.cpp.cpp => gtest_rw_lock.cpp} | 2 +- .../PushingToViewsBlockOutputStream.cpp | 4 +-- .../Interpreters/InterpreterCreateQuery.cpp | 2 +- .../Interpreters/InterpreterDescribeQuery.cpp | 2 +- .../src/Interpreters/InterpreterDropQuery.cpp | 10 +++---- .../Interpreters/InterpreterInsertQuery.cpp | 2 +- .../Interpreters/InterpreterOptimizeQuery.cpp | 2 +- .../Interpreters/InterpreterRenameQuery.cpp | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 2 +- .../Interpreters/InterpreterSystemQuery.cpp | 2 +- dbms/src/Storages/IStorage.cpp | 6 ++-- dbms/src/Storages/IStorage.h | 23 +++++++-------- .../Storages/MergeTree/DataPartsExchange.cpp | 2 +- .../ReplicatedMergeTreeAlterThread.cpp | 4 +-- .../ReplicatedMergeTreePartCheckThread.cpp | 2 +- dbms/src/Storages/StorageBuffer.cpp | 2 +- dbms/src/Storages/StorageDistributed.cpp | 2 +- dbms/src/Storages/StorageMerge.cpp | 6 ++-- dbms/src/Storages/StorageMergeTree.cpp | 22 +++++++------- dbms/src/Storages/StorageNull.cpp | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 24 +++++++-------- .../Storages/System/StorageSystemColumns.cpp | 2 +- .../System/StorageSystemPartsBase.cpp | 2 +- suppress_base.json | 4 +-- 26 files changed, 81 insertions(+), 104 deletions(-) rename dbms/src/Common/tests/{gtest_rw_lock_fifo.cpp.cpp => gtest_rw_lock.cpp} (98%) diff --git a/dbms/src/Common/RWLock.cpp b/dbms/src/Common/RWLock.cpp index 6890da7281..10a15cd702 100644 --- a/dbms/src/Common/RWLock.cpp +++ b/dbms/src/Common/RWLock.cpp @@ -53,7 +53,7 @@ public: }; -RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::Client client) +RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type) { Stopwatch watch(CLOCK_MONOTONIC_COARSE); CurrentMetrics::Increment waiting_client_increment((type == Read) ? CurrentMetrics::RWLockWaitingReaders @@ -78,15 +78,16 @@ RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::C { auto handler_ptr = it_handler->second.lock(); - if (!handler_ptr) - throw Exception("Lock handler cannot be nullptr. This is a bug", ErrorCodes::LOGICAL_ERROR); + /// Lock may be released in another thread, but not yet deleted inside |~LogHandlerImpl()| - if (type != Read || handler_ptr->it_group->type != Read) - throw Exception("Attempt to acquire exclusive lock recursively", ErrorCodes::LOGICAL_ERROR); + if (handler_ptr) + { + /// XXX: it means we can't upgrade lock from read to write - with proper waiting! + if (type != Read || handler_ptr->it_group->type != Read) + throw Exception("Attempt to acquire exclusive lock recursively", ErrorCodes::LOGICAL_ERROR); - handler_ptr->it_client->info += "; " + client.info; - - return handler_ptr; + return handler_ptr; + } } if (type == Type::Write || queue.empty() || queue.back().type == Type::Write) @@ -104,7 +105,7 @@ RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::C auto & clients = it_group->clients; try { - it_client = clients.emplace(clients.end(), std::move(client)); + it_client = clients.emplace(clients.end(), type); } catch (...) { @@ -114,10 +115,6 @@ RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::C throw; } - it_client->thread_number = Poco::ThreadNumber::get(); - it_client->enqueue_time = time(nullptr); - it_client->type = type; - LockHandler res(new LockHandlerImpl(shared_from_this(), it_group, it_client)); /// Insert myself (weak_ptr to the handler) to threads set to implement recursive lock @@ -128,7 +125,6 @@ RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::C /// If we are not the first client in the group, a notification could be already sent if (it_group == queue.begin()) { - it_client->start_time = it_client->enqueue_time; finalize_metrics(); return res; } @@ -136,7 +132,6 @@ RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::C /// Wait a notification it_group->cv.wait(lock, [&] () { return it_group == queue.begin(); }); - it_client->start_time = time(nullptr); finalize_metrics(); return res; } @@ -169,8 +164,8 @@ RWLockImpl::LockHandlerImpl::~LockHandlerImpl() RWLockImpl::LockHandlerImpl::LockHandlerImpl(RWLock && parent, RWLockImpl::GroupsContainer::iterator it_group, RWLockImpl::ClientsContainer::iterator it_client) : parent{std::move(parent)}, it_group{it_group}, it_client{it_client}, - active_client_increment{(it_client->type == RWLockImpl::Read) ? CurrentMetrics::RWLockActiveReaders - : CurrentMetrics::RWLockActiveWriters} + active_client_increment{(*it_client == RWLockImpl::Read) ? CurrentMetrics::RWLockActiveReaders + : CurrentMetrics::RWLockActiveWriters} {} } diff --git a/dbms/src/Common/RWLock.h b/dbms/src/Common/RWLock.h index d993705472..fd95ed48e2 100644 --- a/dbms/src/Common/RWLock.h +++ b/dbms/src/Common/RWLock.h @@ -27,23 +27,6 @@ public: Write, }; -private: - /// Client is that who wants to acquire the lock. - struct Client - { - explicit Client(const std::string & info = {}) : info{info} {} - - bool isStarted() { return start_time != 0; } - - /// TODO: delete extra info below if there is no need fot it already. - std::string info; - int thread_number = 0; - std::time_t enqueue_time = 0; - std::time_t start_time = 0; - Type type = Read; - }; - -public: static RWLock create() { return RWLock(new RWLockImpl); } /// Just use LockHandler::reset() to release the lock @@ -53,21 +36,21 @@ public: /// Waits in the queue and returns appropriate lock - LockHandler getLock(Type type, Client client = Client{}); - LockHandler getLock(Type type, const std::string & who) { return getLock(type, Client(who)); } + LockHandler getLock(Type type); private: RWLockImpl() = default; struct Group; using GroupsContainer = std::list; - using ClientsContainer = std::list; + using ClientsContainer = std::list; using ThreadToHandler = std::map>; /// Group of clients that should be executed concurrently /// i.e. a group could contain several readers, but only one writer struct Group { + // FIXME: there is only redundant |type| information inside |clients|. const Type type; ClientsContainer clients; diff --git a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp b/dbms/src/Common/tests/gtest_rw_lock.cpp similarity index 98% rename from dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp rename to dbms/src/Common/tests/gtest_rw_lock.cpp index 44e053b61a..5a10c4a0a4 100644 --- a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp +++ b/dbms/src/Common/tests/gtest_rw_lock.cpp @@ -38,7 +38,7 @@ TEST(Common, RWLock_1) auto type = (std::uniform_int_distribution<>(0, 9)(gen) >= round) ? RWLockImpl::Read : RWLockImpl::Write; auto sleep_for = std::chrono::duration(std::uniform_int_distribution<>(1, 100)(gen)); - auto lock = fifo_lock->getLock(type, "RW"); + auto lock = fifo_lock->getLock(type); if (type == RWLockImpl::Write) { diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index f54c62b357..d3e7eaeb9a 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -20,7 +20,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( * 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. */ - addTableLock(storage->lockStructure(true, __PRETTY_FUNCTION__)); + addTableLock(storage->lockStructure(true)); /// If the "root" table deduplactes blocks, there are no need to make deduplication for children /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks @@ -45,7 +45,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( auto & materialized_view = dynamic_cast(*dependent_table); if (StoragePtr inner_table = materialized_view.tryGetTargetTable()) - addTableLock(inner_table->lockStructure(true, __PRETTY_FUNCTION__)); + addTableLock(inner_table->lockStructure(true)); auto query = materialized_view.getInnerQuery(); BlockOutputStreamPtr out = std::make_shared( diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index eac6d2d776..4dbe0bdfff 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -547,7 +547,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!as_table_name.empty()) { as_storage = context.getTable(as_database_name, as_table_name); - as_storage_lock = as_storage->lockStructure(false, __PRETTY_FUNCTION__); + as_storage_lock = as_storage->lockStructure(false); } /// Set and retrieve list of columns. diff --git a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp index db33383b76..b5e62b9b92 100644 --- a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp @@ -102,7 +102,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() table = context.getTable(database_name, table_name); } - auto table_lock = table->lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = table->lockStructure(false); columns = table->getColumns().getAll(); column_defaults = table->getColumns().defaults; column_comments = table->getColumns().comments; diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index f2a3f45b55..2107df3579 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -69,7 +69,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t { database_and_table.second->shutdown(); /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = database_and_table.second->lockForAlter(__PRETTY_FUNCTION__); + auto table_lock = database_and_table.second->lockForAlter(); /// Drop table from memory, don't touch data and metadata database_and_table.first->detachTable(database_and_table.second->getTableName()); } @@ -78,7 +78,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t database_and_table.second->checkTableCanBeDropped(); /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = database_and_table.second->lockForAlter(__PRETTY_FUNCTION__); + auto table_lock = database_and_table.second->lockForAlter(); /// Drop table data, don't touch metadata database_and_table.second->truncate(query_ptr); } @@ -88,7 +88,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t database_and_table.second->shutdown(); /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = database_and_table.second->lockForAlter(__PRETTY_FUNCTION__); + auto table_lock = database_and_table.second->lockForAlter(); /// Delete table metdata and table itself from memory database_and_table.first->removeTable(context, database_and_table.second->getTableName()); /// Delete table data @@ -124,7 +124,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDr if (kind == ASTDropQuery::Kind::Truncate) { /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = table->lockForAlter(__PRETTY_FUNCTION__); + auto table_lock = table->lockForAlter(); /// Drop table data, don't touch metadata table->truncate(query_ptr); } @@ -133,7 +133,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDr context_handle.tryRemoveExternalTable(table_name); table->shutdown(); /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = table->lockForAlter(__PRETTY_FUNCTION__); + auto table_lock = table->lockForAlter(); /// Delete table data table->drop(); table->is_dropped = true; diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index f058550a44..444e4eed7a 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -92,7 +92,7 @@ BlockIO InterpreterInsertQuery::execute() checkAccess(query); StoragePtr table = getTable(query); - auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__); + auto table_lock = table->lockStructure(true); /// We create a pipeline of several streams, into which we will write data. BlockOutputStreamPtr out; diff --git a/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp b/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp index 80a64d83f9..7d46881539 100644 --- a/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -23,7 +23,7 @@ BlockIO InterpreterOptimizeQuery::execute() return executeDDLQueryOnCluster(query_ptr, context, {ast.database}); StoragePtr table = context.getTable(ast.database, ast.table); - auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__); + auto table_lock = table->lockStructure(true); table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context); return {}; } diff --git a/dbms/src/Interpreters/InterpreterRenameQuery.cpp b/dbms/src/Interpreters/InterpreterRenameQuery.cpp index abeb7ff5c1..74dfe7adec 100644 --- a/dbms/src/Interpreters/InterpreterRenameQuery.cpp +++ b/dbms/src/Interpreters/InterpreterRenameQuery.cpp @@ -101,7 +101,7 @@ BlockIO InterpreterRenameQuery::execute() for (const auto & names : unique_tables_from) if (auto table = context.tryGetTable(names.database_name, names.table_name)) - locks.emplace_back(table->lockForAlter(__PRETTY_FUNCTION__)); + locks.emplace_back(table->lockForAlter()); /** All tables are locked. If there are more than one rename in chain, * we need to hold global lock while doing all renames. Order matters to avoid deadlocks. diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 2a139edce9..fe3801149d 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -182,7 +182,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( } if (storage) - table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); + table_lock = storage->lockStructure(false); syntax_analyzer_result = SyntaxAnalyzer(context, storage) .analyze(query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, subquery_depth); diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 749827896a..f91025b11d 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -238,7 +238,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam table->shutdown(); /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = table->lockForAlter(__PRETTY_FUNCTION__); + auto table_lock = table->lockForAlter(); create_ast = system_context.getCreateTableQuery(database_name, table_name); database->detachTable(table_name); diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 498d486b7a..c0487b5ef0 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -4,13 +4,13 @@ namespace DB { -TableStructureReadLock::TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data, const std::string & who) +TableStructureReadLock::TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data) : storage(storage_) { if (lock_data) - data_lock = storage->data_lock->getLock(RWLockImpl::Read, who); + data_lock = storage->data_lock->getLock(RWLockImpl::Read); if (lock_structure) - structure_lock = storage->structure_lock->getLock(RWLockImpl::Read, who); + structure_lock = storage->structure_lock->getLock(RWLockImpl::Read); } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 446f2269f6..a68e036032 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -65,7 +65,7 @@ private: RWLockImpl::LockHandler structure_lock; public: - TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data, const std::string & who); + TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data); }; @@ -115,14 +115,13 @@ public: /** Does not allow you to change the structure or name of the table. * If you change the data in the table, you will need to specify will_modify_data = true. * This will take an extra lock that does not allow starting ALTER MODIFY. - * Parameter 'who' identifies a client of the lock (ALTER query, merge process, etc), used for diagnostic purposes. * * WARNING: You need to call methods from ITableDeclaration under such a lock. Without it, they are not thread safe. * WARNING: To avoid deadlocks, this method must not be called under lock of Context. */ - TableStructureReadLockPtr lockStructure(bool will_modify_data, const std::string & who) + TableStructureReadLockPtr lockStructure(bool will_modify_data) { - TableStructureReadLockPtr res = std::make_shared(shared_from_this(), true, will_modify_data, who); + TableStructureReadLockPtr res = std::make_shared(shared_from_this(), true, will_modify_data); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); return res; @@ -130,11 +129,11 @@ public: /** Does not allow reading the table structure. It is taken for ALTER, RENAME and DROP, TRUNCATE. */ - TableFullWriteLock lockForAlter(const std::string & who = "Alter") + TableFullWriteLock lockForAlter() { /// The calculation order is important. - auto res_data_lock = lockDataForAlter(who); - auto res_structure_lock = lockStructureForAlter(who); + auto res_data_lock = lockDataForAlter(); + auto res_structure_lock = lockStructureForAlter(); return {std::move(res_data_lock), std::move(res_structure_lock)}; } @@ -143,17 +142,17 @@ public: * It is taken during write temporary data in ALTER MODIFY. * Under this lock, you can take lockStructureForAlter() to change the structure of the table. */ - TableDataWriteLock lockDataForAlter(const std::string & who = "Alter") + TableDataWriteLock lockDataForAlter() { - auto res = data_lock->getLock(RWLockImpl::Write, who); + auto res = data_lock->getLock(RWLockImpl::Write); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); return res; } - TableStructureWriteLock lockStructureForAlter(const std::string & who = "Alter") + TableStructureWriteLock lockStructureForAlter() { - auto res = structure_lock->getLock(RWLockImpl::Write, who); + auto res = structure_lock->getLock(RWLockImpl::Write); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); return res; @@ -243,7 +242,7 @@ public: throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto lock = lockStructureForAlter(); auto new_columns = getColumns(); params.apply(new_columns); context.getDatabase(database_name)->alterTable(context, table_name, new_columns, {}); diff --git a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp index ab1d28ee65..a1ffd45c62 100644 --- a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp @@ -79,7 +79,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo try { - auto storage_lock = owned_storage->lockStructure(false, __PRETTY_FUNCTION__); + auto storage_lock = owned_storage->lockStructure(false); MergeTreeData::DataPartPtr part = findPart(part_name); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index a3e4dbb7ba..8444b27505 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -84,7 +84,7 @@ void ReplicatedMergeTreeAlterThread::run() LOG_INFO(log, "Changed version of 'columns' node in ZooKeeper. Waiting for structure write lock."); - auto table_lock = storage.lockStructureForAlter(__PRETTY_FUNCTION__); + auto table_lock = storage.lockStructureForAlter(); if (columns_in_zk != storage.getColumns()) { @@ -113,7 +113,7 @@ void ReplicatedMergeTreeAlterThread::run() /// Update parts. if (changed_version || force_recheck_parts) { - auto table_lock = storage.lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = storage.lockStructure(false); if (changed_version) LOG_INFO(log, "ALTER-ing parts"); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index fb7a2ad073..c115264d39 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -202,7 +202,7 @@ void ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name) else if (part->name == part_name) { auto zookeeper = storage.getZooKeeper(); - auto table_lock = storage.lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = storage.lockStructure(false); /// If the part is in ZooKeeper, check its data with its checksums, and them with ZooKeeper. if (zookeeper->exists(storage.replica_path + "/parts/" + part_name)) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 4ec32883e6..2a81e5d1d1 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -634,7 +634,7 @@ void StorageBuffer::alter(const AlterCommands & params, const String & database_ if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED); - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto lock = lockStructureForAlter(); /// So that no blocks of the old structure remain. optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context); diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 5d3860f449..3961c530c5 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -308,7 +308,7 @@ void StorageDistributed::alter(const AlterCommands & params, const String & data if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED); - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto lock = lockStructureForAlter(); ColumnsDescription new_columns = getColumns(); params.apply(new_columns); diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 6fcdb86e86..11f7e1deb6 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -344,7 +344,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables() const { auto & table = iterator->table(); if (table.get() != this) - selected_tables.emplace_back(table, table->lockStructure(false, __PRETTY_FUNCTION__)); + selected_tables.emplace_back(table, table->lockStructure(false)); } iterator->next(); @@ -374,7 +374,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr if (storage.get() != this) { virtual_column->insert(storage->getTableName()); - selected_tables.emplace_back(storage, get_lock ? storage->lockStructure(false, __PRETTY_FUNCTION__) : TableStructureReadLockPtr{}); + selected_tables.emplace_back(storage, get_lock ? storage->lockStructure(false) : TableStructureReadLockPtr{}); } } @@ -400,7 +400,7 @@ void StorageMerge::alter(const AlterCommands & params, const String & database_n if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED); - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto lock = lockStructureForAlter(); ColumnsDescription new_columns = getColumns(); params.apply(new_columns); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 50d7af076d..3047370510 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -190,7 +190,7 @@ void StorageMergeTree::alter( { if (!params.is_mutable()) { - auto table_soft_lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto table_soft_lock = lockStructureForAlter(); auto new_columns = getColumns(); params.apply(new_columns); context.getDatabase(database_name)->alterTable(context, table_name, new_columns, {}); @@ -201,7 +201,7 @@ void StorageMergeTree::alter( /// NOTE: Here, as in ReplicatedMergeTree, you can do ALTER which does not block the writing of data for a long time. auto merge_blocker = merger_mutator.actions_blocker.cancel(); - auto table_soft_lock = lockDataForAlter(__PRETTY_FUNCTION__); + auto table_soft_lock = lockDataForAlter(); data.checkAlter(params); @@ -234,7 +234,7 @@ void StorageMergeTree::alter( transactions.push_back(std::move(transaction)); } - auto table_hard_lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto table_hard_lock = lockStructureForAlter(); IDatabase::ASTModifier storage_modifier; if (primary_key_is_modified) @@ -400,7 +400,7 @@ bool StorageMergeTree::merge( bool deduplicate, String * out_disable_reason) { - auto structure_lock = lockStructure(true, __PRETTY_FUNCTION__); + auto structure_lock = lockStructure(true); MergeTreeDataMergerMutator::FuturePart future_part; @@ -505,7 +505,7 @@ bool StorageMergeTree::merge( bool StorageMergeTree::tryMutatePart() { - auto structure_lock = lockStructure(true, __PRETTY_FUNCTION__); + auto structure_lock = lockStructure(true); MergeTreeDataMergerMutator::FuturePart future_part; MutationCommands commands; @@ -705,7 +705,7 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi auto merge_blocker = merger_mutator.actions_blocker.cancel(); /// We don't change table structure, only data in some parts, parts are locked inside alterDataPart() function - auto lock_read_structure = lockStructure(false, __PRETTY_FUNCTION__); + auto lock_read_structure = lockStructure(false); String partition_id = data.getPartitionIDFromQuery(partition, context); auto parts = data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); @@ -807,7 +807,7 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma case PartitionCommand::FREEZE_PARTITION: { - auto lock = lockStructure(false, __PRETTY_FUNCTION__); + auto lock = lockStructure(false); data.freezePartition(command.partition, command.with_name, context); } break; @@ -818,7 +818,7 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma case PartitionCommand::FREEZE_ALL_PARTITIONS: { - auto lock = lockStructure(false, __PRETTY_FUNCTION__); + auto lock = lockStructure(false); data.freezeAll(command.with_name, context); } break; @@ -836,7 +836,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons /// This protects against "revival" of data for a removed partition after completion of merge. auto merge_blocker = merger_mutator.actions_blocker.cancel(); /// Waits for completion of merge and does not start new ones. - auto lock = lockForAlter(__PRETTY_FUNCTION__); + auto lock = lockForAlter(); String partition_id = data.getPartitionIDFromQuery(partition, context); @@ -919,8 +919,8 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool part, cons void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) { - auto lock1 = lockStructure(false, __PRETTY_FUNCTION__); - auto lock2 = source_table->lockStructure(false, __PRETTY_FUNCTION__); + auto lock1 = lockStructure(false); + auto lock2 = source_table->lockStructure(false); Stopwatch watch; MergeTreeData * src_data = data.checkStructureAndGetMergeTreeData(source_table); diff --git a/dbms/src/Storages/StorageNull.cpp b/dbms/src/Storages/StorageNull.cpp index 25ecfecf2c..0a1690d5d5 100644 --- a/dbms/src/Storages/StorageNull.cpp +++ b/dbms/src/Storages/StorageNull.cpp @@ -31,7 +31,7 @@ void registerStorageNull(StorageFactory & factory) void StorageNull::alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) { - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto lock = lockStructureForAlter(); ColumnsDescription new_columns = getColumns(); params.apply(new_columns); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 7fe4a72c0e..c0795c5429 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1146,7 +1146,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) /// Can throw an exception. DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, estimated_space_for_merge); - auto table_lock = lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = lockStructure(false); MergeList::EntryPtr merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name, parts); @@ -1276,7 +1276,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM /// Can throw an exception. DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, estimated_space_for_result); - auto table_lock = lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = lockStructure(false); MergeTreeData::MutableDataPartPtr new_part; MergeTreeData::Transaction transaction(data); @@ -1582,7 +1582,7 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & /// We don't change table structure, only data in some parts /// To disable reading from these parts, we will sequentially acquire write lock for each part inside alterDataPart() /// If we will lock the whole table here, a deadlock can occur. For example, if use use Buffer table (CLICKHOUSE-3238) - auto lock_read_structure = lockStructure(false, __PRETTY_FUNCTION__); + auto lock_read_structure = lockStructure(false); auto zookeeper = getZooKeeper(); @@ -1683,7 +1683,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) PartDescriptions parts_to_add; MergeTreeData::DataPartsVector parts_to_remove; - auto structure_lock_dst_table = lockStructure(false, __PRETTY_FUNCTION__); + auto structure_lock_dst_table = lockStructure(false); for (size_t i = 0; i < entry_replace.new_part_names.size(); ++i) { @@ -1745,7 +1745,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) return 0; } - structure_lock_src_table = source_table->lockStructure(false, __PRETTY_FUNCTION__); + structure_lock_src_table = source_table->lockStructure(false); MergeTreeData::DataPartStates valid_states{MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}; @@ -2767,7 +2767,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin TableStructureReadLockPtr table_lock; if (!to_detached) - table_lock = lockStructure(true, __PRETTY_FUNCTION__); + table_lock = lockStructure(true); /// Logging Stopwatch stopwatch; @@ -3130,7 +3130,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, { /// Just to read current structure. Alter will be done in separate thread. - auto table_lock = lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = lockStructure(false); if (is_readonly) throw Exception("Can't ALTER readonly table", ErrorCodes::TABLE_IS_READ_ONLY); @@ -3312,7 +3312,7 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part case PartitionCommand::FREEZE_PARTITION: { - auto lock = lockStructure(false, __PRETTY_FUNCTION__); + auto lock = lockStructure(false); data.freezePartition(command.partition, command.with_name, context); } break; @@ -3323,7 +3323,7 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part case PartitionCommand::FREEZE_ALL_PARTITIONS: { - auto lock = lockStructure(false, __PRETTY_FUNCTION__); + auto lock = lockStructure(false); data.freezeAll(command.with_name, context); } break; @@ -4325,7 +4325,7 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() { /// Critical section is not required (since grabOldParts() returns unique part set on each call) - auto table_lock = lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = lockStructure(false); auto zookeeper = getZooKeeper(); MergeTreeData::DataPartsVector parts = data.grabOldParts(); @@ -4600,8 +4600,8 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) { - auto lock1 = lockStructure(false, __PRETTY_FUNCTION__); - auto lock2 = source_table->lockStructure(false, __PRETTY_FUNCTION__); + auto lock1 = lockStructure(false); + auto lock2 = source_table->lockStructure(false); Stopwatch watch; MergeTreeData * src_data = data.checkStructureAndGetMergeTreeData(source_table); diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index 91c514dba4..449f833e25 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -98,7 +98,7 @@ protected: try { - table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); + table_lock = storage->lockStructure(false); } catch (const Exception & e) { diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index 9d7b274572..5d888b5bd1 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -166,7 +166,7 @@ public: try { /// For table not to be dropped and set of columns to remain constant. - info.table_lock = info.storage->lockStructure(false, __PRETTY_FUNCTION__); + info.table_lock = info.storage->lockStructure(false); } catch (const Exception & e) { diff --git a/suppress_base.json b/suppress_base.json index cc0ba5cbcb..9a68c58fa5 100644 --- a/suppress_base.json +++ b/suppress_base.json @@ -6038,8 +6038,8 @@ "CodeNext": 0, "CodePrev": 0, "ErrorCode": "V001", - "FileName": "gtest_rw_lock_fifo.cpp.cpp", - "Message": "A code fragment from 'gtest_rw_lock_fifo.cpp.cpp' cannot be analyzed." + "FileName": "gtest_rw_lock.cpp", + "Message": "A code fragment from 'gtest_rw_lock.cpp' cannot be analyzed." }, { "CodeCurrent": 0, -- GitLab