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 2016/03/03 03:10:43 UTC

[3/3] mesos git commit: Removed 'Subscribe.force' field from v1 Scheduler API.

Removed 'Subscribe.force' field from v1 Scheduler API.

See the https://issues.apache.org/jira/browse/MESOS-4712 for details.

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


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

Branch: refs/heads/master
Commit: 503fa896def670e0911b6dcd8affb478b82c1c6e
Parents: f0fb404
Author: Vinod Kone <vi...@gmail.com>
Authored: Tue Mar 1 11:13:55 2016 -0800
Committer: Vinod Kone <vi...@gmail.com>
Committed: Wed Mar 2 18:10:17 2016 -0800

----------------------------------------------------------------------
 include/mesos/scheduler/scheduler.proto    | 20 ++---
 include/mesos/v1/scheduler/scheduler.proto | 15 ----
 src/examples/event_call_framework.cpp      |  4 -
 src/tests/scheduler_http_api_tests.cpp     | 98 +------------------------
 src/tests/scheduler_tests.cpp              |  1 -
 5 files changed, 9 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/503fa896/include/mesos/scheduler/scheduler.proto
----------------------------------------------------------------------
diff --git a/include/mesos/scheduler/scheduler.proto b/include/mesos/scheduler/scheduler.proto
index 4aa32e6..0049e13 100644
--- a/include/mesos/scheduler/scheduler.proto
+++ b/include/mesos/scheduler/scheduler.proto
@@ -185,19 +185,13 @@ message Call {
     // 'framework_info.id'.
     required FrameworkInfo framework_info = 1;
 
-    // 'force' field is only relevant when 'framework_info.id' is set.
-    // It tells the master what to do in case an instance of the
-    // scheduler attempts to subscribe when another instance of it is
-    // already connected (e.g., split brain due to network partition).
-    // If 'force' is true, this scheduler instance is allowed and the
-    // old connected scheduler instance is disconnected. If false,
-    // this scheduler instance is disallowed subscription in favor of
-    // the already connected scheduler instance.
-    //
-    // It is recommended to set this to true only when a newly elected
-    // scheduler instance is attempting to subscribe but not when a
-    // scheduler is retrying subscription (e.g., disconnection or
-    // master failover; see sched/sched.cpp for an example).
+    // NOTE: 'force' field is not present in v1/scheduler.proto because it is
+    // only used by the scheduler driver. The driver sets it to true when the
+    // scheduler re-registers for the first time after a failover. Once
+    // re-registered all subsequent re-registration attempts (e.g., due to ZK
+    // blip) will have 'force' set to false. This is important because master
+    // uses this field to know when it needs to send FrameworkRegisteredMessage
+    // vs FrameworkReregisteredMessage.
     optional bool force = 2;
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/503fa896/include/mesos/v1/scheduler/scheduler.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/scheduler/scheduler.proto b/include/mesos/v1/scheduler/scheduler.proto
index 5d0e6c7..09fafed 100644
--- a/include/mesos/v1/scheduler/scheduler.proto
+++ b/include/mesos/v1/scheduler/scheduler.proto
@@ -184,21 +184,6 @@ message Call {
     // See the comments below on 'framework_id' on the semantics for
     // 'framework_info.id'.
     required FrameworkInfo framework_info = 1;
-
-    // 'force' field is only relevant when 'framework_info.id' is set.
-    // It tells the master what to do in case an instance of the
-    // scheduler attempts to subscribe when another instance of it is
-    // already connected (e.g., split brain due to network partition).
-    // If 'force' is true, this scheduler instance is allowed and the
-    // old connected scheduler instance is disconnected. If false,
-    // this scheduler instance is disallowed subscription in favor of
-    // the already connected scheduler instance.
-    //
-    // It is recommended to set this to true only when a newly elected
-    // scheduler instance is attempting to subscribe but not when a
-    // scheduler is retrying subscription (e.g., disconnection or
-    // master failover; see sched/sched.cpp for an example).
-    optional bool force = 2;
   }
 
   // Accepts an offer, performing the specified operations

http://git-wip-us.apache.org/repos/asf/mesos/blob/503fa896/src/examples/event_call_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/event_call_framework.cpp b/src/examples/event_call_framework.cpp
index d07d05d..9bac802 100644
--- a/src/examples/event_call_framework.cpp
+++ b/src/examples/event_call_framework.cpp
@@ -319,10 +319,6 @@ private:
     Call::Subscribe* subscribe = call.mutable_subscribe();
     subscribe->mutable_framework_info()->CopyFrom(framework);
 
-    if (framework.has_id()) {
-      subscribe->set_force(true);
-    }
-
     mesos->send(call);
 
     process::delay(Seconds(1),

http://git-wip-us.apache.org/repos/asf/mesos/blob/503fa896/src/tests/scheduler_http_api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/scheduler_http_api_tests.cpp b/src/tests/scheduler_http_api_tests.cpp
index 428e126..a8d660a 100644
--- a/src/tests/scheduler_http_api_tests.cpp
+++ b/src/tests/scheduler_http_api_tests.cpp
@@ -326,7 +326,7 @@ TEST_P(SchedulerHttpApiTest, Subscribe)
 
 // This test verifies if the scheduler can subscribe on retrying,
 // e.g. after a ZK blip.
-TEST_P(SchedulerHttpApiTest, SubscribedOnRetryWithForce)
+TEST_P(SchedulerHttpApiTest, SubscribedOnRetry)
 {
   // HTTP schedulers cannot yet authenticate.
   master::Flags flags = CreateMasterFlags();
@@ -379,9 +379,6 @@ TEST_P(SchedulerHttpApiTest, SubscribedOnRetryWithForce)
   }
 
   {
-    // Now subscribe again with force set.
-    subscribe->set_force(true);
-
     call.mutable_framework_id()->CopyFrom(frameworkId);
     subscribe->mutable_framework_info()->mutable_id()->CopyFrom(frameworkId);
 
@@ -419,7 +416,7 @@ TEST_P(SchedulerHttpApiTest, SubscribedOnRetryWithForce)
 
 
 // This test verifies if we are able to upgrade from a PID based
-// framework to HTTP when force is set.
+// scheduler to HTTP scheduler.
 TEST_P(SchedulerHttpApiTest, UpdatePidToHttpScheduler)
 {
   // HTTP schedulers cannot yet authenticate.
@@ -464,8 +461,6 @@ TEST_P(SchedulerHttpApiTest, UpdatePidToHttpScheduler)
   subscribe->mutable_framework_info()->mutable_id()->
     CopyFrom(evolve(frameworkId.get()));
 
-  subscribe->set_force(true);
-
   // Retrieve the parameter passed as content type to this test.
   const string contentType = GetParam();
   process::http::Headers headers;
@@ -592,95 +587,6 @@ TEST_P(SchedulerHttpApiTest, UpdateHttpToPidScheduler)
 }
 
 
-// This test verifies that updating a PID based framework to HTTP
-// framework fails when force is not set and the PID based
-// framework is already connected.
-TEST_P(SchedulerHttpApiTest, UpdatePidToHttpSchedulerWithoutForce)
-{
-  // HTTP schedulers cannot yet authenticate.
-  master::Flags flags = CreateMasterFlags();
-  flags.authenticate_frameworks = false;
-
-  Try<PID<Master>> master = StartMaster(flags);
-  ASSERT_SOME(master);
-
-  v1::FrameworkInfo frameworkInfo = DEFAULT_V1_FRAMEWORK_INFO;
-  frameworkInfo.set_failover_timeout(Weeks(2).secs());
-
-  // Start the scheduler without credentials.
-  MockScheduler sched;
-  StandaloneMasterDetector detector(master.get());
-  TestingMesosSchedulerDriver driver(&sched, &detector, devolve(frameworkInfo));
-
-  Future<FrameworkID> frameworkId;
-  EXPECT_CALL(sched, registered(&driver, _, _))
-    .WillOnce(FutureArg<1>(&frameworkId));
-
-  driver.start();
-
-  AWAIT_READY(frameworkId);
-  EXPECT_NE("", frameworkId.get().value());
-
-  // Now try to subscribe using a HTTP framework without setting the
-  // 'force' field.
-  Call call;
-  call.set_type(Call::SUBSCRIBE);
-  call.mutable_framework_id()->CopyFrom(evolve(frameworkId.get()));
-
-  Call::Subscribe* subscribe = call.mutable_subscribe();
-  subscribe->mutable_framework_info()->CopyFrom(frameworkInfo);
-  subscribe->mutable_framework_info()->mutable_id()->
-    CopyFrom(evolve(frameworkId.get()));
-
-  // Retrieve the parameter passed as content type to this test.
-  const string contentType = GetParam();
-  process::http::Headers headers;
-  headers["Accept"] = contentType;
-
-  Future<Response> response = process::http::streaming::post(
-      master.get(),
-      "api/v1/scheduler",
-      headers,
-      serialize(call, contentType),
-      contentType);
-
-  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
-  AWAIT_EXPECT_RESPONSE_HEADER_EQ("chunked", "Transfer-Encoding", response);
-  ASSERT_EQ(Response::PIPE, response.get().type);
-
-  Option<Pipe::Reader> reader = response.get().reader;
-  ASSERT_SOME(reader);
-
-  auto deserializer = lambda::bind(
-      &SchedulerHttpApiTest::deserialize, this, contentType, lambda::_1);
-
-  Reader<Event> responseDecoder(Decoder<Event>(deserializer), reader.get());
-
-  Future<Result<Event>> event = responseDecoder.read();
-  AWAIT_READY(event);
-  ASSERT_SOME(event.get());
-
-  // We should be receiving an error event since the PID framework
-  // was already connected.
-  ASSERT_EQ(Event::ERROR, event.get().get().type());
-
-  // Unsubscribed HTTP framework should not get any heartbeats.
-  Clock::pause();
-  Clock::advance(DEFAULT_HEARTBEAT_INTERVAL);
-  Clock::settle();
-
-  // The next read should be EOF.
-  event = responseDecoder.read();
-  AWAIT_READY(event);
-  EXPECT_NONE(event.get());
-
-  driver.stop();
-  driver.join();
-
-  Shutdown();
-}
-
-
 TEST_P(SchedulerHttpApiTest, NotAcceptable)
 {
   // HTTP schedulers cannot yet authenticate.

http://git-wip-us.apache.org/repos/asf/mesos/blob/503fa896/src/tests/scheduler_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/scheduler_tests.cpp b/src/tests/scheduler_tests.cpp
index 2b1693e..06cef6b 100644
--- a/src/tests/scheduler_tests.cpp
+++ b/src/tests/scheduler_tests.cpp
@@ -244,7 +244,6 @@ TEST_P(SchedulerTest, SchedulerFailover)
     Call::Subscribe* subscribe = call.mutable_subscribe();
     subscribe->mutable_framework_info()->CopyFrom(DEFAULT_V1_FRAMEWORK_INFO);
     subscribe->mutable_framework_info()->mutable_id()->CopyFrom(frameworkId);
-    subscribe->set_force(true);
 
     mesos2.send(call);
   }