Commit f7548ce4 authored by Stepan Palamarchuk's avatar Stepan Palamarchuk Committed by Facebook Github Bot

Use HHWheelTimer in the FiberManager

Summary:
Currently, fibers library has its own class for managing timeouts - TimeoutController. It's optimized for the case when the set of timeouts is fixed and its size is not large. However, it's not efficient when that's not the case (see the benchmark). Since we're starting to use fibers in Thrift, we'll need to make sure that the underlying timeouts management is efficient for unbounded cases.

This kills `TimeoutController` and switches FiberManager to use `HHWheelTimer` from the underlying LoopController (which is in general EventBase).

One important note is that `HHWheelTimer` is **not exact** it rounds up timeouts to the 10ms boundary, so this **will slightly change the behavior**.

The results of FibersBenchmark before & after:

Before:
```
============================================================================
folly/fibers/test/FibersBenchmark.cpp           relative  time/iter  iters/s
============================================================================
FiberManagerCancelledTimeouts_Single_300                    16.34ms    61.18
FiberManagerCancelledTimeouts_Five                          17.54ms    57.00
FiberManagerCancelledTimeouts_TenThousand                  252.06ms     3.97
============================================================================
```

After:
```
============================================================================
folly/fibers/test/FibersBenchmark.cpp           relative  time/iter  iters/s
============================================================================
FiberManagerCancelledTimeouts_Single_300                    22.75ms    43.95
FiberManagerCancelledTimeouts_Five                          21.12ms    47.34
FiberManagerCancelledTimeouts_TenThousand                   19.13ms    52.27
============================================================================
```

This shows that with HHWheelTimer, throughput is unaffected by the number of different values used.

Reviewed By: andriigrynenko

Differential Revision: D13613830

fbshipit-source-id: 7b4662d308a9e1ef232672338a78b79efba46172
parent 3364055a
...@@ -76,31 +76,22 @@ bool Baton::try_wait_for( ...@@ -76,31 +76,22 @@ bool Baton::try_wait_for(
const std::chrono::duration<Rep, Period>& timeout, const std::chrono::duration<Rep, Period>& timeout,
F&& mainContextFunc) { F&& mainContextFunc) {
auto fm = FiberManager::getFiberManagerUnsafe(); auto fm = FiberManager::getFiberManagerUnsafe();
auto timeoutMs =
std::chrono::duration_cast<std::chrono::milliseconds>(timeout);
if (!fm || !fm->activeFiber_) { if (!fm || !fm->activeFiber_) {
mainContextFunc(); mainContextFunc();
return timedWaitThread(timeout); return timedWaitThread(timeoutMs);
} }
auto& baton = *this; auto timeoutFunc = [this]() mutable { this->postHelper(TIMEOUT); };
bool canceled = false; TimeoutHandler handler;
auto timeoutFunc = [&baton, &canceled]() mutable { handler.timeoutFunc_ = std::ref(timeoutFunc);
baton.postHelper(TIMEOUT);
canceled = true;
};
auto id =
fm->timeoutManager_->registerTimeout(std::ref(timeoutFunc), timeout);
fm->loopController_->timer().scheduleTimeout(&handler, timeoutMs);
waitFiber(*fm, static_cast<F&&>(mainContextFunc)); waitFiber(*fm, static_cast<F&&>(mainContextFunc));
auto posted = waiter_ == POSTED; return waiter_ == POSTED;
if (!canceled) {
fm->timeoutManager_->cancel(id);
}
return posted;
} }
template <typename Clock, typename Duration, typename F> template <typename Clock, typename Duration, typename F>
......
...@@ -47,11 +47,10 @@ void Baton::wait() { ...@@ -47,11 +47,10 @@ void Baton::wait() {
} }
void Baton::wait(TimeoutHandler& timeoutHandler) { void Baton::wait(TimeoutHandler& timeoutHandler) {
auto timeoutFunc = [this, &timeoutHandler] { auto timeoutFunc = [this] {
if (!try_wait()) { if (!try_wait()) {
postHelper(TIMEOUT); postHelper(TIMEOUT);
} }
timeoutHandler.timeoutPtr_ = 0;
}; };
timeoutHandler.timeoutFunc_ = std::ref(timeoutFunc); timeoutHandler.timeoutFunc_ = std::ref(timeoutFunc);
timeoutHandler.fiberManager_ = FiberManager::getFiberManagerUnsafe(); timeoutHandler.fiberManager_ = FiberManager::getFiberManagerUnsafe();
...@@ -110,7 +109,7 @@ bool Baton::spinWaitForEarlyPost() { ...@@ -110,7 +109,7 @@ bool Baton::spinWaitForEarlyPost() {
return false; return false;
} }
bool Baton::timedWaitThread(TimeoutController::Duration timeout) { bool Baton::timedWaitThread(std::chrono::milliseconds timeout) {
if (spinWaitForEarlyPost()) { if (spinWaitForEarlyPost()) {
assert(waiter_.load(std::memory_order_acquire) == POSTED); assert(waiter_.load(std::memory_order_acquire) == POSTED);
return true; return true;
...@@ -121,7 +120,7 @@ bool Baton::timedWaitThread(TimeoutController::Duration timeout) { ...@@ -121,7 +120,7 @@ bool Baton::timedWaitThread(TimeoutController::Duration timeout) {
if (LIKELY( if (LIKELY(
waiter == NO_WAITER && waiter == NO_WAITER &&
waiter_.compare_exchange_strong(waiter, THREAD_WAITING))) { waiter_.compare_exchange_strong(waiter, THREAD_WAITING))) {
auto deadline = TimeoutController::Clock::now() + timeout; auto deadline = std::chrono::steady_clock::now() + timeout;
do { do {
auto* futex = &futex_.futex; auto* futex = &futex_.futex;
const auto wait_rv = const auto wait_rv =
...@@ -189,22 +188,14 @@ void Baton::reset() { ...@@ -189,22 +188,14 @@ void Baton::reset() {
waiter_.store(NO_WAITER, std::memory_order_relaxed); waiter_.store(NO_WAITER, std::memory_order_relaxed);
} }
void Baton::TimeoutHandler::scheduleTimeout( void Baton::TimeoutHandler::scheduleTimeout(std::chrono::milliseconds timeout) {
TimeoutController::Duration timeout) {
assert(fiberManager_ != nullptr); assert(fiberManager_ != nullptr);
assert(timeoutFunc_ != nullptr); assert(timeoutFunc_ != nullptr);
assert(timeoutPtr_ == 0);
if (timeout.count() > 0) { if (timeout.count() > 0) {
timeoutPtr_ = fiberManager_->loopController_->timer().scheduleTimeout(this, timeout);
fiberManager_->timeoutManager_->registerTimeout(timeoutFunc_, timeout);
} }
} }
void Baton::TimeoutHandler::cancelTimeout() {
if (timeoutPtr_) {
fiberManager_->timeoutManager_->cancel(timeoutPtr_);
}
}
} // namespace fibers } // namespace fibers
} // namespace folly } // namespace folly
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
#include <folly/Portability.h> #include <folly/Portability.h>
#include <folly/detail/Futex.h> #include <folly/detail/Futex.h>
#include <folly/fibers/TimeoutController.h> #include <folly/io/async/HHWheelTimer.h>
#if FOLLY_HAS_COROUTINES #if FOLLY_HAS_COROUTINES
#include <experimental/coroutine> #include <experimental/coroutine>
...@@ -215,19 +215,22 @@ class Baton { ...@@ -215,19 +215,22 @@ class Baton {
* scheduleTimeout() may only be called once prior to the end of the * scheduleTimeout() may only be called once prior to the end of the
* associated Baton's life. * associated Baton's life.
*/ */
class TimeoutHandler { class TimeoutHandler final : private HHWheelTimer::Callback {
public: public:
void scheduleTimeout(TimeoutController::Duration timeoutMs); void scheduleTimeout(std::chrono::milliseconds timeout);
private: private:
friend class Baton; friend class Baton;
void cancelTimeout();
std::function<void()> timeoutFunc_{nullptr}; std::function<void()> timeoutFunc_{nullptr};
FiberManager* fiberManager_{nullptr}; FiberManager* fiberManager_{nullptr};
intptr_t timeoutPtr_{0}; void timeoutExpired() noexcept override {
assert(timeoutFunc_ != nullptr);
timeoutFunc_();
}
void callbackCanceled() noexcept override {}
}; };
private: private:
...@@ -268,7 +271,7 @@ class Baton { ...@@ -268,7 +271,7 @@ class Baton {
*/ */
bool spinWaitForEarlyPost(); bool spinWaitForEarlyPost();
bool timedWaitThread(TimeoutController::Duration timeout); bool timedWaitThread(std::chrono::milliseconds timeout);
static constexpr intptr_t NO_WAITER = 0; static constexpr intptr_t NO_WAITER = 0;
static constexpr intptr_t POSTED = -1; static constexpr intptr_t POSTED = -1;
......
...@@ -104,19 +104,10 @@ inline void EventBaseLoopController::scheduleThreadSafe() { ...@@ -104,19 +104,10 @@ inline void EventBaseLoopController::scheduleThreadSafe() {
}); });
} }
inline void EventBaseLoopController::timedSchedule( inline HHWheelTimer& EventBaseLoopController::timer() {
std::function<void()> func,
TimePoint time) {
assert(eventBaseAttached_); assert(eventBaseAttached_);
// We want upper bound for the cast, thus we just add 1 return eventBase_->timer();
auto delay_ms =
std::chrono::duration_cast<std::chrono::milliseconds>(time - Clock::now())
.count() +
1;
// If clock is not monotonic
delay_ms = std::max<decltype(delay_ms)>(delay_ms, 0);
eventBase_->tryRunAfterDelay(func, uint32_t(delay_ms));
} }
} // namespace fibers } // namespace fibers
} // namespace folly } // namespace folly
...@@ -71,7 +71,7 @@ class EventBaseLoopController : public LoopController { ...@@ -71,7 +71,7 @@ class EventBaseLoopController : public LoopController {
void schedule() override; void schedule() override;
void runLoop() override; void runLoop() override;
void scheduleThreadSafe() override; void scheduleThreadSafe() override;
void timedSchedule(std::function<void()> func, TimePoint time) override; HHWheelTimer& timer() override;
friend class FiberManager; friend class FiberManager;
}; };
......
...@@ -58,9 +58,7 @@ inline void ExecutorLoopController::scheduleThreadSafe() { ...@@ -58,9 +58,7 @@ inline void ExecutorLoopController::scheduleThreadSafe() {
}); });
} }
inline void ExecutorLoopController::timedSchedule( inline HHWheelTimer& ExecutorLoopController::timer() {
std::function<void()>,
TimePoint) {
throw std::logic_error("Time schedule isn't supported by asyncio executor"); throw std::logic_error("Time schedule isn't supported by asyncio executor");
} }
......
...@@ -42,7 +42,7 @@ class ExecutorLoopController : public fibers::LoopController { ...@@ -42,7 +42,7 @@ class ExecutorLoopController : public fibers::LoopController {
void schedule() override; void schedule() override;
void runLoop() override; void runLoop() override;
void scheduleThreadSafe() override; void scheduleThreadSafe() override;
void timedSchedule(std::function<void()> func, TimePoint time) override; HHWheelTimer& timer() override;
friend class fibers::FiberManager; friend class fibers::FiberManager;
}; };
......
...@@ -105,7 +105,7 @@ Fiber* FiberManager::getFiber() { ...@@ -105,7 +105,7 @@ Fiber* FiberManager::getFiber() {
Fiber* fiber = nullptr; Fiber* fiber = nullptr;
if (options_.fibersPoolResizePeriodMs > 0 && !fibersPoolResizerScheduled_) { if (options_.fibersPoolResizePeriodMs > 0 && !fibersPoolResizerScheduled_) {
fibersPoolResizer_(); fibersPoolResizer_.run();
fibersPoolResizerScheduled_ = true; fibersPoolResizerScheduled_ = true;
} }
...@@ -181,10 +181,10 @@ void FiberManager::doFibersPoolResizing() { ...@@ -181,10 +181,10 @@ void FiberManager::doFibersPoolResizing() {
maxFibersActiveLastPeriod_ = fibersActive_; maxFibersActiveLastPeriod_ = fibersActive_;
} }
void FiberManager::FibersPoolResizer::operator()() { void FiberManager::FibersPoolResizer::run() {
fiberManager_.doFibersPoolResizing(); fiberManager_.doFibersPoolResizing();
fiberManager_.timeoutManager_->registerTimeout( fiberManager_.loopController_->timer().scheduleTimeout(
*this, this,
std::chrono::milliseconds( std::chrono::milliseconds(
fiberManager_.options_.fibersPoolResizePeriodMs)); fiberManager_.options_.fibersPoolResizePeriodMs));
} }
......
...@@ -591,7 +591,6 @@ FiberManager::FiberManager( ...@@ -591,7 +591,6 @@ FiberManager::FiberManager(
<< "context '" << context << "'"; << "context '" << context << "'";
} }
}), }),
timeoutManager_(std::make_shared<TimeoutController>(*loopController_)),
fibersPoolResizer_(*this), fibersPoolResizer_(*this),
localType_(typeid(LocalT)) { localType_(typeid(LocalT)) {
loopController_->setFiberManager(this); loopController_->setFiberManager(this);
......
...@@ -31,13 +31,14 @@ ...@@ -31,13 +31,14 @@
#include <folly/Likely.h> #include <folly/Likely.h>
#include <folly/Try.h> #include <folly/Try.h>
#include <folly/functional/Invoke.h> #include <folly/functional/Invoke.h>
#include <folly/io/async/HHWheelTimer.h>
#include <folly/io/async/Request.h> #include <folly/io/async/Request.h>
#include <folly/experimental/ExecutionObserver.h> #include <folly/experimental/ExecutionObserver.h>
#include <folly/fibers/BoostContextCompatibility.h> #include <folly/fibers/BoostContextCompatibility.h>
#include <folly/fibers/Fiber.h> #include <folly/fibers/Fiber.h>
#include <folly/fibers/GuardPageAllocator.h> #include <folly/fibers/GuardPageAllocator.h>
#include <folly/fibers/TimeoutController.h> #include <folly/fibers/LoopController.h>
#include <folly/fibers/traits.h> #include <folly/fibers/traits.h>
namespace folly { namespace folly {
...@@ -49,8 +50,6 @@ namespace fibers { ...@@ -49,8 +50,6 @@ namespace fibers {
class Baton; class Baton;
class Fiber; class Fiber;
class LoopController;
class TimeoutController;
template <typename T> template <typename T>
class LocalType {}; class LocalType {};
...@@ -513,14 +512,17 @@ class FiberManager : public ::folly::Executor { ...@@ -513,14 +512,17 @@ class FiberManager : public ::folly::Executor {
ssize_t remoteCount_{0}; ssize_t remoteCount_{0};
std::shared_ptr<TimeoutController> timeoutManager_; class FibersPoolResizer final : private HHWheelTimer::Callback {
public:
struct FibersPoolResizer {
explicit FibersPoolResizer(FiberManager& fm) : fiberManager_(fm) {} explicit FibersPoolResizer(FiberManager& fm) : fiberManager_(fm) {}
void operator()(); void run();
private: private:
FiberManager& fiberManager_; FiberManager& fiberManager_;
void timeoutExpired() noexcept {
run();
}
void callbackCanceled() noexcept {}
}; };
FibersPoolResizer fibersPoolResizer_; FibersPoolResizer fibersPoolResizer_;
......
...@@ -19,6 +19,9 @@ ...@@ -19,6 +19,9 @@
#include <functional> #include <functional>
namespace folly { namespace folly {
class HHWheelTimer;
namespace fibers { namespace fibers {
class FiberManager; class FiberManager;
...@@ -53,9 +56,9 @@ class LoopController { ...@@ -53,9 +56,9 @@ class LoopController {
virtual void scheduleThreadSafe() = 0; virtual void scheduleThreadSafe() = 0;
/** /**
* Called by FiberManager to schedule some function to be run at some time. * Used by FiberManager to schedule some function to be run at some time.
*/ */
virtual void timedSchedule(std::function<void()> func, TimePoint time) = 0; virtual HHWheelTimer& timer() = 0;
}; };
} // namespace fibers } // namespace fibers
} // namespace folly } // namespace folly
/*
* Copyright 2004-present Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <folly/fibers/SimpleLoopController.h>
#include <folly/io/async/TimeoutManager.h>
namespace folly {
namespace fibers {
/**
* A simple version of TimeoutManager that maintains only a single AsyncTimeout
* object that is used by HHWheelTimer in SimpleLoopController.
*/
class SimpleLoopController::SimpleTimeoutManager : public TimeoutManager {
public:
explicit SimpleTimeoutManager(SimpleLoopController& loopController)
: loopController_(loopController) {}
void attachTimeoutManager(
AsyncTimeout* /* unused */,
InternalEnum /* unused */) final {}
void detachTimeoutManager(AsyncTimeout* /* unused */) final {}
bool scheduleTimeout(AsyncTimeout* obj, timeout_type timeout) final {
// Make sure that we don't try to use this manager with two timeouts.
CHECK(!timeout_ || timeout_->first == obj);
timeout_.emplace(obj, std::chrono::steady_clock::now() + timeout);
return true;
}
void cancelTimeout(AsyncTimeout* obj) final {
CHECK(timeout_ && timeout_->first == obj);
timeout_.clear();
}
void bumpHandlingTime() final {}
bool isInTimeoutManagerThread() final {
return loopController_.isInLoopThread();
}
void runTimeouts() {
std::chrono::steady_clock::time_point tp = std::chrono::steady_clock::now();
if (!timeout_ || tp < timeout_->second) {
return;
}
auto* timeout = timeout_->first;
timeout_.clear();
timeout->timeoutExpired();
}
private:
SimpleLoopController& loopController_;
folly::Optional<
std::pair<AsyncTimeout*, std::chrono::steady_clock::time_point>>
timeout_;
};
SimpleLoopController::SimpleLoopController()
: fm_(nullptr),
stopRequested_(false),
loopThread_(),
timeoutManager_(std::make_unique<SimpleTimeoutManager>(*this)),
timer_(HHWheelTimer::newTimer(timeoutManager_.get())) {}
SimpleLoopController::~SimpleLoopController() {
scheduled_ = false;
}
void SimpleLoopController::runTimeouts() {
timeoutManager_->runTimeouts();
}
} // namespace fibers
} // namespace folly
...@@ -30,15 +30,8 @@ class FiberManager; ...@@ -30,15 +30,8 @@ class FiberManager;
class SimpleLoopController : public LoopController { class SimpleLoopController : public LoopController {
public: public:
SimpleLoopController() : fm_(nullptr), stopRequested_(false) {} SimpleLoopController();
~SimpleLoopController();
~SimpleLoopController() {
scheduled_ = false;
}
void setTimeFunc(Function<TimePoint()> timeFunc) {
timeFunc_ = std::move(timeFunc);
}
/** /**
* Run FiberManager loop; if no ready task are present, * Run FiberManager loop; if no ready task are present,
...@@ -47,29 +40,22 @@ class SimpleLoopController : public LoopController { ...@@ -47,29 +40,22 @@ class SimpleLoopController : public LoopController {
*/ */
template <typename F> template <typename F>
void loop(F&& func) { void loop(F&& func) {
loopThread_.store(std::this_thread::get_id(), std::memory_order_release);
bool waiting = false; bool waiting = false;
stopRequested_ = false; stopRequested_ = false;
while (LIKELY(waiting || !stopRequested_)) { while (LIKELY(waiting || !stopRequested_)) {
func(); func();
runTimeouts();
auto time = timeFunc_();
for (size_t i = 0; i < scheduledFuncs_.size(); ++i) {
if (scheduledFuncs_[i].first <= time) {
scheduledFuncs_[i].second();
swap(scheduledFuncs_[i], scheduledFuncs_.back());
scheduledFuncs_.pop_back();
--i;
}
}
if (scheduled_) { if (scheduled_) {
scheduled_ = false; scheduled_ = false;
runLoop(); runLoop();
waiting = fm_->hasTasks(); waiting = fm_->hasTasks();
} }
} }
loopThread_.store({}, std::memory_order_release);
} }
/** /**
...@@ -101,8 +87,13 @@ class SimpleLoopController : public LoopController { ...@@ -101,8 +87,13 @@ class SimpleLoopController : public LoopController {
scheduled_ = true; scheduled_ = true;
} }
void timedSchedule(std::function<void()> func, TimePoint time) override { HHWheelTimer& timer() override {
scheduledFuncs_.emplace_back(time, std::move(func)); return *timer_;
}
bool isInLoopThread() const {
auto tid = loopThread_.load(std::memory_order_relaxed);
return tid == std::thread::id() || tid == std::this_thread::get_id();
} }
private: private:
...@@ -111,8 +102,11 @@ class SimpleLoopController : public LoopController { ...@@ -111,8 +102,11 @@ class SimpleLoopController : public LoopController {
bool stopRequested_; bool stopRequested_;
std::atomic<int> remoteScheduleCalled_{0}; std::atomic<int> remoteScheduleCalled_{0};
int remoteLoopRun_{0}; int remoteLoopRun_{0};
std::vector<std::pair<TimePoint, std::function<void()>>> scheduledFuncs_; std::atomic<std::thread::id> loopThread_;
Function<TimePoint()> timeFunc_{[] { return Clock::now(); }};
class SimpleTimeoutManager;
std::unique_ptr<SimpleTimeoutManager> timeoutManager_;
std::shared_ptr<HHWheelTimer> timer_;
/* LoopController interface */ /* LoopController interface */
...@@ -125,6 +119,8 @@ class SimpleLoopController : public LoopController { ...@@ -125,6 +119,8 @@ class SimpleLoopController : public LoopController {
scheduled_ = true; scheduled_ = true;
} }
void runTimeouts();
friend class FiberManager; friend class FiberManager;
}; };
} // namespace fibers } // namespace fibers
......
/*
* Copyright 2014-present Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <folly/fibers/TimeoutController.h>
#include <folly/Memory.h>
namespace folly {
namespace fibers {
TimeoutController::TimeoutController(LoopController& loopController)
: nextTimeout_(TimePoint::max()), loopController_(loopController) {}
intptr_t TimeoutController::registerTimeout(
std::function<void()> f,
Duration duration) {
auto& list = [&]() -> TimeoutHandleList& {
for (auto& bucket : timeoutHandleBuckets_) {
if (bucket.first == duration) {
return *bucket.second;
}
}
timeoutHandleBuckets_.emplace_back(
duration, std::make_unique<TimeoutHandleList>());
return *timeoutHandleBuckets_.back().second;
}();
auto timeout = Clock::now() + duration;
list.emplace(std::move(f), timeout, list);
if (timeout < nextTimeout_) {
nextTimeout_ = timeout;
scheduleRun();
}
return reinterpret_cast<intptr_t>(&list.back());
}
void TimeoutController::runTimeouts(TimePoint time) {
auto now = Clock::now();
// Make sure we don't skip some events if function was run before actual time.
if (time < now) {
time = now;
}
if (nextTimeout_ > time) {
return;
}
nextTimeout_ = TimePoint::max();
for (auto& bucket : timeoutHandleBuckets_) {
auto& list = *bucket.second;
while (!list.empty()) {
if (!list.front().canceled) {
if (list.front().timeout > time) {
nextTimeout_ = std::min(nextTimeout_, list.front().timeout);
break;
}
list.front().func();
}
list.pop();
}
}
if (nextTimeout_ != TimePoint::max()) {
scheduleRun();
}
}
void TimeoutController::scheduleRun() {
auto time = nextTimeout_;
std::weak_ptr<TimeoutController> timeoutControllerWeak = shared_from_this();
loopController_.timedSchedule(
[timeoutControllerWeak, time]() {
if (auto timeoutController = timeoutControllerWeak.lock()) {
timeoutController->runTimeouts(time);
}
},
time);
}
void TimeoutController::cancel(intptr_t p) {
auto handle = reinterpret_cast<TimeoutHandle*>(p);
handle->canceled = true;
auto& list = handle->list;
while (!list.empty() && list.front().canceled) {
list.pop();
}
}
} // namespace fibers
} // namespace folly
/*
* Copyright 2014-present Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#include <chrono>
#include <functional>
#include <memory>
#include <queue>
#include <boost/intrusive/list.hpp>
#include <folly/Likely.h>
#include <folly/fibers/LoopController.h>
namespace folly {
namespace fibers {
class TimeoutController
: public std::enable_shared_from_this<TimeoutController> {
public:
typedef std::chrono::steady_clock Clock;
typedef std::chrono::time_point<Clock> TimePoint;
typedef Clock::duration Duration;
explicit TimeoutController(LoopController& loopController);
intptr_t registerTimeout(std::function<void()> f, Duration duration);
void cancel(intptr_t id);
void runTimeouts(TimePoint time);
private:
void scheduleRun();
struct TimeoutHandle;
typedef std::queue<TimeoutHandle> TimeoutHandleList;
typedef std::unique_ptr<TimeoutHandleList> TimeoutHandleListPtr;
struct TimeoutHandle {
TimeoutHandle(
std::function<void()> func_,
TimePoint timeout_,
TimeoutHandleList& list_)
: func(std::move(func_)), timeout(timeout_), list(list_) {}
std::function<void()> func;
bool canceled{false};
TimePoint timeout;
TimeoutHandleList& list;
};
std::vector<std::pair<Duration, TimeoutHandleListPtr>> timeoutHandleBuckets_;
TimePoint nextTimeout_;
LoopController& loopController_;
};
} // namespace fibers
} // namespace folly
...@@ -42,41 +42,31 @@ using folly::Try; ...@@ -42,41 +42,31 @@ using folly::Try;
TEST(FiberManager, batonTimedWaitTimeout) { TEST(FiberManager, batonTimedWaitTimeout) {
bool taskAdded = false; bool taskAdded = false;
size_t iterations = 0;
FiberManager manager(std::make_unique<SimpleLoopController>()); FiberManager manager(std::make_unique<SimpleLoopController>());
auto& loopController = auto& loopController =
dynamic_cast<SimpleLoopController&>(manager.loopController()); dynamic_cast<SimpleLoopController&>(manager.loopController());
std::chrono::steady_clock::time_point start;
auto now = SimpleLoopController::Clock::now();
loopController.setTimeFunc([&] { return now; });
auto loopFunc = [&]() { auto loopFunc = [&]() {
if (!taskAdded) { if (!taskAdded) {
manager.addTask([&]() { manager.addTask([&]() {
Baton baton; Baton baton;
auto res = baton.try_wait_for(std::chrono::milliseconds(230)); start = std::chrono::steady_clock::now();
constexpr auto kTimeout = std::chrono::milliseconds(230);
auto res = baton.try_wait_for(kTimeout);
auto elapsedMs = std::chrono::duration_cast<std::chrono::milliseconds>(
std::chrono::steady_clock::now() - start);
EXPECT_FALSE(res); EXPECT_FALSE(res);
EXPECT_EQ(5, iterations); EXPECT_LE(kTimeout, elapsedMs);
loopController.stop();
});
manager.addTask([&]() {
Baton baton;
auto res = baton.try_wait_for(std::chrono::milliseconds(130));
EXPECT_FALSE(res);
EXPECT_EQ(3, iterations);
loopController.stop(); loopController.stop();
}); });
taskAdded = true; taskAdded = true;
} else { } else {
now += std::chrono::milliseconds(50); std::this_thread::sleep_for(std::chrono::milliseconds(50));
iterations++;
} }
}; };
......
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