Commit ffff09dd authored by Jon Maltiel Swenson's avatar Jon Maltiel Swenson Committed by Facebook Github Bot

Make EventBase destruction callbacks safely cancellable

Summary: Currently, `runOnDestruction` aims to be thread-safe; new callbacks are added to the `onDestructionCallbacks_` list while the associated mutex is held. However, the caller may own the `LoopCallback` and wish to destroy/cancel it before the `EventBase` destructor runs, and this callback cancellation is not thread-safe, since unlinking does not happen under the lock protecting `onDestructionCallbacks_`. The primary motivation of this diff is to make on-destruction callback cancellation thread-safe; in particular, it is safe to cancel an on-destruction callback concurrently with `~EventBase()`.

Reviewed By: spalamarchuk

Differential Revision: D13440552

fbshipit-source-id: 65cee1e361d37647920baaad4490dd26b791315d
parent fd241ffa
...@@ -18,6 +18,7 @@ ...@@ -18,6 +18,7 @@
#include <memory> #include <memory>
#include <unordered_map> #include <unordered_map>
#include <folly/Function.h>
#include <folly/Synchronized.h> #include <folly/Synchronized.h>
#include <folly/ThreadLocal.h> #include <folly/ThreadLocal.h>
...@@ -27,14 +28,7 @@ namespace fibers { ...@@ -27,14 +28,7 @@ namespace fibers {
namespace { namespace {
template <typename EventBaseT> template <typename EventBaseT>
class EventBaseOnDestructionCallback : public EventBase::LoopCallback { Function<void()> makeOnEventBaseDestructionCallback(EventBaseT& evb);
public:
explicit EventBaseOnDestructionCallback(EventBaseT& evb) : evb_(evb) {}
void runLoopCallback() noexcept override;
private:
EventBaseT& evb_;
};
template <typename EventBaseT> template <typename EventBaseT>
class GlobalCache { class GlobalCache {
...@@ -65,8 +59,7 @@ class GlobalCache { ...@@ -65,8 +59,7 @@ class GlobalCache {
if (!fmPtrRef) { if (!fmPtrRef) {
auto loopController = std::make_unique<EventBaseLoopController>(); auto loopController = std::make_unique<EventBaseLoopController>();
loopController->attachEventBase(evb); loopController->attachEventBase(evb);
evb.runOnDestruction(new EventBaseOnDestructionCallback<EventBaseT>(evb)); evb.runOnDestruction(makeOnEventBaseDestructionCallback(evb));
fmPtrRef = fmPtrRef =
std::make_unique<FiberManager>(std::move(loopController), opts); std::make_unique<FiberManager>(std::move(loopController), opts);
} }
...@@ -170,12 +163,12 @@ class ThreadLocalCache { ...@@ -170,12 +163,12 @@ class ThreadLocalCache {
}; };
template <typename EventBaseT> template <typename EventBaseT>
void EventBaseOnDestructionCallback<EventBaseT>::runLoopCallback() noexcept { Function<void()> makeOnEventBaseDestructionCallback(EventBaseT& evb) {
auto fm = GlobalCache<EventBaseT>::erase(evb_); return [&evb] {
auto fm = GlobalCache<EventBaseT>::erase(evb);
DCHECK(fm.get() != nullptr); DCHECK(fm.get() != nullptr);
ThreadLocalCache<EventBaseT>::erase(evb_); ThreadLocalCache<EventBaseT>::erase(evb);
};
delete this;
} }
} // namespace } // namespace
......
...@@ -164,10 +164,14 @@ EventBase::~EventBase() { ...@@ -164,10 +164,14 @@ EventBase::~EventBase() {
} }
// Call all destruction callbacks, before we start cleaning up our state. // Call all destruction callbacks, before we start cleaning up our state.
while (!onDestructionCallbacks_.empty()) { while (!onDestructionCallbacks_.rlock()->empty()) {
LoopCallback* callback = &onDestructionCallbacks_.front(); OnDestructionCallback::List callbacks;
onDestructionCallbacks_.pop_front(); onDestructionCallbacks_.swap(callbacks);
callback->runLoopCallback(); while (!callbacks.empty()) {
auto& callback = callbacks.front();
callbacks.pop_front();
callback.runCallback();
}
} }
clearCobTimeouts(); clearCobTimeouts();
...@@ -541,10 +545,18 @@ void EventBase::runInLoop(Func cob, bool thisIteration) { ...@@ -541,10 +545,18 @@ void EventBase::runInLoop(Func cob, bool thisIteration) {
} }
} }
void EventBase::runOnDestruction(LoopCallback* callback) { void EventBase::runOnDestruction(OnDestructionCallback& callback) {
std::lock_guard<std::mutex> lg(onDestructionCallbacksMutex_); callback.schedule(
callback->cancelLoopCallback(); [this](auto& cb) { onDestructionCallbacks_.wlock()->push_back(cb); },
onDestructionCallbacks_.push_back(*callback); [this](auto& cb) {
onDestructionCallbacks_.withWLock(
[&](auto& list) { list.erase(list.iterator_to(cb)); });
});
}
void EventBase::runOnDestruction(Func f) {
auto* callback = new FunctionOnDestructionCallback(std::move(f));
runOnDestruction(*callback);
} }
void EventBase::runBeforeLoop(LoopCallback* callback) { void EventBase::runBeforeLoop(LoopCallback* callback) {
...@@ -801,5 +813,49 @@ EventBase* EventBase::getEventBase() { ...@@ -801,5 +813,49 @@ EventBase* EventBase::getEventBase() {
return this; return this;
} }
EventBase::OnDestructionCallback::~OnDestructionCallback() {
if (*scheduled_.rlock()) {
LOG(FATAL)
<< "OnDestructionCallback must be canceled if needed prior to destruction";
}
}
void EventBase::OnDestructionCallback::runCallback() noexcept {
scheduled_.withWLock([&](bool& scheduled) {
CHECK(scheduled);
scheduled = false;
// run can only be called by EventBase and VirtualEventBase, and it's called
// after the callback has been popped off the list.
eraser_ = nullptr;
// Note that the exclusive lock on shared state is held while the callback
// runs. This ensures concurrent callers to cancel() block until the
// callback finishes.
onEventBaseDestruction();
});
}
void EventBase::OnDestructionCallback::schedule(
FunctionRef<void(OnDestructionCallback&)> linker,
Function<void(OnDestructionCallback&)> eraser) {
eraser_ = std::move(eraser);
scheduled_.withWLock([](bool& scheduled) { scheduled = true; });
linker(*this);
}
bool EventBase::OnDestructionCallback::cancel() {
return scheduled_.withWLock([this](bool& scheduled) {
const bool wasScheduled = std::exchange(scheduled, false);
if (wasScheduled) {
auto eraser = std::move(eraser_);
CHECK(eraser);
eraser(*this);
}
return wasScheduled;
});
}
constexpr std::chrono::milliseconds EventBase::SmoothLoopTime::buffer_interval_; constexpr std::chrono::milliseconds EventBase::SmoothLoopTime::buffer_interval_;
} // namespace folly } // namespace folly
...@@ -22,7 +22,6 @@ ...@@ -22,7 +22,6 @@
#include <functional> #include <functional>
#include <list> #include <list>
#include <memory> #include <memory>
#include <mutex>
#include <queue> #include <queue>
#include <set> #include <set>
#include <stack> #include <stack>
...@@ -37,6 +36,7 @@ ...@@ -37,6 +36,7 @@
#include <folly/Function.h> #include <folly/Function.h>
#include <folly/Portability.h> #include <folly/Portability.h>
#include <folly/ScopeGuard.h> #include <folly/ScopeGuard.h>
#include <folly/Synchronized.h>
#include <folly/executors/DrivableExecutor.h> #include <folly/executors/DrivableExecutor.h>
#include <folly/executors/IOExecutor.h> #include <folly/executors/IOExecutor.h>
#include <folly/executors/ScheduledExecutor.h> #include <folly/executors/ScheduledExecutor.h>
...@@ -202,6 +202,79 @@ class EventBase : private boost::noncopyable, ...@@ -202,6 +202,79 @@ class EventBase : private boost::noncopyable,
Func function_; Func function_;
}; };
// Base class for user callbacks to be run during EventBase destruction. As
// with LoopCallback, users may inherit from this class and provide a concrete
// implementation of onEventBaseDestruction(). (Alternatively, users may use
// the convenience method EventBase::runOnDestruction(Function<void()> f) to
// schedule a function f to be run on EventBase destruction.)
//
// The only thread-safety guarantees of OnDestructionCallback are as follows:
// - Users may call runOnDestruction() from any thread, provided the caller
// is the only user of the callback, i.e., the callback is not already
// scheduled and there are no concurrent calls to schedule or cancel the
// callback.
// - Users may safely cancel() from any thread. Multiple calls to cancel()
// may execute concurrently. The only caveat is that it is not safe to
// call cancel() within the onEventBaseDestruction() callback.
class OnDestructionCallback {
public:
OnDestructionCallback() = default;
OnDestructionCallback(OnDestructionCallback&&) = default;
OnDestructionCallback& operator=(OnDestructionCallback&&) = default;
virtual ~OnDestructionCallback();
// Attempt to cancel the callback. If the callback is running or has already
// finished running, cancellation will fail. If the callback is running when
// cancel() is called, cancel() will block until the callback completes.
bool cancel();
// Callback to be invoked during ~EventBase()
virtual void onEventBaseDestruction() noexcept = 0;
private:
boost::intrusive::list_member_hook<
boost::intrusive::link_mode<boost::intrusive::normal_link>>
listHook_;
Function<void(OnDestructionCallback&)> eraser_;
Synchronized<bool> scheduled_{in_place, false};
using List = boost::intrusive::list<
OnDestructionCallback,
boost::intrusive::member_hook<
OnDestructionCallback,
decltype(listHook_),
&OnDestructionCallback::listHook_>>;
void schedule(
FunctionRef<void(OnDestructionCallback&)> linker,
Function<void(OnDestructionCallback&)> eraser);
friend class EventBase;
friend class VirtualEventBase;
protected:
virtual void runCallback() noexcept;
};
class FunctionOnDestructionCallback : public OnDestructionCallback {
public:
explicit FunctionOnDestructionCallback(Function<void()> f)
: f_(std::move(f)) {}
void onEventBaseDestruction() noexcept final {
f_();
}
protected:
void runCallback() noexcept override {
OnDestructionCallback::runCallback();
delete this;
}
private:
Function<void()> f_;
};
/** /**
* Create a new EventBase object. * Create a new EventBase object.
* *
...@@ -364,13 +437,19 @@ class EventBase : private boost::noncopyable, ...@@ -364,13 +437,19 @@ class EventBase : private boost::noncopyable,
* Adds the given callback to a queue of things run before destruction * Adds the given callback to a queue of things run before destruction
* of current EventBase. * of current EventBase.
* *
* This allows users of EventBase that run in it, but don't control it, * This allows users of EventBase that run in it, but don't control it, to be
* to be notified before EventBase gets destructed. * notified before EventBase gets destructed.
* *
* Note: will be called from the thread that invoked EventBase destructor, * Note: will be called from the thread that invoked EventBase destructor,
* before the final run of loop callbacks. * before the final run of loop callbacks.
*/ */
void runOnDestruction(LoopCallback* callback); void runOnDestruction(OnDestructionCallback& callback);
/**
* Convenience function that allows users to pass in a Function<void()> to be
* run on EventBase destruction.
*/
void runOnDestruction(Func f);
/** /**
* Adds a callback that will run immediately *before* the event loop. * Adds a callback that will run immediately *before* the event loop.
...@@ -757,7 +836,7 @@ class EventBase : private boost::noncopyable, ...@@ -757,7 +836,7 @@ class EventBase : private boost::noncopyable,
LoopCallbackList loopCallbacks_; LoopCallbackList loopCallbacks_;
LoopCallbackList runBeforeLoopCallbacks_; LoopCallbackList runBeforeLoopCallbacks_;
LoopCallbackList onDestructionCallbacks_; Synchronized<OnDestructionCallback::List> onDestructionCallbacks_;
// This will be null most of the time, but point to currentCallbacks // This will be null most of the time, but point to currentCallbacks
// if we are in the middle of running loop callbacks, such that // if we are in the middle of running loop callbacks, such that
...@@ -823,9 +902,6 @@ class EventBase : private boost::noncopyable, ...@@ -823,9 +902,6 @@ class EventBase : private boost::noncopyable,
// Name of the thread running this EventBase // Name of the thread running this EventBase
std::string name_; std::string name_;
// allow runOnDestruction() to be called from any threads
std::mutex onDestructionCallbacksMutex_;
// see EventBaseLocal // see EventBaseLocal
friend class detail::EventBaseLocalBase; friend class detail::EventBaseLocalBase;
template <typename T> template <typename T>
......
...@@ -40,8 +40,7 @@ static void run( ...@@ -40,8 +40,7 @@ static void run(
eb->loopForever(); eb->loopForever();
// must destruct in io thread for on-destruction callbacks // must destruct in io thread for on-destruction callbacks
EventBase::StackFunctionLoopCallback cb([=] { ebm->clearEventBase(); }); eb->runOnDestruction([=] { ebm->clearEventBase(); });
eb->runOnDestruction(&cb);
// wait until terminateLoopSoon() is complete // wait until terminateLoopSoon() is complete
stop->wait(); stop->wait();
eb->~EventBase(); eb->~EventBase();
......
...@@ -37,14 +37,16 @@ void VirtualEventBase::destroyImpl() { ...@@ -37,14 +37,16 @@ void VirtualEventBase::destroyImpl() {
clearCobTimeouts(); clearCobTimeouts();
while (!onDestructionCallbacks_.rlock()->empty()) {
// To avoid potential deadlock, do not hold the mutex while invoking // To avoid potential deadlock, do not hold the mutex while invoking
// user-supplied callbacks. // user-supplied callbacks.
LoopCallbackList callbacks; EventBase::OnDestructionCallback::List callbacks;
onDestructionCallbacks_.swap(callbacks); onDestructionCallbacks_.swap(callbacks);
while (!callbacks.empty()) { while (!callbacks.empty()) {
auto& callback = callbacks.front(); auto& callback = callbacks.front();
callbacks.pop_front(); callbacks.pop_front();
callback.runLoopCallback(); callback.runCallback();
}
} }
} }
...@@ -62,10 +64,19 @@ VirtualEventBase::~VirtualEventBase() { ...@@ -62,10 +64,19 @@ VirtualEventBase::~VirtualEventBase() {
destroy().get(); destroy().get();
} }
void VirtualEventBase::runOnDestruction(EventBase::LoopCallback* callback) { void VirtualEventBase::runOnDestruction(
onDestructionCallbacks_.withWLock([&](LoopCallbackList& callbacks) { EventBase::OnDestructionCallback& callback) {
callback->cancelLoopCallback(); callback.schedule(
callbacks.push_back(*callback); [this](auto& cb) { onDestructionCallbacks_.wlock()->push_back(cb); },
[this](auto& cb) {
onDestructionCallbacks_.withWLock(
[&](auto& list) { list.erase(list.iterator_to(cb)); });
}); });
} }
void VirtualEventBase::runOnDestruction(Func f) {
auto* callback = new EventBase::FunctionOnDestructionCallback(std::move(f));
runOnDestruction(*callback);
}
} // namespace folly } // namespace folly
...@@ -19,6 +19,7 @@ ...@@ -19,6 +19,7 @@
#include <future> #include <future>
#include <folly/Executor.h> #include <folly/Executor.h>
#include <folly/Function.h>
#include <folly/Synchronized.h> #include <folly/Synchronized.h>
#include <folly/io/async/EventBase.h> #include <folly/io/async/EventBase.h>
#include <folly/synchronization/Baton.h> #include <folly/synchronization/Baton.h>
...@@ -60,7 +61,8 @@ class VirtualEventBase : public folly::Executor, public folly::TimeoutManager { ...@@ -60,7 +61,8 @@ class VirtualEventBase : public folly::Executor, public folly::TimeoutManager {
* Note: this will be called from the loop of the EventBase, backing this * Note: this will be called from the loop of the EventBase, backing this
* VirtualEventBase * VirtualEventBase
*/ */
void runOnDestruction(EventBase::LoopCallback* callback); void runOnDestruction(EventBase::OnDestructionCallback& callback);
void runOnDestruction(Func f);
/** /**
* VirtualEventBase destructor blocks until all tasks scheduled through its * VirtualEventBase destructor blocks until all tasks scheduled through its
...@@ -169,6 +171,6 @@ class VirtualEventBase : public folly::Executor, public folly::TimeoutManager { ...@@ -169,6 +171,6 @@ class VirtualEventBase : public folly::Executor, public folly::TimeoutManager {
KeepAlive<VirtualEventBase> loopKeepAlive_{ KeepAlive<VirtualEventBase> loopKeepAlive_{
makeKeepAlive<VirtualEventBase>(this)}; makeKeepAlive<VirtualEventBase>(this)};
folly::Synchronized<LoopCallbackList> onDestructionCallbacks_; Synchronized<EventBase::OnDestructionCallback::List> onDestructionCallbacks_;
}; };
} // namespace folly } // namespace folly
...@@ -2076,3 +2076,55 @@ TEST(EventBaseTest, TestStarvation) { ...@@ -2076,3 +2076,55 @@ TEST(EventBaseTest, TestStarvation) {
EXPECT_EQ(1000, num); EXPECT_EQ(1000, num);
t.join(); t.join();
} }
TEST(EventBaseTest, RunOnDestructionBasic) {
bool ranOnDestruction = false;
{
EventBase evb;
evb.runOnDestruction([&ranOnDestruction] { ranOnDestruction = true; });
}
EXPECT_TRUE(ranOnDestruction);
}
TEST(EventBaseTest, RunOnDestructionCancelled) {
struct Callback : EventBase::OnDestructionCallback {
bool ranOnDestruction{false};
void onEventBaseDestruction() noexcept final {
ranOnDestruction = true;
}
};
auto cb = std::make_unique<Callback>();
{
EventBase evb;
evb.runOnDestruction(*cb);
EXPECT_TRUE(cb->cancel());
}
EXPECT_FALSE(cb->ranOnDestruction);
EXPECT_FALSE(cb->cancel());
}
TEST(EventBaseTest, RunOnDestructionAfterHandleDestroyed) {
EventBase evb;
{
bool ranOnDestruction = false;
auto* cb = new EventBase::FunctionOnDestructionCallback(
[&ranOnDestruction] { ranOnDestruction = true; });
evb.runOnDestruction(*cb);
EXPECT_TRUE(cb->cancel());
delete cb;
}
}
TEST(EventBaseTest, RunOnDestructionAddCallbackWithinCallback) {
size_t callbacksCalled = 0;
{
EventBase evb;
evb.runOnDestruction([&] {
++callbacksCalled;
evb.runOnDestruction([&] { ++callbacksCalled; });
});
}
EXPECT_EQ(2, callbacksCalled);
}
...@@ -83,8 +83,7 @@ TEST_F(ScopedEventBaseThreadTest, eb_dtor_in_io_thread) { ...@@ -83,8 +83,7 @@ TEST_F(ScopedEventBaseThreadTest, eb_dtor_in_io_thread) {
auto const eb = sebt->getEventBase(); auto const eb = sebt->getEventBase();
thread::id eb_dtor_thread_id; thread::id eb_dtor_thread_id;
eb->runOnDestruction(new EventBase::FunctionLoopCallback( eb->runOnDestruction([&] { eb_dtor_thread_id = std::this_thread::get_id(); });
[&] { eb_dtor_thread_id = this_thread::get_id(); }));
sebt.clear(); sebt.clear();
EXPECT_EQ(io_thread_id, eb_dtor_thread_id); EXPECT_EQ(io_thread_id, eb_dtor_thread_id);
} }
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