You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by vi...@apache.org on 2014/01/15 23:02:22 UTC

[1/5] git commit: When the detectors fail, subsequent detect() calls now get failed Futures directly.

Updated Branches:
  refs/heads/master cd17c425e -> bef470090


When the detectors fail, subsequent detect() calls now get failed
Futures directly.

From: Jiang Yan Xu <ya...@jxu.me>
Review: https://reviews.apache.org/r/16586


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/fe86c8b4
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/fe86c8b4
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/fe86c8b4

Branch: refs/heads/master
Commit: fe86c8b4e58945283c8c585e2e9af863ed840c4f
Parents: cd17c42
Author: Vinod Kone <vi...@twitter.com>
Authored: Wed Jan 15 12:49:57 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Wed Jan 15 12:54:48 2014 -0800

----------------------------------------------------------------------
 src/master/detector.cpp                       | 19 +++++++++++++++++--
 src/tests/master_contender_detector_tests.cpp |  6 ++++++
 src/zookeeper/detector.cpp                    | 18 ++++++++++++++++--
 3 files changed, 39 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fe86c8b4/src/master/detector.cpp
----------------------------------------------------------------------
diff --git a/src/master/detector.cpp b/src/master/detector.cpp
index 532e048..cf337cf 100644
--- a/src/master/detector.cpp
+++ b/src/master/detector.cpp
@@ -78,7 +78,7 @@ public:
 
 private:
   // Invoked when the group leadership has changed.
-  void detected(Future<Option<Group::Membership> > leader);
+  void detected(const Future<Option<Group::Membership> >& leader);
 
   // Invoked when we have fetched the data associated with the leader.
   void fetched(const Future<string>& data);
@@ -89,6 +89,9 @@ private:
   // The leading Master.
   Option<UPID> leader;
   set<Promise<Option<UPID> >*> promises;
+
+  // Potential non-retryable error.
+  Option<Error> error;
 };
 
 
@@ -243,6 +246,12 @@ void ZooKeeperMasterDetectorProcess::initialize()
 Future<Option<UPID> > ZooKeeperMasterDetectorProcess::detect(
     const Option<UPID>& previous)
 {
+  // Return immediately if the detector is no longer operational due
+  // to a non-retryable error.
+  if (error.isSome()) {
+    return Failure(error.get().message);
+  }
+
   if (leader != previous) {
     return leader;
   }
@@ -254,11 +263,17 @@ Future<Option<UPID> > ZooKeeperMasterDetectorProcess::detect(
 
 
 void ZooKeeperMasterDetectorProcess::detected(
-    Future<Option<Group::Membership> > _leader)
+    const Future<Option<Group::Membership> >& _leader)
 {
   CHECK(!_leader.isDiscarded());
 
   if (_leader.isFailed()) {
+    LOG(ERROR) << "Failed to detect the leader: " << _leader.failure();
+
+    // Setting this error stops the detection loop and the detector
+    // transitions to an erroneous state. Further calls to detect()
+    // will directly fail as a result.
+    error = Error(_leader.failure());
     leader = None();
     foreach (Promise<Option<UPID> >* promise, promises) {
       promise->fail(_leader.failure());

http://git-wip-us.apache.org/repos/asf/mesos/blob/fe86c8b4/src/tests/master_contender_detector_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_contender_detector_tests.cpp b/src/tests/master_contender_detector_tests.cpp
index d532c17..951d720 100644
--- a/src/tests/master_contender_detector_tests.cpp
+++ b/src/tests/master_contender_detector_tests.cpp
@@ -296,6 +296,9 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, NonRetryableFrrors)
   // Fails due to authentication error.
   AWAIT_FAILED(contender.contend());
 
+  // Subsequent call should also fail.
+  AWAIT_FAILED(contender.contend());
+
   // Now test non-retryable failures in detection.
   ZooKeeperTest::TestWatcher watcher;
   ZooKeeper authenticatedZk(server->connectString(), NO_TIMEOUT, &watcher);
@@ -321,6 +324,9 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, NonRetryableFrrors)
 
   ZooKeeperMasterDetector detector(group3);
   AWAIT_FAILED(detector.detect());
+
+  // Subsequent call should also fail.
+  AWAIT_FAILED(detector.detect());
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/fe86c8b4/src/zookeeper/detector.cpp
----------------------------------------------------------------------
diff --git a/src/zookeeper/detector.cpp b/src/zookeeper/detector.cpp
index 6723d03..2759a2f 100644
--- a/src/zookeeper/detector.cpp
+++ b/src/zookeeper/detector.cpp
@@ -36,11 +36,14 @@ private:
   void watch(const set<Group::Membership>& expected);
 
   // Invoked when the group memberships have changed.
-  void watched(Future<set<Group::Membership> > memberships);
+  void watched(const Future<set<Group::Membership> >& memberships);
 
   Group* group;
   Option<Group::Membership> leader;
   set<Promise<Option<Group::Membership> >*> promises;
+
+  // Potential non-retryable error.
+  Option<Error> error;
 };
 
 
@@ -67,6 +70,12 @@ void LeaderDetectorProcess::initialize()
 Future<Option<Group::Membership> > LeaderDetectorProcess::detect(
     const Option<Group::Membership>& previous)
 {
+  // Return immediately if the detector is no longer operational due
+  // to the non-retryable error.
+  if (error.isSome()) {
+    return Failure(error.get().message);
+  }
+
   // Return immediately if the incumbent leader is different from the
   // expected.
   if (leader != previous) {
@@ -88,12 +97,17 @@ void LeaderDetectorProcess::watch(const set<Group::Membership>& expected)
 }
 
 
-void LeaderDetectorProcess::watched(Future<set<Group::Membership> > memberships)
+void LeaderDetectorProcess::watched(const Future<set<Group::Membership> >& memberships)
 {
   CHECK(!memberships.isDiscarded());
 
   if (memberships.isFailed()) {
     LOG(ERROR) << "Failed to watch memberships: " << memberships.failure();
+
+    // Setting this error stops the watch loop and the detector
+    // transitions to an erroneous state. Further calls to detect()
+    // will directly fail as a result.
+    error = Error(memberships.failure());
     leader = None();
     foreach (Promise<Option<Group::Membership> >* promise, promises) {
       promise->fail(memberships.failure());


[3/5] git commit: Made ZooKeeperMasterContenderProcess::master an option to codify the fact that it can be unset after construction.

Posted by vi...@apache.org.
Made ZooKeeperMasterContenderProcess::master an option to codify the
fact that it can be unset after construction.

From: Jiang Yan Xu <ya...@jxu.me>
Review: https://reviews.apache.org/r/16588


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/e03700a9
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/e03700a9
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/e03700a9

Branch: refs/heads/master
Commit: e03700a92a1c7945da1dbbffa45733f92d5ed65b
Parents: fe86c8b
Author: Vinod Kone <vi...@twitter.com>
Authored: Wed Jan 15 12:50:44 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Wed Jan 15 12:54:57 2014 -0800

----------------------------------------------------------------------
 src/master/contender.cpp | 15 +++++++++++----
 src/master/contender.hpp |  2 ++
 2 files changed, 13 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e03700a9/src/master/contender.cpp
----------------------------------------------------------------------
diff --git a/src/master/contender.cpp b/src/master/contender.cpp
index 72a7796..7bd25b5 100644
--- a/src/master/contender.cpp
+++ b/src/master/contender.cpp
@@ -62,7 +62,9 @@ public:
 private:
   Owned<zookeeper::Group> group;
   LeaderContender* contender;
-  PID<Master> master;
+
+  // The master this contender contends on behalf of.
+  Option<PID<Master> > master;
 };
 
 
@@ -115,9 +117,12 @@ void StandaloneMasterContender::initialize(
   initialized = true;
 }
 
+
 Future<Future<Nothing> > StandaloneMasterContender::contend()
 {
-  CHECK(initialized) << "Initialize the contender first";
+  if (!initialized) {
+    return Failure("Initialize the contender first");
+  }
 
   if (promise != NULL) {
     LOG(INFO) << "Withdrawing the previous membership before recontending";
@@ -194,14 +199,16 @@ void ZooKeeperMasterContenderProcess::initialize(
 
 Future<Future<Nothing> > ZooKeeperMasterContenderProcess::contend()
 {
-  CHECK(master) << "Initialize the contender first";
+  if (master.isNone()) {
+    return Failure("Initialize the contender first");
+  }
 
   if (contender != NULL) {
     LOG(INFO) << "Withdrawing the previous membership before recontending";
     delete contender;
   }
 
-  contender = new LeaderContender(group.get(), master);
+  contender = new LeaderContender(group.get(), master.get());
   return contender->contend();
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/e03700a9/src/master/contender.hpp
----------------------------------------------------------------------
diff --git a/src/master/contender.hpp b/src/master/contender.hpp
index 7764edd..0156369 100644
--- a/src/master/contender.hpp
+++ b/src/master/contender.hpp
@@ -69,6 +69,8 @@ public:
 
   // Returns a Future<Nothing> once the contender has entered the
   // contest (by obtaining a membership) and an error otherwise.
+  // A failed future is returned if this method is called before
+  // initialize().
   // The inner Future returns Nothing when the contender is out of
   // the contest (i.e. its membership is lost).
   //


[2/5] git commit: ZooKeeperMasterContenderProcess::contend now checks if the future for the previous contend() is still pending and if so it directly returns that future without recontending.

Posted by vi...@apache.org.
ZooKeeperMasterContenderProcess::contend now checks if the future for
the previous contend() is still pending and if so it directly returns
that future without recontending.

From: Jiang Yan Xu <ya...@jxu.me>
Review: https://reviews.apache.org/r/16589


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/b2516db5
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/b2516db5
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/b2516db5

Branch: refs/heads/master
Commit: b2516db5595dbe2bbb8b0d32972c670143191d08
Parents: e03700a
Author: Vinod Kone <vi...@twitter.com>
Authored: Wed Jan 15 12:51:17 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Wed Jan 15 12:54:57 2014 -0800

----------------------------------------------------------------------
 src/master/contender.cpp                      |  9 +++-
 src/master/contender.hpp                      |  8 ++--
 src/tests/master_contender_detector_tests.cpp | 52 ++++++++++++++++++++++
 3 files changed, 65 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b2516db5/src/master/contender.cpp
----------------------------------------------------------------------
diff --git a/src/master/contender.cpp b/src/master/contender.cpp
index 7bd25b5..89e368b 100644
--- a/src/master/contender.cpp
+++ b/src/master/contender.cpp
@@ -65,6 +65,7 @@ private:
 
   // The master this contender contends on behalf of.
   Option<PID<Master> > master;
+  Option<Future<Future<Nothing> > > candidacy;
 };
 
 
@@ -203,13 +204,19 @@ Future<Future<Nothing> > ZooKeeperMasterContenderProcess::contend()
     return Failure("Initialize the contender first");
   }
 
+  // Should not recontend if the last election is still ongoing.
+  if (candidacy.isSome() && candidacy.get().isPending()) {
+    return candidacy.get();
+  }
+
   if (contender != NULL) {
     LOG(INFO) << "Withdrawing the previous membership before recontending";
     delete contender;
   }
 
   contender = new LeaderContender(group.get(), master.get());
-  return contender->contend();
+  candidacy = contender->contend();
+  return candidacy.get();
 }
 
 } // namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/b2516db5/src/master/contender.hpp
----------------------------------------------------------------------
diff --git a/src/master/contender.hpp b/src/master/contender.hpp
index 0156369..2a7e7c4 100644
--- a/src/master/contender.hpp
+++ b/src/master/contender.hpp
@@ -74,9 +74,11 @@ public:
   // The inner Future returns Nothing when the contender is out of
   // the contest (i.e. its membership is lost).
   //
-  // This method can be used to contend again. Each call to this
-  // method causes the previous candidacy to be withdrawn before
-  // re-contending.
+  // This method can be used to contend again after candidacy is
+  // obtained (the outer future satisfied), otherwise the future for
+  // the pending election is returned.
+  // Recontending after candidacy is obtained causes the previous
+  // candidacy to be withdrawn.
   virtual process::Future<process::Future<Nothing> > contend() = 0;
 };
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/b2516db5/src/tests/master_contender_detector_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_contender_detector_tests.cpp b/src/tests/master_contender_detector_tests.cpp
index 951d720..9cd576f 100644
--- a/src/tests/master_contender_detector_tests.cpp
+++ b/src/tests/master_contender_detector_tests.cpp
@@ -213,6 +213,58 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterContender)
 }
 
 
+// Verifies that contender does not recontend if the current election
+// is still pending.
+TEST_F(ZooKeeperMasterContenderDetectorTest, ContenderPendingElection)
+{
+  Try<zookeeper::URL> url = zookeeper::URL::parse(
+      "zk://" + server->connectString() + "/mesos");
+
+  ASSERT_SOME(url);
+
+  ZooKeeperMasterContender contender(url.get());
+
+  PID<Master> master;
+  master.ip = 10000000;
+  master.port = 10000;
+
+  contender.initialize(master);
+
+  // Drop Group::join so that 'contended' will stay pending.
+  Future<Nothing> join = DROP_DISPATCH(_, &GroupProcess::join);
+
+  Future<Future<Nothing> > contended = contender.contend();
+  AWAIT_READY(join);
+
+  Clock::pause();
+
+  // Make sure GroupProcess::join is dispatched (and dropped).
+  Clock::settle();
+
+  EXPECT_TRUE(contended.isPending());
+
+  process::filter(NULL);
+
+  process::TestsFilter* filter =
+    process::FilterTestEventListener::instance()->install();
+  pthread_mutex_lock(&filter->mutex);
+
+  // Expect GroupProcess::join not getting called because
+  // ZooKeeperMasterContender directly returns.
+  EXPECT_CALL(filter->mock, filter(testing::A<const process::DispatchEvent&>()))
+    .With(DispatchMatcher(_, &GroupProcess::join))
+    .Times(0);
+  pthread_mutex_lock(&filter->mutex);
+
+  // Recontend and settle so that if ZooKeeperMasterContender is not
+  // directly returning, GroupProcess::join is dispatched.
+  contender.contend();
+  Clock::settle();
+
+  Clock::resume();
+}
+
+
 // Two contenders, the first wins. Kill the first, then the second
 // is elected.
 TEST_F(ZooKeeperMasterContenderDetectorTest, MasterContenders)


[4/5] git commit: LeaderContender::withdraw() now always returns false if there is no membership to withdraw.

Posted by vi...@apache.org.
LeaderContender::withdraw() now always returns false if there is no
membership to withdraw.

This includes when:
- withdraw() is called before contend()
- withdraw() is called after contend() but before the contender learns that it fails to obtain the candidacy
- withdraw() is called after the contender has learned that it had failed to obtain the candidacy

From: Jiang Yan Xu <ya...@jxu.me>
Review: https://reviews.apache.org/r/16590


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/682b0d99
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/682b0d99
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/682b0d99

Branch: refs/heads/master
Commit: 682b0d9917960fd4f976eb26dc5fdb1d9957c692
Parents: b2516db
Author: Vinod Kone <vi...@twitter.com>
Authored: Wed Jan 15 12:51:49 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Wed Jan 15 12:54:58 2014 -0800

----------------------------------------------------------------------
 src/tests/zookeeper_tests.cpp |  9 ++++++-
 src/zookeeper/contender.cpp   | 50 ++++++++++++++++++--------------------
 src/zookeeper/contender.hpp   |  4 +--
 3 files changed, 33 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/682b0d99/src/tests/zookeeper_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/zookeeper_tests.cpp b/src/tests/zookeeper_tests.cpp
index 94d324a..656afde 100644
--- a/src/tests/zookeeper_tests.cpp
+++ b/src/tests/zookeeper_tests.cpp
@@ -267,11 +267,18 @@ TEST_F(ZooKeeperTest, LeaderContender)
 
   Owned<LeaderContender> contender(
       new LeaderContender(&group, "candidate 1"));
+
+  // Calling withdraw before contending returns 'false' because there
+  // is nothing to withdraw.
+  Future<bool> withdrawn = contender->withdraw();
+  AWAIT_READY(withdrawn);
+  EXPECT_FALSE(withdrawn.get());
+
   contender->contend();
 
   // Immediately withdrawing after contending leads to delayed
   // cancellation.
-  Future<bool> withdrawn = contender->withdraw();
+  withdrawn = contender->withdraw();
   AWAIT_READY(withdrawn);
   EXPECT_TRUE(withdrawn.get());
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/682b0d99/src/zookeeper/contender.cpp
----------------------------------------------------------------------
diff --git a/src/zookeeper/contender.cpp b/src/zookeeper/contender.cpp
index e857afa..d8a5201 100644
--- a/src/zookeeper/contender.cpp
+++ b/src/zookeeper/contender.cpp
@@ -40,9 +40,6 @@ private:
   // Invoked when the group membership is cancelled.
   void cancelled(const Future<bool>& result);
 
-  // Helper for setting error and failing pending promises.
-  void fail(const string& message);
-
   // Helper for cancelling the Group membership.
   void cancel();
 
@@ -104,8 +101,8 @@ void LeaderContenderProcess::finalize()
 {
   // We do not wait for the result here because the Group keeps
   // retrying (even after the contender is destroyed) until it
-  // either succeeds or its session times out. In either case the
-  // old membership is eventually cancelled.
+  // succeeds so the old membership is eventually going to be
+  // cancelled.
   // There is a tricky situation where the contender terminates after
   // it has contended but before it is notified of the obtained
   // membership. In this case the membership is not cancelled during
@@ -135,7 +132,8 @@ Future<Future<Nothing> > LeaderContenderProcess::contend()
 Future<bool> LeaderContenderProcess::withdraw()
 {
   if (contending.isNone()) {
-    return Failure("Can only withdraw after the contender has contended");
+    // Nothing to withdraw because the contender has not contended.
+    return false;
   }
 
   if (withdrawing.isSome()) {
@@ -169,6 +167,9 @@ void LeaderContenderProcess::cancel()
 {
   if (!candidacy.isReady()) {
     // Nothing to cancel.
+    if (withdrawing.isSome()) {
+      withdrawing.get()->set(false);
+    }
     return;
   }
 
@@ -191,7 +192,13 @@ void LeaderContenderProcess::cancelled(const Future<bool>& result)
   CHECK(!result.isDiscarded());
 
   if (result.isFailed()) {
-    fail(result.failure());
+    if (withdrawing.isSome()) {
+      withdrawing.get()->fail(result.failure());
+    }
+
+    if (watching.isSome()) {
+      watching.get()->fail(result.failure());
+    }
   } else {
     if (withdrawing.isSome()) {
       withdrawing.get()->set(result);
@@ -208,13 +215,22 @@ void LeaderContenderProcess::joined()
 {
   CHECK(!candidacy.isDiscarded());
 
+  // Cannot be watching because the candidacy is not obtained yet.
+  CHECK(watching.isNone());
+
+  CHECK_SOME(contending);
+
   if (candidacy.isFailed()) {
-    fail(candidacy.failure());
+    // The promise 'withdrawing' will be set to false in cancel().
+    contending.get()->fail(candidacy.failure());
     return;
   }
 
   if (withdrawing.isSome()) {
     LOG(INFO) << "Joined group after the contender started withdrawing";
+
+    // The promise 'withdrawing' will be set to 'false' in subsequent
+    // 'cancel()' call.
     return;
   }
 
@@ -222,11 +238,9 @@ void LeaderContenderProcess::joined()
             << data << "') has entered the contest for leadership";
 
   // Transition to 'watching' state.
-  CHECK(watching.isNone());
   watching = new Promise<Nothing>();
 
   // Notify the client.
-  CHECK_SOME(contending);
   if (contending.get()->set(watching.get()->future())) {
     // Continue to watch that our membership is not removed (if the
     // client still cares about it).
@@ -236,22 +250,6 @@ void LeaderContenderProcess::joined()
 }
 
 
-void LeaderContenderProcess::fail(const string& message)
-{
-  if (contending.isSome()) {
-    contending.get()->fail(message);
-  }
-
-  if (watching.isSome()) {
-    watching.get()->fail(message);
-  }
-
-  if (withdrawing.isSome()) {
-    withdrawing.get()->fail(message);
-  }
-}
-
-
 LeaderContender::LeaderContender(Group* group, const string& data)
 {
   process = new LeaderContenderProcess(group, data);

http://git-wip-us.apache.org/repos/asf/mesos/blob/682b0d99/src/zookeeper/contender.hpp
----------------------------------------------------------------------
diff --git a/src/zookeeper/contender.hpp b/src/zookeeper/contender.hpp
index f45c0df..e526e2d 100644
--- a/src/zookeeper/contender.hpp
+++ b/src/zookeeper/contender.hpp
@@ -45,11 +45,9 @@ public:
   // Returns true if successfully withdrawn from the contest (either
   // while contending or has already contended and is watching for
   // membership loss).
-  // It should only be called after contend() is called, otherwise a
-  // failure is returned.
   // A false return value implies that there was no valid group
   // membership to cancel, which may be a result of a race to cancel
-  // an expired membership.
+  // an expired membership or because there is nothing to withdraw.
   // A failed future is returned if the contender is unable to
   // withdraw.
   process::Future<bool> withdraw();


[5/5] git commit: GroupProcess::abort() now also sets all owned memberships as cancelled.

Posted by vi...@apache.org.
GroupProcess::abort() now also sets all owned memberships as cancelled.

From: Jiang Yan Xu <ya...@jxu.me>
Review: https://reviews.apache.org/r/16591


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/bef47009
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/bef47009
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/bef47009

Branch: refs/heads/master
Commit: bef47009015b976be0a73b013d77b9ce9505a463
Parents: 682b0d9
Author: Vinod Kone <vi...@twitter.com>
Authored: Wed Jan 15 12:54:10 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Wed Jan 15 12:54:58 2014 -0800

----------------------------------------------------------------------
 src/zookeeper/group.cpp | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/bef47009/src/zookeeper/group.cpp
----------------------------------------------------------------------
diff --git a/src/zookeeper/group.cpp b/src/zookeeper/group.cpp
index 47d599d..e21dc6f 100644
--- a/src/zookeeper/group.cpp
+++ b/src/zookeeper/group.cpp
@@ -848,6 +848,14 @@ void GroupProcess::abort(const string& message)
   fail(&pending.datas, message);
   fail(&pending.watches, message);
 
+  // Set all owned memberships as cancelled.
+  foreachvalue (Promise<bool>* cancelled, owned) {
+    cancelled->set(false); // Since this was not requested.
+    delete cancelled;
+  }
+
+  owned.clear();
+
   // Since we decided to abort, we expire the session to clean up
   // ephemeral ZNodes as necessary.
   delete CHECK_NOTNULL(zk);