2019-01-14 19:22:09 +00:00
|
|
|
#include <Common/ThreadPool.h>
|
|
|
|
#include <Common/Exception.h>
|
2020-07-16 23:12:47 +00:00
|
|
|
#include <Common/getNumberOfPhysicalCPUCores.h>
|
2019-01-14 19:22:09 +00:00
|
|
|
|
2020-06-22 19:04:12 +00:00
|
|
|
#include <cassert>
|
2019-01-14 19:22:09 +00:00
|
|
|
#include <type_traits>
|
|
|
|
|
2020-06-14 06:43:01 +00:00
|
|
|
#include <Poco/Util/Application.h>
|
|
|
|
#include <Poco/Util/LayeredConfiguration.h>
|
|
|
|
|
2019-01-14 19:22:09 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int CANNOT_SCHEDULE_TASK;
|
|
|
|
}
|
|
|
|
}
|
2019-01-14 10:59:58 +00:00
|
|
|
|
2019-08-01 20:09:38 +00:00
|
|
|
namespace CurrentMetrics
|
|
|
|
{
|
|
|
|
extern const Metric GlobalThread;
|
|
|
|
extern const Metric GlobalThreadActive;
|
|
|
|
extern const Metric LocalThread;
|
|
|
|
extern const Metric LocalThreadActive;
|
|
|
|
}
|
|
|
|
|
2019-01-14 10:59:58 +00:00
|
|
|
|
2020-07-16 23:12:47 +00:00
|
|
|
template <typename Thread>
|
|
|
|
ThreadPoolImpl<Thread>::ThreadPoolImpl()
|
|
|
|
: ThreadPoolImpl(getNumberOfPhysicalCPUCores())
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-01-14 10:59:58 +00:00
|
|
|
template <typename Thread>
|
2019-08-03 11:02:40 +00:00
|
|
|
ThreadPoolImpl<Thread>::ThreadPoolImpl(size_t max_threads_)
|
|
|
|
: ThreadPoolImpl(max_threads_, max_threads_, max_threads_)
|
2019-01-14 10:59:58 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Thread>
|
Do not shutdown global thread pool on exception
Otherwise GlobalThreadPool can be terminated (for example due to an
exception from the ParallelInputsHandler::onFinish/onFinishThread, from
ParallelAggregatingBlockInputStream::Handler::onFinish/onFinishThread,
since writeToTemporaryFile() can definitelly throw) and the server will
not accept new connections (or/and execute queries) anymore.
Here is possible stacktrace (it is a bit inaccurate, due to
optimizations I guess, and it had been obtained with the
DB::tryLogCurrentException() in the catch block of the
ThreadPoolImpl::worker()):
2020.02.16 22:30:40.415246 [ 45909 ] {} <Error> ThreadPool: Unhandled exception in the ThreadPool(10000,1000,10000) the loop will be shutted down: Code: 241, e.displayText() = DB::Exception: Memory limit (total) exceeded: would use 279.40 GiB (attempt to allocate chunk of 4205536 bytes), maximum: 279.40 GiB, Stack trace (when copying this message, always include the lines below):
1. Common/Exception.cpp:35: DB::Exception::Exception(...)
...
6. Common/Allocator.h:102: void DB::PODArrayBase<8ul, 4096ul, Allocator<false, false>, 15ul, 16ul>::reserve<>(unsigned long) (.part.0)
7. Interpreters/Aggregator.cpp:1040: void DB::Aggregator::writeToTemporaryFileImpl<...>(...)
8. Interpreters/Aggregator.cpp:719: DB::Aggregator::writeToTemporaryFile(...)
9. include/memory:4206: DB::Aggregator::writeToTemporaryFile(...)
10. DataStreams/ParallelInputsProcessor.h:223: DB::ParallelInputsProcessor<DB::ParallelAggregatingBlockInputStream::Handler>::thread(...)
Refs: https://github.com/ClickHouse/ClickHouse/issues/6833#issuecomment-579221732
(Reference to particular comment, since I'm not sure about the initial issue)
2020-02-17 20:15:29 +00:00
|
|
|
ThreadPoolImpl<Thread>::ThreadPoolImpl(size_t max_threads_, size_t max_free_threads_, size_t queue_size_, bool shutdown_on_exception_)
|
|
|
|
: max_threads(max_threads_)
|
|
|
|
, max_free_threads(max_free_threads_)
|
|
|
|
, queue_size(queue_size_)
|
|
|
|
, shutdown_on_exception(shutdown_on_exception_)
|
2019-01-14 10:59:58 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2019-08-02 17:14:04 +00:00
|
|
|
template <typename Thread>
|
|
|
|
void ThreadPoolImpl<Thread>::setMaxThreads(size_t value)
|
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
max_threads = value;
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Thread>
|
|
|
|
void ThreadPoolImpl<Thread>::setMaxFreeThreads(size_t value)
|
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
max_free_threads = value;
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Thread>
|
|
|
|
void ThreadPoolImpl<Thread>::setQueueSize(size_t value)
|
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
queue_size = value;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-01-14 10:59:58 +00:00
|
|
|
template <typename Thread>
|
2019-01-14 19:22:09 +00:00
|
|
|
template <typename ReturnType>
|
|
|
|
ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::optional<uint64_t> wait_microseconds)
|
2019-01-14 10:59:58 +00:00
|
|
|
{
|
2019-06-29 22:37:46 +00:00
|
|
|
auto on_error = [&]
|
2019-01-14 19:22:09 +00:00
|
|
|
{
|
|
|
|
if constexpr (std::is_same_v<ReturnType, void>)
|
2019-06-29 22:37:46 +00:00
|
|
|
{
|
|
|
|
if (first_exception)
|
|
|
|
{
|
|
|
|
std::exception_ptr exception;
|
|
|
|
std::swap(exception, first_exception);
|
|
|
|
std::rethrow_exception(exception);
|
|
|
|
}
|
2019-01-14 19:22:09 +00:00
|
|
|
throw DB::Exception("Cannot schedule a task", DB::ErrorCodes::CANNOT_SCHEDULE_TASK);
|
2019-06-29 22:37:46 +00:00
|
|
|
}
|
2019-01-14 19:22:09 +00:00
|
|
|
else
|
|
|
|
return false;
|
|
|
|
};
|
|
|
|
|
2019-01-14 10:59:58 +00:00
|
|
|
{
|
2019-01-14 19:22:09 +00:00
|
|
|
std::unique_lock lock(mutex);
|
|
|
|
|
|
|
|
auto pred = [this] { return !queue_size || scheduled_jobs < queue_size || shutdown; };
|
|
|
|
|
2019-08-02 17:14:04 +00:00
|
|
|
if (wait_microseconds) /// Check for optional. Condition is true if the optional is set and the value is zero.
|
2019-01-14 19:22:09 +00:00
|
|
|
{
|
|
|
|
if (!job_finished.wait_for(lock, std::chrono::microseconds(*wait_microseconds), pred))
|
|
|
|
return on_error();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
job_finished.wait(lock, pred);
|
|
|
|
|
2019-01-14 10:59:58 +00:00
|
|
|
if (shutdown)
|
2019-01-14 19:22:09 +00:00
|
|
|
return on_error();
|
2019-01-14 10:59:58 +00:00
|
|
|
|
|
|
|
jobs.emplace(std::move(job), priority);
|
2019-01-14 19:22:09 +00:00
|
|
|
++scheduled_jobs;
|
2019-01-14 10:59:58 +00:00
|
|
|
|
2019-01-14 19:22:09 +00:00
|
|
|
if (threads.size() < std::min(max_threads, scheduled_jobs))
|
|
|
|
{
|
|
|
|
threads.emplace_front();
|
|
|
|
try
|
|
|
|
{
|
|
|
|
threads.front() = Thread([this, it = threads.begin()] { worker(it); });
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
threads.pop_front();
|
2019-08-02 17:14:04 +00:00
|
|
|
|
|
|
|
/// Remove the job and return error to caller.
|
|
|
|
/// Note that if we have allocated at least one thread, we may continue
|
|
|
|
/// (one thread is enough to process all jobs).
|
|
|
|
/// But this condition indicate an error nevertheless and better to refuse.
|
|
|
|
|
|
|
|
jobs.pop();
|
|
|
|
--scheduled_jobs;
|
|
|
|
return on_error();
|
2019-01-14 19:22:09 +00:00
|
|
|
}
|
|
|
|
}
|
2019-01-14 10:59:58 +00:00
|
|
|
}
|
|
|
|
new_job_or_shutdown.notify_one();
|
2019-01-14 19:22:09 +00:00
|
|
|
return ReturnType(true);
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Thread>
|
2019-10-17 14:41:27 +00:00
|
|
|
void ThreadPoolImpl<Thread>::scheduleOrThrowOnError(Job job, int priority)
|
2019-01-14 19:22:09 +00:00
|
|
|
{
|
|
|
|
scheduleImpl<void>(std::move(job), priority, std::nullopt);
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Thread>
|
2019-10-17 14:41:27 +00:00
|
|
|
bool ThreadPoolImpl<Thread>::trySchedule(Job job, int priority, uint64_t wait_microseconds) noexcept
|
2019-01-14 19:22:09 +00:00
|
|
|
{
|
|
|
|
return scheduleImpl<bool>(std::move(job), priority, wait_microseconds);
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Thread>
|
|
|
|
void ThreadPoolImpl<Thread>::scheduleOrThrow(Job job, int priority, uint64_t wait_microseconds)
|
|
|
|
{
|
|
|
|
scheduleImpl<void>(std::move(job), priority, wait_microseconds);
|
2019-01-14 10:59:58 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Thread>
|
|
|
|
void ThreadPoolImpl<Thread>::wait()
|
|
|
|
{
|
|
|
|
{
|
2019-01-14 19:22:09 +00:00
|
|
|
std::unique_lock lock(mutex);
|
|
|
|
job_finished.wait(lock, [this] { return scheduled_jobs == 0; });
|
2019-01-14 10:59:58 +00:00
|
|
|
|
|
|
|
if (first_exception)
|
|
|
|
{
|
|
|
|
std::exception_ptr exception;
|
|
|
|
std::swap(exception, first_exception);
|
|
|
|
std::rethrow_exception(exception);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Thread>
|
|
|
|
ThreadPoolImpl<Thread>::~ThreadPoolImpl()
|
|
|
|
{
|
|
|
|
finalize();
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Thread>
|
|
|
|
void ThreadPoolImpl<Thread>::finalize()
|
|
|
|
{
|
|
|
|
{
|
2019-01-14 19:22:09 +00:00
|
|
|
std::unique_lock lock(mutex);
|
2019-01-14 10:59:58 +00:00
|
|
|
shutdown = true;
|
|
|
|
}
|
|
|
|
|
|
|
|
new_job_or_shutdown.notify_all();
|
|
|
|
|
|
|
|
for (auto & thread : threads)
|
|
|
|
thread.join();
|
|
|
|
|
|
|
|
threads.clear();
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Thread>
|
|
|
|
size_t ThreadPoolImpl<Thread>::active() const
|
|
|
|
{
|
2019-01-14 19:22:09 +00:00
|
|
|
std::unique_lock lock(mutex);
|
|
|
|
return scheduled_jobs;
|
2019-01-14 10:59:58 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Thread>
|
2019-01-14 19:22:09 +00:00
|
|
|
void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_it)
|
2019-01-14 10:59:58 +00:00
|
|
|
{
|
2019-08-01 20:09:38 +00:00
|
|
|
CurrentMetrics::Increment metric_all_threads(
|
|
|
|
std::is_same_v<Thread, std::thread> ? CurrentMetrics::GlobalThread : CurrentMetrics::LocalThread);
|
|
|
|
|
2019-01-14 10:59:58 +00:00
|
|
|
while (true)
|
|
|
|
{
|
|
|
|
Job job;
|
|
|
|
bool need_shutdown = false;
|
|
|
|
|
|
|
|
{
|
2019-01-14 19:22:09 +00:00
|
|
|
std::unique_lock lock(mutex);
|
2019-01-14 10:59:58 +00:00
|
|
|
new_job_or_shutdown.wait(lock, [this] { return shutdown || !jobs.empty(); });
|
|
|
|
need_shutdown = shutdown;
|
|
|
|
|
|
|
|
if (!jobs.empty())
|
|
|
|
{
|
|
|
|
job = jobs.top().job;
|
|
|
|
jobs.pop();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2019-03-05 20:49:21 +00:00
|
|
|
/// shutdown is true, simply finish the thread.
|
2019-01-14 10:59:58 +00:00
|
|
|
return;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!need_shutdown)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2019-08-01 20:09:38 +00:00
|
|
|
CurrentMetrics::Increment metric_active_threads(
|
|
|
|
std::is_same_v<Thread, std::thread> ? CurrentMetrics::GlobalThreadActive : CurrentMetrics::LocalThreadActive);
|
|
|
|
|
2019-01-14 10:59:58 +00:00
|
|
|
job();
|
Fix possible UAF from jobs in the ThreadPool on shutdown
ThreadPoolImpl<>::worker signaling that job is done while still storing
std::function<> object, and this can lead to problems on shutdown, since
in in this cast std::function<> can refers to some global/static object
that had been already destroyed (typical example is Logger).
I believe that this is exactly what TSAN reports about (decoded
manually, since llvm-symbolizer does not work in the test env):
<details>
```
- 2020-09-20 17:44:43 Write of size 8 at 0x7b1000008f78 by main thread (mutexes: write M1432):
operator delete(void*, unsigned long)
??:0:0
Poco::Logger::~Logger()
/build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Logger.cpp:39:1
non-virtual thunk to Poco::Logger::~Logger()
/build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Logger.cpp:0:0
Poco::RefCountedObject::release() const
/build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/include/Poco/RefCountedObject.h:82:24
Poco::Logger::shutdown()
/build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Logger.cpp:346:16
Poco::AutoLoggerShutdown::~AutoLoggerShutdown()
/build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Logger.cpp:459:4
cxa_at_exit_wrapper(void*)
??:0:0
```
```
- 2020-09-20 17:44:43 Previous atomic read of size 4 at 0x7b1000008f78 by thread T116:
__tsan_atomic32_load
??:0:0
int std::__1::__cxx_atomic_load<int>(std::__1::__cxx_atomic_base_impl<int> const*, std::__1::memory_order)
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/atomic:970:12
std::__1::__atomic_base<int, false>::load(std::__1::memory_order) const
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/atomic:1487:17
std::__1::__atomic_base<int, false>::operator int() const
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/atomic:1491:53
Poco::Logger::is(int) const
/build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/include/Poco/Logger.h:1264:9
MemoryTracker::logPeakMemoryUsage() const
/build/obj-x86_64-linux-gnu/../src/Common/MemoryTracker.cpp:59:5
MemoryTracker::~MemoryTracker()
/build/obj-x86_64-linux-gnu/../src/Common/MemoryTracker.cpp:46:13
DB::ThreadGroupStatus::~ThreadGroupStatus()
/build/obj-x86_64-linux-gnu/../src/Common/ThreadStatus.h:51:7
std::__1::__shared_ptr_emplace<DB::ThreadGroupStatus, std::__1::allocator<DB::ThreadGroupStatus> >::__on_zero_shared()
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3602:23
std::__1::__shared_count::__release_shared()
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3440:9
std::__1::__shared_weak_count::__release_shared()
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3482:27
std::__1::shared_ptr<DB::ThreadGroupStatus>::~shared_ptr()
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:4207:19
DB::PipelineExecutor::executeImpl(unsigned long)::$_4::~$_4()
/build/obj-x86_64-linux-gnu/../src/Processors/Executors/PipelineExecutor.cpp:720:34
ThreadFromGlobalPool::ThreadFromGlobalPool<DB::PipelineExecutor::executeImpl(unsigned long)::$_4>(DB::PipelineExecutor::executeImpl(unsigned long)::$_4&&)::'lambda'()::~()
/build/obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:161:54
std::__1::__compressed_pair_elem<ThreadFromGlobalPool::ThreadFromGlobalPool<DB::PipelineExecutor::executeImpl(unsigned long)::$_4>(DB::PipelineExecutor::executeImpl(unsigned long)::$_4&&)::'lambda'(), 0, false>::~__compressed_pair_elem()
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2188:8
std::__1::__function::__alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<DB::PipelineExecutor::executeImpl(unsigned long)::$_4>(DB::PipelineExecutor::executeImpl(unsigned long)::$_4&&)::'lambda'(), std::__1::allocator<ThreadFromGlobalPool::ThreadFromGlobalPool<DB::Pip>
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1559:38
std::__1::__function::__func<ThreadFromGlobalPool::ThreadFromGlobalPool<DB::PipelineExecutor::executeImpl(unsigned long)::$_4>(DB::PipelineExecutor::executeImpl(unsigned long)::$_4&&)::'lambda'(), std::__1::allocator<ThreadFromGlobalPool::ThreadFromGlobalPool<DB::PipelineE>
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1706:10
std::__1::__function::__value_func<void ()>::~__value_func()
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1828:19
std::__1::function<void ()>::~function()
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2460:43
ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>)
/build/obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:268:5
void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const
/build/obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:116:73
decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleI>
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3519:1
void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'(>
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:273:5
void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned lon>
/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:284:5
__tsan_thread_start_func
??:0:0
```
```
- 2020-09-20 17:44:43 Mutex M1432 (0x0000181213a8) created at:
pthread_mutex_init
??:0:0
Poco::MutexImpl::MutexImpl()
/build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Mutex_POSIX.cpp:64:6
Poco::Mutex::Mutex()
/build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Mutex.cpp:34:8
__cxx_global_var_init
/build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Logger.cpp:28:15
_GLOBAL__sub_I_Logger.cpp
/build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Logger.cpp:0:0
__libc_csu_init
??:0:0
```
</details>
2020-09-20 20:41:36 +00:00
|
|
|
job = Job();
|
2019-01-14 10:59:58 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
{
|
2019-01-14 19:22:09 +00:00
|
|
|
std::unique_lock lock(mutex);
|
2019-01-14 10:59:58 +00:00
|
|
|
if (!first_exception)
|
2020-03-18 02:02:24 +00:00
|
|
|
first_exception = std::current_exception(); // NOLINT
|
Do not shutdown global thread pool on exception
Otherwise GlobalThreadPool can be terminated (for example due to an
exception from the ParallelInputsHandler::onFinish/onFinishThread, from
ParallelAggregatingBlockInputStream::Handler::onFinish/onFinishThread,
since writeToTemporaryFile() can definitelly throw) and the server will
not accept new connections (or/and execute queries) anymore.
Here is possible stacktrace (it is a bit inaccurate, due to
optimizations I guess, and it had been obtained with the
DB::tryLogCurrentException() in the catch block of the
ThreadPoolImpl::worker()):
2020.02.16 22:30:40.415246 [ 45909 ] {} <Error> ThreadPool: Unhandled exception in the ThreadPool(10000,1000,10000) the loop will be shutted down: Code: 241, e.displayText() = DB::Exception: Memory limit (total) exceeded: would use 279.40 GiB (attempt to allocate chunk of 4205536 bytes), maximum: 279.40 GiB, Stack trace (when copying this message, always include the lines below):
1. Common/Exception.cpp:35: DB::Exception::Exception(...)
...
6. Common/Allocator.h:102: void DB::PODArrayBase<8ul, 4096ul, Allocator<false, false>, 15ul, 16ul>::reserve<>(unsigned long) (.part.0)
7. Interpreters/Aggregator.cpp:1040: void DB::Aggregator::writeToTemporaryFileImpl<...>(...)
8. Interpreters/Aggregator.cpp:719: DB::Aggregator::writeToTemporaryFile(...)
9. include/memory:4206: DB::Aggregator::writeToTemporaryFile(...)
10. DataStreams/ParallelInputsProcessor.h:223: DB::ParallelInputsProcessor<DB::ParallelAggregatingBlockInputStream::Handler>::thread(...)
Refs: https://github.com/ClickHouse/ClickHouse/issues/6833#issuecomment-579221732
(Reference to particular comment, since I'm not sure about the initial issue)
2020-02-17 20:15:29 +00:00
|
|
|
if (shutdown_on_exception)
|
|
|
|
shutdown = true;
|
2019-01-14 19:22:09 +00:00
|
|
|
--scheduled_jobs;
|
2019-01-14 10:59:58 +00:00
|
|
|
}
|
Do not shutdown global thread pool on exception
Otherwise GlobalThreadPool can be terminated (for example due to an
exception from the ParallelInputsHandler::onFinish/onFinishThread, from
ParallelAggregatingBlockInputStream::Handler::onFinish/onFinishThread,
since writeToTemporaryFile() can definitelly throw) and the server will
not accept new connections (or/and execute queries) anymore.
Here is possible stacktrace (it is a bit inaccurate, due to
optimizations I guess, and it had been obtained with the
DB::tryLogCurrentException() in the catch block of the
ThreadPoolImpl::worker()):
2020.02.16 22:30:40.415246 [ 45909 ] {} <Error> ThreadPool: Unhandled exception in the ThreadPool(10000,1000,10000) the loop will be shutted down: Code: 241, e.displayText() = DB::Exception: Memory limit (total) exceeded: would use 279.40 GiB (attempt to allocate chunk of 4205536 bytes), maximum: 279.40 GiB, Stack trace (when copying this message, always include the lines below):
1. Common/Exception.cpp:35: DB::Exception::Exception(...)
...
6. Common/Allocator.h:102: void DB::PODArrayBase<8ul, 4096ul, Allocator<false, false>, 15ul, 16ul>::reserve<>(unsigned long) (.part.0)
7. Interpreters/Aggregator.cpp:1040: void DB::Aggregator::writeToTemporaryFileImpl<...>(...)
8. Interpreters/Aggregator.cpp:719: DB::Aggregator::writeToTemporaryFile(...)
9. include/memory:4206: DB::Aggregator::writeToTemporaryFile(...)
10. DataStreams/ParallelInputsProcessor.h:223: DB::ParallelInputsProcessor<DB::ParallelAggregatingBlockInputStream::Handler>::thread(...)
Refs: https://github.com/ClickHouse/ClickHouse/issues/6833#issuecomment-579221732
(Reference to particular comment, since I'm not sure about the initial issue)
2020-02-17 20:15:29 +00:00
|
|
|
|
2019-01-14 10:59:58 +00:00
|
|
|
job_finished.notify_all();
|
|
|
|
new_job_or_shutdown.notify_all();
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
{
|
2019-01-14 19:22:09 +00:00
|
|
|
std::unique_lock lock(mutex);
|
|
|
|
--scheduled_jobs;
|
|
|
|
|
|
|
|
if (threads.size() > scheduled_jobs + max_free_threads)
|
|
|
|
{
|
2019-02-25 15:45:07 +00:00
|
|
|
thread_it->detach();
|
2019-01-14 19:22:09 +00:00
|
|
|
threads.erase(thread_it);
|
|
|
|
job_finished.notify_all();
|
|
|
|
return;
|
|
|
|
}
|
2019-01-14 10:59:58 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
job_finished.notify_all();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template class ThreadPoolImpl<std::thread>;
|
|
|
|
template class ThreadPoolImpl<ThreadFromGlobalPool>;
|
|
|
|
|
2020-06-22 19:04:12 +00:00
|
|
|
std::unique_ptr<GlobalThreadPool> GlobalThreadPool::the_instance;
|
2019-01-14 10:59:58 +00:00
|
|
|
|
2020-06-22 19:04:12 +00:00
|
|
|
void GlobalThreadPool::initialize(size_t max_threads)
|
2019-08-22 03:24:05 +00:00
|
|
|
{
|
2020-06-23 17:31:46 +00:00
|
|
|
assert(!the_instance);
|
2020-06-14 06:43:01 +00:00
|
|
|
|
2020-06-22 19:04:12 +00:00
|
|
|
the_instance.reset(new GlobalThreadPool(max_threads,
|
|
|
|
1000 /*max_free_threads*/, 10000 /*max_queue_size*/,
|
|
|
|
false /*shutdown_on_exception*/));
|
|
|
|
}
|
2020-06-14 06:43:01 +00:00
|
|
|
|
2020-06-22 19:04:12 +00:00
|
|
|
GlobalThreadPool & GlobalThreadPool::instance()
|
|
|
|
{
|
|
|
|
if (!the_instance)
|
|
|
|
{
|
2020-06-23 17:31:46 +00:00
|
|
|
// Allow implicit initialization. This is needed for old code that is
|
|
|
|
// impractical to redo now, especially Arcadia users and unit tests.
|
|
|
|
initialize();
|
2020-06-22 19:04:12 +00:00
|
|
|
}
|
2020-06-14 06:43:01 +00:00
|
|
|
|
2020-06-22 19:04:12 +00:00
|
|
|
return *the_instance;
|
2019-08-22 03:24:05 +00:00
|
|
|
}
|