mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-14 19:45:11 +00:00
Make thread analyzer happy
This commit is contained in:
parent
2d8060d843
commit
c0fb4ddc3e
@ -54,6 +54,7 @@
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/QueryPlan/QueryIdHolder.h>
|
||||
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/Freeze.h>
|
||||
@ -7763,6 +7764,25 @@ void MergeTreeData::removeQueryIdNoLock(const String & query_id) const
|
||||
query_id_set.erase(query_id);
|
||||
}
|
||||
|
||||
std::shared_ptr<QueryIdHolder> MergeTreeData::getQueryIdHolder(const String & query_id, UInt64 max_concurrent_queries) const
|
||||
{
|
||||
auto lock = std::lock_guard<std::mutex>(query_id_set_mutex);
|
||||
if (insertQueryIdOrThrowNoLock(query_id, max_concurrent_queries))
|
||||
{
|
||||
try
|
||||
{
|
||||
return std::make_shared<QueryIdHolder>(query_id, *this);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// If we fail to construct the holder, remove query_id explicitly to avoid leak.
|
||||
removeQueryIdNoLock(query_id);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
ReservationPtr MergeTreeData::balancedReservation(
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
size_t part_size,
|
||||
|
@ -958,7 +958,7 @@ public:
|
||||
/// section from config.xml.
|
||||
CompressionCodecPtr getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const;
|
||||
|
||||
std::lock_guard<std::mutex> getQueryIdSetLock() const { return std::lock_guard<std::mutex>(query_id_set_mutex); }
|
||||
std::shared_ptr<QueryIdHolder> getQueryIdHolder(const String & query_id, UInt64 max_concurrent_queries) const;
|
||||
|
||||
/// Record current query id where querying the table. Throw if there are already `max_queries` queries accessing the same table.
|
||||
/// Returns false if the `query_id` already exists in the running set, otherwise return true.
|
||||
|
@ -1198,7 +1198,6 @@ std::shared_ptr<QueryIdHolder> MergeTreeDataSelectExecutor::checkLimits(
|
||||
const MergeTreeData & data,
|
||||
const ReadFromMergeTree::AnalysisResult & result,
|
||||
const ContextPtr & context)
|
||||
TSA_NO_THREAD_SAFETY_ANALYSIS // disabled because TSA is confused by guaranteed copy elision in data.getQueryIdSetLock()
|
||||
{
|
||||
const auto & settings = context->getSettingsRef();
|
||||
const auto data_settings = data.getSettings();
|
||||
@ -1222,22 +1221,7 @@ std::shared_ptr<QueryIdHolder> MergeTreeDataSelectExecutor::checkLimits(
|
||||
{
|
||||
auto query_id = context->getCurrentQueryId();
|
||||
if (!query_id.empty())
|
||||
{
|
||||
auto lock = data.getQueryIdSetLock();
|
||||
if (data.insertQueryIdOrThrowNoLock(query_id, data_settings->max_concurrent_queries))
|
||||
{
|
||||
try
|
||||
{
|
||||
return std::make_shared<QueryIdHolder>(query_id, data);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// If we fail to construct the holder, remove query_id explicitly to avoid leak.
|
||||
data.removeQueryIdNoLock(query_id);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
return data.getQueryIdHolder(query_id, data_settings->max_concurrent_queries);
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user