You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@mesos.apache.org by "Alexander Rukletsov (JIRA)" <ji...@apache.org> on 2017/02/06 09:30:41 UTC

[jira] [Created] (MESOS-7065) Deadlock is possible when a pointer to an actor is captured in this actor's callback.

Alexander Rukletsov created MESOS-7065:
------------------------------------------

             Summary: Deadlock is possible when a pointer to an actor is captured in this actor's callback.
                 Key: MESOS-7065
                 URL: https://issues.apache.org/jira/browse/MESOS-7065
             Project: Mesos
          Issue Type: Improvement
          Components: libprocess
    Affects Versions: 1.1.0, 1.0.2, 1.2.0
            Reporter: Alexander Rukletsov


A deadlock is triggered if a callback of the actor {{A a}} contains a {{shared_ptr}} to itself. In this case, if the last copy of {{shared_ptr<A>}} is held by the callback, {{clearAllCallbacks()}} called on {{a}}'s context will try to terminate {{a}} and will never succeed.

I've put up a [test demonstrating a deadlock|https://gist.github.com/rukletsov/4ba5a70e74c50ae8229a19f7558055ff] triggered by removing callbacks from a satisfied future: 
{noformat}
#include <iostream>
#include <process/clock.hpp>
#include <process/future.hpp>
#include <process/id.hpp>
#include <process/process.hpp>
#include <process/shared.hpp>
#include <stout/os.hpp>

class SatisfierProcess : public process::Process<SatisfierProcess>
{
public:
  SatisfierProcess() : ProcessBase(process::ID::generate("__satisfier__")) {}
  virtual void finalize() { promise.discard(); }
 process::Future<Nothing> future() { return promise.future(); }

  void satisfy()
  {
    std::cout << " >> SatisfierProcess: satisfying the promise" << std::endl;
    promise.set(Nothing());
  }

  process::Promise<Nothing> promise;
};

class Satisfier
{
public:
  Satisfier()
  {
    process = new SatisfierProcess();
    process::spawn(process);
  }

  ~Satisfier()
  {
    std::cout << " > ~Satisfier()" << std::endl;
    process::terminate(process);
    process::wait(process);
    delete process;
  }

  process::Future<Nothing> future() const { return dispatch(process, &SatisfierProcess::future); }
  void satisfy() const { dispatch(process->self(), &SatisfierProcess::satisfy); }

  SatisfierProcess* process;
};


TEST(CircularDependencyTest, FutureCallbackRemovalTriggersDeadlock)
{
  {
    // This shared pointer will go out of scope and will
    // only be referenced from the `.onAny` callback.
    process::Shared<Satisfier> s(new Satisfier);

    s->future()
      // Callback with a circular dependency. When it finishes,
      // the only reference to `s` is in the callback itself.
      .onAny(process::defer([s](const Future<Nothing>&) {
        std::cout << " | First callback finished" << std::endl;
        return;
      }))
      // Callback that takes some time to process. Note the absence
      // of `defer` here: this callback is executed in the same context
      // where the promise was set, i.e. `s->process->self()` and ensures
      // the first callback has already finished when `clearAllCallbacks`
      // is called. `clearAllCallbacks` removes the last reference to the
      // `Satisfier` instance and hence calls its d-tor, i.e. `Satisfier`'s
      // d-tor is called from the `Satisfier` context => deadlock.
      .onAny([](const Future<Nothing>&) {
        std::cout << " | Second callback started" << std::endl;
        os::sleep(Seconds(2));
        std::cout << " | Second callback finished" << std::endl;
        return;
      });

    s->satisfy();
  }

  // Wait for all actors to process messages in their queues. Hint:
  // this will not happen because one actor is waits on himself.
  process::Clock::pause();
  process::Clock::settle();
}
{noformat}

The [output of the test|https://gist.github.com/rukletsov/334f74e7a5ed3c2942f4ce95a3574cef] looks like: 
{noformat}
[ RUN      ] CircularDependencyTest.FutureCallbackRemovalTriggersDeadlock
 >> SatisfierProcess: satisfying the promise
 | Second callback started
 | First callback finished
 | Second callback finished
 > ~Satisfier()

**** DEADLOCK DETECTED! ****
You are waiting on process __satisfier__(1)@192.168.9.40:61447 that it is currently executing.
*** Aborted at 1485916555 (unix time) try "date -d @1485916555" if you are using GNU date ***
PC: @        0x10c2479a8 process::Future<>::Future()
*** SIGSEGV (@0xdeb5) received by PID 39047 (TID 0x70000028d000) stack trace: ***
    @     0x7fff85acc52a _sigtramp
    @     0x7fff6754a5c8 (unknown)
    @        0x10c24795d process::Future<>::Future()
    @        0x10c246990 process::Promise<>::future()
    @        0x10f429a0d process::wait()
    @        0x106f062cf process::wait()
    @        0x1075a4b65 mesos::internal::tests::Satisfier::~Satisfier()
    @        0x1075a1c15 mesos::internal::tests::Satisfier::~Satisfier()
    @        0x1075a1b1d process::Shared<>::Data::~Data()
    @        0x1075a19f5 process::Shared<>::Data::~Data()
    @        0x1075a18f1 std::__1::__shared_ptr_pointer<>::__on_zero_shared()
    @     0x7fff8515fcb8 std::__1::__shared_weak_count::__release_shared()
    @        0x106f071bc std::__1::shared_ptr<>::~shared_ptr()
    @        0x106f07185 std::__1::shared_ptr<>::~shared_ptr()
    @        0x1075971c5 process::Shared<>::~Shared()
    @        0x107590ce5 process::Shared<>::~Shared()
    @        0x107586c25 mesos::internal::tests::CircularDependencyTest_FutureCallbackRemovalTriggersDeadlock_Test::TestBody()::$_0::~TestBody()
    @        0x107586bb5 mesos::internal::tests::CircularDependencyTest_FutureCallbackRemovalTriggersDeadlock_Test::TestBody()::$_0::~TestBody()
    @        0x10758b80c _ZZNK7process9_DeferredIZN5mesos8internal5tests65CircularDependencyTest_FutureCallbackRemovalTriggersDeadlock_Test8TestBodyEvE3$_0EcvNSt3__18functionIFvT_EEEIRKNS_6FutureI7NothingEEEEvENUlSH_E_D2Ev
    @        0x107587175 _ZZNK7process9_DeferredIZN5mesos8internal5tests65CircularDependencyTest_FutureCallbackRemovalTriggersDeadlock_Test8TestBodyEvE3$_0EcvNSt3__18functionIFvT_EEEIRKNS_6FutureI7NothingEEEEvENUlSH_E_D1Ev
    @        0x107589c95 _ZNSt3__128__libcpp_compressed_pair_impIZNK7process9_DeferredIZN5mesos8internal5tests65CircularDependencyTest_FutureCallbackRemovalTriggersDeadlock_Test8TestBodyEvE3$_0EcvNS_8functionIFvT_EEEIRKNS1_6FutureI7NothingEEEEvEUlSI_E_NS_9allocatorISJ_EELj2EED2Ev
    @        0x107589c75 _ZNSt3__117__compressed_pairIZNK7process9_DeferredIZN5mesos8internal5tests65CircularDependencyTest_FutureCallbackRemovalTriggersDeadlock_Test8TestBodyEvE3$_0EcvNS_8functionIFvT_EEEIRKNS1_6FutureI7NothingEEEEvEUlSI_E_NS_9allocatorISJ_EEED2Ev
    @        0x107589c55 _ZNSt3__117__compressed_pairIZNK7process9_DeferredIZN5mesos8internal5tests65CircularDependencyTest_FutureCallbackRemovalTriggersDeadlock_Test8TestBodyEvE3$_0EcvNS_8functionIFvT_EEEIRKNS1_6FutureI7NothingEEEEvEUlSI_E_NS_9allocatorISJ_EEED1Ev
    @        0x107589a44 _ZNSt3__110__function6__funcIZNK7process9_DeferredIZN5mesos8internal5tests65CircularDependencyTest_FutureCallbackRemovalTriggersDeadlock_Test8TestBodyEvE3$_0EcvNS_8functionIFvT_EEEIRKNS2_6FutureI7NothingEEEEvEUlSJ_E_NS_9allocatorISK_EEFvSJ_EE18destroy_deallocateEv
    @        0x106f075fa std::__1::function<>::~function()
    @        0x106f06395 std::__1::function<>::~function()
    @        0x106f8b16a process::Future<>::Data::clearAllCallbacks()
    @        0x106f8ae80 process::Future<>::_set<>()
    @        0x106f8818d process::Future<>::set()
    @        0x106f8f41a _ZZNK7process6FutureI7NothingE7onReadyINSt3__16__bindIRMS2_FbRKS1_EJRS2_RNS4_12placeholders4__phILi1EEEEEEbEERKS2_OT_NS2_6PreferEENUlS7_E_clES7_
    @        0x106f8f1fd _ZNSt3__128__invoke_void_return_wrapperIvE6__callIJRZNK7process6FutureI7NothingE7onReadyINS_6__bindIRMS6_FbRKS5_EJRS6_RNS_12placeholders4__phILi1EEEEEEbEERKS6_OT_NS6_6PreferEEUlSA_E_SA_EEEvDpOT_
    @        0x106f8ee09 _ZNSt3__110__function6__funcIZNK7process6FutureI7NothingE7onReadyINS_6__bindIRMS5_FbRKS4_EJRS5_RNS_12placeholders4__phILi1EEEEEEbEERKS5_OT_NS5_6PreferEEUlS9_E_NS_9allocatorISO_EEFvS9_EEclES9_
make[3]: *** [check-local] Segmentation fault: 11
make[2]: *** [check-am] Error 2
make[1]: *** [check] Error 2
make: *** [check-recursive] Error 1
{noformat}

Note that I've introduced a [segfault in {{"process.cpp"}}|https://gist.github.com/rukletsov/929d86d0dc6e3ecbb50f10667c4dfef2] to get the stack trace:
{noformat}
bool wait(const UPID& pid, const Duration& duration)
{
  process::initialize();

  if (!pid) {
    return false;
  }

  // This could result in a deadlock if some code decides to wait on a
  // process that has invoked that code!
  if (__process__ != nullptr && __process__->self() == pid) {
    std::cerr << "\n**** DEADLOCK DETECTED! ****\nYou are waiting on process "
              << pid << " that it is currently executing." << std::endl;

    ((process::Promise<Nothing>*)0xDEAD)->future();
  }

  ...
}
{noformat}

It would be great if we can help users prevent from getting into this deadlock by either rejecting or circumventing this pattern altogether.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)