Safer SCOPE_EXIT

It executes the code in the dtor, that should never throw.
This commit is contained in:
Azat Khuzhin 2021-04-04 12:23:40 +03:00
parent 9394e6e5ed
commit f157278b72
14 changed files with 44 additions and 40 deletions

View File

@ -21,7 +21,7 @@
#include <unordered_set>
#include <algorithm>
#include <optional>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
#include <boost/program_options.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <Poco/String.h>
@ -1610,7 +1610,7 @@ private:
{
/// Temporarily apply query settings to context.
std::optional<Settings> old_settings;
SCOPE_EXIT({ if (old_settings) context.setSettings(*old_settings); });
SCOPE_EXIT_SAFE({ if (old_settings) context.setSettings(*old_settings); });
auto apply_query_settings = [&](const IAST & settings_ast)
{
if (!old_settings)

View File

@ -3,6 +3,7 @@
#include <Common/TerminalSize.h>
#include <IO/ConnectionTimeoutsContext.h>
#include <Formats/registerFormats.h>
#include <ext/scope_guard_safe.h>
#include <unistd.h>
@ -112,7 +113,7 @@ void ClusterCopierApp::mainImpl()
SharedContextHolder shared_context = Context::createShared();
auto context = std::make_unique<Context>(Context::createGlobal(shared_context.get()));
context->makeGlobalContext();
SCOPE_EXIT(context->shutdown());
SCOPE_EXIT_SAFE(context->shutdown());
context->setConfig(loaded_config.configuration);
context->setApplicationType(Context::ApplicationType::LOCAL);

View File

@ -10,7 +10,7 @@
#include <Storages/IStorage.h>
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
#include <iomanip>
#include <Poco/File.h>
@ -61,7 +61,7 @@ void DatabaseLazy::createTable(
const StoragePtr & table,
const ASTPtr & query)
{
SCOPE_EXIT({ clearExpiredTables(); });
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
if (!endsWith(table->getName(), "Log"))
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
DatabaseOnDisk::createTable(context, table_name, table, query);
@ -78,7 +78,7 @@ void DatabaseLazy::dropTable(
const String & table_name,
bool no_delay)
{
SCOPE_EXIT({ clearExpiredTables(); });
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
DatabaseOnDisk::dropTable(context, table_name, no_delay);
}
@ -90,7 +90,7 @@ void DatabaseLazy::renameTable(
bool exchange,
bool dictionary)
{
SCOPE_EXIT({ clearExpiredTables(); });
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary);
}
@ -115,14 +115,14 @@ void DatabaseLazy::alterTable(
bool DatabaseLazy::isTableExist(const String & table_name) const
{
SCOPE_EXIT({ clearExpiredTables(); });
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
std::lock_guard lock(mutex);
return tables_cache.find(table_name) != tables_cache.end();
}
StoragePtr DatabaseLazy::tryGetTable(const String & table_name) const
{
SCOPE_EXIT({ clearExpiredTables(); });
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
{
std::lock_guard lock(mutex);
auto it = tables_cache.find(table_name);
@ -224,7 +224,7 @@ DatabaseLazy::~DatabaseLazy()
StoragePtr DatabaseLazy::loadTable(const String & table_name) const
{
SCOPE_EXIT({ clearExpiredTables(); });
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
LOG_DEBUG(log, "Load table {} to cache.", backQuote(table_name));

View File

@ -11,7 +11,7 @@
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
#include <Common/Arena.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
namespace DB
@ -172,7 +172,7 @@ ColumnPtr FunctionArrayReduce::executeImpl(const ColumnsWithTypeAndName & argume
}
}
SCOPE_EXIT({
SCOPE_EXIT_MEMORY_SAFE({
for (size_t i = 0; i < input_rows_count; ++i)
agg_func.destroy(places[i]);
});

View File

@ -13,7 +13,7 @@
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
#include <Common/Arena.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
namespace DB
@ -252,7 +252,7 @@ ColumnPtr FunctionArrayReduceInRanges::executeImpl(const ColumnsWithTypeAndName
}
}
SCOPE_EXIT({
SCOPE_EXIT_MEMORY_SAFE({
for (size_t j = 0; j < place_total; ++j)
agg_func.destroy(places[j]);
});
@ -331,7 +331,7 @@ ColumnPtr FunctionArrayReduceInRanges::executeImpl(const ColumnsWithTypeAndName
AggregateDataPtr place = arena->alignedAlloc(agg_func.sizeOfData(), agg_func.alignOfData());
agg_func.create(place);
SCOPE_EXIT({
SCOPE_EXIT_MEMORY_SAFE({
agg_func.destroy(place);
});

View File

@ -9,7 +9,7 @@
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
#include <Common/Arena.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
namespace DB
@ -132,7 +132,7 @@ ColumnPtr FunctionInitializeAggregation::executeImpl(const ColumnsWithTypeAndNam
}
}
SCOPE_EXIT({
SCOPE_EXIT_MEMORY_SAFE({
for (size_t i = 0; i < input_rows_count; ++i)
agg_func.destroy(places[i]);
});

View File

@ -5,7 +5,7 @@
#include <DataTypes/DataTypeAggregateFunction.h>
#include <Common/AlignedBuffer.h>
#include <Common/Arena.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
namespace DB
@ -104,7 +104,7 @@ public:
const auto & states = column_with_states->getData();
bool state_created = false;
SCOPE_EXIT({
SCOPE_EXIT_MEMORY_SAFE({
if (state_created)
agg_func.destroy(place.data());
});

View File

@ -29,6 +29,7 @@
#include <common/logger_useful.h>
#include <random>
#include <pcg_random.hpp>
#include <ext/scope_guard_safe.h>
namespace fs = std::filesystem;
@ -820,7 +821,7 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica(
zookeeper->set(tries_to_execute_path, toString(counter + 1));
task.ops.push_back(create_shard_flag);
SCOPE_EXIT({ if (!executed_by_us && !task.ops.empty()) task.ops.pop_back(); });
SCOPE_EXIT_MEMORY({ if (!executed_by_us && !task.ops.empty()) task.ops.pop_back(); });
/// If the leader will unexpectedly changed this method will return false
/// and on the next iteration new leader will take lock

View File

@ -82,7 +82,7 @@
#include <Common/typeid_cast.h>
#include <Common/checkStackSize.h>
#include <ext/map.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
#include <memory>
@ -1401,7 +1401,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
AggregateDataPtr place = state.data();
agg_count.create(place);
SCOPE_EXIT(agg_count.destroy(place));
SCOPE_EXIT_MEMORY_SAFE(agg_count.destroy(place));
agg_count.set(place, *num_rows);

View File

@ -1,14 +1,15 @@
#include <Processors/Executors/PipelineExecutor.h>
#include <queue>
#include <IO/WriteBufferFromString.h>
#include <Processors/printPipeline.h>
#include <Common/EventCounter.h>
#include <ext/scope_guard.h>
#include <Common/CurrentThread.h>
#include <Processors/ISource.h>
#include <Common/setThreadName.h>
#include <Common/MemoryTracker.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Processors/printPipeline.h>
#include <Processors/ISource.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/OpenTelemetrySpanLog.h>
#include <ext/scope_guard_safe.h>
#ifndef NDEBUG
#include <Common/Stopwatch.h>
@ -740,7 +741,7 @@ void PipelineExecutor::executeImpl(size_t num_threads)
bool finished_flag = false;
SCOPE_EXIT(
SCOPE_EXIT_SAFE(
if (!finished_flag)
{
finish();
@ -766,9 +767,9 @@ void PipelineExecutor::executeImpl(size_t num_threads)
if (thread_group)
CurrentThread::attachTo(thread_group);
SCOPE_EXIT(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);
try

View File

@ -5,7 +5,7 @@
#include <Processors/QueryPipeline.h>
#include <Common/setThreadName.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
namespace DB
{
@ -72,7 +72,7 @@ static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGrou
if (thread_group)
CurrentThread::attachTo(thread_group);
SCOPE_EXIT(
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);

View File

@ -2,14 +2,14 @@
#include <IO/ReadHelpers.h>
#include <Common/CurrentThread.h>
#include <Common/setThreadName.h>
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
namespace DB
{
void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr thread_group)
{
SCOPE_EXIT(
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);
@ -60,7 +60,7 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr
void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupStatusPtr thread_group, size_t current_ticket_number)
{
SCOPE_EXIT(
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);

View File

@ -1,5 +1,5 @@
#include <boost/rational.hpp> /// For calculations related to sampling coefficients.
#include <ext/scope_guard.h>
#include <ext/scope_guard_safe.h>
#include <optional>
#include <unordered_set>
@ -704,7 +704,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
for (size_t part_index = 0; part_index < parts.size(); ++part_index)
pool.scheduleOrThrowOnError([&, part_index, thread_group = CurrentThread::getGroup()] {
SCOPE_EXIT(
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);

View File

@ -26,6 +26,7 @@
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h>
#include <Storages/MergeTree/ReplicatedMergeTreePartHeader.h>
#include <Storages/VirtualColumnUtils.h>
#include <Storages/MergeTree/MergeTreeReaderCompact.h>
#include <Databases/IDatabase.h>
@ -59,7 +60,7 @@
#include <ext/range.h>
#include <ext/scope_guard.h>
#include "Storages/MergeTree/MergeTreeReaderCompact.h"
#include <ext/scope_guard_safe.h>
#include <ctime>
#include <thread>
@ -3693,7 +3694,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora
}
}
SCOPE_EXIT
SCOPE_EXIT_MEMORY
({
std::lock_guard lock(currently_fetching_parts_mutex);
currently_fetching_parts.erase(part_name);
@ -3901,7 +3902,7 @@ bool StorageReplicatedMergeTree::fetchExistsPart(const String & part_name, const
}
}
SCOPE_EXIT
SCOPE_EXIT_MEMORY
({
std::lock_guard lock(currently_fetching_parts_mutex);
currently_fetching_parts.erase(part_name);