Commit 68a6b5b5 authored by Dave Watson's avatar Dave Watson Committed by Facebook Github Bot

dynamic CPUThreadPoolExecutor

Summary:
This diff adds dynamic thread creation/destruction to ThreadPoolExecutor (mostly CPU).

1) Threads are lazily created when the first task is add()ed (or getEventBase is called), applies to both IO and CPU

and

2) after a timeout in LifoSem, threads are joined in CPUThreadPoolExecutor.

Most of the logic is in ThreadPoolExecutor.  A separate dynamic IOThreadPoolExecutor will be a follow on diff.

To keep the previous behavior, users can set minthread == maxthreads, or gflags --dynamic_cputhreadpoolexecutor=false

Reviewed By: magedm

Differential Revision: D7477390

fbshipit-source-id: 855a35380861111de33656e8fb1b681ae8cd2807
parent 484f75dd
...@@ -17,6 +17,11 @@ ...@@ -17,6 +17,11 @@
#include <folly/executors/CPUThreadPoolExecutor.h> #include <folly/executors/CPUThreadPoolExecutor.h>
#include <folly/executors/task_queue/PriorityLifoSemMPMCQueue.h> #include <folly/executors/task_queue/PriorityLifoSemMPMCQueue.h>
DEFINE_bool(
dynamic_cputhreadpoolexecutor,
true,
"CPUThreadPoolExecutor will dynamically create and destroy threads");
namespace folly { namespace folly {
const size_t CPUThreadPoolExecutor::kDefaultMaxQueueSize = 1 << 14; const size_t CPUThreadPoolExecutor::kDefaultMaxQueueSize = 1 << 14;
...@@ -25,11 +30,26 @@ CPUThreadPoolExecutor::CPUThreadPoolExecutor( ...@@ -25,11 +30,26 @@ CPUThreadPoolExecutor::CPUThreadPoolExecutor(
size_t numThreads, size_t numThreads,
std::unique_ptr<BlockingQueue<CPUTask>> taskQueue, std::unique_ptr<BlockingQueue<CPUTask>> taskQueue,
std::shared_ptr<ThreadFactory> threadFactory) std::shared_ptr<ThreadFactory> threadFactory)
: ThreadPoolExecutor(numThreads, std::move(threadFactory)), : ThreadPoolExecutor(
numThreads,
FLAGS_dynamic_cputhreadpoolexecutor ? 0 : numThreads,
std::move(threadFactory)),
taskQueue_(std::move(taskQueue)) { taskQueue_(std::move(taskQueue)) {
setNumThreads(numThreads); setNumThreads(numThreads);
} }
CPUThreadPoolExecutor::CPUThreadPoolExecutor(
std::pair<size_t, size_t> numThreads,
std::unique_ptr<BlockingQueue<CPUTask>> taskQueue,
std::shared_ptr<ThreadFactory> threadFactory)
: ThreadPoolExecutor(
numThreads.first,
numThreads.second,
std::move(threadFactory)),
taskQueue_(std::move(taskQueue)) {
setNumThreads(numThreads.first);
}
CPUThreadPoolExecutor::CPUThreadPoolExecutor( CPUThreadPoolExecutor::CPUThreadPoolExecutor(
size_t numThreads, size_t numThreads,
std::shared_ptr<ThreadFactory> threadFactory) std::shared_ptr<ThreadFactory> threadFactory)
...@@ -81,9 +101,10 @@ void CPUThreadPoolExecutor::add( ...@@ -81,9 +101,10 @@ void CPUThreadPoolExecutor::add(
Func func, Func func,
std::chrono::milliseconds expiration, std::chrono::milliseconds expiration,
Func expireCallback) { Func expireCallback) {
// TODO handle enqueue failure, here and in other add() callsites if (!taskQueue_->add(
taskQueue_->add( CPUTask(std::move(func), expiration, std::move(expireCallback)))) {
CPUTask(std::move(func), expiration, std::move(expireCallback))); ensureActiveThreads();
}
} }
void CPUThreadPoolExecutor::addWithPriority(Func func, int8_t priority) { void CPUThreadPoolExecutor::addWithPriority(Func func, int8_t priority) {
...@@ -96,9 +117,11 @@ void CPUThreadPoolExecutor::add( ...@@ -96,9 +117,11 @@ void CPUThreadPoolExecutor::add(
std::chrono::milliseconds expiration, std::chrono::milliseconds expiration,
Func expireCallback) { Func expireCallback) {
CHECK(getNumPriorities() > 0); CHECK(getNumPriorities() > 0);
taskQueue_->addWithPriority( if (!taskQueue_->addWithPriority(
CPUTask(std::move(func), expiration, std::move(expireCallback)), CPUTask(std::move(func), expiration, std::move(expireCallback)),
priority); priority)) {
ensureActiveThreads();
}
} }
uint8_t CPUThreadPoolExecutor::getNumPriorities() const { uint8_t CPUThreadPoolExecutor::getNumPriorities() const {
...@@ -114,33 +137,71 @@ CPUThreadPoolExecutor::getTaskQueue() { ...@@ -114,33 +137,71 @@ CPUThreadPoolExecutor::getTaskQueue() {
return taskQueue_.get(); return taskQueue_.get();
} }
bool CPUThreadPoolExecutor::tryDecrToStop() {
while (true) {
auto toStop = threadsToStop_.load(std::memory_order_relaxed);
if (toStop <= 0) {
return false;
}
if (threadsToStop_.compare_exchange_strong(
toStop, toStop - 1, std::memory_order_relaxed)) {
return true;
}
}
}
bool CPUThreadPoolExecutor::taskShouldStop(folly::Optional<CPUTask>& task) {
if (task) {
if (!tryDecrToStop()) {
// Some other thread beat us to it.
return false;
}
} else {
// Try to stop based on idle thread timeout (try_take_for),
// if there are at least minThreads running.
if (!minActive()) {
return false;
}
// If this is based on idle thread timeout, then
// adjust vars appropriately (otherwise stop() or join()
// does this).
activeThreads_.fetch_sub(1, std::memory_order_relaxed);
threadsToJoin_.fetch_add(1, std::memory_order_relaxed);
}
return true;
}
void CPUThreadPoolExecutor::threadRun(ThreadPtr thread) { void CPUThreadPoolExecutor::threadRun(ThreadPtr thread) {
this->threadPoolHook_.registerThread(); this->threadPoolHook_.registerThread();
thread->startupBaton.post(); thread->startupBaton.post();
while (true) { while (true) {
auto task = taskQueue_->take(); auto task = taskQueue_->try_take_for(threadTimeout_);
if (UNLIKELY(task.poison)) { // Handle thread stopping, either by task timeout, or
CHECK(threadsToStop_-- > 0); // by 'poison' task added in join() or stop().
if (UNLIKELY(!task || task.value().poison)) {
if (taskShouldStop(task)) {
for (auto& o : observers_) { for (auto& o : observers_) {
o->threadStopped(thread.get()); o->threadStopped(thread.get());
} }
// Actually remove the thread from the list.
folly::RWSpinLock::WriteHolder w{&threadListLock_}; folly::RWSpinLock::WriteHolder w{&threadListLock_};
threadList_.remove(thread); threadList_.remove(thread);
stoppedThreads_.add(thread); stoppedThreads_.add(thread);
return; return;
} else { } else {
runTask(thread, std::move(task)); continue;
} }
}
runTask(thread, std::move(task.value()));
if (UNLIKELY(threadsToStop_ > 0 && !isJoin_)) { if (UNLIKELY(threadsToStop_ > 0 && !isJoin_)) {
if (--threadsToStop_ >= 0) { if (tryDecrToStop()) {
folly::RWSpinLock::WriteHolder w{&threadListLock_}; folly::RWSpinLock::WriteHolder w{&threadListLock_};
threadList_.remove(thread); threadList_.remove(thread);
stoppedThreads_.add(thread); stoppedThreads_.add(thread);
return; return;
} else {
threadsToStop_++;
} }
} }
} }
...@@ -153,7 +214,7 @@ void CPUThreadPoolExecutor::stopThreads(size_t n) { ...@@ -153,7 +214,7 @@ void CPUThreadPoolExecutor::stopThreads(size_t n) {
} }
} }
// threadListLock_ is readlocked // threadListLock_ is read (or write) locked.
uint64_t CPUThreadPoolExecutor::getPendingTaskCountImpl() { uint64_t CPUThreadPoolExecutor::getPendingTaskCountImpl() {
return taskQueue_->size(); return taskQueue_->size();
} }
......
...@@ -72,6 +72,12 @@ class CPUThreadPoolExecutor : public ThreadPoolExecutor, ...@@ -72,6 +72,12 @@ class CPUThreadPoolExecutor : public ThreadPoolExecutor,
std::shared_ptr<ThreadFactory> threadFactory = std::shared_ptr<ThreadFactory> threadFactory =
std::make_shared<NamedThreadFactory>("CPUThreadPool")); std::make_shared<NamedThreadFactory>("CPUThreadPool"));
CPUThreadPoolExecutor(
std::pair<size_t, size_t> numThreads,
std::unique_ptr<BlockingQueue<CPUTask>> taskQueue,
std::shared_ptr<ThreadFactory> threadFactory =
std::make_shared<NamedThreadFactory>("CPUThreadPool"));
explicit CPUThreadPoolExecutor(size_t numThreads); explicit CPUThreadPoolExecutor(size_t numThreads);
CPUThreadPoolExecutor( CPUThreadPoolExecutor(
...@@ -135,6 +141,9 @@ class CPUThreadPoolExecutor : public ThreadPoolExecutor, ...@@ -135,6 +141,9 @@ class CPUThreadPoolExecutor : public ThreadPoolExecutor,
void stopThreads(size_t n) override; void stopThreads(size_t n) override;
uint64_t getPendingTaskCountImpl() override; uint64_t getPendingTaskCountImpl() override;
bool tryDecrToStop();
bool taskShouldStop(folly::Optional<CPUTask>&);
std::unique_ptr<BlockingQueue<CPUTask>> taskQueue_; std::unique_ptr<BlockingQueue<CPUTask>> taskQueue_;
std::atomic<ssize_t> threadsToStop_{0}; std::atomic<ssize_t> threadsToStop_{0};
}; };
......
...@@ -66,7 +66,7 @@ IOThreadPoolExecutor::IOThreadPoolExecutor( ...@@ -66,7 +66,7 @@ IOThreadPoolExecutor::IOThreadPoolExecutor(
std::shared_ptr<ThreadFactory> threadFactory, std::shared_ptr<ThreadFactory> threadFactory,
EventBaseManager* ebm, EventBaseManager* ebm,
bool waitForAll) bool waitForAll)
: ThreadPoolExecutor(numThreads, std::move(threadFactory), waitForAll), : ThreadPoolExecutor(numThreads, 0, std::move(threadFactory), waitForAll),
nextThread_(0), nextThread_(0),
eventBaseManager_(ebm) { eventBaseManager_(ebm) {
setNumThreads(numThreads); setNumThreads(numThreads);
...@@ -85,6 +85,7 @@ void IOThreadPoolExecutor::add( ...@@ -85,6 +85,7 @@ void IOThreadPoolExecutor::add(
Func func, Func func,
std::chrono::milliseconds expiration, std::chrono::milliseconds expiration,
Func expireCallback) { Func expireCallback) {
ensureActiveThreads();
RWSpinLock::ReadHolder r{&threadListLock_}; RWSpinLock::ReadHolder r{&threadListLock_};
if (threadList_.get().empty()) { if (threadList_.get().empty()) {
throw std::runtime_error("No threads available"); throw std::runtime_error("No threads available");
...@@ -125,6 +126,7 @@ IOThreadPoolExecutor::pickThread() { ...@@ -125,6 +126,7 @@ IOThreadPoolExecutor::pickThread() {
} }
EventBase* IOThreadPoolExecutor::getEventBase() { EventBase* IOThreadPoolExecutor::getEventBase() {
ensureActiveThreads();
RWSpinLock::ReadHolder r{&threadListLock_}; RWSpinLock::ReadHolder r{&threadListLock_};
return pickThread()->eventBase; return pickThread()->eventBase;
} }
......
...@@ -28,14 +28,22 @@ SyncVecThreadPoolExecutors& getSyncVecThreadPoolExecutors() { ...@@ -28,14 +28,22 @@ SyncVecThreadPoolExecutors& getSyncVecThreadPoolExecutors() {
return *storage; return *storage;
} }
DEFINE_int64(
threadtimeout_ms,
60000,
"Idle time before ThreadPoolExecutor threads are joined");
ThreadPoolExecutor::ThreadPoolExecutor( ThreadPoolExecutor::ThreadPoolExecutor(
size_t /* numThreads */, size_t /* maxThreads */,
size_t minThreads,
std::shared_ptr<ThreadFactory> threadFactory, std::shared_ptr<ThreadFactory> threadFactory,
bool isWaitForAll) bool isWaitForAll)
: threadFactory_(std::move(threadFactory)), : threadFactory_(std::move(threadFactory)),
isWaitForAll_(isWaitForAll), isWaitForAll_(isWaitForAll),
taskStatsCallbacks_(std::make_shared<TaskStatsCallbackRegistry>()), taskStatsCallbacks_(std::make_shared<TaskStatsCallbackRegistry>()),
threadPoolHook_("folly::ThreadPoolExecutor") { threadPoolHook_("folly::ThreadPoolExecutor"),
minThreads_(minThreads),
threadTimeout_(FLAGS_threadtimeout_ms) {
getSyncVecThreadPoolExecutors()->push_back(this); getSyncVecThreadPoolExecutors()->push_back(this);
} }
...@@ -104,25 +112,62 @@ void ThreadPoolExecutor::runTask(const ThreadPtr& thread, Task&& task) { ...@@ -104,25 +112,62 @@ void ThreadPoolExecutor::runTask(const ThreadPtr& thread, Task&& task) {
} }
size_t ThreadPoolExecutor::numThreads() { size_t ThreadPoolExecutor::numThreads() {
RWSpinLock::ReadHolder r{&threadListLock_}; return maxThreads_.load(std::memory_order_relaxed);
return threadList_.get().size();
} }
void ThreadPoolExecutor::setNumThreads(size_t n) { // Set the maximum number of running threads.
void ThreadPoolExecutor::setNumThreads(size_t numThreads) {
/* Since ThreadPoolExecutor may be dynamically adjusting the number of
threads, we adjust the relevant variables instead of changing
the number of threads directly. Roughly:
If numThreads < minthreads reset minThreads to numThreads.
If numThreads < active threads, reduce number of running threads.
If the number of pending tasks is > 0, then increase the currently
active number of threads such that we can run all the tasks, or reach
numThreads.
Note that if there are observers, we actually have to create all
the threads, because some observer implementations need to 'observe'
all thread creation (see tests for an example of this)
*/
size_t numThreadsToJoin = 0; size_t numThreadsToJoin = 0;
{ {
RWSpinLock::WriteHolder w{&threadListLock_}; RWSpinLock::WriteHolder w{&threadListLock_};
const auto current = threadList_.get().size(); auto pending = getPendingTaskCountImpl();
if (n > current) { maxThreads_.store(numThreads, std::memory_order_relaxed);
addThreads(n - current); auto active = activeThreads_.load(std::memory_order_relaxed);
} else if (n < current) { auto minthreads = minThreads_.load(std::memory_order_relaxed);
numThreadsToJoin = current - n; if (numThreads < minthreads) {
removeThreads(numThreadsToJoin, true); minthreads = numThreads;
minThreads_.store(numThreads, std::memory_order_relaxed);
}
if (active > numThreads) {
numThreadsToJoin = active - numThreads;
if (numThreadsToJoin > active - minthreads) {
numThreadsToJoin = active - minthreads;
}
ThreadPoolExecutor::removeThreads(numThreadsToJoin, false);
activeThreads_.store(
active - numThreadsToJoin, std::memory_order_relaxed);
} else if (pending > 0 || observers_.size() > 0 || active < minthreads) {
size_t numToAdd = std::min(pending, numThreads - active);
if (observers_.size() > 0) {
numToAdd = numThreads - active;
} }
if (active + numToAdd < minthreads) {
numToAdd = minthreads - active;
} }
ThreadPoolExecutor::addThreads(numToAdd);
activeThreads_.store(active + numToAdd, std::memory_order_relaxed);
}
}
/* We may have removed some threads, attempt to join them */
joinStoppedThreads(numThreadsToJoin); joinStoppedThreads(numThreadsToJoin);
CHECK_EQ(n, threadList_.get().size());
CHECK_EQ(0, stoppedThreads_.size());
} }
// threadListLock_ is writelocked // threadListLock_ is writelocked
...@@ -150,7 +195,6 @@ void ThreadPoolExecutor::addThreads(size_t n) { ...@@ -150,7 +195,6 @@ void ThreadPoolExecutor::addThreads(size_t n) {
// threadListLock_ is writelocked // threadListLock_ is writelocked
void ThreadPoolExecutor::removeThreads(size_t n, bool isJoin) { void ThreadPoolExecutor::removeThreads(size_t n, bool isJoin) {
CHECK_LE(n, threadList_.get().size());
isJoin_ = isJoin; isJoin_ = isJoin;
stopThreads(n); stopThreads(n);
} }
...@@ -163,6 +207,13 @@ void ThreadPoolExecutor::joinStoppedThreads(size_t n) { ...@@ -163,6 +207,13 @@ void ThreadPoolExecutor::joinStoppedThreads(size_t n) {
} }
void ThreadPoolExecutor::stop() { void ThreadPoolExecutor::stop() {
{
folly::RWSpinLock::WriteHolder w{&threadListLock_};
maxThreads_.store(0, std::memory_order_release);
activeThreads_.store(0, std::memory_order_release);
}
ensureJoined();
size_t n = 0; size_t n = 0;
{ {
RWSpinLock::WriteHolder w{&threadListLock_}; RWSpinLock::WriteHolder w{&threadListLock_};
...@@ -175,6 +226,13 @@ void ThreadPoolExecutor::stop() { ...@@ -175,6 +226,13 @@ void ThreadPoolExecutor::stop() {
} }
void ThreadPoolExecutor::join() { void ThreadPoolExecutor::join() {
{
folly::RWSpinLock::WriteHolder w{&threadListLock_};
maxThreads_.store(0, std::memory_order_release);
activeThreads_.store(0, std::memory_order_release);
}
ensureJoined();
size_t n = 0; size_t n = 0;
{ {
RWSpinLock::WriteHolder w{&threadListLock_}; RWSpinLock::WriteHolder w{&threadListLock_};
...@@ -198,18 +256,24 @@ ThreadPoolExecutor::PoolStats ThreadPoolExecutor::getPoolStats() { ...@@ -198,18 +256,24 @@ ThreadPoolExecutor::PoolStats ThreadPoolExecutor::getPoolStats() {
const auto now = std::chrono::steady_clock::now(); const auto now = std::chrono::steady_clock::now();
RWSpinLock::ReadHolder r{&threadListLock_}; RWSpinLock::ReadHolder r{&threadListLock_};
ThreadPoolExecutor::PoolStats stats; ThreadPoolExecutor::PoolStats stats;
stats.threadCount = threadList_.get().size(); size_t activeTasks = 0;
size_t idleAlive = 0;
for (auto thread : threadList_.get()) { for (auto thread : threadList_.get()) {
if (thread->idle) { if (thread->idle) {
stats.idleThreadCount++;
const std::chrono::nanoseconds idleTime = now - thread->lastActiveTime; const std::chrono::nanoseconds idleTime = now - thread->lastActiveTime;
stats.maxIdleTime = std::max(stats.maxIdleTime, idleTime); stats.maxIdleTime = std::max(stats.maxIdleTime, idleTime);
idleAlive++;
} else { } else {
stats.activeThreadCount++; activeTasks++;
} }
} }
stats.pendingTaskCount = getPendingTaskCountImpl(); stats.pendingTaskCount = getPendingTaskCountImpl();
stats.totalTaskCount = stats.pendingTaskCount + stats.activeThreadCount; stats.totalTaskCount = stats.pendingTaskCount + activeTasks;
stats.threadCount = maxThreads_.load(std::memory_order_relaxed);
stats.activeThreadCount =
activeThreads_.load(std::memory_order_relaxed) - idleAlive;
stats.idleThreadCount = stats.threadCount - stats.activeThreadCount;
return stats; return stats;
} }
...@@ -281,11 +345,17 @@ size_t ThreadPoolExecutor::StoppedThreadQueue::size() { ...@@ -281,11 +345,17 @@ size_t ThreadPoolExecutor::StoppedThreadQueue::size() {
} }
void ThreadPoolExecutor::addObserver(std::shared_ptr<Observer> o) { void ThreadPoolExecutor::addObserver(std::shared_ptr<Observer> o) {
{
RWSpinLock::ReadHolder r{&threadListLock_}; RWSpinLock::ReadHolder r{&threadListLock_};
observers_.push_back(o); observers_.push_back(o);
for (auto& thread : threadList_.get()) { for (auto& thread : threadList_.get()) {
o->threadPreviouslyStarted(thread.get()); o->threadPreviouslyStarted(thread.get());
} }
}
while (activeThreads_.load(std::memory_order_relaxed) <
maxThreads_.load(std::memory_order_relaxed)) {
ensureActiveThreads();
}
} }
void ThreadPoolExecutor::removeObserver(std::shared_ptr<Observer> o) { void ThreadPoolExecutor::removeObserver(std::shared_ptr<Observer> o) {
...@@ -303,4 +373,46 @@ void ThreadPoolExecutor::removeObserver(std::shared_ptr<Observer> o) { ...@@ -303,4 +373,46 @@ void ThreadPoolExecutor::removeObserver(std::shared_ptr<Observer> o) {
DCHECK(false); DCHECK(false);
} }
// Idle threads may have destroyed themselves, attempt to join
// them here
void ThreadPoolExecutor::ensureJoined() {
auto tojoin = threadsToJoin_.load(std::memory_order_relaxed);
if (tojoin) {
tojoin = threadsToJoin_.exchange(0, std::memory_order_relaxed);
joinStoppedThreads(tojoin);
}
}
// If we can't ensure that we were able to hand off a task to a thread,
// attempt to start a thread that handled the task, if we aren't already
// running the maximum number of threads.
void ThreadPoolExecutor::ensureActiveThreads() {
ensureJoined();
// Fast path assuming we are already at max threads.
auto active = activeThreads_.load(std::memory_order_relaxed);
auto total = maxThreads_.load(std::memory_order_relaxed);
if (active >= total) {
return;
}
RWSpinLock::WriteHolder w{&threadListLock_};
// Double check behind lock.
active = activeThreads_.load(std::memory_order_relaxed);
total = maxThreads_.load(std::memory_order_relaxed);
if (active >= total) {
return;
}
ThreadPoolExecutor::addThreads(1);
activeThreads_.store(active + 1, std::memory_order_relaxed);
}
// If an idle thread times out, only join it if there are at least
// minThreads threads.
bool ThreadPoolExecutor::minActive() {
return activeThreads_.load(std::memory_order_relaxed) >
minThreads_.load(std::memory_order_relaxed);
}
} // namespace folly } // namespace folly
...@@ -31,10 +31,31 @@ ...@@ -31,10 +31,31 @@
namespace folly { namespace folly {
/* Base class for implementing threadpool based executors.
*
* Dynamic thread behavior:
*
* ThreadPoolExecutors may vary their actual running number of threads
* between minThreads_ and maxThreads_, tracked by activeThreads_.
* The actual implementation of joining an idle thread is left to the
* ThreadPoolExecutors' subclass (typically by LifoSem try_take_for
* timing out). Idle threads should be removed from threadList_, and
* threadsToJoin incremented, and activeThreads_ decremented.
*
* On task add(), if an executor can garantee there is an active
* thread that will handle the task, then nothing needs to be done.
* If not, then ensureActiveThreads() should be called to possibly
* start another pool thread, up to maxThreads_.
*
* ensureJoined() is called on add(), such that we can join idle
* threads that were destroyed (which can't be joined from
* themselves).
*/
class ThreadPoolExecutor : public virtual folly::Executor { class ThreadPoolExecutor : public virtual folly::Executor {
public: public:
explicit ThreadPoolExecutor( explicit ThreadPoolExecutor(
size_t numThreads, size_t maxThreads,
size_t minThreads,
std::shared_ptr<ThreadFactory> threadFactory, std::shared_ptr<ThreadFactory> threadFactory,
bool isWaitForAll = false); bool isWaitForAll = false);
...@@ -127,6 +148,10 @@ class ThreadPoolExecutor : public virtual folly::Executor { ...@@ -127,6 +148,10 @@ class ThreadPoolExecutor : public virtual folly::Executor {
void addObserver(std::shared_ptr<Observer>); void addObserver(std::shared_ptr<Observer>);
void removeObserver(std::shared_ptr<Observer>); void removeObserver(std::shared_ptr<Observer>);
void setThreadDeathTimeout(std::chrono::milliseconds timeout) {
threadTimeout_ = timeout;
}
protected: protected:
// Prerequisite: threadListLock_ writelocked // Prerequisite: threadListLock_ writelocked
void addThreads(size_t n); void addThreads(size_t n);
...@@ -259,7 +284,7 @@ class ThreadPoolExecutor : public virtual folly::Executor { ...@@ -259,7 +284,7 @@ class ThreadPoolExecutor : public virtual folly::Executor {
ThreadList threadList_; ThreadList threadList_;
folly::RWSpinLock threadListLock_; folly::RWSpinLock threadListLock_;
StoppedThreadQueue stoppedThreads_; StoppedThreadQueue stoppedThreads_;
std::atomic<bool> isJoin_; // whether the current downsizing is a join std::atomic<bool> isJoin_{false}; // whether the current downsizing is a join
struct TaskStatsCallbackRegistry { struct TaskStatsCallbackRegistry {
folly::ThreadLocal<bool> inCallback; folly::ThreadLocal<bool> inCallback;
...@@ -268,6 +293,20 @@ class ThreadPoolExecutor : public virtual folly::Executor { ...@@ -268,6 +293,20 @@ class ThreadPoolExecutor : public virtual folly::Executor {
std::shared_ptr<TaskStatsCallbackRegistry> taskStatsCallbacks_; std::shared_ptr<TaskStatsCallbackRegistry> taskStatsCallbacks_;
std::vector<std::shared_ptr<Observer>> observers_; std::vector<std::shared_ptr<Observer>> observers_;
folly::ThreadPoolListHook threadPoolHook_; folly::ThreadPoolListHook threadPoolHook_;
// Dynamic thread sizing functions and variables
void ensureActiveThreads();
void ensureJoined();
bool minActive();
// These are only modified while holding threadListLock_, but
// are read without holding the lock.
std::atomic<size_t> maxThreads_{0};
std::atomic<size_t> minThreads_{0};
std::atomic<size_t> activeThreads_{0};
std::atomic<size_t> threadsToJoin_{0};
std::chrono::milliseconds threadTimeout_{0};
}; };
} // namespace folly } // namespace folly
...@@ -160,8 +160,8 @@ static void poolStats() { ...@@ -160,8 +160,8 @@ static void poolStats() {
folly::Baton<> startBaton, endBaton; folly::Baton<> startBaton, endBaton;
TPE tpe(1); TPE tpe(1);
auto stats = tpe.getPoolStats(); auto stats = tpe.getPoolStats();
EXPECT_EQ(1, stats.threadCount); EXPECT_GE(1, stats.threadCount);
EXPECT_EQ(1, stats.idleThreadCount); EXPECT_GE(1, stats.idleThreadCount);
EXPECT_EQ(0, stats.activeThreadCount); EXPECT_EQ(0, stats.activeThreadCount);
EXPECT_EQ(0, stats.pendingTaskCount); EXPECT_EQ(0, stats.pendingTaskCount);
EXPECT_EQ(0, tpe.getPendingTaskCount()); EXPECT_EQ(0, tpe.getPendingTaskCount());
...@@ -617,7 +617,7 @@ static void resizeThreadWhileExecutingTest() { ...@@ -617,7 +617,7 @@ static void resizeThreadWhileExecutingTest() {
EXPECT_EQ(5, tpe.numThreads()); EXPECT_EQ(5, tpe.numThreads());
tpe.setNumThreads(15); tpe.setNumThreads(15);
EXPECT_EQ(15, tpe.numThreads()); EXPECT_EQ(15, tpe.numThreads());
tpe.stop(); tpe.join();
EXPECT_EQ(1000, completed); EXPECT_EQ(1000, completed);
} }
...@@ -695,3 +695,16 @@ TEST(ThreadPoolExecutorTest, testUsesNameFromNamedThreadFactoryIO) { ...@@ -695,3 +695,16 @@ TEST(ThreadPoolExecutorTest, testUsesNameFromNamedThreadFactoryIO) {
TEST(ThreadPoolExecutorTest, testUsesNameFromNamedThreadFactoryCPU) { TEST(ThreadPoolExecutorTest, testUsesNameFromNamedThreadFactoryCPU) {
testUsesNameFromNamedThreadFactory<CPUThreadPoolExecutor>(); testUsesNameFromNamedThreadFactory<CPUThreadPoolExecutor>();
} }
TEST(ThreadPoolExecutorTest, DynamicThreadsTest) {
CPUThreadPoolExecutor e(2);
e.setThreadDeathTimeout(std::chrono::milliseconds(100));
e.add([] { /* sleep override */ usleep(1000); });
e.add([] { /* sleep override */ usleep(1000); });
auto stats = e.getPoolStats();
EXPECT_GE(2, stats.activeThreadCount);
/* sleep override */ sleep(1);
e.add([] {});
stats = e.getPoolStats();
EXPECT_LE(stats.activeThreadCount, 0);
}
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment