mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Miscellaneous [#CLICKHOUSE-2]
This commit is contained in:
parent
531560a6a5
commit
e33e5150b7
@ -21,7 +21,7 @@ MetricsTransmitter::~MetricsTransmitter()
|
||||
try
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
std::lock_guard lock{mutex};
|
||||
quit = true;
|
||||
}
|
||||
|
||||
@ -56,7 +56,7 @@ void MetricsTransmitter::run()
|
||||
|
||||
std::vector<ProfileEvents::Count> prev_counters(ProfileEvents::end());
|
||||
|
||||
std::unique_lock<std::mutex> lock{mutex};
|
||||
std::unique_lock lock{mutex};
|
||||
|
||||
while (true)
|
||||
{
|
||||
|
@ -52,7 +52,7 @@ MultiplexedConnections::MultiplexedConnections(
|
||||
|
||||
void MultiplexedConnections::sendExternalTablesData(std::vector<ExternalTablesData> & data)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
if (!sent_query)
|
||||
throw Exception("Cannot send external tables data: query not yet sent.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -79,7 +79,7 @@ void MultiplexedConnections::sendQuery(
|
||||
const ClientInfo * client_info,
|
||||
bool with_pending_data)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
if (sent_query)
|
||||
throw Exception("Query already sent.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -121,14 +121,14 @@ void MultiplexedConnections::sendQuery(
|
||||
|
||||
Connection::Packet MultiplexedConnections::receivePacket()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
Connection::Packet packet = receivePacketUnlocked();
|
||||
return packet;
|
||||
}
|
||||
|
||||
void MultiplexedConnections::disconnect()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
for (ReplicaState & state : replica_states)
|
||||
{
|
||||
@ -143,7 +143,7 @@ void MultiplexedConnections::disconnect()
|
||||
|
||||
void MultiplexedConnections::sendCancel()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
if (!sent_query || cancelled)
|
||||
throw Exception("Cannot cancel. Either no query sent or already cancelled.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -160,7 +160,7 @@ void MultiplexedConnections::sendCancel()
|
||||
|
||||
Connection::Packet MultiplexedConnections::drain()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
if (!cancelled)
|
||||
throw Exception("Cannot drain connections: cancel first.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -195,7 +195,7 @@ Connection::Packet MultiplexedConnections::drain()
|
||||
|
||||
std::string MultiplexedConnections::dumpAddresses() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
return dumpAddressesUnlocked();
|
||||
}
|
||||
|
||||
|
@ -231,7 +231,7 @@ public:
|
||||
|
||||
~Holder()
|
||||
{
|
||||
std::lock_guard<std::mutex> cache_lock(cache.mutex);
|
||||
std::lock_guard cache_lock(cache.mutex);
|
||||
if (--region.refcount == 0)
|
||||
cache.lru_list.push_back(region);
|
||||
cache.total_size_in_use -= region.size;
|
||||
@ -279,14 +279,14 @@ private:
|
||||
|
||||
InsertTokenHolder() = default;
|
||||
|
||||
void acquire(const Key * key_, const std::shared_ptr<InsertToken> & token_, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
|
||||
void acquire(const Key * key_, const std::shared_ptr<InsertToken> & token_, [[maybe_unused]] std::lock_guard & cache_lock)
|
||||
{
|
||||
key = key_;
|
||||
token = token_;
|
||||
++token->refcount;
|
||||
}
|
||||
|
||||
void cleanup([[maybe_unused]] std::lock_guard<std::mutex> & token_lock, [[maybe_unused]] std::lock_guard<std::mutex> & cache_lock)
|
||||
void cleanup([[maybe_unused]] std::lock_guard & token_lock, [[maybe_unused]] std::lock_guard & cache_lock)
|
||||
{
|
||||
token->cache.insert_tokens.erase(*key);
|
||||
token->cleaned_up = true;
|
||||
@ -301,12 +301,12 @@ private:
|
||||
if (cleaned_up)
|
||||
return;
|
||||
|
||||
std::lock_guard<std::mutex> token_lock(token->mutex);
|
||||
std::lock_guard token_lock(token->mutex);
|
||||
|
||||
if (token->cleaned_up)
|
||||
return;
|
||||
|
||||
std::lock_guard<std::mutex> cache_lock(token->cache.mutex);
|
||||
std::lock_guard cache_lock(token->cache.mutex);
|
||||
|
||||
--token->refcount;
|
||||
if (token->refcount == 0)
|
||||
@ -536,7 +536,7 @@ public:
|
||||
|
||||
~ArrayCache()
|
||||
{
|
||||
std::lock_guard<std::mutex> cache_lock(mutex);
|
||||
std::lock_guard cache_lock(mutex);
|
||||
|
||||
key_map.clear();
|
||||
lru_list.clear();
|
||||
@ -563,7 +563,7 @@ public:
|
||||
{
|
||||
InsertTokenHolder token_holder;
|
||||
{
|
||||
std::lock_guard<std::mutex> cache_lock(mutex);
|
||||
std::lock_guard cache_lock(mutex);
|
||||
|
||||
auto it = key_map.find(key, RegionCompareByKey());
|
||||
if (key_map.end() != it)
|
||||
@ -584,7 +584,7 @@ public:
|
||||
|
||||
InsertToken * token = token_holder.token.get();
|
||||
|
||||
std::lock_guard<std::mutex> token_lock(token->mutex);
|
||||
std::lock_guard token_lock(token->mutex);
|
||||
|
||||
token_holder.cleaned_up = token->cleaned_up;
|
||||
|
||||
@ -605,7 +605,7 @@ public:
|
||||
|
||||
RegionMetadata * region;
|
||||
{
|
||||
std::lock_guard<std::mutex> cache_lock(mutex);
|
||||
std::lock_guard cache_lock(mutex);
|
||||
region = allocate(size);
|
||||
}
|
||||
|
||||
@ -626,14 +626,14 @@ public:
|
||||
catch (...)
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> cache_lock(mutex);
|
||||
std::lock_guard cache_lock(mutex);
|
||||
freeRegion(*region);
|
||||
}
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
std::lock_guard<std::mutex> cache_lock(mutex);
|
||||
std::lock_guard cache_lock(mutex);
|
||||
|
||||
try
|
||||
{
|
||||
@ -692,7 +692,7 @@ public:
|
||||
|
||||
Statistics getStatistics() const
|
||||
{
|
||||
std::lock_guard<std::mutex> cache_lock(mutex);
|
||||
std::lock_guard cache_lock(mutex);
|
||||
Statistics res;
|
||||
|
||||
res.total_chunks_size = total_chunks_size;
|
||||
|
@ -78,7 +78,7 @@ void ConfigReloader::run()
|
||||
|
||||
void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallback_to_preprocessed)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(reload_mutex);
|
||||
std::lock_guard lock(reload_mutex);
|
||||
|
||||
FilesChangesTracker new_files = getNewFileList();
|
||||
if (force || need_reload_from_zk || new_files.isDifferOrNewerThan(files))
|
||||
|
@ -54,7 +54,7 @@ public:
|
||||
template <typename Callback>
|
||||
Int64 add(Int64 delta, Callback && locked_callback, bool create_if_need = false)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
Int64 res = -1;
|
||||
|
||||
|
@ -48,7 +48,7 @@ public:
|
||||
|
||||
MappedPtr get(const Key & key)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
auto res = getImpl(key, lock);
|
||||
if (res)
|
||||
@ -61,7 +61,7 @@ public:
|
||||
|
||||
void set(const Key & key, const MappedPtr & mapped)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
setImpl(key, mapped, lock);
|
||||
}
|
||||
@ -79,7 +79,7 @@ public:
|
||||
{
|
||||
InsertTokenHolder token_holder;
|
||||
{
|
||||
std::lock_guard<std::mutex> cache_lock(mutex);
|
||||
std::lock_guard cache_lock(mutex);
|
||||
|
||||
auto val = getImpl(key, cache_lock);
|
||||
if (val)
|
||||
@ -97,7 +97,7 @@ public:
|
||||
|
||||
InsertToken * token = token_holder.token.get();
|
||||
|
||||
std::lock_guard<std::mutex> token_lock(token->mutex);
|
||||
std::lock_guard token_lock(token->mutex);
|
||||
|
||||
token_holder.cleaned_up = token->cleaned_up;
|
||||
|
||||
@ -111,7 +111,7 @@ public:
|
||||
++misses;
|
||||
token->value = load_func();
|
||||
|
||||
std::lock_guard<std::mutex> cache_lock(mutex);
|
||||
std::lock_guard cache_lock(mutex);
|
||||
|
||||
/// Insert the new value only if the token is still in present in insert_tokens.
|
||||
/// (The token may be absent because of a concurrent reset() call).
|
||||
@ -131,26 +131,26 @@ public:
|
||||
|
||||
void getStats(size_t & out_hits, size_t & out_misses) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
out_hits = hits;
|
||||
out_misses = misses;
|
||||
}
|
||||
|
||||
size_t weight() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
return current_size;
|
||||
}
|
||||
|
||||
size_t count() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
return cells.size();
|
||||
}
|
||||
|
||||
void reset()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
queue.clear();
|
||||
cells.clear();
|
||||
insert_tokens.clear();
|
||||
@ -234,12 +234,12 @@ private:
|
||||
if (cleaned_up)
|
||||
return;
|
||||
|
||||
std::lock_guard<std::mutex> token_lock(token->mutex);
|
||||
std::lock_guard token_lock(token->mutex);
|
||||
|
||||
if (token->cleaned_up)
|
||||
return;
|
||||
|
||||
std::lock_guard<std::mutex> cache_lock(token->cache.mutex);
|
||||
std::lock_guard cache_lock(token->cache.mutex);
|
||||
|
||||
--token->refcount;
|
||||
if (token->refcount == 0)
|
||||
|
@ -38,7 +38,7 @@ protected:
|
||||
|
||||
void operator()(T * owning_ptr) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{parent->mutex};
|
||||
std::lock_guard lock{parent->mutex};
|
||||
parent->stack.emplace(owning_ptr);
|
||||
}
|
||||
};
|
||||
@ -51,7 +51,7 @@ public:
|
||||
template <typename Factory>
|
||||
Pointer get(Factory && f)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
std::unique_lock lock(mutex);
|
||||
|
||||
if (stack.empty())
|
||||
{
|
||||
@ -94,7 +94,7 @@ public:
|
||||
template <typename Factory>
|
||||
Pointer get(const Key & key, Factory && f)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
std::unique_lock lock(mutex);
|
||||
|
||||
auto it = container.find(key);
|
||||
if (container.end() == it)
|
||||
|
@ -128,7 +128,7 @@ protected:
|
||||
c_end = c_start + end_diff;
|
||||
c_end_of_storage = c_start + bytes - pad_right - pad_left;
|
||||
|
||||
if (pad_left)
|
||||
if (pad_left) /// TODO Do we need it?
|
||||
memset(c_start - ELEMENT_SIZE, 0, ELEMENT_SIZE);
|
||||
}
|
||||
|
||||
|
@ -54,7 +54,7 @@ private:
|
||||
PoolEntryHelper(PooledObject & data_) : data(data_) { data.in_use = true; }
|
||||
~PoolEntryHelper()
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(data.pool.mutex);
|
||||
std::unique_lock lock(data.pool.mutex);
|
||||
data.in_use = false;
|
||||
data.pool.available.notify_one();
|
||||
}
|
||||
@ -107,7 +107,7 @@ public:
|
||||
/** Allocates the object. Wait for free object in pool for 'timeout'. With 'timeout' < 0, the timeout is infinite. */
|
||||
Entry get(Poco::Timespan::TimeDiff timeout)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
std::unique_lock lock(mutex);
|
||||
|
||||
while (true)
|
||||
{
|
||||
@ -133,7 +133,7 @@ public:
|
||||
|
||||
void reserve(size_t count)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
while (items.size() < count)
|
||||
items.emplace_back(std::make_shared<PooledObject>(allocObject(), *this));
|
||||
|
@ -195,7 +195,7 @@ PoolWithFailoverBase<TNestedPool>::getMany(
|
||||
/// At exit update shared error counts with error counts occured during this call.
|
||||
SCOPE_EXIT(
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(pool_states_mutex);
|
||||
std::lock_guard lock(pool_states_mutex);
|
||||
for (const ShuffledPool & pool: shuffled_pools)
|
||||
shared_pool_states[pool.index].error_count += pool.error_count;
|
||||
});
|
||||
@ -300,7 +300,7 @@ void PoolWithFailoverBase<TNestedPool>::reportError(const Entry & entry)
|
||||
{
|
||||
if (nested_pools[i]->contains(entry))
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(pool_states_mutex);
|
||||
std::lock_guard lock(pool_states_mutex);
|
||||
++shared_pool_states[i].error_count;
|
||||
return;
|
||||
}
|
||||
@ -338,7 +338,7 @@ PoolWithFailoverBase<TNestedPool>::updatePoolStates()
|
||||
result.reserve(nested_pools.size());
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(pool_states_mutex);
|
||||
std::lock_guard lock(pool_states_mutex);
|
||||
|
||||
for (auto & state : shared_pool_states)
|
||||
state.randomize();
|
||||
|
@ -70,7 +70,7 @@ RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type)
|
||||
GroupsContainer::iterator it_group;
|
||||
ClientsContainer::iterator it_client;
|
||||
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
std::unique_lock lock(mutex);
|
||||
|
||||
/// Check if the same thread is acquiring previously acquired lock
|
||||
auto it_handler = thread_to_handler.find(this_thread_id);
|
||||
@ -139,7 +139,7 @@ RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type)
|
||||
|
||||
RWLockImpl::LockHandlerImpl::~LockHandlerImpl()
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(parent->mutex);
|
||||
std::unique_lock lock(parent->mutex);
|
||||
|
||||
/// Remove weak_ptr to the handler, since there are no owners of the current lock
|
||||
parent->thread_to_handler.erase(it_handler);
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
Result operator() (Args &&... args)
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
Key key{std::forward<Args>(args)...};
|
||||
auto it = cache.find(key);
|
||||
@ -45,7 +45,7 @@ public:
|
||||
Result res = f(std::forward<Args>(args)...);
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
cache.emplace(std::forward_as_tuple(args...), res);
|
||||
}
|
||||
@ -55,7 +55,7 @@ public:
|
||||
|
||||
void drop()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
cache.clear();
|
||||
}
|
||||
};
|
||||
|
@ -32,11 +32,10 @@ StackTrace::StackTrace()
|
||||
std::string StackTrace::toStringImpl(const Frames & frames, size_t frames_size)
|
||||
{
|
||||
char ** symbols = backtrace_symbols(frames.data(), frames_size);
|
||||
std::stringstream res;
|
||||
|
||||
if (!symbols)
|
||||
return "Cannot get symbols for stack trace.\n";
|
||||
|
||||
std::stringstream res;
|
||||
try
|
||||
{
|
||||
for (size_t i = 0, size = frames_size; i < size; ++i)
|
||||
|
@ -14,7 +14,7 @@ using namespace zkutil;
|
||||
|
||||
ZooKeeperHolder::UnstorableZookeeperHandler ZooKeeperHolder::getZooKeeper()
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
std::unique_lock lock(mutex);
|
||||
return UnstorableZookeeperHandler(ptr);
|
||||
}
|
||||
|
||||
@ -25,7 +25,7 @@ void ZooKeeperHolder::initFromInstance(const ZooKeeper::Ptr & zookeeper_ptr)
|
||||
|
||||
bool ZooKeeperHolder::replaceZooKeeperSessionToNewOne()
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
std::unique_lock lock(mutex);
|
||||
|
||||
if (ptr.unique())
|
||||
{
|
||||
|
@ -22,7 +22,7 @@ ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Coor
|
||||
{
|
||||
std::unordered_set<std::string> invalidated_paths;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(context->mutex);
|
||||
std::lock_guard lock(context->mutex);
|
||||
|
||||
if (context->all_paths_invalidated)
|
||||
{
|
||||
@ -57,7 +57,7 @@ ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Coor
|
||||
|
||||
bool changed = false;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(owned_context->mutex);
|
||||
std::lock_guard lock(owned_context->mutex);
|
||||
|
||||
if (response.type != Coordination::SESSION)
|
||||
changed = owned_context->invalidated_paths.emplace(response.path).second;
|
||||
|
@ -127,7 +127,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::cancel(bool kill)
|
||||
if (parallel_merge_data)
|
||||
{
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(parallel_merge_data->merged_blocks_mutex);
|
||||
std::unique_lock lock(parallel_merge_data->merged_blocks_mutex);
|
||||
parallel_merge_data->finish = true;
|
||||
}
|
||||
parallel_merge_data->merged_blocks_changed.notify_one(); /// readImpl method must stop waiting and exit.
|
||||
@ -219,7 +219,7 @@ Block MergingAggregatedMemoryEfficientBlockInputStream::readImpl()
|
||||
|
||||
while (true)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(parallel_merge_data->merged_blocks_mutex);
|
||||
std::unique_lock lock(parallel_merge_data->merged_blocks_mutex);
|
||||
|
||||
parallel_merge_data->merged_blocks_changed.wait(lock, [this]
|
||||
{
|
||||
@ -323,7 +323,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(ThreadGroupSt
|
||||
* - or, if no next blocks, set 'exhausted' flag.
|
||||
*/
|
||||
{
|
||||
std::lock_guard<std::mutex> lock_next_blocks(parallel_merge_data->get_next_blocks_mutex);
|
||||
std::lock_guard lock_next_blocks(parallel_merge_data->get_next_blocks_mutex);
|
||||
|
||||
if (parallel_merge_data->exhausted || parallel_merge_data->finish)
|
||||
break;
|
||||
@ -333,7 +333,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(ThreadGroupSt
|
||||
if (!blocks_to_merge || blocks_to_merge->empty())
|
||||
{
|
||||
{
|
||||
std::unique_lock<std::mutex> lock_merged_blocks(parallel_merge_data->merged_blocks_mutex);
|
||||
std::unique_lock lock_merged_blocks(parallel_merge_data->merged_blocks_mutex);
|
||||
parallel_merge_data->exhausted = true;
|
||||
}
|
||||
|
||||
@ -347,7 +347,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(ThreadGroupSt
|
||||
: blocks_to_merge->front().info.bucket_num;
|
||||
|
||||
{
|
||||
std::unique_lock<std::mutex> lock_merged_blocks(parallel_merge_data->merged_blocks_mutex);
|
||||
std::unique_lock lock_merged_blocks(parallel_merge_data->merged_blocks_mutex);
|
||||
|
||||
parallel_merge_data->have_space.wait(lock_merged_blocks, [this]
|
||||
{
|
||||
@ -370,7 +370,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(ThreadGroupSt
|
||||
Block res = aggregator.mergeBlocks(*blocks_to_merge, final);
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parallel_merge_data->merged_blocks_mutex);
|
||||
std::lock_guard lock(parallel_merge_data->merged_blocks_mutex);
|
||||
|
||||
if (parallel_merge_data->finish)
|
||||
break;
|
||||
@ -385,7 +385,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(ThreadGroupSt
|
||||
catch (...)
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parallel_merge_data->merged_blocks_mutex);
|
||||
std::lock_guard lock(parallel_merge_data->merged_blocks_mutex);
|
||||
parallel_merge_data->exception = std::current_exception();
|
||||
parallel_merge_data->finish = true;
|
||||
}
|
||||
|
@ -191,7 +191,7 @@ private:
|
||||
{
|
||||
InputData unprepared_input;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(unprepared_inputs_mutex);
|
||||
std::lock_guard lock(unprepared_inputs_mutex);
|
||||
|
||||
if (unprepared_inputs.empty())
|
||||
break;
|
||||
@ -203,7 +203,7 @@ private:
|
||||
unprepared_input.in->readPrefix();
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(available_inputs_mutex);
|
||||
std::lock_guard lock(available_inputs_mutex);
|
||||
available_inputs.push(unprepared_input);
|
||||
}
|
||||
}
|
||||
@ -257,7 +257,7 @@ private:
|
||||
|
||||
/// Select the next source.
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(available_inputs_mutex);
|
||||
std::lock_guard lock(available_inputs_mutex);
|
||||
|
||||
/// If there are no free sources, then this thread is no longer needed. (But other threads can work with their sources.)
|
||||
if (available_inputs.empty())
|
||||
@ -278,7 +278,7 @@ private:
|
||||
|
||||
/// If this source is not run out yet, then put the resulting block in the ready queue.
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(available_inputs_mutex);
|
||||
std::lock_guard lock(available_inputs_mutex);
|
||||
|
||||
if (block)
|
||||
{
|
||||
|
@ -104,7 +104,7 @@ void RemoteBlockInputStream::cancel(bool kill)
|
||||
return;
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(external_tables_mutex);
|
||||
std::lock_guard lock(external_tables_mutex);
|
||||
|
||||
/// Stop sending external data.
|
||||
for (auto & vec : external_tables_data)
|
||||
@ -124,7 +124,7 @@ void RemoteBlockInputStream::sendExternalTables()
|
||||
size_t count = multiplexed_connections->size();
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(external_tables_mutex);
|
||||
std::lock_guard lock(external_tables_mutex);
|
||||
|
||||
external_tables_data.reserve(count);
|
||||
|
||||
|
@ -273,7 +273,7 @@ void DatabaseOrdinary::createTable(
|
||||
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
if (tables.find(table_name) != tables.end())
|
||||
throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
@ -298,7 +298,7 @@ void DatabaseOrdinary::createTable(
|
||||
{
|
||||
/// Add a table to the map of known tables.
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
if (!tables.emplace(table_name, table).second)
|
||||
throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
@ -492,7 +492,7 @@ void DatabaseOrdinary::shutdown()
|
||||
|
||||
Tables tables_snapshot;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
tables_snapshot = tables;
|
||||
}
|
||||
|
||||
@ -501,7 +501,7 @@ void DatabaseOrdinary::shutdown()
|
||||
kv.second->shutdown();
|
||||
}
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
tables.clear();
|
||||
}
|
||||
|
||||
|
@ -88,7 +88,7 @@ bool DatabaseWithOwnTablesBase::isTableExist(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
return tables.find(table_name) != tables.end();
|
||||
}
|
||||
|
||||
@ -96,7 +96,7 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTable(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
return {};
|
||||
@ -105,13 +105,13 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTable(
|
||||
|
||||
DatabaseIteratorPtr DatabaseWithOwnTablesBase::getIterator(const Context & /*context*/)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
return std::make_unique<DatabaseSnapshotIterator>(tables);
|
||||
}
|
||||
|
||||
bool DatabaseWithOwnTablesBase::empty(const Context & /*context*/) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
return tables.empty();
|
||||
}
|
||||
|
||||
@ -119,7 +119,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
|
||||
{
|
||||
StoragePtr res;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
@ -132,7 +132,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
|
||||
|
||||
void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
if (!tables.emplace(table_name, table).second)
|
||||
throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
@ -144,7 +144,7 @@ void DatabaseWithOwnTablesBase::shutdown()
|
||||
|
||||
Tables tables_snapshot;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
tables_snapshot = tables;
|
||||
}
|
||||
|
||||
@ -153,7 +153,7 @@ void DatabaseWithOwnTablesBase::shutdown()
|
||||
kv.second->shutdown();
|
||||
}
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
tables.clear();
|
||||
}
|
||||
|
||||
|
@ -742,7 +742,7 @@ private:
|
||||
if (0 == size)
|
||||
throw Exception{"Empty arrays are illegal in function " + getName(), ErrorCodes::BAD_ARGUMENTS};
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (initialized)
|
||||
return;
|
||||
|
@ -78,7 +78,7 @@ void AIOContextPool::fulfillPromises(const io_event events[], const int num_even
|
||||
if (num_events == 0)
|
||||
return;
|
||||
|
||||
const std::lock_guard<std::mutex> lock{mutex};
|
||||
const std::lock_guard lock{mutex};
|
||||
|
||||
/// look at returned events and find corresponding promise, set result and erase promise from map
|
||||
for (const auto & event : boost::make_iterator_range(events, events + num_events))
|
||||
@ -114,7 +114,7 @@ void AIOContextPool::notifyProducers(const int num_producers) const
|
||||
|
||||
void AIOContextPool::reportExceptionToAnyProducer()
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{mutex};
|
||||
const std::lock_guard lock{mutex};
|
||||
|
||||
const auto any_promise_it = std::begin(promises);
|
||||
any_promise_it->second.set_exception(std::current_exception());
|
||||
@ -123,7 +123,7 @@ void AIOContextPool::reportExceptionToAnyProducer()
|
||||
|
||||
std::future<AIOContextPool::BytesRead> AIOContextPool::post(struct iocb & iocb)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock{mutex};
|
||||
std::unique_lock lock{mutex};
|
||||
|
||||
/// get current id and increment it by one
|
||||
const auto request_id = next_id;
|
||||
|
@ -141,7 +141,7 @@ namespace
|
||||
public:
|
||||
Entry getSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t max_connections_per_endpoint)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
std::unique_lock lock(mutex);
|
||||
const std::string & host = uri.getHost();
|
||||
UInt16 port = uri.getPort();
|
||||
bool https = isHTTPS(uri);
|
||||
|
@ -68,7 +68,7 @@ void WriteBufferFromHTTPServerResponse::finishSendHeaders()
|
||||
void WriteBufferFromHTTPServerResponse::nextImpl()
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
startSendHeaders();
|
||||
|
||||
@ -147,7 +147,7 @@ WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse(
|
||||
|
||||
void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
/// Cannot add new headers if body was started to send.
|
||||
if (headers_finished_sending)
|
||||
@ -181,7 +181,7 @@ void WriteBufferFromHTTPServerResponse::finalize()
|
||||
else
|
||||
{
|
||||
/// If no remaining data, just send headers.
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
startSendHeaders();
|
||||
finishSendHeaders();
|
||||
}
|
||||
|
@ -33,7 +33,7 @@ void ActionLocksManager::add(StorageActionBlockType action_type)
|
||||
|
||||
if (!action_lock.expired())
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
storage_locks[table.get()][action_type] = std::move(action_lock);
|
||||
}
|
||||
});
|
||||
@ -47,7 +47,7 @@ void ActionLocksManager::add(const String & database_name, const String & table_
|
||||
|
||||
if (!action_lock.expired())
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
storage_locks[table.get()][action_type] = std::move(action_lock);
|
||||
}
|
||||
}
|
||||
@ -55,7 +55,7 @@ void ActionLocksManager::add(const String & database_name, const String & table_
|
||||
|
||||
void ActionLocksManager::remove(StorageActionBlockType action_type)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
for (auto & storage_elem : storage_locks)
|
||||
storage_elem.second.erase(action_type);
|
||||
@ -65,7 +65,7 @@ void ActionLocksManager::remove(const String & database_name, const String & tab
|
||||
{
|
||||
if (auto table = global_context.tryGetTable(database_name, table_name))
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (storage_locks.count(table.get()))
|
||||
storage_locks[table.get()].erase(action_type);
|
||||
@ -74,7 +74,7 @@ void ActionLocksManager::remove(const String & database_name, const String & tab
|
||||
|
||||
void ActionLocksManager::cleanExpired()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
for (auto it_storage = storage_locks.begin(); it_storage != storage_locks.end();)
|
||||
{
|
||||
|
@ -196,7 +196,7 @@ Aggregator::Aggregator(const Params & params_)
|
||||
|
||||
void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (compiled_if_possible)
|
||||
return;
|
||||
@ -966,7 +966,7 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants)
|
||||
double uncompressed_bytes = compressed_buf.count();
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(temporary_files.mutex);
|
||||
std::lock_guard lock(temporary_files.mutex);
|
||||
temporary_files.files.emplace_back(std::move(file));
|
||||
temporary_files.sum_size_uncompressed += uncompressed_bytes;
|
||||
temporary_files.sum_size_compressed += compressed_bytes;
|
||||
@ -1819,7 +1819,7 @@ protected:
|
||||
|
||||
while (true)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(parallel_merge_data->mutex);
|
||||
std::unique_lock lock(parallel_merge_data->mutex);
|
||||
|
||||
if (parallel_merge_data->exception)
|
||||
std::rethrow_exception(parallel_merge_data->exception);
|
||||
@ -1909,12 +1909,12 @@ private:
|
||||
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
||||
#undef M
|
||||
|
||||
std::lock_guard<std::mutex> lock(parallel_merge_data->mutex);
|
||||
std::lock_guard lock(parallel_merge_data->mutex);
|
||||
parallel_merge_data->ready_blocks[bucket_num] = std::move(block);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parallel_merge_data->mutex);
|
||||
std::lock_guard lock(parallel_merge_data->mutex);
|
||||
if (!parallel_merge_data->exception)
|
||||
parallel_merge_data->exception = std::current_exception();
|
||||
}
|
||||
|
@ -1486,7 +1486,7 @@ public:
|
||||
|
||||
bool empty() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
return files.empty();
|
||||
}
|
||||
};
|
||||
|
@ -42,7 +42,7 @@ AsynchronousMetrics::~AsynchronousMetrics()
|
||||
try
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{wait_mutex};
|
||||
std::lock_guard lock{wait_mutex};
|
||||
quit = true;
|
||||
}
|
||||
|
||||
@ -58,14 +58,14 @@ AsynchronousMetrics::~AsynchronousMetrics()
|
||||
|
||||
AsynchronousMetrics::Container AsynchronousMetrics::getValues() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{container_mutex};
|
||||
std::lock_guard lock{container_mutex};
|
||||
return container;
|
||||
}
|
||||
|
||||
|
||||
void AsynchronousMetrics::set(const std::string & name, Value value)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{container_mutex};
|
||||
std::lock_guard lock{container_mutex};
|
||||
container[name] = value;
|
||||
}
|
||||
|
||||
@ -74,7 +74,7 @@ void AsynchronousMetrics::run()
|
||||
{
|
||||
setThreadName("AsyncMetrics");
|
||||
|
||||
std::unique_lock<std::mutex> lock{wait_mutex};
|
||||
std::unique_lock lock{wait_mutex};
|
||||
|
||||
/// Next minute + 30 seconds. To be distant with moment of transmission of metrics, see MetricsTransmitter.
|
||||
const auto get_next_minute = []
|
||||
|
@ -488,7 +488,7 @@ std::shared_ptr<CatBoostLibHolder> getCatBoostWrapperHolder(const std::string &
|
||||
static std::weak_ptr<CatBoostLibHolder> ptr;
|
||||
static std::mutex mutex;
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
auto result = ptr.lock();
|
||||
|
||||
if (!result || result->getCurrentPath() != lib_path)
|
||||
|
@ -87,7 +87,7 @@ SharedLibraryPtr Compiler::getOrCount(
|
||||
{
|
||||
HashedKey hashed_key = getHash(key);
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
UInt32 count = ++counts[hashed_key];
|
||||
|
||||
@ -306,7 +306,7 @@ void Compiler::compile(
|
||||
SharedLibraryPtr lib(new SharedLibrary(so_file_path));
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
libraries[hashed_key] = lib;
|
||||
}
|
||||
|
||||
|
@ -908,7 +908,7 @@ DDLGuard::DDLGuard(Map & map_, std::unique_lock<std::mutex> guards_lock_, const
|
||||
it = map.emplace(elem, Entry{std::make_unique<std::mutex>(), 0}).first;
|
||||
++it->second.counter;
|
||||
guards_lock.unlock();
|
||||
table_lock = std::unique_lock<std::mutex>(*it->second.mutex);
|
||||
table_lock = std::unique_lock(*it->second.mutex);
|
||||
}
|
||||
|
||||
DDLGuard::~DDLGuard()
|
||||
@ -924,7 +924,7 @@ DDLGuard::~DDLGuard()
|
||||
|
||||
std::unique_ptr<DDLGuard> Context::getDDLGuard(const String & database, const String & table) const
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(shared->ddl_guards_mutex);
|
||||
std::unique_lock lock(shared->ddl_guards_mutex);
|
||||
return std::make_unique<DDLGuard>(shared->ddl_guards[database], std::move(lock), table);
|
||||
}
|
||||
|
||||
@ -1177,7 +1177,7 @@ ExternalModels & Context::getExternalModels()
|
||||
|
||||
EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_error) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(shared->embedded_dictionaries_mutex);
|
||||
std::lock_guard lock(shared->embedded_dictionaries_mutex);
|
||||
|
||||
if (!shared->embedded_dictionaries)
|
||||
{
|
||||
@ -1195,7 +1195,7 @@ EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_
|
||||
|
||||
ExternalDictionaries & Context::getExternalDictionariesImpl(const bool throw_on_error) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(shared->external_dictionaries_mutex);
|
||||
std::lock_guard lock(shared->external_dictionaries_mutex);
|
||||
|
||||
if (!shared->external_dictionaries)
|
||||
{
|
||||
@ -1215,7 +1215,7 @@ ExternalDictionaries & Context::getExternalDictionariesImpl(const bool throw_on_
|
||||
|
||||
ExternalModels & Context::getExternalModelsImpl(bool throw_on_error) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(shared->external_models_mutex);
|
||||
std::lock_guard lock(shared->external_models_mutex);
|
||||
|
||||
if (!shared->external_models)
|
||||
{
|
||||
@ -1372,7 +1372,7 @@ DDLWorker & Context::getDDLWorker() const
|
||||
|
||||
zkutil::ZooKeeperPtr Context::getZooKeeper() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(shared->zookeeper_mutex);
|
||||
std::lock_guard lock(shared->zookeeper_mutex);
|
||||
|
||||
if (!shared->zookeeper)
|
||||
shared->zookeeper = std::make_shared<zkutil::ZooKeeper>(getConfigRef(), "zookeeper");
|
||||
@ -1465,7 +1465,7 @@ void Context::reloadClusterConfig()
|
||||
{
|
||||
ConfigurationPtr cluster_config;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(shared->clusters_mutex);
|
||||
std::lock_guard lock(shared->clusters_mutex);
|
||||
cluster_config = shared->clusters_config;
|
||||
}
|
||||
|
||||
@ -1473,7 +1473,7 @@ void Context::reloadClusterConfig()
|
||||
auto new_clusters = std::make_unique<Clusters>(config, settings);
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(shared->clusters_mutex);
|
||||
std::lock_guard lock(shared->clusters_mutex);
|
||||
if (shared->clusters_config.get() == cluster_config.get())
|
||||
{
|
||||
shared->clusters = std::move(new_clusters);
|
||||
@ -1488,7 +1488,7 @@ void Context::reloadClusterConfig()
|
||||
|
||||
Clusters & Context::getClusters() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(shared->clusters_mutex);
|
||||
std::lock_guard lock(shared->clusters_mutex);
|
||||
if (!shared->clusters)
|
||||
{
|
||||
auto & config = shared->clusters_config ? *shared->clusters_config : getConfigRef();
|
||||
@ -1502,7 +1502,7 @@ Clusters & Context::getClusters() const
|
||||
/// On repeating calls updates existing clusters and adds new clusters, doesn't delete old clusters
|
||||
void Context::setClustersConfig(const ConfigurationPtr & config, const String & config_name)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(shared->clusters_mutex);
|
||||
std::lock_guard lock(shared->clusters_mutex);
|
||||
|
||||
shared->clusters_config = config;
|
||||
|
||||
@ -1515,7 +1515,7 @@ void Context::setClustersConfig(const ConfigurationPtr & config, const String &
|
||||
|
||||
void Context::setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(shared->clusters_mutex);
|
||||
std::lock_guard lock(shared->clusters_mutex);
|
||||
|
||||
if (!shared->clusters)
|
||||
throw Exception("Clusters are not set", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -1846,7 +1846,7 @@ SessionCleaner::~SessionCleaner()
|
||||
try
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
std::lock_guard lock{mutex};
|
||||
quit = true;
|
||||
}
|
||||
|
||||
@ -1864,7 +1864,7 @@ void SessionCleaner::run()
|
||||
{
|
||||
setThreadName("HTTPSessionCleaner");
|
||||
|
||||
std::unique_lock<std::mutex> lock{mutex};
|
||||
std::unique_lock lock{mutex};
|
||||
|
||||
while (true)
|
||||
{
|
||||
|
@ -64,7 +64,7 @@ bool EmbeddedDictionaries::reloadDictionary(
|
||||
|
||||
bool EmbeddedDictionaries::reloadImpl(const bool throw_on_error, const bool force_reload)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
std::unique_lock lock(mutex);
|
||||
|
||||
/** If you can not update the directories, then despite this, do not throw an exception (use the old directories).
|
||||
* If there are no old correct directories, then when using functions that depend on them,
|
||||
|
@ -626,7 +626,7 @@ size_t CompiledExpressionCache::weight() const
|
||||
{
|
||||
|
||||
#if LLVM_VERSION_MAJOR >= 6
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
size_t result{0};
|
||||
std::unordered_set<size_t> seen;
|
||||
for (const auto & cell : cells)
|
||||
|
@ -92,7 +92,7 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error)
|
||||
/// list of recreated loadable objects to perform delayed removal from unordered_map
|
||||
std::list<std::string> recreated_failed_loadable_objects;
|
||||
|
||||
std::unique_lock<std::mutex> all_lock(all_mutex);
|
||||
std::unique_lock all_lock(all_mutex);
|
||||
|
||||
/// retry loading failed loadable objects
|
||||
for (auto & failed_loadable_object : failed_loadable_objects)
|
||||
@ -122,7 +122,7 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error)
|
||||
}
|
||||
else
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{map_mutex};
|
||||
const std::lock_guard lock{map_mutex};
|
||||
|
||||
const auto & lifetime = loadable_ptr->getLifetime();
|
||||
std::uniform_int_distribution<UInt64> distribution{lifetime.min_sec, lifetime.max_sec};
|
||||
@ -253,7 +253,7 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const
|
||||
}
|
||||
else
|
||||
{
|
||||
std::unique_lock<std::mutex> all_lock(all_mutex);
|
||||
std::unique_lock all_lock(all_mutex);
|
||||
|
||||
auto modification_time_it = last_modification_times.find(config_path);
|
||||
if (modification_time_it == std::end(last_modification_times))
|
||||
@ -305,7 +305,7 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const
|
||||
|
||||
decltype(loadable_objects.begin()) object_it;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{map_mutex};
|
||||
std::lock_guard lock{map_mutex};
|
||||
object_it = loadable_objects.find(name);
|
||||
}
|
||||
|
||||
@ -342,7 +342,7 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const
|
||||
}
|
||||
}
|
||||
|
||||
const std::lock_guard<std::mutex> lock{map_mutex};
|
||||
const std::lock_guard lock{map_mutex};
|
||||
|
||||
/// add new loadable object or update an existing version
|
||||
if (object_it == std::end(loadable_objects))
|
||||
@ -365,7 +365,7 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const
|
||||
/// If the loadable object could not load data or even failed to initialize from the config.
|
||||
/// - all the same we insert information into the `loadable_objects`, with the zero pointer `loadable`.
|
||||
|
||||
const std::lock_guard<std::mutex> lock{map_mutex};
|
||||
const std::lock_guard lock{map_mutex};
|
||||
|
||||
const auto exception_ptr = std::current_exception();
|
||||
const auto loadable_it = loadable_objects.find(name);
|
||||
@ -397,14 +397,14 @@ void ExternalLoader::reload(const std::string & name)
|
||||
reloadFromConfigFiles(true, true, name);
|
||||
|
||||
/// Check that specified object was loaded
|
||||
const std::lock_guard<std::mutex> lock{map_mutex};
|
||||
const std::lock_guard lock{map_mutex};
|
||||
if (!loadable_objects.count(name))
|
||||
throw Exception("Failed to load " + object_name + " '" + name + "' during the reload process", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
ExternalLoader::LoadablePtr ExternalLoader::getLoadableImpl(const std::string & name, bool throw_on_error) const
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{map_mutex};
|
||||
const std::lock_guard lock{map_mutex};
|
||||
|
||||
const auto it = loadable_objects.find(name);
|
||||
if (it == std::end(loadable_objects))
|
||||
|
@ -84,7 +84,7 @@ class InterserverIOHandler
|
||||
public:
|
||||
void addEndpoint(const String & name, InterserverIOEndpointPtr endpoint)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
bool inserted = endpoint_map.try_emplace(name, std::move(endpoint)).second;
|
||||
if (!inserted)
|
||||
throw Exception("Duplicate interserver IO endpoint: " + name, ErrorCodes::DUPLICATE_INTERSERVER_IO_ENDPOINT);
|
||||
@ -92,7 +92,7 @@ public:
|
||||
|
||||
void removeEndpoint(const String & name)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
if (!endpoint_map.erase(name))
|
||||
throw Exception("No interserver IO endpoint named " + name, ErrorCodes::NO_SUCH_INTERSERVER_IO_ENDPOINT);
|
||||
}
|
||||
@ -100,7 +100,7 @@ public:
|
||||
InterserverIOEndpointPtr getEndpoint(const String & name)
|
||||
try
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
return endpoint_map.at(name);
|
||||
}
|
||||
catch (...)
|
||||
|
@ -283,7 +283,7 @@ QueryStatus::~QueryStatus() = default;
|
||||
|
||||
void QueryStatus::setQueryStreams(const BlockIO & io)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(query_streams_mutex);
|
||||
std::lock_guard lock(query_streams_mutex);
|
||||
|
||||
query_stream_in = io.in;
|
||||
query_stream_out = io.out;
|
||||
@ -296,7 +296,7 @@ void QueryStatus::releaseQueryStreams()
|
||||
BlockOutputStreamPtr out;
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(query_streams_mutex);
|
||||
std::lock_guard lock(query_streams_mutex);
|
||||
|
||||
query_streams_status = QueryStreamsStatus::Released;
|
||||
in = std::move(query_stream_in);
|
||||
@ -308,14 +308,14 @@ void QueryStatus::releaseQueryStreams()
|
||||
|
||||
bool QueryStatus::streamsAreReleased()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(query_streams_mutex);
|
||||
std::lock_guard lock(query_streams_mutex);
|
||||
|
||||
return query_streams_status == QueryStreamsStatus::Released;
|
||||
}
|
||||
|
||||
bool QueryStatus::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(query_streams_mutex);
|
||||
std::lock_guard lock(query_streams_mutex);
|
||||
|
||||
if (query_streams_status != QueryStreamsStatus::Initialized)
|
||||
return false;
|
||||
@ -358,7 +358,7 @@ QueryStatus * ProcessList::tryGetProcessListElement(const String & current_query
|
||||
|
||||
ProcessList::CancellationCode ProcessList::sendCancelToQuery(const String & current_query_id, const String & current_user, bool kill)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
QueryStatus * elem = tryGetProcessListElement(current_query_id, current_user);
|
||||
|
||||
@ -431,7 +431,7 @@ ProcessList::Info ProcessList::getInfo(bool get_thread_list, bool get_profile_ev
|
||||
{
|
||||
Info per_query_infos;
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
per_query_infos.reserve(processes.size());
|
||||
for (const auto & process : processes)
|
||||
|
@ -308,7 +308,7 @@ public:
|
||||
|
||||
void setMaxSize(size_t max_size_)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
max_size = max_size_;
|
||||
}
|
||||
|
||||
|
@ -60,7 +60,7 @@ private:
|
||||
std::chrono::nanoseconds cur_timeout = timeout;
|
||||
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
|
||||
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
std::unique_lock lock(mutex);
|
||||
|
||||
while (true)
|
||||
{
|
||||
@ -109,7 +109,7 @@ public:
|
||||
~HandleImpl()
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parent.mutex);
|
||||
std::lock_guard lock(parent.mutex);
|
||||
--value.second;
|
||||
}
|
||||
parent.condvar.notify_all();
|
||||
@ -132,7 +132,7 @@ public:
|
||||
if (0 == priority)
|
||||
return {};
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = container.emplace(priority, 0).first;
|
||||
++it->second;
|
||||
return std::make_shared<HandleImpl>(*this, *it);
|
||||
|
@ -298,7 +298,7 @@ QuotaForIntervalsPtr Quota::get(const String & quota_key, const String & user_na
|
||||
? quota_key
|
||||
: user_name));
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
Container::iterator it = quota_for_keys.find(quota_key_hashed);
|
||||
if (quota_for_keys.end() == it)
|
||||
|
@ -52,7 +52,7 @@ void do_io(size_t id)
|
||||
TaskStatsInfoGetter get_info;
|
||||
|
||||
get_info.getStat(stat, tid);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
std::cerr << "#" << id << ", tid " << tid << ", intitial\n" << stat << "\n";
|
||||
|
||||
size_t copy_size = 1048576 * (1 + id);
|
||||
@ -67,7 +67,7 @@ void do_io(size_t id)
|
||||
|
||||
get_info.getStat(stat, tid);
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
std::cerr << "#" << id << ", tid " << tid << ", step1\n" << stat << "\n";
|
||||
}
|
||||
|
||||
@ -79,7 +79,7 @@ void do_io(size_t id)
|
||||
|
||||
get_info.getStat(stat, tid);
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
std::cerr << "#" << id << ", tid " << tid << ", step2\n" << stat << "\n";
|
||||
}
|
||||
|
||||
@ -91,7 +91,7 @@ void do_io(size_t id)
|
||||
|
||||
get_info.getStat(stat, tid);
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
std::cerr << "#" << id << ", tid " << tid << ", step3\n" << stat << "\n";
|
||||
}
|
||||
|
||||
|
@ -109,7 +109,7 @@ StorageDistributedDirectoryMonitor::~StorageDistributedDirectoryMonitor()
|
||||
{
|
||||
{
|
||||
quit = true;
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
std::lock_guard lock{mutex};
|
||||
}
|
||||
cond.notify_one();
|
||||
thread.join();
|
||||
@ -123,7 +123,7 @@ void StorageDistributedDirectoryMonitor::shutdownAndDropAllData()
|
||||
{
|
||||
{
|
||||
quit = true;
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
std::lock_guard lock{mutex};
|
||||
}
|
||||
cond.notify_one();
|
||||
thread.join();
|
||||
@ -137,7 +137,7 @@ void StorageDistributedDirectoryMonitor::run()
|
||||
{
|
||||
setThreadName("DistrDirMonitor");
|
||||
|
||||
std::unique_lock<std::mutex> lock{mutex};
|
||||
std::unique_lock lock{mutex};
|
||||
|
||||
const auto quit_requested = [this] { return quit.load(std::memory_order_relaxed); };
|
||||
|
||||
|
@ -427,7 +427,7 @@ StorageKafka::ConsumerPtr StorageKafka::tryClaimConsumer(long wait_ms)
|
||||
semaphore.wait();
|
||||
|
||||
// Take the first available consumer from the list
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
auto consumer = consumers.back();
|
||||
consumers.pop_back();
|
||||
return consumer;
|
||||
@ -435,7 +435,7 @@ StorageKafka::ConsumerPtr StorageKafka::tryClaimConsumer(long wait_ms)
|
||||
|
||||
void StorageKafka::pushConsumer(StorageKafka::ConsumerPtr c)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
consumers.push_back(c);
|
||||
semaphore.set();
|
||||
}
|
||||
|
@ -41,7 +41,7 @@ public:
|
||||
{
|
||||
try
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex);
|
||||
std::lock_guard lock(DiskSpaceMonitor::mutex);
|
||||
if (DiskSpaceMonitor::reserved_bytes < size)
|
||||
{
|
||||
DiskSpaceMonitor::reserved_bytes = 0;
|
||||
@ -70,7 +70,7 @@ public:
|
||||
/// Change amount of reserved space. When new_size is greater than before, availability of free space is not checked.
|
||||
void update(UInt64 new_size)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex);
|
||||
std::lock_guard lock(DiskSpaceMonitor::mutex);
|
||||
DiskSpaceMonitor::reserved_bytes -= size;
|
||||
size = new_size;
|
||||
DiskSpaceMonitor::reserved_bytes += size;
|
||||
@ -84,7 +84,7 @@ public:
|
||||
Reservation(UInt64 size_)
|
||||
: size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex);
|
||||
std::lock_guard lock(DiskSpaceMonitor::mutex);
|
||||
DiskSpaceMonitor::reserved_bytes += size;
|
||||
++DiskSpaceMonitor::reservation_count;
|
||||
}
|
||||
@ -108,7 +108,7 @@ public:
|
||||
/// Heuristic by Michael Kolupaev: reserve 30 MB more, because statvfs shows few megabytes more space than df.
|
||||
res -= std::min(res, static_cast<UInt64>(30 * (1ul << 20)));
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (reserved_bytes > res)
|
||||
res = 0;
|
||||
@ -120,13 +120,13 @@ public:
|
||||
|
||||
static UInt64 getReservedSpace()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
return reserved_bytes;
|
||||
}
|
||||
|
||||
static UInt64 getReservationCount()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
return reservation_count;
|
||||
}
|
||||
|
||||
|
@ -124,13 +124,13 @@ public:
|
||||
template <typename... Args>
|
||||
EntryPtr insert(Args &&... args)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
std::lock_guard lock{mutex};
|
||||
return std::make_unique<Entry>(*this, merges.emplace(merges.end(), std::forward<Args>(args)...));
|
||||
}
|
||||
|
||||
info_container_t get() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
std::lock_guard lock{mutex};
|
||||
info_container_t res;
|
||||
for (const auto & merge_element : merges)
|
||||
res.emplace_back(merge_element.getInfo());
|
||||
@ -141,7 +141,7 @@ public:
|
||||
|
||||
inline MergeListEntry::~MergeListEntry()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{list.mutex};
|
||||
std::lock_guard lock{list.mutex};
|
||||
list.merges.erase(it);
|
||||
}
|
||||
|
||||
|
@ -558,7 +558,7 @@ String MergeTreeData::MergingParams::getModeName() const
|
||||
|
||||
Int64 MergeTreeData::getMaxBlockNumber()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock_all(data_parts_mutex);
|
||||
std::lock_guard lock_all(data_parts_mutex);
|
||||
|
||||
Int64 max_block_num = 0;
|
||||
for (const DataPartPtr & part : data_parts_by_info)
|
||||
@ -587,7 +587,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
|
||||
DataPartsVector broken_parts_to_detach;
|
||||
size_t suspicious_broken_parts = 0;
|
||||
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
std::lock_guard lock(data_parts_mutex);
|
||||
data_parts_indexes.clear();
|
||||
|
||||
for (const String & file_name : part_file_names)
|
||||
@ -766,7 +766,7 @@ static bool isOldPartDirectory(Poco::File & directory, time_t threshold)
|
||||
void MergeTreeData::clearOldTemporaryDirectories(ssize_t custom_directories_lifetime_seconds)
|
||||
{
|
||||
/// If the method is already called from another thread, then we don't need to do anything.
|
||||
std::unique_lock<std::mutex> lock(clear_old_temporary_directories_mutex, std::defer_lock);
|
||||
std::unique_lock lock(clear_old_temporary_directories_mutex, std::defer_lock);
|
||||
if (!lock.try_lock())
|
||||
return;
|
||||
|
||||
@ -805,7 +805,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts()
|
||||
DataPartsVector res;
|
||||
|
||||
/// If the method is already called from another thread, then we don't need to do anything.
|
||||
std::unique_lock<std::mutex> lock(grab_old_parts_mutex, std::defer_lock);
|
||||
std::unique_lock lock(grab_old_parts_mutex, std::defer_lock);
|
||||
if (!lock.try_lock())
|
||||
return res;
|
||||
|
||||
@ -813,7 +813,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts()
|
||||
std::vector<DataPartIteratorByStateAndInfo> parts_to_delete;
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock_parts(data_parts_mutex);
|
||||
std::lock_guard lock_parts(data_parts_mutex);
|
||||
|
||||
auto outdated_parts_range = getDataPartsStateRange(DataPartState::Outdated);
|
||||
for (auto it = outdated_parts_range.begin(); it != outdated_parts_range.end(); ++it)
|
||||
@ -847,7 +847,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts()
|
||||
|
||||
void MergeTreeData::rollbackDeletingParts(const MergeTreeData::DataPartsVector & parts)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
std::lock_guard lock(data_parts_mutex);
|
||||
for (auto & part : parts)
|
||||
{
|
||||
/// We should modify it under data_parts_mutex
|
||||
@ -859,7 +859,7 @@ void MergeTreeData::rollbackDeletingParts(const MergeTreeData::DataPartsVector &
|
||||
void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & parts)
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
std::lock_guard lock(data_parts_mutex);
|
||||
|
||||
/// TODO: use data_parts iterators instead of pointers
|
||||
for (auto & part : parts)
|
||||
@ -926,7 +926,7 @@ void MergeTreeData::dropAllData()
|
||||
{
|
||||
LOG_TRACE(log, "dropAllData: waiting for locks.");
|
||||
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
std::lock_guard lock(data_parts_mutex);
|
||||
|
||||
LOG_TRACE(log, "dropAllData: removing data from memory.");
|
||||
|
||||
@ -1630,7 +1630,7 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace(
|
||||
|
||||
DataPartsVector covered_parts;
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(data_parts_mutex);
|
||||
std::unique_lock lock(data_parts_mutex);
|
||||
renameTempPartAndReplace(part, increment, out_transaction, lock, &covered_parts);
|
||||
}
|
||||
return covered_parts;
|
||||
@ -1844,7 +1844,7 @@ void MergeTreeData::tryRemovePartImmediately(DataPartPtr && part)
|
||||
{
|
||||
DataPartPtr part_to_delete;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock_parts(data_parts_mutex);
|
||||
std::lock_guard lock_parts(data_parts_mutex);
|
||||
|
||||
LOG_TRACE(log, "Trying to immediately remove part " << part->getNameWithState());
|
||||
|
||||
@ -1880,7 +1880,7 @@ size_t MergeTreeData::getTotalActiveSizeInBytes() const
|
||||
{
|
||||
size_t res = 0;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
std::lock_guard lock(data_parts_mutex);
|
||||
|
||||
for (auto & part : getDataPartsStateRange(DataPartState::Committed))
|
||||
res += part->bytes_on_disk;
|
||||
@ -1892,7 +1892,7 @@ size_t MergeTreeData::getTotalActiveSizeInBytes() const
|
||||
|
||||
size_t MergeTreeData::getMaxPartsCountForPartition() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
std::lock_guard lock(data_parts_mutex);
|
||||
|
||||
size_t res = 0;
|
||||
size_t cur_count = 0;
|
||||
@ -2016,7 +2016,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorInPartition(Merg
|
||||
{
|
||||
DataPartStateAndPartitionID state_with_partition{state, partition_id};
|
||||
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
std::lock_guard lock(data_parts_mutex);
|
||||
return DataPartsVector(
|
||||
data_parts_by_state_and_info.lower_bound(state_with_partition),
|
||||
data_parts_by_state_and_info.upper_bound(state_with_partition));
|
||||
@ -2025,7 +2025,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorInPartition(Merg
|
||||
|
||||
MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const MergeTreePartInfo & part_info, const MergeTreeData::DataPartStates & valid_states)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
std::lock_guard lock(data_parts_mutex);
|
||||
|
||||
auto it = data_parts_by_info.find(part_info);
|
||||
if (it == data_parts_by_info.end())
|
||||
@ -2266,7 +2266,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector(const DataPartS
|
||||
DataPartsVector res;
|
||||
DataPartsVector buf;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
std::lock_guard lock(data_parts_mutex);
|
||||
|
||||
for (auto state : affordable_states)
|
||||
{
|
||||
@ -2292,7 +2292,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getAllDataPartsVector(MergeTreeDat
|
||||
{
|
||||
DataPartsVector res;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
std::lock_guard lock(data_parts_mutex);
|
||||
res.assign(data_parts_by_info.begin(), data_parts_by_info.end());
|
||||
|
||||
if (out_states != nullptr)
|
||||
@ -2310,7 +2310,7 @@ MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affo
|
||||
{
|
||||
DataParts res;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(data_parts_mutex);
|
||||
std::lock_guard lock(data_parts_mutex);
|
||||
for (auto state : affordable_states)
|
||||
{
|
||||
auto range = getDataPartsStateRange(state);
|
||||
|
@ -531,7 +531,7 @@ public:
|
||||
|
||||
size_t getColumnCompressedSize(const std::string & name) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{data_parts_mutex};
|
||||
std::lock_guard lock{data_parts_mutex};
|
||||
|
||||
const auto it = column_sizes.find(name);
|
||||
return it == std::end(column_sizes) ? 0 : it->second.data_compressed;
|
||||
@ -540,14 +540,14 @@ public:
|
||||
using ColumnSizeByName = std::unordered_map<std::string, DataPart::ColumnSize>;
|
||||
ColumnSizeByName getColumnSizes() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{data_parts_mutex};
|
||||
std::lock_guard lock{data_parts_mutex};
|
||||
return column_sizes;
|
||||
}
|
||||
|
||||
/// Calculates column sizes in compressed form for the current state of data_parts.
|
||||
void recalculateColumnSizes()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{data_parts_mutex};
|
||||
std::lock_guard lock{data_parts_mutex};
|
||||
calculateColumnSizesImpl();
|
||||
}
|
||||
|
||||
|
@ -41,7 +41,7 @@ MergeTreeReadPool::MergeTreeReadPool(
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, const size_t thread, const Names & ordered_names)
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{mutex};
|
||||
const std::lock_guard lock{mutex};
|
||||
|
||||
/// If number of threads was lowered due to backoff, then will assign work only for maximum 'backoff_state.current_threads' threads.
|
||||
if (thread >= backoff_state.current_threads)
|
||||
@ -164,7 +164,7 @@ void MergeTreeReadPool::profileFeedback(const ReadBufferFromFileBase::ProfileInf
|
||||
if (info.nanoseconds < backoff_settings.min_read_latency_ms * 1000000)
|
||||
return;
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (backoff_state.current_threads <= 1)
|
||||
return;
|
||||
|
@ -38,7 +38,7 @@ ReplicatedMergeTreePartCheckThread::~ReplicatedMergeTreePartCheckThread()
|
||||
|
||||
void ReplicatedMergeTreePartCheckThread::start()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(start_stop_mutex);
|
||||
std::lock_guard lock(start_stop_mutex);
|
||||
need_stop = false;
|
||||
task->activateAndSchedule();
|
||||
}
|
||||
@ -48,14 +48,14 @@ void ReplicatedMergeTreePartCheckThread::stop()
|
||||
//based on discussion on https://github.com/yandex/ClickHouse/pull/1489#issuecomment-344756259
|
||||
//using the schedule pool there is no problem in case stop is called two time in row and the start multiple times
|
||||
|
||||
std::lock_guard<std::mutex> lock(start_stop_mutex);
|
||||
std::lock_guard lock(start_stop_mutex);
|
||||
need_stop = true;
|
||||
task->deactivate();
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreePartCheckThread::enqueuePart(const String & name, time_t delay_to_check_seconds)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parts_mutex);
|
||||
std::lock_guard lock(parts_mutex);
|
||||
|
||||
if (parts_set.count(name))
|
||||
return;
|
||||
@ -68,7 +68,7 @@ void ReplicatedMergeTreePartCheckThread::enqueuePart(const String & name, time_t
|
||||
|
||||
size_t ReplicatedMergeTreePartCheckThread::size() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parts_mutex);
|
||||
std::lock_guard lock(parts_mutex);
|
||||
return parts_set.size();
|
||||
}
|
||||
|
||||
@ -295,7 +295,7 @@ void ReplicatedMergeTreePartCheckThread::run()
|
||||
time_t min_check_time = std::numeric_limits<time_t>::max();
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parts_mutex);
|
||||
std::lock_guard lock(parts_mutex);
|
||||
|
||||
if (parts_queue.empty())
|
||||
{
|
||||
@ -331,7 +331,7 @@ void ReplicatedMergeTreePartCheckThread::run()
|
||||
|
||||
/// Remove the part from check queue.
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parts_mutex);
|
||||
std::lock_guard lock(parts_mutex);
|
||||
|
||||
if (parts_queue.empty())
|
||||
{
|
||||
|
@ -294,7 +294,7 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep
|
||||
size_t queue_size = 0;
|
||||
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(state_mutex);
|
||||
std::unique_lock lock(state_mutex);
|
||||
|
||||
/// Remove the job from the queue in the RAM.
|
||||
/// You can not just refer to a pre-saved iterator, because someone else might be able to delete the task.
|
||||
@ -335,7 +335,7 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri
|
||||
std::optional<time_t> max_processed_insert_time_changed;
|
||||
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(state_mutex);
|
||||
std::unique_lock lock(state_mutex);
|
||||
|
||||
virtual_parts.remove(part_name);
|
||||
|
||||
@ -370,7 +370,7 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri
|
||||
|
||||
bool ReplicatedMergeTreeQueue::removeFromVirtualParts(const MergeTreePartInfo & part_info)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(state_mutex);
|
||||
std::unique_lock lock(state_mutex);
|
||||
return virtual_parts.remove(part_info);
|
||||
}
|
||||
|
||||
@ -711,7 +711,7 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange(zkutil::ZooKeeperPt
|
||||
std::optional<time_t> max_processed_insert_time_changed;
|
||||
|
||||
/// Remove operations with parts, contained in the range to be deleted, from the queue.
|
||||
std::unique_lock<std::mutex> lock(state_mutex);
|
||||
std::unique_lock lock(state_mutex);
|
||||
for (Queue::iterator it = queue.begin(); it != queue.end();)
|
||||
{
|
||||
auto type = (*it)->type;
|
||||
@ -785,7 +785,7 @@ size_t ReplicatedMergeTreeQueue::getConflictsCountForRange(
|
||||
void ReplicatedMergeTreeQueue::checkThereAreNoConflictsInRange(const MergeTreePartInfo & range, const LogEntry & entry)
|
||||
{
|
||||
String conflicts_description;
|
||||
std::lock_guard<std::mutex> lock(state_mutex);
|
||||
std::lock_guard lock(state_mutex);
|
||||
|
||||
if (0 != getConflictsCountForRange(range, entry, &conflicts_description, lock))
|
||||
throw Exception(conflicts_description, ErrorCodes::UNFINISHED);
|
||||
@ -1013,7 +1013,7 @@ ReplicatedMergeTreeQueue::SelectedEntry ReplicatedMergeTreeQueue::selectEntryToP
|
||||
{
|
||||
LogEntryPtr entry;
|
||||
|
||||
std::lock_guard<std::mutex> lock(state_mutex);
|
||||
std::lock_guard lock(state_mutex);
|
||||
|
||||
for (auto it = queue.begin(); it != queue.end(); ++it)
|
||||
{
|
||||
@ -1635,8 +1635,8 @@ bool ReplicatedMergeTreeMergePredicate::isMutationFinished(const ReplicatedMerge
|
||||
ReplicatedMergeTreeQueue::SubscriberHandler
|
||||
ReplicatedMergeTreeQueue::addSubscriber(ReplicatedMergeTreeQueue::SubscriberCallBack && callback)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(state_mutex);
|
||||
std::lock_guard<std::mutex> lock_subscribers(subscribers_mutex);
|
||||
std::lock_guard lock(state_mutex);
|
||||
std::lock_guard lock_subscribers(subscribers_mutex);
|
||||
|
||||
auto it = subscribers.emplace(subscribers.end(), std::move(callback));
|
||||
|
||||
@ -1648,13 +1648,13 @@ ReplicatedMergeTreeQueue::addSubscriber(ReplicatedMergeTreeQueue::SubscriberCall
|
||||
|
||||
ReplicatedMergeTreeQueue::SubscriberHandler::~SubscriberHandler()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(queue.subscribers_mutex);
|
||||
std::lock_guard lock(queue.subscribers_mutex);
|
||||
queue.subscribers.erase(it);
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeQueue::notifySubscribers(size_t new_queue_size)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock_subscribers(subscribers_mutex);
|
||||
std::lock_guard lock_subscribers(subscribers_mutex);
|
||||
for (auto & subscriber_callback : subscribers)
|
||||
subscriber_callback(new_queue_size);
|
||||
}
|
||||
|
@ -99,7 +99,7 @@ protected:
|
||||
return res;
|
||||
has_been_read = true;
|
||||
|
||||
std::lock_guard<std::mutex> lock(buffer.mutex);
|
||||
std::lock_guard lock(buffer.mutex);
|
||||
|
||||
if (!buffer.data.rows())
|
||||
return res;
|
||||
@ -336,7 +336,7 @@ public:
|
||||
|
||||
for (size_t try_no = 0; try_no < storage.num_shards; ++try_no)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(storage.buffers[shard_num].mutex, std::try_to_lock);
|
||||
std::unique_lock lock(storage.buffers[shard_num].mutex, std::try_to_lock);
|
||||
|
||||
if (lock.owns_lock())
|
||||
{
|
||||
@ -356,7 +356,7 @@ public:
|
||||
if (!least_busy_buffer)
|
||||
{
|
||||
least_busy_buffer = &storage.buffers[start_shard_num];
|
||||
least_busy_lock = std::unique_lock<std::mutex>(least_busy_buffer->mutex);
|
||||
least_busy_lock = std::unique_lock(least_busy_buffer->mutex);
|
||||
}
|
||||
insertIntoBuffer(block, *least_busy_buffer);
|
||||
}
|
||||
@ -527,7 +527,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc
|
||||
size_t bytes = 0;
|
||||
time_t time_passed = 0;
|
||||
|
||||
std::unique_lock<std::mutex> lock(buffer.mutex, std::defer_lock);
|
||||
std::unique_lock lock(buffer.mutex, std::defer_lock);
|
||||
if (!locked)
|
||||
lock.lock();
|
||||
|
||||
|
@ -66,7 +66,7 @@ public:
|
||||
void write(const Block & block) override
|
||||
{
|
||||
storage.check(block, true);
|
||||
std::lock_guard<std::mutex> lock(storage.mutex);
|
||||
std::lock_guard lock(storage.mutex);
|
||||
storage.data.push_back(block);
|
||||
}
|
||||
private:
|
||||
@ -90,7 +90,7 @@ BlockInputStreams StorageMemory::read(
|
||||
{
|
||||
check(column_names);
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
size_t size = data.size();
|
||||
|
||||
@ -123,13 +123,13 @@ BlockOutputStreamPtr StorageMemory::write(
|
||||
|
||||
void StorageMemory::drop()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
data.clear();
|
||||
}
|
||||
|
||||
void StorageMemory::truncate(const ASTPtr &, const Context &)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
data.clear();
|
||||
}
|
||||
|
||||
|
@ -277,7 +277,7 @@ struct CurrentlyMergingPartsTagger
|
||||
|
||||
~CurrentlyMergingPartsTagger()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(storage->currently_merging_mutex);
|
||||
std::lock_guard lock(storage->currently_merging_mutex);
|
||||
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
@ -386,7 +386,7 @@ bool StorageMergeTree::merge(
|
||||
std::optional<CurrentlyMergingPartsTagger> merging_tagger;
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(currently_merging_mutex);
|
||||
std::lock_guard lock(currently_merging_mutex);
|
||||
|
||||
auto can_merge = [this, &lock] (const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, String *)
|
||||
{
|
||||
@ -492,7 +492,7 @@ bool StorageMergeTree::tryMutatePart()
|
||||
{
|
||||
auto disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
|
||||
|
||||
std::lock_guard<std::mutex> lock(currently_merging_mutex);
|
||||
std::lock_guard lock(currently_merging_mutex);
|
||||
|
||||
if (current_mutations_by_version.empty())
|
||||
return false;
|
||||
|
@ -174,13 +174,13 @@ thread_local
|
||||
|
||||
void StorageReplicatedMergeTree::setZooKeeper(zkutil::ZooKeeperPtr zookeeper)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(current_zookeeper_mutex);
|
||||
std::lock_guard lock(current_zookeeper_mutex);
|
||||
current_zookeeper = zookeeper;
|
||||
}
|
||||
|
||||
zkutil::ZooKeeperPtr StorageReplicatedMergeTree::tryGetZooKeeper()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(current_zookeeper_mutex);
|
||||
std::lock_guard lock(current_zookeeper_mutex);
|
||||
return current_zookeeper;
|
||||
}
|
||||
|
||||
@ -2144,7 +2144,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask()
|
||||
{
|
||||
/// We must select parts for merge under merge_selecting_mutex because other threads
|
||||
/// (OPTIMIZE queries) can assign new merges.
|
||||
std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
|
||||
std::lock_guard merge_selecting_lock(merge_selecting_mutex);
|
||||
|
||||
auto zookeeper = getZooKeeper();
|
||||
|
||||
@ -2644,7 +2644,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
|
||||
}
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(currently_fetching_parts_mutex);
|
||||
std::lock_guard lock(currently_fetching_parts_mutex);
|
||||
if (!currently_fetching_parts.insert(part_name).second)
|
||||
{
|
||||
LOG_DEBUG(log, "Part " << part_name << " is already fetching right now");
|
||||
@ -2654,7 +2654,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
|
||||
|
||||
SCOPE_EXIT
|
||||
({
|
||||
std::lock_guard<std::mutex> lock(currently_fetching_parts_mutex);
|
||||
std::lock_guard lock(currently_fetching_parts_mutex);
|
||||
currently_fetching_parts.erase(part_name);
|
||||
});
|
||||
|
||||
@ -2948,7 +2948,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
|
||||
{
|
||||
/// We must select parts for merge under merge_selecting_mutex because other threads
|
||||
/// (merge_selecting_thread or OPTIMIZE queries) could assign new merges.
|
||||
std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
|
||||
std::lock_guard merge_selecting_lock(merge_selecting_mutex);
|
||||
|
||||
auto zookeeper = getZooKeeper();
|
||||
ReplicatedMergeTreeMergePredicate can_merge = queue.getMergePredicate(zookeeper);
|
||||
@ -3620,7 +3620,7 @@ void StorageReplicatedMergeTree::rename(const String & new_path_to_db, const Str
|
||||
bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path)
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(existing_nodes_cache_mutex);
|
||||
std::lock_guard lock(existing_nodes_cache_mutex);
|
||||
if (existing_nodes_cache.count(path))
|
||||
return true;
|
||||
}
|
||||
@ -3629,7 +3629,7 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path)
|
||||
|
||||
if (res)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(existing_nodes_cache_mutex);
|
||||
std::lock_guard lock(existing_nodes_cache_mutex);
|
||||
existing_nodes_cache.insert(path);
|
||||
}
|
||||
|
||||
@ -4666,7 +4666,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
|
||||
/// It does not provides strong guarantees, but is suitable for intended use case (assume merges are quite rare).
|
||||
|
||||
{
|
||||
std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
|
||||
std::lock_guard merge_selecting_lock(merge_selecting_mutex);
|
||||
queue.disableMergesInRange(drop_range_fake_part_name);
|
||||
}
|
||||
}
|
||||
@ -4913,7 +4913,7 @@ bool StorageReplicatedMergeTree::dropPartsInPartition(
|
||||
*/
|
||||
String drop_range_fake_part_name = getPartNamePossiblyFake(data.format_version, drop_range_info);
|
||||
{
|
||||
std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
|
||||
std::lock_guard merge_selecting_lock(merge_selecting_mutex);
|
||||
queue.disableMergesInRange(drop_range_fake_part_name);
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user