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 2015/07/17 19:44:41 UTC

[7/9] mesos git commit: Updated scheduler driver to send TEARDOWN call.

Updated scheduler driver to send TEARDOWN call.

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


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

Branch: refs/heads/master
Commit: 51c2b523d0b548b5882371d3a87dab13f9aec4dc
Parents: cf485e2
Author: Vinod Kone <vi...@gmail.com>
Authored: Mon Jul 13 15:51:17 2015 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Jul 17 10:44:00 2015 -0700

----------------------------------------------------------------------
 src/sched/sched.cpp                  | 10 +++++++---
 src/tests/exception_tests.cpp        | 10 ++++++----
 src/tests/fault_tolerance_tests.cpp  | 13 +++++++++++--
 src/tests/master_allocator_tests.cpp |  4 +++-
 src/tests/master_tests.cpp           | 23 ++++++++++-------------
 src/tests/rate_limiting_tests.cpp    | 10 +++++-----
 src/tests/slave_recovery_tests.cpp   |  8 ++++----
 7 files changed, 46 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/51c2b523/src/sched/sched.cpp
----------------------------------------------------------------------
diff --git a/src/sched/sched.cpp b/src/sched/sched.cpp
index c563c44..8163796 100644
--- a/src/sched/sched.cpp
+++ b/src/sched/sched.cpp
@@ -990,10 +990,14 @@ protected:
     terminate(self());
 
     if (connected && !failover) {
-      UnregisterFrameworkMessage message;
-      message.mutable_framework_id()->MergeFrom(framework.id());
+      Call call;
+
+      CHECK(framework.has_id());
+      call.mutable_framework_id()->CopyFrom(framework.id());
+      call.set_type(Call::TEARDOWN);
+
       CHECK_SOME(master);
-      send(master.get(), message);
+      send(master.get(), call);
     }
 
     synchronized (mutex) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/51c2b523/src/tests/exception_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/exception_tests.cpp b/src/tests/exception_tests.cpp
index 9af1674..658e485 100644
--- a/src/tests/exception_tests.cpp
+++ b/src/tests/exception_tests.cpp
@@ -21,6 +21,8 @@
 #include <mesos/executor.hpp>
 #include <mesos/scheduler.hpp>
 
+#include <mesos/scheduler/scheduler.hpp>
+
 #include <process/gmock.hpp>
 #include <process/pid.hpp>
 #include <process/process.hpp>
@@ -190,13 +192,13 @@ TEST_F(ExceptionTest, DisallowSchedulerCallbacksOnAbort)
 
   AWAIT_READY(rescindMsg);
 
-  Future<UnregisterFrameworkMessage> unregisterMsg =
-    FUTURE_PROTOBUF(UnregisterFrameworkMessage(), _, _);
+  Future<mesos::scheduler::Call> teardownCall = FUTURE_CALL(
+      mesos::scheduler::Call(), mesos::scheduler::Call::TEARDOWN, _, _);
 
   driver.stop();
 
-  //Ensures reception of RescindResourceOfferMessage.
-  AWAIT_READY(unregisterMsg);
+  // Ensures reception of RescindResourceOfferMessage.
+  AWAIT_READY(teardownCall);
 
   Shutdown();
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/51c2b523/src/tests/fault_tolerance_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fault_tolerance_tests.cpp b/src/tests/fault_tolerance_tests.cpp
index 60ca523..72f4cab 100644
--- a/src/tests/fault_tolerance_tests.cpp
+++ b/src/tests/fault_tolerance_tests.cpp
@@ -1310,8 +1310,7 @@ TEST_F(FaultToleranceTest, SchedulerFailoverFrameworkToExecutorMessage)
 
   MockScheduler sched2;
 
-  FrameworkInfo framework2; // Bug in gcc 4.1.*, must assign on next line.
-  framework2 = DEFAULT_FRAMEWORK_INFO;
+  FrameworkInfo framework2 = DEFAULT_FRAMEWORK_INFO;
   framework2.mutable_id()->MergeFrom(frameworkId);
 
   MesosSchedulerDriver driver2(
@@ -1335,9 +1334,19 @@ TEST_F(FaultToleranceTest, SchedulerFailoverFrameworkToExecutorMessage)
   EXPECT_CALL(exec, frameworkMessage(_, _))
     .WillOnce(FutureArg<1>(&frameworkMessage));
 
+  // Since 'sched2' doesn't receive any offers the framework message
+  // should go through the master.
+  Future<mesos::scheduler::Call> messageCall = FUTURE_CALL(
+      mesos::scheduler::Call(),
+      mesos::scheduler::Call::MESSAGE,
+      _,
+      master.get());
+
   driver2.sendFrameworkMessage(
       DEFAULT_EXECUTOR_ID, offers.get()[0].slave_id(), "hello world");
 
+  AWAIT_READY(messageCall);
+
   AWAIT_EQ("hello world", frameworkMessage);
 
   EXPECT_CALL(exec, shutdown(_))

http://git-wip-us.apache.org/repos/asf/mesos/blob/51c2b523/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index 534b248..147f510 100644
--- a/src/tests/master_allocator_tests.cpp
+++ b/src/tests/master_allocator_tests.cpp
@@ -29,6 +29,8 @@
 
 #include <mesos/module/allocator.hpp>
 
+#include <mesos/scheduler/scheduler.hpp>
+
 #include <process/clock.hpp>
 #include <process/future.hpp>
 #include <process/gmock.hpp>
@@ -440,7 +442,7 @@ TYPED_TEST(MasterAllocatorTest, SchedulerFailover)
   // When we shut down the first framework, we don't want it to tell
   // the master it's shutting down so that the master will wait to see
   // if it fails over.
-  DROP_PROTOBUFS(UnregisterFrameworkMessage(), _, _);
+  DROP_CALLS(mesos::scheduler::Call(), mesos::scheduler::Call::TEARDOWN, _, _);
 
   Future<Nothing> deactivateFramework;
   EXPECT_CALL(allocator, deactivateFramework(_))

http://git-wip-us.apache.org/repos/asf/mesos/blob/51c2b523/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index fdee267..9205ec4 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -241,13 +241,10 @@ TEST_F(MasterTest, ShutdownFrameworkWhileTaskRunning)
 
   AWAIT_READY(update);
 
-  // Set expectation that Master receives UnregisterFrameworkMessage,
-  // which triggers marking running tasks as killed.
-  UnregisterFrameworkMessage message;
-  message.mutable_framework_id()->MergeFrom(offer.framework_id());
-
-  Future<UnregisterFrameworkMessage> unregisterFrameworkMessage =
-    FUTURE_PROTOBUF(message, _, master.get());
+  // Set expectation that Master receives teardown call, which
+  // triggers marking running tasks as killed.
+  Future<mesos::scheduler::Call> teardownCall = FUTURE_CALL(
+      mesos::scheduler::Call(), mesos::scheduler::Call::TEARDOWN, _, _);
 
   // Set expectation that Executor's shutdown callback is invoked.
   Future<Nothing> shutdown;
@@ -258,14 +255,14 @@ TEST_F(MasterTest, ShutdownFrameworkWhileTaskRunning)
   driver.stop();
   driver.join();
 
-  // Wait for UnregisterFrameworkMessage message to be dispatched and
-  // executor's shutdown callback to be called.
-  AWAIT_READY(unregisterFrameworkMessage);
+  // Wait for teardown call to be dispatched and executor's shutdown
+  // callback to be called.
+  AWAIT_READY(teardownCall);
   AWAIT_READY(shutdown);
 
-  // We have to be sure the UnregisterFrameworkMessage is processed
-  // completely and running tasks enter a terminal state before we
-  // request the master state.
+  // We have to be sure the teardown call is processed completely and
+  // running tasks enter a terminal state before we request the master
+  // state.
   Clock::pause();
   Clock::settle();
   Clock::resume();

http://git-wip-us.apache.org/repos/asf/mesos/blob/51c2b523/src/tests/rate_limiting_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/rate_limiting_tests.cpp b/src/tests/rate_limiting_tests.cpp
index 49d907b..6a93df0 100644
--- a/src/tests/rate_limiting_tests.cpp
+++ b/src/tests/rate_limiting_tests.cpp
@@ -178,10 +178,10 @@ TEST_F(RateLimitingTest, NoRateLimiting)
   driver->join();
   delete driver;
 
-  // The fact that UnregisterFrameworkMessage (the 2nd message from
-  // 'sched' that reaches Master after its registration) gets
-  // processed without Clock advances proves that the framework is
-  // given unlimited rate.
+  // The fact that the teardown call (the 2nd call from the scheduler
+  // that reaches Master after its registration) gets processed
+  // without Clock advances proves that the framework is given
+  // unlimited rate.
   AWAIT_READY(removeFramework);
 
   // For metrics endpoint.
@@ -724,7 +724,7 @@ TEST_F(RateLimitingTest, SamePrincipalFrameworks)
   driver1->join();
   delete driver1;
 
-  // Advance to let UnregisterFrameworkMessage come through.
+  // Advance to let the teardown call come through.
   Clock::settle();
   Clock::advance(Seconds(1));
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/51c2b523/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index ff7aaf9..de2fc28 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -2151,15 +2151,15 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileShutdownFramework)
   this->Stop(slave.get());
   delete containerizer1.get();
 
-  Future<UnregisterFrameworkMessage> unregisterFrameworkMessage =
-    FUTURE_PROTOBUF(UnregisterFrameworkMessage(), _, _);
+  Future<mesos::scheduler::Call> teardownCall = FUTURE_CALL(
+      mesos::scheduler::Call(), mesos::scheduler::Call::TEARDOWN, _, _);
 
   // Now stop the framework.
   driver.stop();
   driver.join();
 
-  // Wait util the framework is removed.
-  AWAIT_READY(unregisterFrameworkMessage);
+  // Wait until the framework is removed.
+  AWAIT_READY(teardownCall);
 
   Future<ShutdownFrameworkMessage> shutdownFrameworkMessage =
     FUTURE_PROTOBUF(ShutdownFrameworkMessage(), _, _);