You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2013/05/26 18:57:51 UTC

[26/28] git commit: Fixed bug where we didn't stop all MesosExecutorDrivers when using the TestingIsolator (potentially causing a segfault when a driver attempts to use a MockExecutor that had been deallocated).

Fixed bug where we didn't stop all MesosExecutorDrivers when using the
TestingIsolator (potentially causing a segfault when a driver attempts
to use a MockExecutor that had been deallocated).

Review: https://reviews.apache.org/r/11352


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

Branch: refs/heads/master
Commit: bc9cb875536dd2ee349dfc8b917322d77a865a04
Parents: 6958e6d
Author: Benjamin Hindman <be...@twitter.com>
Authored: Tue May 21 23:35:11 2013 -0700
Committer: Benjamin Hindman <be...@twitter.com>
Committed: Sun May 26 09:28:38 2013 -0700

----------------------------------------------------------------------
 src/tests/allocator_tests.cpp             |   38 ++-------
 src/tests/allocator_zookeeper_tests.cpp   |  116 +++++++++---------------
 src/tests/fault_tolerance_tests.cpp       |   51 ++++-------
 src/tests/gc_tests.cpp                    |   20 ++---
 src/tests/isolator.hpp                    |   10 ++-
 src/tests/master_tests.cpp                |   42 ++-------
 src/tests/resource_offers_tests.cpp       |    5 +-
 src/tests/status_update_manager_tests.cpp |   21 +----
 8 files changed, 105 insertions(+), 198 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/bc9cb875/src/tests/allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/allocator_tests.cpp b/src/tests/allocator_tests.cpp
index 8b868bc..52e2b03 100644
--- a/src/tests/allocator_tests.cpp
+++ b/src/tests/allocator_tests.cpp
@@ -666,15 +666,12 @@ TYPED_TEST(AllocatorTest, SchedulerFailover)
   EXPECT_CALL(this->allocator, frameworkRemoved(_))
     .Times(AtMost(1));
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver2.stop();
   driver2.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   EXPECT_CALL(this->allocator, slaveRemoved(_))
     .Times(AtMost(1));
 
@@ -796,25 +793,20 @@ TYPED_TEST(AllocatorTest, FrameworkExited)
   EXPECT_CALL(sched2, resourceOffers(_, OfferEq(2, 768)))
     .WillOnce(FutureSatisfy(&resourceOffers));
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver1.stop();
   driver1.join();
 
   AWAIT_READY(resourceOffers);
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver2.stop();
   driver2.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   EXPECT_CALL(this->allocator, slaveRemoved(_))
     .Times(AtMost(1));
 
@@ -881,9 +873,8 @@ TYPED_TEST(AllocatorTest, SlaveLost)
     .WillOnce(DoAll(InvokeSlaveRemoved(&this->allocator),
                     FutureSatisfy(&slaveRemoved)));
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   EXPECT_CALL(sched, slaveLost(_, _));
 
@@ -891,10 +882,6 @@ TYPED_TEST(AllocatorTest, SlaveLost)
 
   AWAIT_READY(slaveRemoved);
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
-  MockExecutor exec2(DEFAULT_EXECUTOR_ID);
-
   slave::Flags flags2 = this->CreateSlaveFlags();
   flags2.resources = Option<string>("cpus:3;mem:256");
 
@@ -907,7 +894,7 @@ TYPED_TEST(AllocatorTest, SlaveLost)
   EXPECT_CALL(sched, resourceOffers(_, OfferEq(3, 256)))
     .WillOnce(FutureSatisfy(&resourceOffers));
 
-  Try<PID<Slave> > slave2 = this->StartSlave(&exec2, flags2);
+  Try<PID<Slave> > slave2 = this->StartSlave(flags2);
   ASSERT_SOME(slave2);
 
   AWAIT_READY(resourceOffers);
@@ -922,14 +909,9 @@ TYPED_TEST(AllocatorTest, SlaveLost)
   EXPECT_CALL(this->allocator, frameworkRemoved(_))
     .Times(AtMost(1));
 
-  EXPECT_CALL(exec2, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
-
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   EXPECT_CALL(this->allocator, slaveRemoved(_))
     .Times(AtMost(1));
 
@@ -1025,15 +1007,12 @@ TYPED_TEST(AllocatorTest, SlaveAdded)
   EXPECT_CALL(this->allocator, frameworkRemoved(_))
     .Times(AtMost(1));
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   EXPECT_CALL(this->allocator, slaveRemoved(_))
     .Times(AtMost(2));
 
@@ -1135,15 +1114,12 @@ TYPED_TEST(AllocatorTest, TaskFinished)
   EXPECT_CALL(this->allocator, frameworkRemoved(_))
     .Times(AtMost(1));
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   EXPECT_CALL(this->allocator, slaveRemoved(_))
     .Times(AtMost(1));
 

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/bc9cb875/src/tests/allocator_zookeeper_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/allocator_zookeeper_tests.cpp b/src/tests/allocator_zookeeper_tests.cpp
index 31f4dc6..1034d72 100644
--- a/src/tests/allocator_zookeeper_tests.cpp
+++ b/src/tests/allocator_zookeeper_tests.cpp
@@ -121,47 +121,28 @@ TYPED_TEST(AllocatorZooKeeperTest, FrameworkReregistersFirst)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, stringify(this->url.get()));
 
-  Future<Nothing> registered;
-  EXPECT_CALL(sched, registered(&driver, _, _))
-    .WillOnce(FutureSatisfy(&registered));
+  EXPECT_CALL(sched, registered(&driver, _, _));
 
-  Future<vector<Offer> > resourceOffers1;
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(DoAll(LaunchTasks(1, 1, 500),
-                    FutureArg<1>(&resourceOffers1)))
+  // The framework should be offered all of the resources on the slave
+  // since it is the only framework running.
+  EXPECT_CALL(sched, resourceOffers(&driver, OfferEq(2, 1024)))
+    .WillOnce(LaunchTasks(1, 1, 500))
     .WillRepeatedly(DeclineOffers());
 
-  Future<TaskStatus> statusUpdate;
-  EXPECT_CALL(sched, statusUpdate(&driver, _))
-    .WillOnce(FutureArg<1>(&statusUpdate));
-
-  EXPECT_CALL(sched, disconnected(_))
-    .WillRepeatedly(DoDefault());
-
   EXPECT_CALL(exec, registered(_, _, _, _));
 
   EXPECT_CALL(exec, launchTask(_, _))
     .WillOnce(SendStatusUpdateFromTask(TASK_RUNNING));
 
-  EXPECT_CALL(exec, disconnected(_))
-    .WillRepeatedly(DoDefault());
-
-  EXPECT_CALL(exec, shutdown(_))
-    .WillRepeatedly(DoDefault());
+  Future<TaskStatus> status;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&status));
 
   driver.start();
 
-  AWAIT_READY(registered);
+  AWAIT_READY(status);
 
-  AWAIT_READY(resourceOffers1);
-
-  // The framework will be offered all of the resources on the slave,
-  // since it is the only framework running.
-  EXPECT_THAT(resourceOffers1.get(), OfferEq(2, 1024));
-
-  AWAIT_READY(statusUpdate);
-
-  EXPECT_EQ(TASK_RUNNING, statusUpdate.get().state());
+  EXPECT_EQ(TASK_RUNNING, status.get().state());
 
   // Stop the failing master from telling the slave to shut down when
   // it is killed.
@@ -172,6 +153,16 @@ TYPED_TEST(AllocatorZooKeeperTest, FrameworkReregistersFirst)
   // framework has reregistered.
   DROP_PROTOBUFS(ReregisterSlaveMessage(), _, _);
 
+  // Shutting down the masters will cause the scheduler to get
+  // disconnected.
+  EXPECT_CALL(sched, disconnected(_));
+
+  // Shutting down the masters will also cause the slave to shutdown
+  // frameworks that are not checkpointing, thus causing the executor
+  // to get shutdown.
+  EXPECT_CALL(exec, shutdown(_))
+    .Times(AtMost(1));
+
   this->ShutdownMasters();
 
   AWAIT_READY(shutdownMessage);
@@ -213,20 +204,15 @@ TYPED_TEST(AllocatorZooKeeperTest, FrameworkReregistersFirst)
   EXPECT_CALL(allocator2, resourcesRecovered(_, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(allocator2, frameworkDeactivated(_));
+  EXPECT_CALL(allocator2, frameworkDeactivated(_))
+    .Times(AtMost(1));
 
-  Future<Nothing> frameworkRemoved;
   EXPECT_CALL(allocator2, frameworkRemoved(_))
-    .WillOnce(FutureSatisfy(&frameworkRemoved));
-
-  EXPECT_CALL(exec, shutdown(_))
     .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(frameworkRemoved);
-
   EXPECT_CALL(allocator2, slaveRemoved(_))
     .Times(AtMost(1));
 
@@ -257,47 +243,28 @@ TYPED_TEST(AllocatorZooKeeperTest, SlaveReregistersFirst)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, stringify(this->url.get()));
 
-  Future<Nothing> registered;
-  EXPECT_CALL(sched, registered(&driver, _, _))
-    .WillOnce(FutureSatisfy(&registered));
+  EXPECT_CALL(sched, registered(&driver, _, _));
 
-  Future<vector<Offer> > resourceOffers1;
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(DoAll(LaunchTasks(1, 1, 500),
-                    FutureArg<1>(&resourceOffers1)))
+  // The framework should be offered all of the resources on the slave
+  // since it is the only framework running.
+  EXPECT_CALL(sched, resourceOffers(&driver, OfferEq(2, 1024)))
+    .WillOnce(LaunchTasks(1, 1, 500))
     .WillRepeatedly(DeclineOffers());
 
-  Future<TaskStatus> statusUpdate;
-  EXPECT_CALL(sched, statusUpdate(&driver, _))
-    .WillOnce(FutureArg<1>(&statusUpdate));
-
-  EXPECT_CALL(sched, disconnected(_))
-    .WillRepeatedly(DoDefault());
-
   EXPECT_CALL(exec, registered(_, _, _, _));
 
   EXPECT_CALL(exec, launchTask(_, _))
     .WillOnce(SendStatusUpdateFromTask(TASK_RUNNING));
 
-  EXPECT_CALL(exec, disconnected(_))
-    .WillRepeatedly(DoDefault());
-
-  EXPECT_CALL(exec, shutdown(_))
-    .WillRepeatedly(DoDefault());
+  Future<TaskStatus> status;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&status));
 
   driver.start();
 
-  AWAIT_READY(registered);
+  AWAIT_READY(status);
 
-  AWAIT_READY(resourceOffers1);
-
-  // The framework will be offered all of the resources on the slave,
-  // since it is the only framework running.
-  EXPECT_THAT(resourceOffers1.get(), OfferEq(2, 1024));
-
-  AWAIT_READY(statusUpdate);
-
-  EXPECT_EQ(TASK_RUNNING, statusUpdate.get().state());
+  EXPECT_EQ(TASK_RUNNING, status.get().state());
 
   // Stop the failing master from telling the slave to shut down when
   // it is killed.
@@ -308,6 +275,16 @@ TYPED_TEST(AllocatorZooKeeperTest, SlaveReregistersFirst)
   // slave has reregistered.
   DROP_PROTOBUFS(ReregisterFrameworkMessage(), _, _);
 
+  // Shutting down the masters will cause the scheduler to get
+  // disconnected.
+  EXPECT_CALL(sched, disconnected(_));
+
+  // Shutting down the masters will also cause the slave to shutdown
+  // frameworks that are not checkpointing, thus causing the executor
+  // to get shutdown.
+  EXPECT_CALL(exec, shutdown(_))
+    .Times(AtMost(1));
+
   this->ShutdownMasters();
 
   AWAIT_READY(shutdownMessage);
@@ -349,20 +326,15 @@ TYPED_TEST(AllocatorZooKeeperTest, SlaveReregistersFirst)
   EXPECT_CALL(allocator2, resourcesRecovered(_, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(allocator2, frameworkDeactivated(_));
+  EXPECT_CALL(allocator2, frameworkDeactivated(_))
+    .Times(AtMost(1));
 
-  Future<Nothing> frameworkRemoved;
   EXPECT_CALL(allocator2, frameworkRemoved(_))
-    .WillOnce(FutureSatisfy(&frameworkRemoved));
-
-  EXPECT_CALL(exec, shutdown(_))
     .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(frameworkRemoved);
-
   EXPECT_CALL(allocator2, slaveRemoved(_))
     .Times(AtMost(1));
 

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/bc9cb875/src/tests/fault_tolerance_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fault_tolerance_tests.cpp b/src/tests/fault_tolerance_tests.cpp
index 287453d..c8fad7c 100644
--- a/src/tests/fault_tolerance_tests.cpp
+++ b/src/tests/fault_tolerance_tests.cpp
@@ -274,7 +274,7 @@ TEST_F(FaultToleranceTest, PartitionedSlaveReregistration)
   // Drop the first shutdown message from the master (simulated
   // partition), allow the second shutdown message to pass when
   // the slave re-registers.
-  Future<ShutdownMessage> shutdownSlave =
+  Future<ShutdownMessage> shutdownMessage =
     DROP_PROTOBUF(ShutdownMessage(), _, slave.get());
 
   Future<TaskStatus> lostStatus;
@@ -309,7 +309,7 @@ TEST_F(FaultToleranceTest, PartitionedSlaveReregistration)
   EXPECT_EQ(TASK_LOST, lostStatus.get().state());
 
   // Wait for the master to attempt to shut down the slave.
-  AWAIT_READY(shutdownSlave);
+  AWAIT_READY(shutdownMessage);
 
   // The master will notify the framework that the slave was lost.
   AWAIT_READY(slaveLost);
@@ -319,11 +319,11 @@ TEST_F(FaultToleranceTest, PartitionedSlaveReregistration)
   // normally occur during a network partition.
   process::post(slave.get(), NoMasterDetectedMessage());
 
-  Future<Nothing> shutdownExecutor;
+  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdownExecutor));
+    .WillOnce(FutureSatisfy(&shutdown));
 
-  shutdownSlave = FUTURE_PROTOBUF(ShutdownMessage(), _, slave.get());
+  shutdownMessage = FUTURE_PROTOBUF(ShutdownMessage(), _, slave.get());
 
   // Have the slave re-register with the master.
   NewMasterDetectedMessage newMasterDetectedMessage;
@@ -332,8 +332,8 @@ TEST_F(FaultToleranceTest, PartitionedSlaveReregistration)
 
   // Upon re-registration, the master will shutdown the slave.
   // The slave will then shut down the executor.
-  AWAIT_READY(shutdownSlave);
-  AWAIT_READY(shutdownExecutor);
+  AWAIT_READY(shutdownMessage);
+  AWAIT_READY(shutdown);
 
   Clock::resume();
 
@@ -392,7 +392,7 @@ TEST_F(FaultToleranceTest, PartitionedSlaveStatusUpdates)
   // Drop the first shutdown message from the master (simulated
   // partition), allow the second shutdown message to pass when
   // the slave sends an update.
-  Future<ShutdownMessage> shutdownSlave =
+  Future<ShutdownMessage> shutdownMessage =
     DROP_PROTOBUF(ShutdownMessage(), _, slave.get());
 
   EXPECT_CALL(sched, offerRescinded(&driver, _))
@@ -422,12 +422,12 @@ TEST_F(FaultToleranceTest, PartitionedSlaveStatusUpdates)
   Clock::settle();
 
   // Wait for the master to attempt to shut down the slave.
-  AWAIT_READY(shutdownSlave);
+  AWAIT_READY(shutdownMessage);
 
   // The master will notify the framework that the slave was lost.
   AWAIT_READY(slaveLost);
 
-  shutdownSlave = FUTURE_PROTOBUF(ShutdownMessage(), _, slave.get());
+  shutdownMessage = FUTURE_PROTOBUF(ShutdownMessage(), _, slave.get());
 
   // At this point, the slave still thinks it's registered, so we
   // simulate a status update coming from the slave.
@@ -445,7 +445,7 @@ TEST_F(FaultToleranceTest, PartitionedSlaveStatusUpdates)
   process::post(master.get(), statusUpdate);
 
   // The master should shutdown the slave upon receiving the update.
-  AWAIT_READY(shutdownSlave);
+  AWAIT_READY(shutdownMessage);
 
   Clock::resume();
 
@@ -519,6 +519,7 @@ TEST_F(FaultToleranceTest, PartitionedSlaveExitedExecutor)
 
   // Set up the expectations for launching the task.
   EXPECT_CALL(exec, registered(_, _, _, _));
+
   EXPECT_CALL(exec, launchTask(_, _))
     .WillOnce(SendStatusUpdateFromTask(TASK_RUNNING));
 
@@ -532,7 +533,7 @@ TEST_F(FaultToleranceTest, PartitionedSlaveExitedExecutor)
   // Drop the first shutdown message from the master (simulated
   // partition) and allow the second shutdown message to pass when
   // triggered by the ExitedExecutorMessage.
-  Future<ShutdownMessage> shutdownSlave =
+  Future<ShutdownMessage> shutdownMessage =
     DROP_PROTOBUF(ShutdownMessage(), _, slave.get());
 
   Future<TaskStatus> lostStatus;
@@ -567,12 +568,12 @@ TEST_F(FaultToleranceTest, PartitionedSlaveExitedExecutor)
   EXPECT_EQ(TASK_LOST, lostStatus.get().state());
 
   // Wait for the master to attempt to shut down the slave.
-  AWAIT_READY(shutdownSlave);
+  AWAIT_READY(shutdownMessage);
 
   // The master will notify the framework that the slave was lost.
   AWAIT_READY(slaveLost);
 
-  shutdownSlave = FUTURE_PROTOBUF(ShutdownMessage(), _, slave.get());
+  shutdownMessage = FUTURE_PROTOBUF(ShutdownMessage(), _, slave.get());
 
   // Induce an ExitedExecutorMessage from the slave.
   dispatch(isolator,
@@ -581,7 +582,7 @@ TEST_F(FaultToleranceTest, PartitionedSlaveExitedExecutor)
            DEFAULT_EXECUTOR_INFO.executor_id());
 
   // Upon receiving the message, the master will shutdown the slave.
-  AWAIT_READY(shutdownSlave);
+  AWAIT_READY(shutdownMessage);
 
   Clock::resume();
 
@@ -964,9 +965,8 @@ TEST_F(FaultToleranceTest, SchedulerFailoverStatusUpdate)
 
   AWAIT_READY(statusUpdate);
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver1.stop();
   driver2.stop();
@@ -974,8 +974,6 @@ TEST_F(FaultToleranceTest, SchedulerFailoverStatusUpdate)
   driver1.join();
   driver2.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   Shutdown();
 
   Clock::resume();
@@ -1052,15 +1050,12 @@ TEST_F(FaultToleranceTest, ForwardStatusUpdateUnknownExecutor)
   EXPECT_EQ(taskId, status.get().task_id());
   EXPECT_EQ(TASK_RUNNING, status.get().state());
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   Shutdown();
 }
 
@@ -1144,9 +1139,8 @@ TEST_F(FaultToleranceTest, SchedulerFailoverFrameworkMessage)
 
   AWAIT_READY(frameworkMessage);
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver1.stop();
   driver2.stop();
@@ -1154,8 +1148,6 @@ TEST_F(FaultToleranceTest, SchedulerFailoverFrameworkMessage)
   driver1.join();
   driver2.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   Shutdown();
 }
 
@@ -1212,15 +1204,12 @@ TEST_F(FaultToleranceTest, SchedulerExit)
   AWAIT_READY(status);
   EXPECT_EQ(TASK_RUNNING, status.get().state());
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown);
-
   Shutdown();
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/bc9cb875/src/tests/gc_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/gc_tests.cpp b/src/tests/gc_tests.cpp
index 43c3e5d..1745800 100644
--- a/src/tests/gc_tests.cpp
+++ b/src/tests/gc_tests.cpp
@@ -313,9 +313,8 @@ TEST_F(GarbageCollectorIntegrationTest, Restart)
 
   Clock::pause();
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   EXPECT_CALL(sched, statusUpdate(_, _))
     .Times(AtMost(1)); // Ignore TASK_LOST from killed executor.
@@ -326,8 +325,6 @@ TEST_F(GarbageCollectorIntegrationTest, Restart)
 
   Stop(slave.get());
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   AWAIT_READY(slaveLost);
 
   Future<Nothing> schedule =
@@ -727,21 +724,22 @@ TEST_F(GarbageCollectorIntegrationTest, Unschedule)
   Future<Nothing> unscheduleFrameworkWork =
     FUTURE_DISPATCH(_, &GarbageCollectorProcess::unschedule);
 
-  // Launch the next run of the executor on the receipt of next offer.
-  EXPECT_CALL(sched, resourceOffers(_, _))
-    .WillOnce(LaunchTasks(1, cpus, mem));
+  // We ask the isolator to kill the exector below.
+  EXPECT_CALL(exec, shutdown(_))
+    .Times(AtMost(1));
 
   EXPECT_CALL(sched, statusUpdate(_, _))
-    .WillRepeatedly(Return());            // Ignore subsequent updates.
+    .Times(AtMost(2)); // Once for a TASK_LOST then once for TASK_RUNNING.
+
+  // We use the killed executor/tasks resources to run another task.
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(LaunchTasks(1, cpus, mem));
 
   EXPECT_CALL(exec, registered(_, _, _, _));
 
   EXPECT_CALL(exec, launchTask(_, _))
     .WillOnce(SendStatusUpdateFromTask(TASK_RUNNING));
 
-  EXPECT_CALL(exec, shutdown(_))
-    .WillRepeatedly(Return());
-
   Clock::pause();
 
   // Kill the first executor.

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/bc9cb875/src/tests/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator.hpp b/src/tests/isolator.hpp
index ebfc485..fe6b38d 100644
--- a/src/tests/isolator.hpp
+++ b/src/tests/isolator.hpp
@@ -67,7 +67,15 @@ public:
     setup();
   }
 
-  virtual ~TestingIsolator() {}
+  virtual ~TestingIsolator()
+  {
+    foreachvalue (MesosExecutorDriver* driver, drivers) {
+      driver->stop();
+      driver->join();
+      delete driver;
+    }
+    drivers.clear();
+  }
 
   virtual void initialize(
       const slave::Flags& flags,

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/bc9cb875/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index 1d26eeb..5ac4d5f 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -134,15 +134,12 @@ TEST_F(MasterTest, TaskRunning)
 
   AWAIT_READY(resourcesChanged);
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   Shutdown(); // Must shutdown before 'isolator' gets deallocated.
 }
 
@@ -209,15 +206,12 @@ TEST_F(MasterTest, ShutdownFrameworkWhileTaskRunning)
 
   AWAIT_READY(resourcesChanged);
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   Shutdown(); // Must shutdown before 'isolator' gets deallocated.
 }
 
@@ -287,15 +281,12 @@ TEST_F(MasterTest, KillTask)
   AWAIT_READY(status);
   EXPECT_EQ(TASK_KILLED, status.get().state());
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // To ensure can deallocate MockExecutor.
-
   Shutdown();
 }
 
@@ -357,15 +348,12 @@ TEST_F(MasterTest, StatusUpdateAck)
   // Ensure the slave gets a status update ACK.
   AWAIT_READY(acknowledgement);
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   Shutdown();
 }
 
@@ -465,6 +453,9 @@ TEST_F(MasterTest, RecoverResources)
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
 
+  EXPECT_CALL(exec, shutdown(_))
+    .Times(AtMost(1));
+
   // Now kill the executor, scheduler should get an offer it's resources.
   // TODO(benh): WTF? Why aren't we dispatching?
   isolator.killExecutor(offer.framework_id(), executorInfo.executor_id());
@@ -482,11 +473,6 @@ TEST_F(MasterTest, RecoverResources)
   driver.stop();
   driver.join();
 
-  // Terminating the slave might cause the mock executor to get a
-  // shutdown since the executor driver "links" the slave.
-  EXPECT_CALL(exec, shutdown(_))
-    .Times(AtMost(1));
-
   Shutdown(); // Must shutdown before 'isolator' gets deallocated.
 }
 
@@ -562,15 +548,12 @@ TEST_F(MasterTest, FrameworkMessage)
   AWAIT_READY(schedData);
   EXPECT_EQ("world", schedData.get());
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   schedDriver.stop();
   schedDriver.join();
 
-  AWAIT_READY(shutdown); // To ensure can deallocate MockExecutor.
-
   Shutdown();
 }
 
@@ -673,20 +656,15 @@ TEST_F(MasterTest, MultipleExecutors)
   AWAIT_READY(status2);
   EXPECT_EQ(TASK_RUNNING, status2.get().state());
 
-  Future<Nothing> shutdown1;
   EXPECT_CALL(exec1, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown1));
+    .Times(AtMost(1));
 
-  Future<Nothing> shutdown2;
   EXPECT_CALL(exec2, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown2));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown1); // To ensure can deallocate MockExecutor.
-  AWAIT_READY(shutdown2); // To ensure can deallocate MockExecutor.
-
   Shutdown(); // Must shutdown before 'isolator' gets deallocated.
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/bc9cb875/src/tests/resource_offers_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resource_offers_tests.cpp b/src/tests/resource_offers_tests.cpp
index e030d3d..b066403 100644
--- a/src/tests/resource_offers_tests.cpp
+++ b/src/tests/resource_offers_tests.cpp
@@ -573,14 +573,11 @@ TEST_F(MultipleExecutorsTest, TasksExecutorInfoDiffers)
             " with same ExecutorID is not compatible)",
             status.get().message());
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillOnce(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // To ensure can deallocate MockExecutor.
-
   Shutdown();
 }

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/bc9cb875/src/tests/status_update_manager_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/status_update_manager_tests.cpp b/src/tests/status_update_manager_tests.cpp
index 892a3f1..61ccfcc 100644
--- a/src/tests/status_update_manager_tests.cpp
+++ b/src/tests/status_update_manager_tests.cpp
@@ -64,6 +64,7 @@ using std::string;
 using std::vector;
 
 using testing::_;
+using testing::AtMost;
 using testing::Return;
 using testing::SaveArg;
 
@@ -176,15 +177,12 @@ TEST_F(StatusUpdateManagerTest, CheckpointStatusUpdate)
 
   close(fd.get());
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillRepeatedly(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   Shutdown();
 }
 
@@ -249,15 +247,12 @@ TEST_F(StatusUpdateManagerTest, RetryStatusUpdate)
 
   Clock::resume();
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillRepeatedly(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   Shutdown();
 }
 
@@ -369,15 +364,12 @@ TEST_F(StatusUpdateManagerTest, IgnoreDuplicateStatusUpdateAck)
 
   Clock::resume();
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillRepeatedly(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   Shutdown();
 }
 
@@ -464,14 +456,11 @@ TEST_F(StatusUpdateManagerTest, IgnoreUnexpectedStatusUpdateAck)
   // it is 'false'.
   AWAIT_READY(unexpectedAck);
 
-  Future<Nothing> shutdown;
   EXPECT_CALL(exec, shutdown(_))
-    .WillRepeatedly(FutureSatisfy(&shutdown));
+    .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
-
   Shutdown();
 }