Make thread analyzer happy

This commit is contained in:
Amos Bird 2023-02-28 22:48:14 +08:00
parent 2d8060d843
commit c0fb4ddc3e
No known key found for this signature in database
GPG Key ID: 80D430DCBECFEDB4
3 changed files with 22 additions and 18 deletions

View File

@ -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,

View File

@ -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.

View File

@ -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;
}