You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ya...@apache.org on 2014/05/27 19:01:12 UTC

git commit: Keep track of the principals for authenticated pids in Master and verify FramworkInfo::principal during (re-)registration.

Repository: mesos
Updated Branches:
  refs/heads/master 7026fc7b7 -> 70725089e


Keep track of the principals for authenticated pids in Master and verify FramworkInfo::principal during (re-)registration.

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


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

Branch: refs/heads/master
Commit: 70725089e9c20c2d0a85ad9b24bb018a01181ae3
Parents: 7026fc7
Author: Jiang Yan Xu <ya...@jxu.me>
Authored: Wed May 21 14:54:07 2014 -0700
Committer: Jiang Yan Xu <ya...@jxu.me>
Committed: Tue May 27 08:13:51 2014 -0700

----------------------------------------------------------------------
 include/mesos/mesos.proto          |   5 ++
 src/master/master.cpp              |  83 +++++++++++++------
 src/master/master.hpp              |   4 +-
 src/tests/authentication_tests.cpp | 141 ++++++++++++++++++++++++++++++++
 src/tests/mesos.hpp                |  13 +--
 5 files changed, 214 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/70725089/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index ce780ca..82388e1 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -111,6 +111,10 @@ message ContainerID {
  * decisions, depending on the allocation policy being used.
  * If the hostname field is set to an empty string Mesos will
  * automagically set it to the current hostname.
+ * The principal field should match the credential the framework uses
+ * in authentication. This field is used for framework API rate
+ * exporting and limiting and should be set even if authentication is
+ * not enabled if these features are desired.
  */
 message FrameworkInfo {
   required string user = 1;
@@ -120,6 +124,7 @@ message FrameworkInfo {
   optional bool checkpoint = 5 [default = false];
   optional string role = 6 [default = "*"];
   optional string hostname = 7;
+  optional string principal = 8;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/70725089/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index e5d8edf..ae738bb 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -980,17 +980,34 @@ void Master::registerFramework(
     return;
   }
 
-  if (flags.authenticate_frameworks && !authenticated.contains(from)) {
-    // This could happen if another authentication request came
-    // through before we are here or if a framework tried to register
-    // without authentication.
-    LOG(WARNING) << "Refusing registration of framework at " << from
-                 << " because it is not authenticated";
-    FrameworkErrorMessage message;
-    message.set_message("Framework at " + stringify(from) +
-                        " is not authenticated.");
-    send(from, message);
-    return;
+  if (flags.authenticate_frameworks) {
+    if (!authenticated.contains(from)) {
+      // This could happen if another authentication request came
+      // through before we are here or if a framework tried to register
+      // without authentication.
+      LOG(WARNING) << "Refusing registration of framework at " << from
+                   << " because it is not authenticated";
+      FrameworkErrorMessage message;
+      message.set_message("Framework at " + stringify(from) +
+                          " is not authenticated.");
+      send(from, message);
+      return;
+    } else if (frameworkInfo.has_principal() &&
+               frameworkInfo.principal() != authenticated[from]) {
+      LOG(WARNING) << "Refusing registration of framework at " << from
+                   << " because its principal '" << frameworkInfo.principal()
+                   << "' does not match what it used in authentication: '"
+                   << authenticated[from] << "'";
+      FrameworkErrorMessage message;
+      message.set_message("Framework principal " + frameworkInfo.principal() +
+                          " does not match what was used in authentication: " +
+                          authenticated[from]);
+      send(from, message);
+      return;
+    } else if (!frameworkInfo.has_principal()) {
+      LOG(WARNING) << "Framework at " << from
+                   << " does not specify principal in its FrameworkInfo";
+    }
   }
 
   if (!roles.contains(frameworkInfo.role())) {
@@ -1064,17 +1081,34 @@ void Master::reregisterFramework(
     return;
   }
 
-  if (flags.authenticate_frameworks && !authenticated.contains(from)) {
-    // This could happen if another authentication request came
-    // through before we are here or if a framework tried to
-    // re-register without authentication.
-    LOG(WARNING) << "Refusing re-registration of framework at " << from
-                 << " because it is not authenticated";
-    FrameworkErrorMessage message;
-    message.set_message("Framework '" + frameworkInfo.id().value() + "' at " +
-                        stringify(from) + " is not authenticated.");
-    send(from, message);
-    return;
+  if (flags.authenticate_frameworks) {
+    if (!authenticated.contains(from)) {
+      // This could happen if another authentication request came
+      // through before we are here or if a framework tried to
+      // re-register without authentication.
+      LOG(WARNING) << "Refusing re-registration of framework at " << from
+                   << " because it is not authenticated";
+      FrameworkErrorMessage message;
+      message.set_message("Framework '" + frameworkInfo.id().value() + "' at " +
+                          stringify(from) + " is not authenticated.");
+      send(from, message);
+      return;
+    } else if (frameworkInfo.has_principal() &&
+               frameworkInfo.principal() != authenticated[from]) {
+      LOG(WARNING) << "Refusing re-registration of framework at " << from
+                   << " because its principal '" << frameworkInfo.principal()
+                   << "' does not match what it used in authentication: '"
+                   << authenticated[from] << "'";
+      FrameworkErrorMessage message;
+      message.set_message("Framework principal " + frameworkInfo.principal() +
+                          " does not match what was used in authentication: " +
+                          authenticated[from]);
+      send(from, message);
+      return;
+    } else if (!frameworkInfo.has_principal()) {
+      LOG(WARNING) << "Framework at " << from
+                   << " does not specify principal in its FrameworkInfo";
+    }
   }
 
   if (!roles.contains(frameworkInfo.role())) {
@@ -2899,10 +2933,11 @@ void Master::_authenticate(
 
     promise->fail(error);
   } else {
-    LOG(INFO) << "Successfully authenticated " << pid;
+    LOG(INFO) << "Successfully authenticated principal '" << future.get().get()
+              << "' at " << pid;
 
     promise->set(Nothing());
-    authenticated.insert(pid);
+    authenticated.put(pid, future.get().get());
   }
 
   authenticators.erase(pid);

http://git-wip-us.apache.org/repos/asf/mesos/blob/70725089/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 99c7063..847d7c7 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -456,8 +456,8 @@ private:
 
   hashmap<process::UPID, process::Owned<sasl::Authenticator> > authenticators;
 
-  // Authenticated frameworks/slaves keyed by PID.
-  hashset<process::UPID> authenticated;
+  // Principals of authenticated frameworks/slaves keyed by PID.
+  hashmap<process::UPID, std::string> authenticated;
 
   Option<process::Owned<Authorizer> > authorizer;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/70725089/src/tests/authentication_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/authentication_tests.cpp b/src/tests/authentication_tests.cpp
index b60e595..5cf2da4 100644
--- a/src/tests/authentication_tests.cpp
+++ b/src/tests/authentication_tests.cpp
@@ -157,6 +157,76 @@ TEST_F(AuthenticationTest, DisableSlaveAuthentication)
 }
 
 
+// This test verifies that an authenticated framework is denied
+// registration by the master if it uses a different
+// FrameworkInfo::principal.
+TEST_F(AuthenticationTest, MismatchedFrameworkInfoPrincipal)
+{
+  Try<PID<Master> > master = StartMaster();
+  ASSERT_SOME(master);
+
+  MockScheduler sched;
+  FrameworkInfo frameworkInfo; // Bug in gcc 4.1.*, must assign on next line.
+  frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.set_principal("mismatched-principal");
+
+  MesosSchedulerDriver driver(
+      &sched,
+      frameworkInfo,
+      master.get(),
+      DEFAULT_CREDENTIAL);
+
+  Future<Nothing> error;
+  EXPECT_CALL(sched, error(&driver, _))
+    .WillOnce(FutureSatisfy(&error));
+
+  driver.start();
+
+  // Scheduler should get error message from the master.
+  AWAIT_READY(error);
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
+
+// This test verifies that if a Framework successfully authenticates
+// but does not set FrameworkInfo::principal, it is allowed to
+// register.
+TEST_F(AuthenticationTest, UnspecifiedFrameworkInfoPrincipal)
+{
+  Try<PID<Master> > master = StartMaster();
+  ASSERT_SOME(master);
+
+  MockScheduler sched;
+  FrameworkInfo frameworkInfo; // Bug in gcc 4.1.*, must assign on next line.
+  frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.clear_principal();
+
+  MesosSchedulerDriver driver(
+      &sched,
+      frameworkInfo,
+      master.get(),
+      DEFAULT_CREDENTIAL);
+
+  Future<Nothing> registered;
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .WillOnce(FutureSatisfy(&registered));
+
+  driver.start();
+
+  // Scheduler should be able to get registered.
+  AWAIT_READY(registered);
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
+
 // This test verifies that when the master is started with
 // authentication disabled, it registers authenticated frameworks.
 TEST_F(AuthenticationTest, AuthenticatedFramework)
@@ -690,3 +760,74 @@ TEST_F(AuthenticationTest, SchedulerFailover)
 
   Shutdown();
 }
+
+
+// This test verifies that a scheduler's re-registration will be
+// rejected if it specifies a principal different from what's used in
+// authentication.
+TEST_F(AuthenticationTest, RejectedSchedulerFailover)
+{
+  Try<PID<Master> > master = StartMaster();
+  ASSERT_SOME(master);
+
+  // Launch the first scheduler.
+  MockScheduler sched1;
+  Owned<StandaloneMasterDetector> detector(
+      new StandaloneMasterDetector(master.get()));
+  TestingMesosSchedulerDriver driver1(&sched1, detector.get());
+
+  Future<FrameworkID> frameworkId;
+  EXPECT_CALL(sched1, registered(&driver1, _, _))
+    .WillOnce(FutureArg<1>(&frameworkId));
+
+  driver1.start();
+
+  AWAIT_READY(frameworkId);
+
+  // Drop the AuthenticationStepMessage from authenticator
+  // to stop authentication from succeeding.
+  Future<AuthenticationStepMessage> authenticationStepMessage =
+    DROP_PROTOBUF(AuthenticationStepMessage(), _, _);
+
+  EXPECT_CALL(sched1, disconnected(&driver1));
+
+  // Appoint a new master and inform the scheduler about it.
+  detector->appoint(master.get());
+
+  AWAIT_READY(authenticationStepMessage);
+
+  // Attempt to failover to scheduler 2 while scheduler 1 is still
+  // up. We use the framework id recorded from scheduler 1 but change
+  // the principal in FrameworInfo and it will be denied. Scheduler 1
+  // will not be asked to shutdown.
+  MockScheduler sched2;
+
+  FrameworkInfo framework2; // Bug in gcc 4.1.*, must assign on next line.
+  framework2 = DEFAULT_FRAMEWORK_INFO;
+  framework2.mutable_id()->MergeFrom(frameworkId.get());
+  framework2.set_principal("mismatched-principal");
+
+  MesosSchedulerDriver driver2(
+      &sched2, framework2, master.get(), DEFAULT_CREDENTIAL);
+
+  Future<Nothing> sched1Error;
+  EXPECT_CALL(sched1, error(&driver1, _))
+    .Times(0);
+
+  Future<Nothing> sched2Error;
+  EXPECT_CALL(sched2, error(&driver2, _))
+    .WillOnce(FutureSatisfy(&sched2Error));
+
+  driver2.start();
+
+  // Scheduler 2 should get error message from the master.
+  AWAIT_READY(sched2Error);
+
+  driver2.stop();
+  driver2.join();
+
+  driver1.stop();
+  driver1.join();
+
+  Shutdown();
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/70725089/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index a1a7d79..c31199a 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -263,12 +263,6 @@ protected:
         executor; })
 
 
-#define DEFAULT_FRAMEWORK_INFO                                          \
-     ({ FrameworkInfo framework;                                        \
-        framework.set_name("default");                                  \
-        framework; })
-
-
 #define DEFAULT_CREDENTIAL                                             \
      ({ Credential credential;                                         \
         credential.set_principal("test-principal");                    \
@@ -276,6 +270,13 @@ protected:
         credential; })
 
 
+#define DEFAULT_FRAMEWORK_INFO                                          \
+     ({ FrameworkInfo framework;                                        \
+        framework.set_name("default");                                  \
+        framework.set_principal(DEFAULT_CREDENTIAL.principal());        \
+        framework; })
+
+
 #define DEFAULT_EXECUTOR_ID           \
       DEFAULT_EXECUTOR_INFO.executor_id()