You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by an...@apache.org on 2017/03/25 19:06:37 UTC

[01/14] mesos git commit: Added support for auth tokens to the V1 executor library.

Repository: mesos
Updated Branches:
  refs/heads/master 1de39e676 -> 02c7083af


Added support for auth tokens to the V1 executor library.

This patch adds support for authentication tokens to the
V1 default executor library. The tokens are loaded from a
pre-determined environment variable, if present.

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


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

Branch: refs/heads/master
Commit: dfdd9bde5f67861a19083d0a173e683ec569fb63
Parents: 1de39e6
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:04:11 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:04:11 2017 -0700

----------------------------------------------------------------------
 src/executor/executor.cpp | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/dfdd9bde/src/executor/executor.cpp
----------------------------------------------------------------------
diff --git a/src/executor/executor.cpp b/src/executor/executor.cpp
index 35378ec..9c14054 100644
--- a/src/executor/executor.cpp
+++ b/src/executor/executor.cpp
@@ -223,6 +223,11 @@ public:
         upid.id +
         "/api/v1/executor");
 
+    value = os::getenv("MESOS_EXECUTOR_AUTHENTICATION_TOKEN");
+    if (value.isSome()) {
+      authenticationToken = value.get();
+    }
+
     // Get checkpointing status from environment.
     value = os::getenv("MESOS_CHECKPOINT");
     checkpoint = value.isSome() && value.get() == "1";
@@ -309,6 +314,10 @@ public:
     request.headers = {{"Accept", stringify(contentType)},
                        {"Content-Type", stringify(contentType)}};
 
+    if (authenticationToken.isSome()) {
+      request.headers["Authorization"] = "Bearer " + authenticationToken.get();
+    }
+
     CHECK_SOME(connections);
 
     Future<Response> response;
@@ -811,6 +820,7 @@ private:
   Option<Duration> maxBackoff;
   Option<Timer> recoveryTimer;
   Duration shutdownGracePeriod;
+  Option<string> authenticationToken;
 };
 
 


[03/14] mesos git commit: Added a output operator overload for Secret::Type.

Posted by an...@apache.org.
Added a output operator overload for Secret::Type.

This patch adds an `operator<<` overload for the `Secret::Type`
protobuf enum to facilitate logging.

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


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

Branch: refs/heads/master
Commit: 2fc8033624b0119e4ef77fa864f7735f07b3175f
Parents: 0fea4c5
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:04:22 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:04:22 2017 -0700

----------------------------------------------------------------------
 include/mesos/type_utils.hpp | 3 +++
 include/mesos/v1/mesos.hpp   | 3 +++
 src/common/type_utils.cpp    | 6 ++++++
 src/v1/mesos.cpp             | 6 ++++++
 4 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2fc80336/include/mesos/type_utils.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/type_utils.hpp b/include/mesos/type_utils.hpp
index 19f34a7..90b0227 100644
--- a/include/mesos/type_utils.hpp
+++ b/include/mesos/type_utils.hpp
@@ -322,6 +322,9 @@ std::ostream& operator<<(
 std::ostream& operator<<(std::ostream& stream, const Image::Type& imageType);
 
 
+std::ostream& operator<<(std::ostream& stream, const Secret::Type& secretType);
+
+
 std::ostream& operator<<(std::ostream& stream, const RLimitInfo& rlimitInfo);
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/2fc80336/include/mesos/v1/mesos.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/v1/mesos.hpp b/include/mesos/v1/mesos.hpp
index 0fecc4a..f91a574 100644
--- a/include/mesos/v1/mesos.hpp
+++ b/include/mesos/v1/mesos.hpp
@@ -322,6 +322,9 @@ std::ostream& operator<<(
 std::ostream& operator<<(std::ostream& stream, const Image::Type& imageType);
 
 
+std::ostream& operator<<(std::ostream& stream, const Secret::Type& secretType);
+
+
 template <typename T>
 inline std::ostream& operator<<(
     std::ostream& stream,

http://git-wip-us.apache.org/repos/asf/mesos/blob/2fc80336/src/common/type_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/type_utils.cpp b/src/common/type_utils.cpp
index 3ab2dc1..9d87a6d 100644
--- a/src/common/type_utils.cpp
+++ b/src/common/type_utils.cpp
@@ -643,6 +643,12 @@ ostream& operator<<(ostream& stream, const Image::Type& imageType)
 }
 
 
+ostream& operator<<(ostream& stream, const Secret::Type& secretType)
+{
+  return stream << Secret::Type_Name(secretType);
+}
+
+
 ostream& operator<<(ostream& stream, const hashmap<string, string>& map)
 {
   return stream << stringify(map);

http://git-wip-us.apache.org/repos/asf/mesos/blob/2fc80336/src/v1/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/v1/mesos.cpp b/src/v1/mesos.cpp
index b21e1e7..85db891 100644
--- a/src/v1/mesos.cpp
+++ b/src/v1/mesos.cpp
@@ -557,6 +557,12 @@ ostream& operator<<(ostream& stream, const Image::Type& imageType)
 }
 
 
+ostream& operator<<(ostream& stream, const Secret::Type& secretType)
+{
+  return stream << Secret::Type_Name(secretType);
+}
+
+
 ostream& operator<<(ostream& stream, const hashmap<string, string>& map)
 {
   return stream << stringify(map);


[07/14] mesos git commit: Turned off executor authentication in the ExecutorHttpApiTests.

Posted by an...@apache.org.
Turned off executor authentication in the ExecutorHttpApiTests.

This patch turns off executor authentication in the
ExecutorHttpApiTests, since authentication is not related
to the functionality they are meant to test. Since executor
tokens depend on the FrameworkID/ExecutorID/ContainerID of
the executor, it would be difficult to enable it in these
tests.

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


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

Branch: refs/heads/master
Commit: 2f6bb1fdd6f4872583e723851bd87ed09378d547
Parents: 95dcb6f
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:05:19 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:05:19 2017 -0700

----------------------------------------------------------------------
 src/tests/executor_http_api_tests.cpp | 95 +++++++++++++++++++++++++-----
 1 file changed, 81 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2f6bb1fd/src/tests/executor_http_api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/executor_http_api_tests.cpp b/src/tests/executor_http_api_tests.cpp
index 13d2e1d..910dbbf 100644
--- a/src/tests/executor_http_api_tests.cpp
+++ b/src/tests/executor_http_api_tests.cpp
@@ -84,7 +84,24 @@ namespace tests {
 
 class ExecutorHttpApiTest
   : public MesosTest,
-    public WithParamInterface<ContentType> {};
+    public WithParamInterface<ContentType>
+{
+protected:
+  slave::Flags CreateSlaveFlags() override
+  {
+    slave::Flags flags = MesosTest::CreateSlaveFlags();
+
+#ifdef USE_SSL_SOCKET
+    // Disable executor authentication on the agent. Executor authentication
+    // currently has SSL as a dependency, so this is only necessary if Mesos was
+    // built with SSL support.
+    flags.authenticate_http_executors = false;
+    flags.executor_secret_key = None();
+#endif // USE_SSL_SOCKET
+
+    return flags;
+  }
+};
 
 
 // The tests are parameterized by the content type of the request.
@@ -112,7 +129,10 @@ TEST_F(ExecutorHttpApiTest, NoContentType)
   Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(__recover);
@@ -149,7 +169,10 @@ TEST_F(ExecutorHttpApiTest, ValidJsonButInvalidProtobuf)
   Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(__recover);
@@ -185,7 +208,10 @@ TEST_P(ExecutorHttpApiTest, MalformedContent)
   Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(__recover);
@@ -221,7 +247,10 @@ TEST_P(ExecutorHttpApiTest, UnsupportedContentMediaType)
   Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(__recover);
@@ -264,7 +293,10 @@ TEST_P(ExecutorHttpApiTest, MessageFromUnknownFramework)
   Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(__recover);
@@ -305,7 +337,10 @@ TEST_F(ExecutorHttpApiTest, GetRequest)
   Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(__recover);
@@ -337,7 +372,13 @@ TEST_P(ExecutorHttpApiTest, DefaultAccept)
   TestContainerizer containerizer(executorId, executor);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), &containerizer);
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(
+      detector.get(),
+      &containerizer,
+      flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -414,7 +455,13 @@ TEST_P(ExecutorHttpApiTest, NoAcceptHeader)
   TestContainerizer containerizer(executorId, executor);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), &containerizer);
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(
+      detector.get(),
+      &containerizer,
+      flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -488,7 +535,10 @@ TEST_P(ExecutorHttpApiTest, NotAcceptable)
   Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(__recover);
@@ -531,7 +581,10 @@ TEST_P(ExecutorHttpApiTest, ValidProtobufInvalidCall)
   Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(__recover);
@@ -609,7 +662,10 @@ TEST_P(ExecutorHttpApiTest, StatusUpdateCallFailedValidation)
   Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(__recover);
@@ -747,7 +803,12 @@ TEST_F(ExecutorHttpApiTest, SubscribeBeforeContainerizerRecovery)
   EXPECT_CALL(mockContainerizer, recover(_))
     .WillOnce(Return(recoveryPromise.future()));
 
-  Try<Owned<cluster::Slave>> slave = StartSlave(&detector, &mockContainerizer);
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(
+      &detector,
+      &mockContainerizer,
+      flags);
   ASSERT_SOME(slave);
 
   // Ensure that the agent has atleast set up HTTP routes upon startup.
@@ -795,7 +856,13 @@ TEST_P(ExecutorHttpApiTest, Subscribe)
   TestContainerizer containerizer(&exec);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), &containerizer);
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(
+      detector.get(),
+      &containerizer,
+      flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;


[11/14] mesos git commit: Added a 'MockSecretGenerator'.

Posted by an...@apache.org.
Added a 'MockSecretGenerator'.

This patch adds a mock secret generator, which will
be used for testing failure scenarios involving
executor secret generation.

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


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

Branch: refs/heads/master
Commit: 8f35d5269d844f361e4a544ce50cf2230c247001
Parents: 3d82286
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:05:58 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:05:58 2017 -0700

----------------------------------------------------------------------
 src/tests/mesos.hpp | 13 +++++++++++++
 1 file changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8f35d526/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index e4a8a42..a7d33ad 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -34,6 +34,8 @@
 
 #include <mesos/v1/scheduler/scheduler.hpp>
 
+#include <mesos/authentication/secret_generator.hpp>
+
 #include <mesos/authorizer/authorizer.hpp>
 
 #include <mesos/fetcher/fetcher.hpp>
@@ -2143,6 +2145,17 @@ public:
 };
 
 
+class MockSecretGenerator : public SecretGenerator
+{
+public:
+  MockSecretGenerator() = default;
+  virtual ~MockSecretGenerator() = default;
+
+  MOCK_METHOD1(generate, process::Future<Secret>(
+      const process::http::authentication::Principal& principal));
+};
+
+
 ACTION_P(SendStatusUpdateFromTask, state)
 {
   TaskStatus status;


[04/14] mesos git commit: Updated the agent to generate executor secrets.

Posted by an...@apache.org.
Updated the agent to generate executor secrets.

This patch updates the agent code to generate executor
authentication tokens when executor authentication is
enabled. For now, the generated `Secret` objects must
be of `VALUE` type, and they're passed directly into the
executor environment.

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


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

Branch: refs/heads/master
Commit: 8b6ddb5fcae38dcfad27cb5dae26b4054773134f
Parents: 2fc8033
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:04:49 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:04:49 2017 -0700

----------------------------------------------------------------------
 src/slave/slave.cpp | 339 +++++++++++++++++++++++++++++++++--------------
 src/slave/slave.hpp |  19 ++-
 2 files changed, 255 insertions(+), 103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8b6ddb5f/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index d68d6b9..5729849 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -73,6 +73,7 @@
 #include "common/protobuf_utils.hpp"
 #include "common/resources_utils.hpp"
 #include "common/status_utils.hpp"
+#include "common/validation.hpp"
 
 #include "credentials/credentials.hpp"
 
@@ -2182,9 +2183,24 @@ void Slave::__run(
   Executor* executor = framework->getExecutor(executorId);
 
   if (executor == nullptr) {
-    executor = framework->launchExecutor(
-        executorInfo,
-        taskGroup.isNone() ? task.get() : Option<TaskInfo>::none());
+    executor = framework->addExecutor(executorInfo);
+
+    if (secretGenerator.get()) {
+      generateSecret(framework->id(), executor->id, executor->containerId)
+        .onAny(defer(
+            self(),
+            &Self::launchExecutor,
+            lambda::_1,
+            frameworkId,
+            executorId,
+            taskGroup.isNone() ? task.get() : Option<TaskInfo>::none()));
+    } else {
+      launchExecutor(
+          None(),
+          frameworkId,
+          executorId,
+          taskGroup.isNone() ? task.get() : Option<TaskInfo>::none());
+    }
   }
 
   CHECK_NOTNULL(executor);
@@ -2302,7 +2318,7 @@ void Slave::__run(
   }
 
   // We don't perform the checks for 'removeFramework' here since
-  // we're guaranteed by 'launchExecutor' that 'framework->executors'
+  // we're guaranteed by 'addExecutor' that 'framework->executors'
   // will be non-empty.
   CHECK(!framework->executors.empty());
 }
@@ -2512,6 +2528,213 @@ void Slave::___run(
 }
 
 
+// Generates a secret for executor authentication.
+Future<Secret> Slave::generateSecret(
+    const FrameworkID& frameworkId,
+    const ExecutorID& executorId,
+    const ContainerID& containerId)
+{
+  Principal principal(
+      Option<string>::none(),
+      {
+        {"fid", frameworkId.value()},
+        {"eid", executorId.value()},
+        {"cid", containerId.value()}
+      });
+
+  return secretGenerator->generate(principal)
+    .then([](const Secret& secret) -> Future<Secret> {
+      Option<Error> error = common::validation::validateSecret(secret);
+
+      if (error.isSome()) {
+        return Failure(
+            "Failed to validate generated secret: " + error->message);
+      } else if (secret.type() != Secret::VALUE) {
+        return Failure(
+            "Expecting generated secret to be of VALUE type instead of " +
+            stringify(secret.type()) + " type; " +
+            "only VALUE type secrets are supported at this time");
+      }
+
+      return secret;
+    });
+}
+
+
+// Launches an executor which was previously created.
+void Slave::launchExecutor(
+    const Option<Future<Secret>>& future,
+    const FrameworkID& frameworkId,
+    const ExecutorID& executorId,
+    const Option<TaskInfo>& taskInfo)
+{
+  Framework* framework = getFramework(frameworkId);
+  if (framework == nullptr) {
+    LOG(WARNING) << "Ignoring launching executor '" << executorId
+                 << "' because the framework " << frameworkId
+                 << " does not exist";
+    return;
+  }
+
+  if (framework->state == Framework::TERMINATING) {
+    LOG(WARNING) << "Ignoring launching executor '" << executorId
+                 << "' of framework " << frameworkId
+                 << " because the framework is terminating";
+    return;
+  }
+
+  Executor* executor = framework->getExecutor(executorId);
+  if (executor == nullptr) {
+    LOG(WARNING) << "Ignoring launching executor '" << executorId
+                 << "' of framework " << frameworkId
+                 << " because the executor does not exist";
+    return;
+  }
+
+  if (executor->state == Executor::TERMINATING ||
+      executor->state == Executor::TERMINATED) {
+    string executorState;
+    if (executor->state == Executor::TERMINATING) {
+      executorState = "terminating";
+    } else {
+      executorState = "terminated";
+    }
+
+    LOG(WARNING) << "Ignoring launching executor " << *executor
+                 << " in container " << executor->containerId
+                 << " because the executor is " << executorState;
+
+    // The framework may have shutdown this executor already, transitioning it
+    // to the TERMINATING/TERMINATED state. However, the executor still exists
+    // in the agent's map, so we must send status updates for any queued tasks
+    // and perform cleanup via `executorTerminated`.
+    ContainerTermination termination;
+    termination.set_state(TASK_FAILED);
+    termination.add_reasons(TaskStatus::REASON_CONTAINER_LAUNCH_FAILED);
+    termination.set_message("Executor " + executorState);
+
+    executorTerminated(frameworkId, executorId, termination);
+
+    return;
+  }
+
+  CHECK_EQ(Executor::REGISTERING, executor->state);
+
+  Option<Secret> authenticationToken;
+
+  if (future.isSome()) {
+    if (!future->isReady()) {
+      LOG(ERROR) << "Failed to launch executor " << *executor
+                 << " in container " << executor->containerId
+                 << " because secret generation failed: "
+                 << (future->isFailed() ? future->failure() : "discarded");
+
+      ContainerTermination termination;
+      termination.set_state(TASK_FAILED);
+      termination.add_reasons(TaskStatus::REASON_CONTAINER_LAUNCH_FAILED);
+      termination.set_message(
+          "Secret generation failed: " +
+          (future->isFailed() ? future->failure() : "discarded"));
+
+      executorTerminated(frameworkId, executorId, termination);
+
+      return;
+    }
+
+    authenticationToken = future->get();
+  }
+
+  // Tell the containerizer to launch the executor.
+  ExecutorInfo executorInfo_ = executor->info;
+
+  // Populate the command info for default executor. We modify the ExecutorInfo
+  // to avoid resetting command info upon re-registering with the master since
+  // the master doesn't store them; they are generated by the slave.
+  if (executorInfo_.has_type() &&
+      executorInfo_.type() == ExecutorInfo::DEFAULT) {
+    CHECK(!executorInfo_.has_command());
+
+    executorInfo_.mutable_command()->CopyFrom(
+        defaultExecutorCommandInfo(flags.launcher_dir, executor->user));
+  }
+
+  Resources resources = executorInfo_.resources();
+
+  // NOTE: We modify the ExecutorInfo to include the task's
+  // resources when launching the executor so that the containerizer
+  // has non-zero resources to work with when the executor has
+  // no resources. This should be revisited after MESOS-600.
+  if (taskInfo.isSome()) {
+    resources += taskInfo->resources();
+  }
+
+  executorInfo_.mutable_resources()->CopyFrom(resources);
+
+  // Prepare environment variables for the executor.
+  map<string, string> environment = executorEnvironment(
+      flags,
+      executorInfo_,
+      executor->directory,
+      info.id(),
+      self(),
+      authenticationToken,
+      framework->info.checkpoint());
+
+  // Launch the container.
+  Future<bool> launch;
+  if (!executor->isCommandExecutor()) {
+    // If the executor is _not_ a command executor, this means that
+    // the task will include the executor to run. The actual task to
+    // run will be enqueued and subsequently handled by the executor
+    // when it has registered to the slave.
+    launch = containerizer->launch(
+        executor->containerId,
+        None(),
+        executorInfo_,
+        executor->directory,
+        executor->user,
+        info.id(),
+        environment,
+        framework->info.checkpoint());
+  } else {
+    // An executor has _not_ been provided by the task and will
+    // instead define a command and/or container to run. Right now,
+    // these tasks will require an executor anyway and the slave
+    // creates a command executor. However, it is up to the
+    // containerizer how to execute those tasks and the generated
+    // executor info works as a placeholder.
+    // TODO(nnielsen): Obsolete the requirement for executors to run
+    // one-off tasks.
+    launch = containerizer->launch(
+        executor->containerId,
+        taskInfo,
+        executorInfo_,
+        executor->directory,
+        executor->user,
+        info.id(),
+        environment,
+        framework->info.checkpoint());
+  }
+
+  launch.onAny(defer(self(),
+                     &Self::executorLaunched,
+                     frameworkId,
+                     executor->id,
+                     executor->containerId,
+                     lambda::_1));
+
+  // Make sure the executor registers within the given timeout.
+  delay(flags.executor_registration_timeout,
+        self(),
+        &Self::registerExecutorTimeout,
+        frameworkId,
+        executor->id,
+        executor->containerId);
+
+  return;
+}
+
+
 void Slave::runTaskGroup(
     const UPID& from,
     const FrameworkInfo& frameworkInfo,
@@ -4855,7 +5078,8 @@ void Slave::executorLaunched(
 }
 
 
-// Called by the isolator when an executor process terminates.
+// Called by the isolator when an executor process terminates, and by
+// `Slave::launchExecutor` when executor secret generation fails.
 void Slave::executorTerminated(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
@@ -6592,10 +6816,7 @@ Framework::~Framework()
 }
 
 
-// Create and launch an executor.
-Executor* Framework::launchExecutor(
-    const ExecutorInfo& executorInfo,
-    const Option<TaskInfo>& taskInfo)
+Executor* Framework::addExecutor(const ExecutorInfo& executorInfo)
 {
   // Verify that Resource.AllocationInfo is set, if coming
   // from a MULTI_ROLE master this will be set, otherwise
@@ -6605,9 +6826,9 @@ Executor* Framework::launchExecutor(
   }
 
   // Generate an ID for the executor's container.
-  // TODO(idownes) This should be done by the containerizer but we
-  // need the ContainerID to create the executor's directory. Fix
-  // this when 'launchExecutor()' is handled asynchronously.
+  // TODO(idownes) This should be done by the containerizer but we need the
+  // ContainerID to create the executor's directory and generate the secret.
+  // Consider fixing this since 'launchExecutor()' is handled asynchronously.
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
@@ -6680,92 +6901,6 @@ Executor* Framework::launchExecutor(
   slave->files->attach(executor->directory, executor->directory, authorize)
     .onAny(defer(slave, &Slave::fileAttached, lambda::_1, executor->directory));
 
-  // Tell the containerizer to launch the executor.
-  ExecutorInfo executorInfo_ = executor->info;
-
-  // Populate the command info for default executor. We modify the ExecutorInfo
-  // to avoid resetting command info upon re-registering with the master since
-  // the master doesn't store them; they are generated by the slave.
-  if (executorInfo_.has_type() &&
-      executorInfo_.type() == ExecutorInfo::DEFAULT) {
-    CHECK(!executorInfo_.has_command());
-
-    executorInfo_.mutable_command()->CopyFrom(
-        defaultExecutorCommandInfo(slave->flags.launcher_dir, user));
-  }
-
-  Resources resources = executorInfo_.resources();
-
-  // NOTE: We modify the ExecutorInfo to include the task's
-  // resources when launching the executor so that the containerizer
-  // has non-zero resources to work with when the executor has
-  // no resources. This should be revisited after MESOS-600.
-  if (taskInfo.isSome()) {
-    resources += taskInfo->resources();
-  }
-
-  executorInfo_.mutable_resources()->CopyFrom(resources);
-
-  // Prepare environment variables for the executor.
-  map<string, string> environment = executorEnvironment(
-      slave->flags,
-      executorInfo_,
-      executor->directory,
-      slave->info.id(),
-      slave->self(),
-      info.checkpoint());
-
-  // Launch the container.
-  Future<bool> launch;
-  if (!executor->isCommandExecutor()) {
-    // If the executor is _not_ a command executor, this means that
-    // the task will include the executor to run. The actual task to
-    // run will be enqueued and subsequently handled by the executor
-    // when it has registered to the slave.
-    launch = slave->containerizer->launch(
-        containerId,
-        None(),
-        executorInfo_,
-        executor->directory,
-        user,
-        slave->info.id(),
-        environment,
-        info.checkpoint());
-  } else {
-    // An executor has _not_ been provided by the task and will
-    // instead define a command and/or container to run. Right now,
-    // these tasks will require an executor anyway and the slave
-    // creates a command executor. However, it is up to the
-    // containerizer how to execute those tasks and the generated
-    // executor info works as a placeholder.
-    // TODO(nnielsen): Obsolete the requirement for executors to run
-    // one-off tasks.
-    launch = slave->containerizer->launch(
-        containerId,
-        taskInfo,
-        executorInfo_,
-        executor->directory,
-        user,
-        slave->info.id(),
-        environment,
-        info.checkpoint());
-  }
-
-  launch.onAny(defer(slave,
-                     &Slave::executorLaunched,
-                     id(),
-                     executor->id,
-                     containerId,
-                     lambda::_1));
-
-  // Make sure the executor registers within the given timeout.
-  delay(slave->flags.executor_registration_timeout,
-        slave,
-        &Slave::registerExecutorTimeout,
-        id(),
-        executor->id,
-        containerId);
-
   return executor;
 }
 
@@ -7342,6 +7477,7 @@ map<string, string> executorEnvironment(
     const string& directory,
     const SlaveID& slaveId,
     const PID<Slave>& slavePid,
+    const Option<Secret>& authenticationToken,
     bool checkpoint)
 {
   map<string, string> environment;
@@ -7434,6 +7570,13 @@ map<string, string> executorEnvironment(
       stringify(EXECUTOR_REREGISTER_TIMEOUT);
   }
 
+  if (authenticationToken.isSome()) {
+    CHECK(authenticationToken->has_value());
+
+    environment["MESOS_EXECUTOR_AUTHENTICATION_TOKEN"] =
+      authenticationToken->value().data();
+  }
+
   if (HookManager::hooksAvailable()) {
     // Include any environment variables from Hooks.
     // TODO(karya): Call environment decorator hook _after_ putting all

http://git-wip-us.apache.org/repos/asf/mesos/blob/8b6ddb5f/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index e06525b..59efa4e 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -357,6 +357,18 @@ public:
       const std::list<TaskInfo>& tasks,
       const std::list<TaskGroupInfo>& taskGroups);
 
+  process::Future<Secret> generateSecret(
+      const FrameworkID& frameworkId,
+      const ExecutorID& executorId,
+      const ContainerID& containerId);
+
+  // If an executor is launched for a task group, `taskInfo` would not be set.
+  void launchExecutor(
+      const Option<process::Future<Secret>>& future,
+      const FrameworkID& frameworkId,
+      const ExecutorID& executorId,
+      const Option<TaskInfo>& taskInfo);
+
   void fileAttached(const process::Future<Nothing>& result,
                     const std::string& path);
 
@@ -1103,11 +1115,7 @@ struct Framework
 
   ~Framework();
 
-  // If an executor is launched for a task group, `taskInfo` would
-  // not be set.
-  Executor* launchExecutor(
-      const ExecutorInfo& executorInfo,
-      const Option<TaskInfo>& taskInfo);
+  Executor* addExecutor(const ExecutorInfo& executorInfo);
   void destroyExecutor(const ExecutorID& executorId);
   Executor* getExecutor(const ExecutorID& executorId) const;
   Executor* getExecutor(const TaskID& taskId) const;
@@ -1206,6 +1214,7 @@ std::map<std::string, std::string> executorEnvironment(
     const std::string& directory,
     const SlaveID& slaveId,
     const process::PID<Slave>& slavePid,
+    const Option<Secret>& authenticationToken,
     bool checkpoint);
 
 


[14/14] mesos git commit: Added new tests for executor secret generation.

Posted by an...@apache.org.
Added new tests for executor secret generation.

This patch adds new tests,
`SlaveTest.RunTaskGroupFailedSecretGeneration` and
`SlaveTest.RunTaskGroupInvalidExecutorSecret`, to
verify the agent's behavior when generation of the
executor secret fails.

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


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

Branch: refs/heads/master
Commit: 02c7083af8d267f5e403b56e392d6ec60fc4dee9
Parents: d11dd0e
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:06:09 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:06:09 2017 -0700

----------------------------------------------------------------------
 src/tests/slave_tests.cpp | 411 +++++++++++++++++++++++++++++++++++++++++
 1 file changed, 411 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/02c7083a/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index c31c670..cd76968 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -46,6 +46,7 @@
 #include <stout/option.hpp>
 #include <stout/os.hpp>
 #include <stout/path.hpp>
+#include <stout/strings.hpp>
 #include <stout/try.hpp>
 
 #include "common/build.hpp"
@@ -5112,6 +5113,416 @@ TEST_F(SlaveTest, RunTaskGroup)
 }
 
 
+// This test verifies that TASK_FAILED updates are sent correctly for all the
+// tasks in a task group when secret generation fails.
+TEST_F(SlaveTest, RunTaskGroupFailedSecretGeneration)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
+  auto executor = std::make_shared<v1::MockHTTPExecutor>();
+
+  v1::Resources resources =
+    v1::Resources::parse("cpus:0.1;mem:32;disk:32").get();
+
+  v1::ExecutorInfo executorInfo = v1::DEFAULT_EXECUTOR_INFO;
+  executorInfo.set_type(v1::ExecutorInfo::CUSTOM);
+
+  executorInfo.mutable_resources()->CopyFrom(resources);
+
+  const v1::ExecutorID& executorId = executorInfo.executor_id();
+  TestContainerizer containerizer(devolve(executorId), executor);
+
+  StandaloneMasterDetector detector(master.get()->pid);
+
+  // This pointer is passed to the agent, which will perform the cleanup.
+  MockSecretGenerator* secretGenerator = new MockSecretGenerator();
+
+  MockSlave slave(
+      CreateSlaveFlags(),
+      &detector,
+      &containerizer,
+      None(),
+      None(),
+      secretGenerator);
+  spawn(slave);
+
+  Future<Nothing> connected;
+  EXPECT_CALL(*scheduler, connected(_))
+    .WillOnce(FutureSatisfy(&connected));
+
+  v1::scheduler::TestMesos mesos(
+      master.get()->pid,
+      ContentType::PROTOBUF,
+      scheduler);
+
+  AWAIT_READY(connected);
+
+  Future<v1::scheduler::Event::Subscribed> subscribed;
+  EXPECT_CALL(*scheduler, subscribed(_, _))
+    .WillOnce(FutureArg<1>(&subscribed));
+
+  Future<v1::scheduler::Event::Offers> offers;
+  EXPECT_CALL(*scheduler, offers(_, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  EXPECT_CALL(*scheduler, heartbeat(_))
+    .WillRepeatedly(Return()); // Ignore heartbeats.
+
+  {
+    Call call;
+    call.set_type(Call::SUBSCRIBE);
+    Call::Subscribe* subscribe = call.mutable_subscribe();
+    subscribe->mutable_framework_info()->CopyFrom(v1::DEFAULT_FRAMEWORK_INFO);
+
+    mesos.send(call);
+  }
+
+  AWAIT_READY(subscribed);
+
+  v1::FrameworkID frameworkId(subscribed->framework_id());
+
+  // Update `executorInfo` with the subscribed `frameworkId`.
+  executorInfo.mutable_framework_id()->CopyFrom(frameworkId);
+
+  AWAIT_READY(offers);
+  ASSERT_NE(0, offers->offers().size());
+
+  const v1::Offer& offer = offers->offers(0);
+  const v1::AgentID agentId = offer.agent_id();
+
+  v1::TaskInfo taskInfo1 = v1::createTask(agentId, resources, "");
+
+  v1::TaskInfo taskInfo2 = v1::createTask(agentId, resources, "");
+
+  v1::TaskGroupInfo taskGroup;
+  taskGroup.add_tasks()->CopyFrom(taskInfo1);
+  taskGroup.add_tasks()->CopyFrom(taskInfo2);
+
+  const hashset<v1::TaskID> tasks{taskInfo1.task_id(), taskInfo2.task_id()};
+
+  // The tasks will fail to launch because the executor secret generation fails.
+  const string failureMessage = "Mock secret generator failed";
+  EXPECT_CALL(*secretGenerator, generate(_))
+    .WillOnce(Return(Failure(failureMessage)));
+
+  EXPECT_CALL(*executor, connected(_))
+    .Times(0);
+
+  EXPECT_CALL(*executor, subscribed(_, _))
+    .Times(0);
+
+  EXPECT_CALL(*executor, shutdown(_))
+    .Times(0);
+
+  EXPECT_CALL(*executor, launchGroup(_, _))
+    .Times(0);
+
+  EXPECT_CALL(*executor, launch(_, _))
+    .Times(0);
+
+  EXPECT_CALL(slave, executorTerminated(_, _, _))
+    .WillOnce(Invoke(&slave, &MockSlave::unmocked_executorTerminated));
+
+  Future<v1::scheduler::Event::Update> update1;
+  Future<v1::scheduler::Event::Update> update2;
+  EXPECT_CALL(*scheduler, update(_, _))
+    .WillOnce(FutureArg<1>(&update1))
+    .WillOnce(FutureArg<1>(&update2));
+
+  Future<Nothing> failure;
+  EXPECT_CALL(*scheduler, failure(_, _))
+    .WillOnce(FutureSatisfy(&failure));
+
+  {
+    Call call;
+    call.mutable_framework_id()->CopyFrom(frameworkId);
+    call.set_type(Call::ACCEPT);
+
+    Call::Accept* accept = call.mutable_accept();
+    accept->add_offer_ids()->CopyFrom(offer.id());
+
+    v1::Offer::Operation* operation = accept->add_operations();
+    operation->set_type(v1::Offer::Operation::LAUNCH_GROUP);
+
+    v1::Offer::Operation::LaunchGroup* launchGroup =
+      operation->mutable_launch_group();
+
+    launchGroup->mutable_executor()->CopyFrom(executorInfo);
+    launchGroup->mutable_task_group()->CopyFrom(taskGroup);
+
+    mesos.send(call);
+  }
+
+  AWAIT_READY(update1);
+  AWAIT_READY(update2);
+
+  AWAIT_READY(failure);
+
+  const hashset<v1::TaskID> failedTasks{
+      update1->status().task_id(), update2->status().task_id()};
+
+  ASSERT_EQ(TASK_FAILED, update1->status().state());
+  ASSERT_EQ(TASK_FAILED, update2->status().state());
+
+  EXPECT_TRUE(strings::contains(update1->status().message(), failureMessage));
+  EXPECT_TRUE(strings::contains(update2->status().message(), failureMessage));
+
+  ASSERT_EQ(tasks, failedTasks);
+
+  // Since this is the only task group for this framework, the
+  // framework should be removed after secret generation fails.
+  Future<Nothing> removeFramework;
+  EXPECT_CALL(slave, removeFramework(_))
+    .WillOnce(DoAll(Invoke(&slave, &MockSlave::unmocked_removeFramework),
+                    FutureSatisfy(&removeFramework)));
+
+  // Acknowledge the status updates so that the agent will remove the framework.
+
+  {
+    Call call;
+    call.mutable_framework_id()->CopyFrom(frameworkId);
+    call.set_type(Call::ACKNOWLEDGE);
+
+    Call::Acknowledge* acknowledge = call.mutable_acknowledge();
+    acknowledge->mutable_task_id()->CopyFrom(update1->status().task_id());
+    acknowledge->mutable_agent_id()->CopyFrom(offer.agent_id());
+    acknowledge->set_uuid(update1->status().uuid());
+
+    mesos.send(call);
+  }
+
+  {
+    Call call;
+    call.mutable_framework_id()->CopyFrom(frameworkId);
+    call.set_type(Call::ACKNOWLEDGE);
+
+    Call::Acknowledge* acknowledge = call.mutable_acknowledge();
+    acknowledge->mutable_task_id()->CopyFrom(update2->status().task_id());
+    acknowledge->mutable_agent_id()->CopyFrom(offer.agent_id());
+    acknowledge->set_uuid(update2->status().uuid());
+
+    mesos.send(call);
+  }
+
+  AWAIT_READY(removeFramework);
+
+  terminate(slave);
+  wait(slave);
+}
+
+
+// This test verifies that TASK_FAILED updates are sent correctly for all the
+// tasks in a task group when the secret generator returns an invalid secret.
+TEST_F(SlaveTest, RunTaskGroupInvalidExecutorSecret)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
+  auto executor = std::make_shared<v1::MockHTTPExecutor>();
+
+  v1::Resources resources =
+    v1::Resources::parse("cpus:0.1;mem:32;disk:32").get();
+
+  v1::ExecutorInfo executorInfo = v1::DEFAULT_EXECUTOR_INFO;
+  executorInfo.set_type(v1::ExecutorInfo::CUSTOM);
+
+  executorInfo.mutable_resources()->CopyFrom(resources);
+
+  const v1::ExecutorID& executorId = executorInfo.executor_id();
+  TestContainerizer containerizer(devolve(executorId), executor);
+
+  StandaloneMasterDetector detector(master.get()->pid);
+
+  // This pointer is passed to the agent, which will perform the cleanup.
+  MockSecretGenerator* secretGenerator = new MockSecretGenerator();
+
+  MockSlave slave(
+      CreateSlaveFlags(),
+      &detector,
+      &containerizer,
+      None(),
+      None(),
+      secretGenerator);
+  spawn(slave);
+
+  Future<Nothing> connected;
+  EXPECT_CALL(*scheduler, connected(_))
+    .WillOnce(FutureSatisfy(&connected));
+
+  v1::scheduler::TestMesos mesos(
+      master.get()->pid,
+      ContentType::PROTOBUF,
+      scheduler);
+
+  AWAIT_READY(connected);
+
+  Future<v1::scheduler::Event::Subscribed> subscribed;
+  EXPECT_CALL(*scheduler, subscribed(_, _))
+    .WillOnce(FutureArg<1>(&subscribed));
+
+  Future<v1::scheduler::Event::Offers> offers;
+  EXPECT_CALL(*scheduler, offers(_, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  EXPECT_CALL(*scheduler, heartbeat(_))
+    .WillRepeatedly(Return()); // Ignore heartbeats.
+
+  {
+    Call call;
+    call.set_type(Call::SUBSCRIBE);
+    Call::Subscribe* subscribe = call.mutable_subscribe();
+    subscribe->mutable_framework_info()->CopyFrom(v1::DEFAULT_FRAMEWORK_INFO);
+
+    mesos.send(call);
+  }
+
+  AWAIT_READY(subscribed);
+
+  v1::FrameworkID frameworkId(subscribed->framework_id());
+
+  // Update `executorInfo` with the subscribed `frameworkId`.
+  executorInfo.mutable_framework_id()->CopyFrom(frameworkId);
+
+  AWAIT_READY(offers);
+  ASSERT_NE(0, offers->offers().size());
+
+  const v1::Offer& offer = offers->offers(0);
+  const v1::AgentID agentId = offer.agent_id();
+
+  v1::TaskInfo taskInfo1 = v1::createTask(agentId, resources, "");
+
+  v1::TaskInfo taskInfo2 = v1::createTask(agentId, resources, "");
+
+  v1::TaskGroupInfo taskGroup;
+  taskGroup.add_tasks()->CopyFrom(taskInfo1);
+  taskGroup.add_tasks()->CopyFrom(taskInfo2);
+
+  const hashset<v1::TaskID> tasks{taskInfo1.task_id(), taskInfo2.task_id()};
+
+  // The tasks will fail to launch because the executor secret is invalid
+  // (VALUE type secrets must not have the `reference` member set).
+  Secret authenticationToken;
+  authenticationToken.set_type(Secret::VALUE);
+  authenticationToken.mutable_reference()->set_name("secret_name");
+  authenticationToken.mutable_reference()->set_key("secret_key");
+
+  EXPECT_CALL(*secretGenerator, generate(_))
+    .WillOnce(Return(authenticationToken));
+
+  EXPECT_CALL(*executor, connected(_))
+    .Times(0);
+
+  EXPECT_CALL(*executor, subscribed(_, _))
+    .Times(0);
+
+  EXPECT_CALL(*executor, shutdown(_))
+    .Times(0);
+
+  EXPECT_CALL(*executor, launchGroup(_, _))
+    .Times(0);
+
+  EXPECT_CALL(*executor, launch(_, _))
+    .Times(0);
+
+  EXPECT_CALL(slave, executorTerminated(_, _, _))
+    .WillOnce(Invoke(&slave, &MockSlave::unmocked_executorTerminated));
+
+  Future<v1::scheduler::Event::Update> update1;
+  Future<v1::scheduler::Event::Update> update2;
+  EXPECT_CALL(*scheduler, update(_, _))
+    .WillOnce(FutureArg<1>(&update1))
+    .WillOnce(FutureArg<1>(&update2));
+
+  Future<Nothing> failure;
+  EXPECT_CALL(*scheduler, failure(_, _))
+    .WillOnce(FutureSatisfy(&failure));
+
+  {
+    Call call;
+    call.mutable_framework_id()->CopyFrom(frameworkId);
+    call.set_type(Call::ACCEPT);
+
+    Call::Accept* accept = call.mutable_accept();
+    accept->add_offer_ids()->CopyFrom(offer.id());
+
+    v1::Offer::Operation* operation = accept->add_operations();
+    operation->set_type(v1::Offer::Operation::LAUNCH_GROUP);
+
+    v1::Offer::Operation::LaunchGroup* launchGroup =
+      operation->mutable_launch_group();
+
+    launchGroup->mutable_executor()->CopyFrom(executorInfo);
+    launchGroup->mutable_task_group()->CopyFrom(taskGroup);
+
+    mesos.send(call);
+  }
+
+  AWAIT_READY(update1);
+  AWAIT_READY(update2);
+
+  AWAIT_READY(failure);
+
+  const hashset<v1::TaskID> failedTasks{
+      update1->status().task_id(), update2->status().task_id()};
+
+  ASSERT_EQ(TASK_FAILED, update1->status().state());
+  ASSERT_EQ(TASK_FAILED, update2->status().state());
+
+  const string failureMessage =
+    "Secret of type VALUE must have the 'value' field set";
+
+  EXPECT_TRUE(strings::contains(update1->status().message(), failureMessage));
+  EXPECT_TRUE(strings::contains(update2->status().message(), failureMessage));
+
+  ASSERT_EQ(tasks, failedTasks);
+
+  // Since this is the only task group for this framework, the
+  // framework should be removed after secret generation fails.
+  Future<Nothing> removeFramework;
+  EXPECT_CALL(slave, removeFramework(_))
+    .WillOnce(DoAll(Invoke(&slave, &MockSlave::unmocked_removeFramework),
+                    FutureSatisfy(&removeFramework)));
+
+  // Acknowledge the status updates so that the agent will remove the framework.
+
+  {
+    Call call;
+    call.mutable_framework_id()->CopyFrom(frameworkId);
+    call.set_type(Call::ACKNOWLEDGE);
+
+    Call::Acknowledge* acknowledge = call.mutable_acknowledge();
+    acknowledge->mutable_task_id()->CopyFrom(update1->status().task_id());
+    acknowledge->mutable_agent_id()->CopyFrom(offer.agent_id());
+    acknowledge->set_uuid(update1->status().uuid());
+
+    mesos.send(call);
+  }
+
+  {
+    Call call;
+    call.mutable_framework_id()->CopyFrom(frameworkId);
+    call.set_type(Call::ACKNOWLEDGE);
+
+    Call::Acknowledge* acknowledge = call.mutable_acknowledge();
+    acknowledge->mutable_task_id()->CopyFrom(update2->status().task_id());
+    acknowledge->mutable_agent_id()->CopyFrom(offer.agent_id());
+    acknowledge->set_uuid(update2->status().uuid());
+
+    mesos.send(call);
+  }
+
+  AWAIT_READY(removeFramework);
+
+  terminate(slave);
+  wait(slave);
+}
+
+
 // This test ensures that a `killTask()` can happen between `runTask()`
 // and `_run()` and then gets "handled properly" for a task group.
 // This should result in TASK_KILLED updates for all the tasks in the


[10/14] mesos git commit: Changed the agent's SecretGenerator from Owned to pointer.

Posted by an...@apache.org.
Changed the agent's SecretGenerator from Owned to pointer.

This patch updates the agent to hold its `SecretGenerator` as a
raw pointer instead of an `Owned` object. This is more consistent
with the other dependencies injected into the agent, and makes it
easier to test the agent with a mock secret generator.

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


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

Branch: refs/heads/master
Commit: 3d822865e84d997e37e4b37cbe8c58531980a9d4
Parents: 3f9bf37
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:05:45 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:05:45 2017 -0700

----------------------------------------------------------------------
 src/slave/slave.cpp | 9 ++++++---
 src/slave/slave.hpp | 2 +-
 2 files changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3d822865/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 5729849..892b66d 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -188,7 +188,8 @@ Slave::Slave(const string& id,
     executorDirectoryMaxAllowedAge(age(0)),
     resourceEstimator(_resourceEstimator),
     qosController(_qosController),
-    authorizer(_authorizer) {}
+    authorizer(_authorizer),
+    secretGenerator(nullptr) {}
 
 
 Slave::~Slave()
@@ -203,6 +204,8 @@ Slave::~Slave()
   }
 
   delete authenticatee;
+
+  delete secretGenerator;
 }
 
 void Slave::signaled(int signal, int uid)
@@ -290,7 +293,7 @@ void Slave::initialize()
 #ifdef USE_SSL_SOCKET
   if (flags.executor_secret_key.isSome()) {
     secretKey = flags.executor_secret_key.get();
-    secretGenerator.reset(new JWTSecretGenerator(secretKey.get()));
+    secretGenerator = new JWTSecretGenerator(secretKey.get());
   }
 
   if (flags.authenticate_http_executors) {
@@ -2185,7 +2188,7 @@ void Slave::__run(
   if (executor == nullptr) {
     executor = framework->addExecutor(executorInfo);
 
-    if (secretGenerator.get()) {
+    if (secretGenerator) {
       generateSecret(framework->id(), executor->id, executor->containerId)
         .onAny(defer(
             self(),

http://git-wip-us.apache.org/repos/asf/mesos/blob/3d822865/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 59efa4e..e4f46d4 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -901,7 +901,7 @@ private:
 
 protected:
   // Made protected for testing purposes.
-  process::Owned<mesos::SecretGenerator> secretGenerator;
+  mesos::SecretGenerator* secretGenerator;
 };
 
 


[06/14] mesos git commit: Enabled executor authentication in the tests.

Posted by an...@apache.org.
Enabled executor authentication in the tests.

This patch sets the `--authenticate_http_executors` and
`--executor_secret_key` flags by default in the tests,
and it updates the test cluster code to load a secret
generator when those flags are set.

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


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

Branch: refs/heads/master
Commit: 95dcb6faf19f4af7850a4cdbe9acdbb8b2aff4a3
Parents: 873497f
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:05:15 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:05:15 2017 -0700

----------------------------------------------------------------------
 src/tests/cluster.cpp | 2 ++
 src/tests/mesos.cpp   | 7 +++++++
 2 files changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/95dcb6fa/src/tests/cluster.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.cpp b/src/tests/cluster.cpp
index 15eaf42..7f09a0c 100644
--- a/src/tests/cluster.cpp
+++ b/src/tests/cluster.cpp
@@ -536,6 +536,8 @@ Slave::~Slave()
       slave::READONLY_HTTP_AUTHENTICATION_REALM);
   process::http::authentication::unsetAuthenticator(
       slave::READWRITE_HTTP_AUTHENTICATION_REALM);
+  process::http::authentication::unsetAuthenticator(
+      slave::EXECUTOR_HTTP_AUTHENTICATION_REALM);
 
   // If either `shutdown()` or `terminate()` were called already,
   // skip the below container cleanup logic.  Additionally, we can skip

http://git-wip-us.apache.org/repos/asf/mesos/blob/95dcb6fa/src/tests/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index 3333c50..099ec37 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -208,6 +208,13 @@ slave::Flags MesosTest::CreateSlaveFlags()
   flags.authenticate_http_readonly = true;
   flags.authenticate_http_readwrite = true;
 
+#ifdef USE_SSL_SOCKET
+  // Executor authentication currently has SSL as a dependency, so we
+  // cannot enable it if Mesos was not built with SSL support.
+  flags.authenticate_http_executors = true;
+  flags.executor_secret_key = "secret_key";
+#endif // USE_SSL_SOCKET
+
   {
     // Create a default HTTP credentials file.
     const string& path = path::join(directory.get(), "http_credentials");


[12/14] mesos git commit: Added SecretGenerator injection to the MockSlave.

Posted by an...@apache.org.
Added SecretGenerator injection to the MockSlave.

This patch adds an overload of `Slave::initialize` to the
mock slave for the purpose of injecting a secret generator.

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


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

Branch: refs/heads/master
Commit: ca58a52346c72cbfed703f2809894bf373f963e3
Parents: 8f35d52
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:06:02 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:06:02 2017 -0700

----------------------------------------------------------------------
 src/tests/mock_slave.cpp | 20 ++++++++++++++++++--
 src/tests/mock_slave.hpp | 11 ++++++++++-
 2 files changed, 28 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ca58a523/src/tests/mock_slave.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mock_slave.cpp b/src/tests/mock_slave.cpp
index 50c04bf..f145c5c 100644
--- a/src/tests/mock_slave.cpp
+++ b/src/tests/mock_slave.cpp
@@ -18,6 +18,8 @@
 
 #include <gmock/gmock.h>
 
+#include <mesos/authentication/secret_generator.hpp>
+
 #include <mesos/slave/qos_controller.hpp>
 #include <mesos/slave/resource_estimator.hpp>
 
@@ -104,7 +106,8 @@ MockSlave::MockSlave(
     MasterDetector* detector,
     slave::Containerizer* containerizer,
     const Option<mesos::slave::QoSController*>& _qosController,
-    const Option<mesos::Authorizer*>& authorizer)
+    const Option<mesos::Authorizer*>& authorizer,
+    const Option<mesos::SecretGenerator*>& _mockSecretGenerator)
   : slave::Slave(
         process::ID::generate("slave"),
         flags,
@@ -116,7 +119,8 @@ MockSlave::MockSlave(
         &resourceEstimator,
         _qosController.isSome() ? _qosController.get() : &qosController,
         authorizer),
-    files(slave::READONLY_HTTP_AUTHENTICATION_REALM)
+    files(slave::READONLY_HTTP_AUTHENTICATION_REALM),
+    mockSecretGenerator(_mockSecretGenerator)
 {
   // Set up default behaviors, calling the original methods.
   EXPECT_CALL(*this, runTask(_, _, _, _, _))
@@ -144,6 +148,18 @@ MockSlave::~MockSlave()
 }
 
 
+void MockSlave::initialize()
+{
+  Slave::initialize();
+
+  if (mockSecretGenerator.isSome()) {
+    delete secretGenerator;
+    secretGenerator = mockSecretGenerator.get();
+    mockSecretGenerator = None();
+  }
+}
+
+
 void MockSlave::unmocked_runTask(
     const UPID& from,
     const FrameworkInfo& frameworkInfo,

http://git-wip-us.apache.org/repos/asf/mesos/blob/ca58a523/src/tests/mock_slave.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mock_slave.hpp b/src/tests/mock_slave.hpp
index 1acb961..979ac59 100644
--- a/src/tests/mock_slave.hpp
+++ b/src/tests/mock_slave.hpp
@@ -22,6 +22,8 @@
 
 #include <gmock/gmock.h>
 
+#include <mesos/authentication/secret_generator.hpp>
+
 #include <mesos/master/detector.hpp>
 
 #include <mesos/slave/qos_controller.hpp>
@@ -108,10 +110,13 @@ public:
       mesos::master::detector::MasterDetector* detector,
       slave::Containerizer* containerizer,
       const Option<mesos::slave::QoSController*>& qosController = None(),
-      const Option<mesos::Authorizer*>& authorizer = None());
+      const Option<mesos::Authorizer*>& authorizer = None(),
+      const Option<mesos::SecretGenerator*>& mockSecretGenerator = None());
 
   virtual ~MockSlave();
 
+  void initialize();
+
   MOCK_METHOD5(runTask, void(
       const process::UPID& from,
       const FrameworkInfo& frameworkInfo,
@@ -196,6 +201,10 @@ private:
   MockResourceEstimator resourceEstimator;
   MockQoSController qosController;
   slave::StatusUpdateManager* statusUpdateManager;
+
+  // Set to the base class `secretGenerator` in `initialize()`. After
+  // `initialize()` has executed, this will be `None()`.
+  Option<mesos::SecretGenerator*> mockSecretGenerator;
 };
 
 } // namespace tests {


[05/14] mesos git commit: Fixed invocations of 'executorEnvironment' in the tests.

Posted by an...@apache.org.
Fixed invocations of 'executorEnvironment' in the tests.

This patch updates invocations of `executorEnvironment` in the tests
to use the function's new signature, which accepts an authentication
token.

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


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

Branch: refs/heads/master
Commit: 873497f2718f91d86746ce2266376c7f442c8c5f
Parents: 8b6ddb5
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:05:09 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:05:09 2017 -0700

----------------------------------------------------------------------
 src/tests/containerizer/mesos_containerizer_tests.cpp | 3 +++
 src/tests/hook_tests.cpp                              | 2 ++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/873497f2/src/tests/containerizer/mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mesos_containerizer_tests.cpp b/src/tests/containerizer/mesos_containerizer_tests.cpp
index ec9f354..9a5cfe4 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -35,6 +35,7 @@
 #include <stout/uuid.hpp>
 
 #include "slave/flags.hpp"
+#include "slave/slave.hpp"
 
 #include "slave/containerizer/fetcher.hpp"
 
@@ -56,6 +57,7 @@ using namespace process;
 
 using mesos::internal::master::Master;
 
+using mesos::internal::slave::executorEnvironment;
 using mesos::internal::slave::Fetcher;
 using mesos::internal::slave::FetcherProcess;
 using mesos::internal::slave::Launcher;
@@ -535,6 +537,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ExecutorEnvironmentVariable)
       directory,
       slaveId,
       PID<Slave>(),
+      None(),
       false);
 
   Future<bool> launch = containerizer->launch(

http://git-wip-us.apache.org/repos/asf/mesos/blob/873497f2/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index 71e619c..02d8f80 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -60,6 +60,7 @@ using mesos::internal::master::Master;
 using mesos::internal::protobuf::createLabel;
 
 using mesos::internal::slave::DockerContainerizer;
+using mesos::internal::slave::executorEnvironment;
 using mesos::internal::slave::Fetcher;
 using mesos::internal::slave::MesosContainerizer;
 using mesos::internal::slave::Slave;
@@ -309,6 +310,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
       directory,
       slaveId,
       PID<Slave>(),
+      None(),
       false);
 
   // Test hook adds a new environment variable "FOO" to the executor


[02/14] mesos git commit: Added support for auth tokens to the default executor.

Posted by an...@apache.org.
Added support for auth tokens to the default executor.

This patch updates the default executor to extract
an authentication token from its environment when present,
and use this token to authenticate with the agent.

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


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

Branch: refs/heads/master
Commit: 0fea4c5fb4ab785cf8c6d6db9e288391a8705e65
Parents: dfdd9bd
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:04:14 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:04:14 2017 -0700

----------------------------------------------------------------------
 src/launcher/default_executor.cpp | 20 +++++++++++++++++---
 1 file changed, 17 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0fea4c5f/src/launcher/default_executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/default_executor.cpp b/src/launcher/default_executor.cpp
index ee24531..606fd9c 100644
--- a/src/launcher/default_executor.cpp
+++ b/src/launcher/default_executor.cpp
@@ -120,7 +120,8 @@ public:
       const ExecutorID& _executorId,
       const ::URL& _agent,
       const string& _sandboxDirectory,
-      const string& _launcherDirectory)
+      const string& _launcherDirectory,
+      const Option<string>& _authenticationToken)
     : ProcessBase(process::ID::generate("default-executor")),
       state(DISCONNECTED),
       contentType(ContentType::PROTOBUF),
@@ -133,7 +134,8 @@ public:
       executorId(_executorId),
       agent(_agent),
       sandboxDirectory(_sandboxDirectory),
-      launcherDirectory(_launcherDirectory) {}
+      launcherDirectory(_launcherDirectory),
+      authenticationToken(_authenticationToken) {}
 
   virtual ~DefaultExecutor() = default;
 
@@ -1191,6 +1193,10 @@ private:
     request.headers = {{"Accept", stringify(contentType)},
                        {"Content-Type", stringify(contentType)}};
 
+    if (authenticationToken.isSome()) {
+      request.headers["Authorization"] = "Bearer " + authenticationToken.get();
+    }
+
     // Only pipeline requests when there is an active connection.
     if (connection.isSome()) {
       request.keepAlive = true;
@@ -1286,6 +1292,7 @@ private:
   const ::URL agent; // Agent API URL.
   const string sandboxDirectory;
   const string launcherDirectory;
+  const Option<string> authenticationToken;
 
   LinkedHashMap<UUID, Call::Update> unacknowledgedUpdates;
 
@@ -1392,13 +1399,20 @@ int main(int argc, char** argv)
   }
   sandboxDirectory = value.get();
 
+  Option<string> authenticationToken;
+  value = os::getenv("MESOS_EXECUTOR_AUTHENTICATION_TOKEN");
+  if (value.isSome()) {
+    authenticationToken = value.get();
+  }
+
   Owned<mesos::internal::DefaultExecutor> executor(
       new mesos::internal::DefaultExecutor(
           frameworkId,
           executorId,
           agent,
           sandboxDirectory,
-          flags.launcher_dir));
+          flags.launcher_dir,
+          authenticationToken));
 
   process::spawn(executor.get());
   process::wait(executor.get());


[13/14] mesos git commit: Added a new unmocked method to the mock agent.

Posted by an...@apache.org.
Added a new unmocked method to the mock agent.

This patch adds a new method to the mock agent,
`MockSlave::unmocked_executorTerminated`. This will
facilitate the testing of executor secret generation,
as it allows us to forward the mock function call to
the base class method.

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


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

Branch: refs/heads/master
Commit: d11dd0e6c1daa9724642ad57db94bdfd12436d22
Parents: ca58a52
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:06:06 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:06:06 2017 -0700

----------------------------------------------------------------------
 src/tests/mock_slave.cpp | 13 +++++++++++++
 src/tests/mock_slave.hpp |  6 ++++++
 2 files changed, 19 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d11dd0e6/src/tests/mock_slave.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mock_slave.cpp b/src/tests/mock_slave.cpp
index f145c5c..35cd13d 100644
--- a/src/tests/mock_slave.cpp
+++ b/src/tests/mock_slave.cpp
@@ -35,6 +35,8 @@
 
 using mesos::master::detector::MasterDetector;
 
+using mesos::slave::ContainerTermination;
+
 using std::list;
 
 using process::Future;
@@ -139,6 +141,8 @@ MockSlave::MockSlave(
     .WillRepeatedly(Invoke(this, &MockSlave::unmocked_qosCorrections));
   EXPECT_CALL(*this, usage())
     .WillRepeatedly(Invoke(this, &MockSlave::unmocked_usage));
+  EXPECT_CALL(*this, executorTerminated(_, _, _))
+    .WillRepeatedly(Invoke(this, &MockSlave::unmocked_executorTerminated));
 }
 
 
@@ -224,6 +228,15 @@ Future<ResourceUsage> MockSlave::unmocked_usage()
   return slave::Slave::usage();
 }
 
+
+void MockSlave::unmocked_executorTerminated(
+    const FrameworkID& frameworkId,
+    const ExecutorID& executorId,
+    const Future<Option<ContainerTermination>>& termination)
+{
+  slave::Slave::executorTerminated(frameworkId, executorId, termination);
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/d11dd0e6/src/tests/mock_slave.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mock_slave.hpp b/src/tests/mock_slave.hpp
index 979ac59..2b30b87 100644
--- a/src/tests/mock_slave.hpp
+++ b/src/tests/mock_slave.hpp
@@ -195,6 +195,12 @@ public:
       const process::Future<Option<
           mesos::slave::ContainerTermination>>& termination));
 
+  void unmocked_executorTerminated(
+      const FrameworkID& frameworkId,
+      const ExecutorID& executorId,
+      const process::Future<Option<
+          mesos::slave::ContainerTermination>>& termination);
+
 private:
   Files files;
   MockGarbageCollector gc;


[09/14] mesos git commit: Turned on executor authentication in the 'DefaultExecutorTests'.

Posted by an...@apache.org.
Turned on executor authentication in the 'DefaultExecutorTests'.

Now that executor authentication has been added, this patch
enables authentication for the agent's operator endpoint in
the DefaultExecutorTests. This is only done when build with
SSL, since executor authentication currently has SSL as a
dependency.

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


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

Branch: refs/heads/master
Commit: 3f9bf3798188c384253ab3c31c3dba4831aec48b
Parents: 4050eb2
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:05:24 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:05:24 2017 -0700

----------------------------------------------------------------------
 src/tests/default_executor_tests.cpp | 54 +++++++++++++++++++------------
 1 file changed, 33 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3f9bf379/src/tests/default_executor_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/default_executor_tests.cpp b/src/tests/default_executor_tests.cpp
index b7440d3..6dadd89 100644
--- a/src/tests/default_executor_tests.cpp
+++ b/src/tests/default_executor_tests.cpp
@@ -80,7 +80,24 @@ namespace tests {
 
 class DefaultExecutorTest
   : public MesosTest,
-    public WithParamInterface<string> {};
+    public WithParamInterface<string>
+{
+protected:
+  slave::Flags CreateSlaveFlags()
+  {
+    slave::Flags flags = MesosTest::CreateSlaveFlags();
+
+#ifndef USE_SSL_SOCKET
+    // Disable operator API authentication for the default executor. Executor
+    // authentication currently has SSL as a dependency, so we cannot require
+    // executors to authenticate with the agent operator API if Mesos was not
+    // built with SSL support.
+    flags.authenticate_http_readwrite = false;
+#endif // USE_SSL_SOCKET
+
+    return flags;
+  }
+};
 
 
 // These tests are parameterized by the containerizers enabled on the agent.
@@ -105,9 +122,7 @@ TEST_P(DefaultExecutorTest, TaskRunning)
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Disable AuthN on the agent.
   slave::Flags flags = CreateSlaveFlags();
-  flags.authenticate_http_readwrite = false;
   flags.containerizers = GetParam();
 
   Owned<MasterDetector> detector = master.get()->createDetector();
@@ -241,9 +256,7 @@ TEST_P(DefaultExecutorTest, KillTask)
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Disable AuthN on the agent.
   slave::Flags flags = CreateSlaveFlags();
-  flags.authenticate_http_readwrite = false;
   flags.containerizers = GetParam();
 
   Owned<MasterDetector> detector = master.get()->createDetector();
@@ -530,9 +543,7 @@ TEST_P(DefaultExecutorTest, KillTaskGroupOnTaskFailure)
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Disable AuthN on the agent.
   slave::Flags flags = CreateSlaveFlags();
-  flags.authenticate_http_readwrite = false;
 
   Owned<MasterDetector> detector = master.get()->createDetector();
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
@@ -710,9 +721,7 @@ TEST_P(DefaultExecutorTest, TaskUsesExecutor)
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Disable AuthN on the agent.
   slave::Flags flags = CreateSlaveFlags();
-  flags.authenticate_http_readwrite = false;
   flags.containerizers = GetParam();
 
   Owned<MasterDetector> detector = master.get()->createDetector();
@@ -819,9 +828,7 @@ TEST_P(DefaultExecutorTest, ROOT_ContainerStatusForTask)
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Disable AuthN on the agent.
   slave::Flags flags = CreateSlaveFlags();
-  flags.authenticate_http_readwrite = false;
   flags.containerizers = GetParam();
 
   Owned<MasterDetector> detector = master.get()->createDetector();
@@ -932,9 +939,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(DefaultExecutorTest, CommitSuicideOnTaskFailure)
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Disable AuthN on the agent.
   slave::Flags flags = CreateSlaveFlags();
-  flags.authenticate_http_readwrite = false;
 
   Owned<MasterDetector> detector = master.get()->createDetector();
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
@@ -1070,9 +1075,7 @@ TEST_P(DefaultExecutorTest, CommitSuicideOnKillTask)
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Disable AuthN on the agent.
   slave::Flags flags = CreateSlaveFlags();
-  flags.authenticate_http_readwrite = false;
 
   Owned<MasterDetector> detector = master.get()->createDetector();
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
@@ -1270,9 +1273,7 @@ TEST_P(DefaultExecutorTest, ReservedResources)
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Disable AuthN on the agent.
   slave::Flags flags = CreateSlaveFlags();
-  flags.authenticate_http_readwrite = false;
 
   Owned<MasterDetector> detector = master.get()->createDetector();
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
@@ -1394,6 +1395,21 @@ public:
   PersistentVolumeDefaultExecutor() : param(GetParam()) {}
 
 protected:
+  slave::Flags CreateSlaveFlags()
+  {
+    slave::Flags flags = MesosTest::CreateSlaveFlags();
+
+#ifndef USE_SSL_SOCKET
+    // Disable operator API authentication for the default executor. Executor
+    // authentication currently has SSL as a dependency, so we cannot require
+    // executors to authenticate with the agent operator API if Mesos was not
+    // built with SSL support.
+    flags.authenticate_http_readwrite = false;
+#endif // USE_SSL_SOCKET
+
+    return flags;
+  }
+
   LauncherAndIsolationParam param;
 };
 
@@ -1416,9 +1432,7 @@ TEST_P(PersistentVolumeDefaultExecutor, ROOT_PersistentResources)
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Disable AuthN on the agent.
   slave::Flags flags = CreateSlaveFlags();
-  flags.authenticate_http_readwrite = false;
   flags.launcher = param.launcher;
   flags.isolation = param.isolation;
 
@@ -1560,9 +1574,7 @@ TEST_P(PersistentVolumeDefaultExecutor, ROOT_TaskSandboxPersistentVolume)
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Disable AuthN on the agent.
   slave::Flags flags = CreateSlaveFlags();
-  flags.authenticate_http_readwrite = false;
   flags.launcher = param.launcher;
   flags.isolation = param.isolation;
 


[08/14] mesos git commit: Updated 'SchedulerTest.TaskGroupRunning'.

Posted by an...@apache.org.
Updated 'SchedulerTest.TaskGroupRunning'.

This patch updates the test `SchedulerTest.TaskGroupRunning`
to confirm that the agent-side code responsible for launching
task groups works correctly. Previously, this test only
verified that the `RunTaskGroupMessage` was sent to the agent.

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


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

Branch: refs/heads/master
Commit: 4050eb2c9d24ebc68fbf2efdc3e0601e50e1aacd
Parents: 2f6bb1f
Author: Greg Mann <gr...@mesosphere.io>
Authored: Sat Mar 25 12:05:22 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Sat Mar 25 12:05:22 2017 -0700

----------------------------------------------------------------------
 src/tests/scheduler_tests.cpp | 94 ++++++++++++++++++++++++++++++++++----
 1 file changed, 85 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/4050eb2c/src/tests/scheduler_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/scheduler_tests.cpp b/src/tests/scheduler_tests.cpp
index 65259b4..0f5d9ad 100644
--- a/src/tests/scheduler_tests.cpp
+++ b/src/tests/scheduler_tests.cpp
@@ -38,6 +38,7 @@
 
 #include <process/metrics/metrics.hpp>
 
+#include <stout/hashset.hpp>
 #include <stout/lambda.hpp>
 #include <stout/try.hpp>
 
@@ -529,9 +530,6 @@ TEST_P(SchedulerTest, TaskRunning)
 
 // Ensures that a task group can be successfully launched
 // on the `DEFAULT` executor.
-//
-// TODO(bmahler): We currently only test the master-side
-// of task group handling, since the rest is unimplemented.
 TEST_P(SchedulerTest, TaskGroupRunning)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
@@ -539,8 +537,16 @@ TEST_P(SchedulerTest, TaskGroupRunning)
 
   auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
 
+  slave::Flags flags = CreateSlaveFlags();
+#ifndef USE_SSL_SOCKET
+  // Executor authentication currently has SSL as a dependency, so we cannot
+  // require executors to authenticate with the agent operator API if Mesos
+  // was not built with SSL support.
+  flags.authenticate_http_readwrite = false;
+#endif // USE_SSL_SOCKET
+
   Owned<MasterDetector> detector = master.get()->createDetector();
-  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
 
   Future<Nothing> connected;
@@ -565,7 +571,8 @@ TEST_P(SchedulerTest, TaskGroupRunning)
 
   Future<Event::Offers> offers;
   EXPECT_CALL(*scheduler, offers(_, _))
-    .WillOnce(FutureArg<1>(&offers));
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
 
   {
     Call call;
@@ -602,6 +609,7 @@ TEST_P(SchedulerTest, TaskGroupRunning)
   task1.mutable_agent_id()->CopyFrom(
       offers->offers(0).agent_id());
   task1.mutable_resources()->CopyFrom(resources);
+  task1.mutable_command()->set_value("exit 0");
 
   v1::TaskInfo task2;
   task2.set_name("2");
@@ -609,11 +617,25 @@ TEST_P(SchedulerTest, TaskGroupRunning)
   task2.mutable_agent_id()->CopyFrom(
       offers->offers(0).agent_id());
   task2.mutable_resources()->CopyFrom(resources);
+  task2.mutable_command()->set_value("exit 0");
 
   v1::TaskGroupInfo taskGroup;
   taskGroup.add_tasks()->CopyFrom(task1);
   taskGroup.add_tasks()->CopyFrom(task2);
 
+  Future<Event::Update> runningUpdate1;
+  Future<Event::Update> runningUpdate2;
+  Future<Event::Update> finishedUpdate1;
+  Future<Event::Update> finishedUpdate2;
+  EXPECT_CALL(*scheduler, update(_, _))
+    .WillOnce(FutureArg<1>(&runningUpdate1))
+    .WillOnce(FutureArg<1>(&runningUpdate2))
+    .WillOnce(FutureArg<1>(&finishedUpdate1))
+    .WillOnce(FutureArg<1>(&finishedUpdate2));
+
+  EXPECT_CALL(*scheduler, failure(_, _))
+    .Times(AtMost(1));
+
   {
     Call call;
     call.mutable_framework_id()->CopyFrom(frameworkId);
@@ -634,10 +656,6 @@ TEST_P(SchedulerTest, TaskGroupRunning)
     mesos.send(call);
   }
 
-  // TODO(bmahler): For now we only ensure that the message is
-  // sent to the agent, since the agent-side of task groups is
-  // not yet implemented.
-
   AWAIT_READY(runTaskGroupMessage);
 
   EXPECT_EQ(devolve(frameworkId), runTaskGroupMessage->framework().id());
@@ -650,6 +668,64 @@ TEST_P(SchedulerTest, TaskGroupRunning)
             runTaskGroupMessage->task_group().tasks(0).task_id());
   EXPECT_EQ(devolve(task2.task_id()),
             runTaskGroupMessage->task_group().tasks(1).task_id());
+
+  AWAIT_READY(runningUpdate1);
+  ASSERT_EQ(v1::TASK_RUNNING, runningUpdate1->status().state());
+
+  AWAIT_READY(runningUpdate2);
+  ASSERT_EQ(v1::TASK_RUNNING, runningUpdate2->status().state());
+
+  const hashset<v1::TaskID> tasks{task1.task_id(), task2.task_id()};
+
+  // TASK_RUNNING updates for the tasks in a
+  // task group can be received in any order.
+  const hashset<v1::TaskID> tasksRunning{
+    runningUpdate1->status().task_id(),
+    runningUpdate2->status().task_id()};
+
+  ASSERT_EQ(tasks, tasksRunning);
+
+  // Acknowledge the TASK_RUNNING updates so
+  // that subsequent updates can be received.
+  {
+    Call call;
+    call.mutable_framework_id()->CopyFrom(frameworkId);
+    call.set_type(Call::ACKNOWLEDGE);
+
+    Call::Acknowledge* acknowledge = call.mutable_acknowledge();
+    acknowledge->mutable_task_id()->CopyFrom(
+        runningUpdate1->status().task_id());
+    acknowledge->mutable_agent_id()->CopyFrom(offers->offers(0).agent_id());
+    acknowledge->set_uuid(runningUpdate1->status().uuid());
+
+    mesos.send(call);
+  }
+
+  {
+    Call call;
+    call.mutable_framework_id()->CopyFrom(frameworkId);
+    call.set_type(Call::ACKNOWLEDGE);
+
+    Call::Acknowledge* acknowledge = call.mutable_acknowledge();
+    acknowledge->mutable_task_id()->CopyFrom(
+        runningUpdate2->status().task_id());
+    acknowledge->mutable_agent_id()->CopyFrom(offers->offers(0).agent_id());
+    acknowledge->set_uuid(runningUpdate2->status().uuid());
+
+    mesos.send(call);
+  }
+
+  AWAIT_READY(finishedUpdate1);
+  EXPECT_EQ(v1::TASK_FINISHED, finishedUpdate1->status().state());
+
+  AWAIT_READY(finishedUpdate2);
+  EXPECT_EQ(v1::TASK_FINISHED, finishedUpdate2->status().state());
+
+  const hashset<v1::TaskID> tasksFinished{
+    finishedUpdate1->status().task_id(),
+    finishedUpdate2->status().task_id()};
+
+  EXPECT_EQ(tasks, tasksFinished);
 }