Apply some CTAD

This commit is contained in:
Robert Schulze 2023-03-02 13:36:47 +00:00
parent f9786c1e32
commit 740aeaba1f
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
30 changed files with 101 additions and 101 deletions

View File

@ -35,26 +35,26 @@ namespace
void CancelToken::Registry::insert(CancelToken * token)
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
threads[token->thread_id] = token;
}
void CancelToken::Registry::remove(CancelToken * token)
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
threads.erase(token->thread_id);
}
void CancelToken::Registry::signal(UInt64 tid)
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
if (auto it = threads.find(tid); it != threads.end())
it->second->signalImpl();
}
void CancelToken::Registry::signal(UInt64 tid, int code, const String & message)
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
if (auto it = threads.find(tid); it != threads.end())
it->second->signalImpl(code, message);
}

View File

@ -163,14 +163,14 @@ public:
/// Returns size of queue
size_t size() const
{
std::lock_guard<std::mutex> lock(queue_mutex);
std::lock_guard lock(queue_mutex);
return queue.size();
}
/// Returns if queue is empty
bool empty() const
{
std::lock_guard<std::mutex> lock(queue_mutex);
std::lock_guard lock(queue_mutex);
return queue.empty();
}
@ -184,7 +184,7 @@ public:
bool was_finished_before = false;
{
std::lock_guard<std::mutex> lock(queue_mutex);
std::lock_guard lock(queue_mutex);
if (is_finished)
return true;
@ -202,14 +202,14 @@ public:
/// Returns if queue is finished
bool isFinished() const
{
std::lock_guard<std::mutex> lock(queue_mutex);
std::lock_guard lock(queue_mutex);
return is_finished;
}
/// Returns if queue is finished and empty
bool isFinishedAndEmpty() const
{
std::lock_guard<std::mutex> lock(queue_mutex);
std::lock_guard lock(queue_mutex);
return is_finished && queue.empty();
}
@ -217,7 +217,7 @@ public:
void clear()
{
{
std::lock_guard<std::mutex> lock(queue_mutex);
std::lock_guard lock(queue_mutex);
if (is_finished)
return;
@ -233,7 +233,7 @@ public:
void clearAndFinish()
{
{
std::lock_guard<std::mutex> lock(queue_mutex);
std::lock_guard lock(queue_mutex);
std::queue<T> empty_queue;
queue.swap(empty_queue);

View File

@ -149,7 +149,7 @@ DateLUT::DateLUT()
const DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone) const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
auto it = impls.emplace(time_zone, nullptr).first;
if (!it->second)

View File

@ -26,13 +26,13 @@ namespace CurrentStatusInfo
inline void set(Status status, Key key, Int8 value)
{
std::lock_guard<std::mutex> lock(locks[status]);
std::lock_guard lock(locks[status]);
values[status][key] = value;
}
inline void unset(Status status, Key key)
{
std::lock_guard<std::mutex> lock(locks[status]);
std::lock_guard lock(locks[status]);
values[status].erase(key);
}
}

View File

@ -109,14 +109,14 @@ size_t TLDListsHolder::parseAndAddTldList(const std::string & name, const std::s
}
size_t tld_list_size = tld_list.size();
std::lock_guard<std::mutex> lock(tld_lists_map_mutex);
std::lock_guard lock(tld_lists_map_mutex);
tld_lists_map.insert(std::make_pair(name, std::move(tld_list)));
return tld_list_size;
}
const TLDList & TLDListsHolder::getTldList(const std::string & name)
{
std::lock_guard<std::mutex> lock(tld_lists_map_mutex);
std::lock_guard lock(tld_lists_map_mutex);
auto it = tld_lists_map.find(name);
if (it == tld_lists_map.end())
throw Exception(ErrorCodes::TLD_LIST_NOT_FOUND, "TLD list {} does not exist", name);

View File

@ -130,7 +130,7 @@ Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & co
Pool::~Pool()
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
for (auto & connection : connections)
delete static_cast<Connection *>(connection);
@ -187,7 +187,7 @@ Pool::Entry Pool::get(uint64_t wait_timeout)
Pool::Entry Pool::tryGet()
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
initialize();
@ -229,7 +229,7 @@ void Pool::removeConnection(Connection* connection)
{
logger.trace("(%s): Removing connection.", getDescription());
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
if (connection)
{
if (connection->ref_count > 0)

View File

@ -76,7 +76,7 @@ PoolWithFailover PoolFactory::get(const Poco::Util::AbstractConfiguration & conf
const std::string & config_name, unsigned default_connections, unsigned max_connections, size_t max_tries)
{
std::lock_guard<std::mutex> lock(impl->mutex);
std::lock_guard lock(impl->mutex);
if (auto entry = impl->pools.find(config_name); entry != impl->pools.end())
{
return *(entry->second);
@ -106,7 +106,7 @@ PoolWithFailover PoolFactory::get(const Poco::Util::AbstractConfiguration & conf
void PoolFactory::reset()
{
std::lock_guard<std::mutex> lock(impl->mutex);
std::lock_guard lock(impl->mutex);
impl->pools.clear();
impl->pools_by_ids.clear();
}

View File

@ -123,7 +123,7 @@ PoolWithFailover::PoolWithFailover(const PoolWithFailover & other)
PoolWithFailover::Entry PoolWithFailover::get()
{
Poco::Util::Application & app = Poco::Util::Application::instance();
std::lock_guard<std::mutex> locker(mutex);
std::lock_guard locker(mutex);
/// If we cannot connect to some replica due to pool overflow, than we will wait and connect.
PoolPtr * full_pool = nullptr;

View File

@ -27,7 +27,7 @@ uint64_t InMemoryLogStore::start_index() const
uint64_t InMemoryLogStore::next_slot() const
{
std::lock_guard<std::mutex> l(logs_lock);
std::lock_guard l(logs_lock);
// Exclude the dummy entry.
return start_idx + logs.size() - 1;
}
@ -35,7 +35,7 @@ uint64_t InMemoryLogStore::next_slot() const
nuraft::ptr<nuraft::log_entry> InMemoryLogStore::last_entry() const
{
uint64_t next_idx = next_slot();
std::lock_guard<std::mutex> lock(logs_lock);
std::lock_guard lock(logs_lock);
auto entry = logs.find(next_idx - 1);
if (entry == logs.end())
entry = logs.find(0);
@ -47,7 +47,7 @@ uint64_t InMemoryLogStore::append(nuraft::ptr<nuraft::log_entry> & entry)
{
ptr<log_entry> clone = makeClone(entry);
std::lock_guard<std::mutex> l(logs_lock);
std::lock_guard l(logs_lock);
uint64_t idx = start_idx + logs.size() - 1;
logs[idx] = clone;
return idx;
@ -58,7 +58,7 @@ void InMemoryLogStore::write_at(uint64_t index, nuraft::ptr<nuraft::log_entry> &
nuraft::ptr<log_entry> clone = makeClone(entry);
// Discard all logs equal to or greater than `index.
std::lock_guard<std::mutex> l(logs_lock);
std::lock_guard l(logs_lock);
auto itr = logs.lower_bound(index);
while (itr != logs.end())
itr = logs.erase(itr);
@ -76,7 +76,7 @@ nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> InMemoryLogStore::log_e
{
nuraft::ptr<nuraft::log_entry> src = nullptr;
{
std::lock_guard<std::mutex> l(logs_lock);
std::lock_guard l(logs_lock);
auto entry = logs.find(i);
if (entry == logs.end())
{
@ -94,7 +94,7 @@ nuraft::ptr<nuraft::log_entry> InMemoryLogStore::entry_at(uint64_t index)
{
nuraft::ptr<nuraft::log_entry> src = nullptr;
{
std::lock_guard<std::mutex> l(logs_lock);
std::lock_guard l(logs_lock);
auto entry = logs.find(index);
if (entry == logs.end())
entry = logs.find(0);
@ -107,7 +107,7 @@ uint64_t InMemoryLogStore::term_at(uint64_t index)
{
uint64_t term = 0;
{
std::lock_guard<std::mutex> l(logs_lock);
std::lock_guard l(logs_lock);
auto entry = logs.find(index);
if (entry == logs.end())
entry = logs.find(0);
@ -125,7 +125,7 @@ nuraft::ptr<nuraft::buffer> InMemoryLogStore::pack(uint64_t index, Int32 cnt)
{
ptr<log_entry> le = nullptr;
{
std::lock_guard<std::mutex> l(logs_lock);
std::lock_guard l(logs_lock);
le = logs[ii];
}
assert(le.get());
@ -162,13 +162,13 @@ void InMemoryLogStore::apply_pack(uint64_t index, nuraft::buffer & pack)
nuraft::ptr<nuraft::log_entry> le = nuraft::log_entry::deserialize(*buf_local);
{
std::lock_guard<std::mutex> l(logs_lock);
std::lock_guard l(logs_lock);
logs[cur_idx] = le;
}
}
{
std::lock_guard<std::mutex> l(logs_lock);
std::lock_guard l(logs_lock);
auto entry = logs.upper_bound(0);
if (entry != logs.end())
start_idx = entry->first;
@ -179,7 +179,7 @@ void InMemoryLogStore::apply_pack(uint64_t index, nuraft::buffer & pack)
bool InMemoryLogStore::compact(uint64_t last_log_index)
{
std::lock_guard<std::mutex> l(logs_lock);
std::lock_guard l(logs_lock);
for (uint64_t ii = start_idx; ii <= last_log_index; ++ii)
{
auto entry = logs.find(ii);

View File

@ -341,7 +341,7 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession
nuraft::ptr<nuraft::snapshot> KeeperStateMachine::last_snapshot()
{
/// Just return the latest snapshot.
std::lock_guard<std::mutex> lock(snapshots_lock);
std::lock_guard lock(snapshots_lock);
return latest_snapshot_meta;
}

View File

@ -37,7 +37,7 @@ nuraft::ptr<nuraft::buffer> SummingStateMachine::commit(const uint64_t log_idx,
bool SummingStateMachine::apply_snapshot(nuraft::snapshot & s)
{
std::lock_guard<std::mutex> ll(snapshots_lock);
std::lock_guard ll(snapshots_lock);
auto entry = snapshots.find(s.get_last_log_idx());
if (entry == snapshots.end())
return false;
@ -50,7 +50,7 @@ bool SummingStateMachine::apply_snapshot(nuraft::snapshot & s)
nuraft::ptr<nuraft::snapshot> SummingStateMachine::last_snapshot()
{
// Just return the latest snapshot.
std::lock_guard<std::mutex> ll(snapshots_lock);
std::lock_guard ll(snapshots_lock);
auto entry = snapshots.rbegin();
if (entry == snapshots.rend())
return nullptr;
@ -100,7 +100,7 @@ void SummingStateMachine::save_logical_snp_obj(
nuraft::buffer_serializer bs(data);
int64_t local_value = static_cast<int64_t>(bs.get_u64());
std::lock_guard<std::mutex> ll(snapshots_lock);
std::lock_guard ll(snapshots_lock);
auto entry = snapshots.find(s.get_last_log_idx());
assert(entry != snapshots.end());
entry->second->value = local_value;
@ -118,7 +118,7 @@ int SummingStateMachine::read_logical_snp_obj(
{
nuraft::ptr<SingleValueSnapshotContext> ctx = nullptr;
{
std::lock_guard<std::mutex> ll(snapshots_lock);
std::lock_guard ll(snapshots_lock);
auto entry = snapshots.find(s.get_last_log_idx());
if (entry == snapshots.end())
{
@ -155,7 +155,7 @@ void SummingStateMachine::create_snapshot(
nuraft::async_result<bool>::handler_type & when_done)
{
{
std::lock_guard<std::mutex> ll(snapshots_lock);
std::lock_guard ll(snapshots_lock);
createSnapshotInternal(s);
}
nuraft::ptr<std::exception> except(nullptr);

View File

@ -82,7 +82,7 @@ DatabaseMySQL::DatabaseMySQL(
bool DatabaseMySQL::empty() const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
fetchTablesIntoLocalCache(getContext());
@ -99,7 +99,7 @@ bool DatabaseMySQL::empty() const
DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(ContextPtr local_context, const FilterByNameFunction & filter_by_table_name) const
{
Tables tables;
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
fetchTablesIntoLocalCache(local_context);
@ -117,7 +117,7 @@ bool DatabaseMySQL::isTableExist(const String & name, ContextPtr local_context)
StoragePtr DatabaseMySQL::tryGetTable(const String & mysql_table_name, ContextPtr local_context) const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
fetchTablesIntoLocalCache(local_context);
@ -129,7 +129,7 @@ StoragePtr DatabaseMySQL::tryGetTable(const String & mysql_table_name, ContextPt
ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, ContextPtr local_context, bool throw_on_error) const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
fetchTablesIntoLocalCache(local_context);
@ -175,7 +175,7 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, Context
time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_name) const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
fetchTablesIntoLocalCache(getContext());
@ -360,7 +360,7 @@ void DatabaseMySQL::cleanOutdatedTables()
void DatabaseMySQL::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &)
{
std::lock_guard<std::mutex> lock{mutex};
std::lock_guard lock{mutex};
if (!local_tables_cache.contains(table_name))
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot attach table {}.{} because it does not exist.",
@ -383,7 +383,7 @@ void DatabaseMySQL::attachTable(ContextPtr /* context_ */, const String & table_
StoragePtr DatabaseMySQL::detachTable(ContextPtr /* context */, const String & table_name)
{
std::lock_guard<std::mutex> lock{mutex};
std::lock_guard lock{mutex};
if (remove_or_detach_tables.contains(table_name))
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped",
@ -405,7 +405,7 @@ String DatabaseMySQL::getMetadataPath() const
void DatabaseMySQL::loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/, bool /* skip_startup_tables */)
{
std::lock_guard<std::mutex> lock{mutex};
std::lock_guard lock{mutex};
fs::directory_iterator iter(getMetadataPath());
for (fs::directory_iterator end; iter != end; ++iter)
@ -421,7 +421,7 @@ void DatabaseMySQL::loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel
void DatabaseMySQL::detachTablePermanently(ContextPtr, const String & table_name)
{
std::lock_guard<std::mutex> lock{mutex};
std::lock_guard lock{mutex};
fs::path remove_flag = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix);

View File

@ -76,7 +76,7 @@ String DatabasePostgreSQL::formatTableName(const String & table_name, bool quote
bool DatabasePostgreSQL::empty() const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
auto connection_holder = pool->get();
auto tables_list = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema);
@ -91,7 +91,7 @@ bool DatabasePostgreSQL::empty() const
DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator(ContextPtr local_context, const FilterByNameFunction & /* filter_by_table_name */) const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
Tables tables;
/// Do not allow to throw here, because this might be, for example, a query to system.tables.
@ -154,7 +154,7 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const
bool DatabasePostgreSQL::isTableExist(const String & table_name, ContextPtr /* context */) const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
if (detached_or_dropped.contains(table_name))
return false;
@ -165,7 +165,7 @@ bool DatabasePostgreSQL::isTableExist(const String & table_name, ContextPtr /* c
StoragePtr DatabasePostgreSQL::tryGetTable(const String & table_name, ContextPtr local_context) const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
if (!detached_or_dropped.contains(table_name))
return fetchTable(table_name, local_context, false);
@ -210,7 +210,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr,
void DatabasePostgreSQL::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &)
{
std::lock_guard<std::mutex> lock{mutex};
std::lock_guard lock{mutex};
if (!checkPostgresTable(table_name))
throw Exception(ErrorCodes::UNKNOWN_TABLE,
@ -235,7 +235,7 @@ void DatabasePostgreSQL::attachTable(ContextPtr /* context_ */, const String & t
StoragePtr DatabasePostgreSQL::detachTable(ContextPtr /* context_ */, const String & table_name)
{
std::lock_guard<std::mutex> lock{mutex};
std::lock_guard lock{mutex};
if (detached_or_dropped.contains(table_name))
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Cannot detach table {}. It is already dropped/detached", getTableNameForLogs(table_name));
@ -266,7 +266,7 @@ void DatabasePostgreSQL::createTable(ContextPtr local_context, const String & ta
void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool /* sync */)
{
std::lock_guard<std::mutex> lock{mutex};
std::lock_guard lock{mutex};
if (!checkPostgresTable(table_name))
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot drop table {} because it does not exist", getTableNameForLogs(table_name));
@ -293,7 +293,7 @@ void DatabasePostgreSQL::drop(ContextPtr /*context*/)
void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, LoadingStrictnessLevel /*mode*/, bool /* skip_startup_tables */)
{
{
std::lock_guard<std::mutex> lock{mutex};
std::lock_guard lock{mutex};
fs::directory_iterator iter(getMetadataPath());
/// Check for previously dropped tables
@ -314,7 +314,7 @@ void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, Load
void DatabasePostgreSQL::removeOutdatedTables()
{
std::lock_guard<std::mutex> lock{mutex};
std::lock_guard lock{mutex};
auto connection_holder = pool->get();
auto actual_tables = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema);

View File

@ -40,14 +40,14 @@ DatabaseSQLite::DatabaseSQLite(
bool DatabaseSQLite::empty() const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
return fetchTablesList().empty();
}
DatabaseTablesIteratorPtr DatabaseSQLite::getTablesIterator(ContextPtr local_context, const IDatabase::FilterByNameFunction &) const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
Tables tables;
auto table_names = fetchTablesList();
@ -120,14 +120,14 @@ bool DatabaseSQLite::checkSQLiteTable(const String & table_name) const
bool DatabaseSQLite::isTableExist(const String & table_name, ContextPtr) const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
return checkSQLiteTable(table_name);
}
StoragePtr DatabaseSQLite::tryGetTable(const String & table_name, ContextPtr local_context) const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
return fetchTable(table_name, local_context, false);
}
@ -175,7 +175,7 @@ ASTPtr DatabaseSQLite::getCreateTableQueryImpl(const String & table_name, Contex
{
StoragePtr storage;
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
storage = fetchTable(table_name, local_context, false);
}
if (!storage)

View File

@ -1289,7 +1289,7 @@ void Context::addQueryAccessInfo(
if (isGlobalContext())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info");
std::lock_guard<std::mutex> lock(query_access_info.mutex);
std::lock_guard lock(query_access_info.mutex);
query_access_info.databases.emplace(quoted_database_name);
query_access_info.tables.emplace(full_quoted_table_name);
for (const auto & column_name : column_names)

View File

@ -34,7 +34,7 @@ std::set<std::string> ExternalLoaderXMLConfigRepository::getAllLoadablesDefiniti
std::unordered_set<std::string> patterns_copy;
{
std::lock_guard<std::mutex> lock(patterns_mutex);
std::lock_guard lock(patterns_mutex);
patterns_copy = patterns;
}
@ -71,7 +71,7 @@ std::set<std::string> ExternalLoaderXMLConfigRepository::getAllLoadablesDefiniti
void ExternalLoaderXMLConfigRepository::updatePatterns(const std::unordered_set<std::string> & patterns_)
{
std::lock_guard<std::mutex> lock(patterns_mutex);
std::lock_guard lock(patterns_mutex);
if (patterns == patterns_)
return;

View File

@ -52,7 +52,7 @@ namespace
Block read()
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
if (eof)
return {};

View File

@ -352,7 +352,7 @@ CHJIT::~CHJIT() = default;
CHJIT::CompiledModule CHJIT::compileModule(std::function<void (llvm::Module &)> compile_function)
{
std::lock_guard<std::mutex> lock(jit_lock);
std::lock_guard lock(jit_lock);
auto module = createModuleForCompilation();
compile_function(*module);
@ -426,7 +426,7 @@ CHJIT::CompiledModule CHJIT::compileModule(std::unique_ptr<llvm::Module> module)
void CHJIT::deleteCompiledModule(const CHJIT::CompiledModule & module)
{
std::lock_guard<std::mutex> lock(jit_lock);
std::lock_guard lock(jit_lock);
auto module_it = module_identifier_to_memory_manager.find(module.identifier);
if (module_it == module_identifier_to_memory_manager.end())
@ -438,7 +438,7 @@ void CHJIT::deleteCompiledModule(const CHJIT::CompiledModule & module)
void CHJIT::registerExternalSymbol(const std::string & symbol_name, void * address)
{
std::lock_guard<std::mutex> lock(jit_lock);
std::lock_guard lock(jit_lock);
symbol_resolver->registerSymbol(symbol_name, address);
}

View File

@ -137,7 +137,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg)
std::shared_ptr<TextLog> text_log_locked{};
{
std::lock_guard<std::mutex> lock(text_log_mutex);
std::lock_guard lock(text_log_mutex);
text_log_locked = text_log.lock();
}
if (text_log_locked)
@ -155,7 +155,7 @@ void OwnSplitChannel::addChannel(Poco::AutoPtr<Poco::Channel> channel, const std
#ifndef WITHOUT_TEXT_LOG
void OwnSplitChannel::addTextLog(std::shared_ptr<DB::TextLog> log, int max_priority)
{
std::lock_guard<std::mutex> lock(text_log_mutex);
std::lock_guard lock(text_log_mutex);
text_log = log;
text_log_max_priority.store(max_priority, std::memory_order_relaxed);
}

View File

@ -15,7 +15,7 @@ namespace DB
collector_finished.wait();
{
std::lock_guard<std::mutex> lock(collector_thread_mutex);
std::lock_guard lock(collector_thread_mutex);
if (collector_thread.joinable())
collector_thread.join();
}
@ -80,7 +80,7 @@ namespace DB
}
{
std::lock_guard<std::mutex> lock(collector_thread_mutex);
std::lock_guard lock(collector_thread_mutex);
if (collector_thread.joinable())
collector_thread.join();
}
@ -137,7 +137,7 @@ namespace DB
{
/// Notify other threads.
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
unit.status = READY_TO_INSERT;
writer_condvar.notify_all();
}
@ -227,7 +227,7 @@ namespace DB
unit.actual_memory_size = out_buffer.getActualSize();
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
unit.status = READY_TO_READ;
collector_condvar.notify_all();
}

View File

@ -292,7 +292,7 @@ private:
{
/// Additionally notify condvars
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
segmentator_condvar.notify_all();
reader_condvar.notify_all();
}

View File

@ -62,7 +62,7 @@ public:
{
assert(!rhs_ports.first->isConnected() && !rhs_ports.second->isConnected());
std::lock_guard<std::mutex> lock(mux);
std::lock_guard lock(mux);
if (input_port || output_port)
{
assert(input_port && output_port);

View File

@ -274,7 +274,7 @@ namespace
}
catch (...)
{
std::lock_guard<std::mutex> lock(send_data_lock);
std::lock_guard lock(send_data_lock);
exception_during_send_data = std::current_exception();
}
});
@ -387,7 +387,7 @@ namespace
void rethrowExceptionDuringSendDataIfNeeded()
{
std::lock_guard<std::mutex> lock(send_data_lock);
std::lock_guard lock(send_data_lock);
if (exception_during_send_data)
{
command_is_invalid = true;

View File

@ -125,7 +125,7 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const
{
for (size_t i = 0, end = CurrentStatusInfo::end(); i < end; ++i)
{
std::lock_guard<std::mutex> lock(CurrentStatusInfo::locks[static_cast<CurrentStatusInfo::Status>(i)]);
std::lock_guard lock(CurrentStatusInfo::locks[static_cast<CurrentStatusInfo::Status>(i)]);
std::string metric_name{CurrentStatusInfo::getName(static_cast<CurrentStatusInfo::Status>(i))};
std::string metric_doc{CurrentStatusInfo::getDocumentation(static_cast<CurrentStatusInfo::Status>(i))};

View File

@ -12,7 +12,7 @@ FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, Stor
FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEventsAndReset()
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
Events res;
res.swap(events);
return res;
@ -20,7 +20,7 @@ FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEventsAndReset()
FileLogDirectoryWatcher::Error FileLogDirectoryWatcher::getErrorAndReset()
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
Error old_error = error;
error = {};
return old_error;
@ -33,7 +33,7 @@ const std::string & FileLogDirectoryWatcher::getPath() const
void FileLogDirectoryWatcher::onItemAdded(DirectoryWatcherBase::DirectoryEvent ev)
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
EventInfo info{ev.event, "onItemAdded"};
std::string event_path = ev.path;
@ -51,7 +51,7 @@ void FileLogDirectoryWatcher::onItemAdded(DirectoryWatcherBase::DirectoryEvent e
void FileLogDirectoryWatcher::onItemRemoved(DirectoryWatcherBase::DirectoryEvent ev)
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
EventInfo info{ev.event, "onItemRemoved"};
std::string event_path = ev.path;
@ -74,7 +74,7 @@ void FileLogDirectoryWatcher::onItemRemoved(DirectoryWatcherBase::DirectoryEvent
/// because it is equal to just record and handle one MODIY event
void FileLogDirectoryWatcher::onItemModified(DirectoryWatcherBase::DirectoryEvent ev)
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
auto event_path = ev.path;
EventInfo info{ev.event, "onItemModified"};
@ -97,7 +97,7 @@ void FileLogDirectoryWatcher::onItemModified(DirectoryWatcherBase::DirectoryEven
void FileLogDirectoryWatcher::onItemMovedFrom(DirectoryWatcherBase::DirectoryEvent ev)
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
EventInfo info{ev.event, "onItemMovedFrom"};
std::string event_path = ev.path;
@ -114,7 +114,7 @@ void FileLogDirectoryWatcher::onItemMovedFrom(DirectoryWatcherBase::DirectoryEve
void FileLogDirectoryWatcher::onItemMovedTo(DirectoryWatcherBase::DirectoryEvent ev)
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
EventInfo info{ev.event, "onItemMovedTo"};
std::string event_path = ev.path;
@ -131,7 +131,7 @@ void FileLogDirectoryWatcher::onItemMovedTo(DirectoryWatcherBase::DirectoryEvent
void FileLogDirectoryWatcher::onError(Exception e)
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
LOG_ERROR(log, "Error happened during watching directory: {}", error.error_msg);
error.has_error = true;
error.error_msg = e.message();

View File

@ -312,7 +312,7 @@ Pipe StorageFileLog::read(
if (mv_attached)
throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageFileLog with attached materialized views");
std::lock_guard<std::mutex> lock(file_infos_mutex);
std::lock_guard lock(file_infos_mutex);
if (running_streams)
{
throw Exception(ErrorCodes::CANNOT_SELECT, "Another select query is running on this table, need to wait it finish.");
@ -659,7 +659,7 @@ void StorageFileLog::threadFunc()
bool StorageFileLog::streamToViews()
{
std::lock_guard<std::mutex> lock(file_infos_mutex);
std::lock_guard lock(file_infos_mutex);
if (running_streams)
{
LOG_INFO(log, "Another select query is running on this table, need to wait it finish.");

View File

@ -856,7 +856,7 @@ HiveFiles StorageHive::collectHiveFiles(
= collectHiveFilesFromPartition(partition, query_info, hive_table_metadata, fs, context_, prune_level);
if (!hive_files_in_partition.empty())
{
std::lock_guard<std::mutex> lock(hive_files_mutex);
std::lock_guard lock(hive_files_mutex);
hit_parttions_num += 1;
if (hive_max_query_partitions > 0 && hit_parttions_num > hive_max_query_partitions)
{
@ -882,7 +882,7 @@ HiveFiles StorageHive::collectHiveFiles(
auto hive_file = getHiveFileIfNeeded(file_info, {}, query_info, hive_table_metadata, context_, prune_level);
if (hive_file)
{
std::lock_guard<std::mutex> lock(hive_files_mutex);
std::lock_guard lock(hive_files_mutex);
hive_files.push_back(hive_file);
}
});

View File

@ -2316,7 +2316,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart(
return false;
}
std::lock_guard<std::mutex> lock(queue.state_mutex);
std::lock_guard lock(queue.state_mutex);
/// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer
/// and it is guaranteed that it will contain all merges assigned before this object is constructed.
@ -2334,7 +2334,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart(
bool ReplicatedMergeTreeMergePredicate::partParticipatesInReplaceRange(const MergeTreeData::DataPartPtr & part, String * out_reason) const
{
std::lock_guard<std::mutex> lock(queue.state_mutex);
std::lock_guard lock(queue.state_mutex);
for (const auto & entry : queue.queue)
{
if (entry->type != ReplicatedMergeTreeLogEntry::REPLACE_RANGE)
@ -2457,7 +2457,7 @@ bool ReplicatedMergeTreeMergePredicate::isGoingToBeDropped(const MergeTreePartIn
String ReplicatedMergeTreeMergePredicate::getCoveringVirtualPart(const String & part_name) const
{
std::lock_guard<std::mutex> lock(queue.state_mutex);
std::lock_guard lock(queue.state_mutex);
return queue.virtual_parts.getContainingPart(MergeTreePartInfo::fromPartName(part_name, queue.format_version));
}

View File

@ -208,13 +208,13 @@ void StorageDictionary::removeDictionaryConfigurationFromRepository()
Poco::Timestamp StorageDictionary::getUpdateTime() const
{
std::lock_guard<std::mutex> lock(dictionary_config_mutex);
std::lock_guard lock(dictionary_config_mutex);
return update_time;
}
LoadablesConfigurationPtr StorageDictionary::getConfiguration() const
{
std::lock_guard<std::mutex> lock(dictionary_config_mutex);
std::lock_guard lock(dictionary_config_mutex);
return configuration;
}
@ -234,7 +234,7 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id)
assert(old_table_id.uuid == new_table_id.uuid || move_to_atomic || move_to_ordinary);
{
std::lock_guard<std::mutex> lock(dictionary_config_mutex);
std::lock_guard lock(dictionary_config_mutex);
configuration->setString("dictionary.database", new_table_id.database_name);
configuration->setString("dictionary.name", new_table_id.table_name);
@ -301,7 +301,7 @@ void StorageDictionary::alter(const AlterCommands & params, ContextPtr alter_con
dictionary_non_const->setDictionaryComment(new_comment);
}
std::lock_guard<std::mutex> lock(dictionary_config_mutex);
std::lock_guard lock(dictionary_config_mutex);
configuration->setString("dictionary.comment", new_comment);
}

View File

@ -732,7 +732,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id)
to_kill->removeFile();
LOG_TRACE(log, "Cancelled part mutations and removed mutation file {}", mutation_id);
{
std::lock_guard<std::mutex> lock(mutation_wait_mutex);
std::lock_guard lock(mutation_wait_mutex);
mutation_wait_event.notify_all();
}
@ -1306,7 +1306,7 @@ size_t StorageMergeTree::clearOldMutations(bool truncate)
std::vector<MergeTreeMutationEntry> mutations_to_delete;
{
std::lock_guard<std::mutex> lock(currently_processing_in_background_mutex);
std::lock_guard lock(currently_processing_in_background_mutex);
if (current_mutations_by_version.size() <= finished_mutations_to_keep)
return 0;