mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
finalize in destructor
This commit is contained in:
parent
b867eabe73
commit
26ab6ebc0a
@ -21,15 +21,9 @@ public:
|
||||
buffer.assign(capacity, {});
|
||||
}
|
||||
|
||||
size_t size() const
|
||||
{
|
||||
return count;
|
||||
}
|
||||
size_t size() const { return count; }
|
||||
|
||||
bool empty() const
|
||||
{
|
||||
return count == 0;
|
||||
}
|
||||
bool empty() const { return count == 0; }
|
||||
|
||||
bool tryPush(T element)
|
||||
{
|
||||
|
@ -111,8 +111,6 @@ void BackgroundJobAssignee::finish()
|
||||
getContext()->getFetchesExecutor()->removeTasksCorrespondingToStorage(storage_id);
|
||||
getContext()->getMergeMutateExecutor()->removeTasksCorrespondingToStorage(storage_id);
|
||||
}
|
||||
|
||||
finished = true;
|
||||
}
|
||||
|
||||
|
||||
@ -141,7 +139,14 @@ catch (...) /// Catch any exception to avoid thread termination.
|
||||
|
||||
BackgroundJobAssignee::~BackgroundJobAssignee()
|
||||
{
|
||||
assert(finished);
|
||||
try
|
||||
{
|
||||
finish();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -48,8 +48,6 @@ private:
|
||||
/// Mutex for thread safety
|
||||
std::mutex holder_mutex;
|
||||
|
||||
bool finished{false};
|
||||
|
||||
public:
|
||||
enum class Type
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user