You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ka...@apache.org on 2017/05/22 21:28:03 UTC

[1/5] mesos git commit: Introduced SecretResolver module interface.

Repository: mesos
Updated Branches:
  refs/heads/master a2d571b10 -> 1c7ffbeb5


Introduced SecretResolver module interface.

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


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

Branch: refs/heads/master
Commit: 9430a3b33b3d5d9955c3a2aed459e66106307bcb
Parents: a2d571b
Author: Kapil Arya <ka...@mesosphere.io>
Authored: Wed Apr 26 10:55:57 2017 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Mon May 22 15:28:37 2017 -0400

----------------------------------------------------------------------
 include/mesos/module.hpp                 | 14 ++++++
 include/mesos/module/secret_resolver.hpp | 63 +++++++++++++++++++++++++++
 include/mesos/secret/resolver.hpp        | 57 ++++++++++++++++++++++++
 src/Makefile.am                          | 12 +++--
 src/module/manager.cpp                   |  1 +
 5 files changed, 144 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9430a3b3/include/mesos/module.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/module.hpp b/include/mesos/module.hpp
index c28d01d..aa6356c 100644
--- a/include/mesos/module.hpp
+++ b/include/mesos/module.hpp
@@ -99,6 +99,20 @@ struct Module;
 template <typename T>
 const char* kind();
 
+// Each module "kind" specialization extends ModuleBase to provide a `create()`
+// method that returns a pointer to an object of the given module "kind".
+//   template <> struct Module<mesos::SecretResolver> : ModuleBase {
+//     Module(..., T* (*_create)(const Parameters&))
+//       : ModuleBase(...), create(_create) {...}
+//     T* (*create)(const Parameters&);
+//   };
+
+// TODO(kapil): Update module interface to return a managed pointer instead of
+// returning raw pointers. This would allow the caller to manage the lifecycle
+// of the dynamically-allocated object. We should also allow passing
+// master/agent flags during module initialization. E.g.,
+//   unique_ptr<T> (*create)(const Parameters&, const Flags&);
+
 } // namespace modules {
 } // namespace mesos {
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/9430a3b3/include/mesos/module/secret_resolver.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/module/secret_resolver.hpp b/include/mesos/module/secret_resolver.hpp
new file mode 100644
index 0000000..a60e661
--- /dev/null
+++ b/include/mesos/module/secret_resolver.hpp
@@ -0,0 +1,63 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef __MESOS_MODULE_SECRET_RESOLVER_HPP__
+#define __MESOS_MODULE_SECRET_RESOLVER_HPP__
+
+#include <mesos/mesos.hpp>
+#include <mesos/module.hpp>
+
+#include <mesos/secret/resolver.hpp>
+
+namespace mesos {
+namespace modules {
+
+template <>
+inline const char* kind<mesos::SecretResolver>()
+{
+  return "SecretResolver";
+}
+
+
+template <>
+struct Module<mesos::SecretResolver> : ModuleBase
+{
+  Module(
+      const char* _moduleApiVersion,
+      const char* _mesosVersion,
+      const char* _authorName,
+      const char* _authorEmail,
+      const char* _description,
+      bool (*_compatible)(),
+      mesos::SecretResolver*
+        (*_create)(const Parameters& parameters))
+    : ModuleBase(
+        _moduleApiVersion,
+        _mesosVersion,
+        mesos::modules::kind<mesos::SecretResolver>(),
+        _authorName,
+        _authorEmail,
+        _description,
+        _compatible),
+      create(_create) {}
+
+  mesos::SecretResolver* (*create)(const Parameters& parameters);
+};
+
+} // namespace modules {
+} // namespace mesos {
+
+#endif // __MESOS_MODULE_SECRET_RESOLVER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/9430a3b3/include/mesos/secret/resolver.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/secret/resolver.hpp b/include/mesos/secret/resolver.hpp
new file mode 100644
index 0000000..244dad2
--- /dev/null
+++ b/include/mesos/secret/resolver.hpp
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef __MESOS_SECRET_RESOLVER_HPP__
+#define __MESOS_SECRET_RESOLVER_HPP__
+
+#include <mesos/mesos.hpp>
+
+#include <process/future.hpp>
+
+namespace mesos {
+
+// This interface is used to resolve a `Secret` type into data bytes.
+//
+// The `resolve()` method takes a `Secret` object, potentially communicates
+// with a secret-store backend, and returns `Future<Secret::Value>`. If the
+// secret cannot be resolved (e.g., secret is invalid), the future fails.
+//
+// NOTE: The `create()` call should return a dynamically allocated object
+// whose lifecycle is then delegated to the master/agent.
+class SecretResolver
+{
+public:
+  // Factory method used to create a SecretResolver instance. If the
+  // `name` parameter is provided, the module is instantiated
+  // using the `ModuleManager`. Otherwise, a "default" secret resolver
+  // instance (defined in `src/secret/resolver.hpp`) is returned.
+  static Try<SecretResolver*> create(const Option<std::string>& name = None());
+
+  virtual ~SecretResolver() {}
+
+  // Validates the given secret, resolves the secret reference (by potentially
+  // querying a secret backend store), and returns the data associated with
+  // the secret.
+  virtual process::Future<Secret::Value> resolve(
+      const Secret& secret) const = 0;
+
+protected:
+  SecretResolver() {}
+};
+
+} // namespace mesos {
+
+#endif // __MESOS_SECRET_RESOLVER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/9430a3b3/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 3e71393..a122a8b 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -653,6 +653,8 @@ module_HEADERS =							\
   $(top_srcdir)/include/mesos/module/authenticator.hpp			\
   $(top_srcdir)/include/mesos/module/authorizer.hpp			\
   $(top_srcdir)/include/mesos/module/container_logger.hpp		\
+  $(top_srcdir)/include/mesos/module/contender.hpp			\
+  $(top_srcdir)/include/mesos/module/detector.hpp			\
   $(top_srcdir)/include/mesos/module/hook.hpp				\
   $(top_srcdir)/include/mesos/module/hook.proto				\
   $(top_srcdir)/include/mesos/module/http_authenticator.hpp		\
@@ -661,9 +663,8 @@ module_HEADERS =							\
   $(top_srcdir)/include/mesos/module/module.proto			\
   $(top_srcdir)/include/mesos/module/qos_controller.hpp			\
   $(top_srcdir)/include/mesos/module/resource_estimator.hpp		\
-  $(top_srcdir)/include/mesos/module/contender.hpp			\
-  $(top_srcdir)/include/mesos/module/detector.hpp			\
-  $(top_srcdir)/include/mesos/module/secret_generator.hpp
+  $(top_srcdir)/include/mesos/module/secret_generator.hpp		\
+  $(top_srcdir)/include/mesos/module/secret_resolver.hpp
 
 nodist_module_HEADERS =							\
   ../include/mesos/module/hook.pb.h					\
@@ -723,6 +724,11 @@ nodist_agent_HEADERS =							\
   ../include/mesos/slave/containerizer.pb.h				\
   ../include/mesos/slave/oversubscription.pb.h
 
+secretdir = $(pkgincludedir)/secret
+
+secret_HEADERS =							\
+  $(top_srcdir)/include/mesos/secret/resolver.hpp
+
 statedir = $(pkgincludedir)/state
 
 state_HEADERS =								\

http://git-wip-us.apache.org/repos/asf/mesos/blob/9430a3b3/src/module/manager.cpp
----------------------------------------------------------------------
diff --git a/src/module/manager.cpp b/src/module/manager.cpp
index 7d875fc..ec5614a 100644
--- a/src/module/manager.cpp
+++ b/src/module/manager.cpp
@@ -78,6 +78,7 @@ void ModuleManager::initialize()
   kindToVersion["MasterDetector"] = MESOS_VERSION;
   kindToVersion["QoSController"] = MESOS_VERSION;
   kindToVersion["ResourceEstimator"] = MESOS_VERSION;
+  kindToVersion["SecretResolver"] = MESOS_VERSION;
   kindToVersion["TestModule"] = MESOS_VERSION;
 
   // What happens then when Mesos is built with a certain version,


[4/5] mesos git commit: Added environment secret isolator.

Posted by ka...@apache.org.
Added environment secret isolator.

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


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

Branch: refs/heads/master
Commit: e9e9e739d4d120c7c7dc9f366b0b1f7d2f1d3154
Parents: 5a630d8
Author: Kapil Arya <ka...@mesosphere.io>
Authored: Mon May 1 18:51:22 2017 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Mon May 22 16:35:15 2017 -0400

----------------------------------------------------------------------
 src/CMakeLists.txt                              |   1 +
 src/Makefile.am                                 |   3 +
 src/launcher/executor.cpp                       |   8 ++
 src/slave/containerizer/mesos/containerizer.cpp |  21 ++-
 .../mesos/isolators/environment_secret.cpp      | 138 +++++++++++++++++++
 .../mesos/isolators/environment_secret.hpp      |  60 ++++++++
 src/tests/CMakeLists.txt                        |   1 +
 .../environment_secret_isolator_tests.cpp       | 130 +++++++++++++++++
 8 files changed, 360 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e9e9e739/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index d71f1c6..fd154e2 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -197,6 +197,7 @@ set(AGENT_SRC
 if (NOT WIN32)
   set(AGENT_SRC
     ${AGENT_SRC}
+    slave/containerizer/mesos/isolators/environment_secret.cpp
     slave/containerizer/mesos/isolators/docker/volume/driver.cpp
     slave/containerizer/mesos/isolators/docker/volume/paths.cpp
     slave/containerizer/mesos/isolators/filesystem/posix.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/e9e9e739/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index e1fdda3..78158a3 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -960,6 +960,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   slave/containerizer/mesos/paths.cpp					\
   slave/containerizer/mesos/utils.cpp					\
   slave/containerizer/mesos/io/switchboard.cpp				\
+  slave/containerizer/mesos/isolators/environment_secret.cpp		\
   slave/containerizer/mesos/isolators/docker/volume/driver.cpp		\
   slave/containerizer/mesos/isolators/docker/volume/paths.cpp		\
   slave/containerizer/mesos/isolators/filesystem/posix.cpp		\
@@ -1090,6 +1091,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   slave/containerizer/mesos/paths.hpp					\
   slave/containerizer/mesos/utils.hpp					\
   slave/containerizer/mesos/io/switchboard.hpp				\
+  slave/containerizer/mesos/isolators/environment_secret.hpp		\
   slave/containerizer/mesos/isolators/posix.hpp				\
   slave/containerizer/mesos/isolators/filesystem/posix.hpp		\
   slave/containerizer/mesos/isolators/filesystem/windows.hpp		\
@@ -2345,6 +2347,7 @@ mesos_tests_SOURCES =						\
   tests/containerizer/docker_containerizer_tests.cpp		\
   tests/containerizer/docker_spec_tests.cpp			\
   tests/containerizer/docker_tests.cpp				\
+  tests/containerizer/environment_secret_isolator_tests.cpp	\
   tests/containerizer/io_switchboard_tests.cpp			\
   tests/containerizer/isolator_tests.cpp			\
   tests/containerizer/launcher.cpp				\

http://git-wip-us.apache.org/repos/asf/mesos/blob/e9e9e739/src/launcher/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.cpp b/src/launcher/executor.cpp
index 9971a67..9ac3c3d 100644
--- a/src/launcher/executor.cpp
+++ b/src/launcher/executor.cpp
@@ -587,6 +587,10 @@ protected:
     if (taskEnvironment.isSome()) {
       foreach (const Environment::Variable& variable,
                taskEnvironment->variables()) {
+        // Skip overwriting if the variable is unresolved secret.
+        if (variable.type() == Environment::Variable::SECRET) {
+          continue;
+        }
         const string& name = variable.name();
         if (environment.contains(name) &&
             environment[name].value() != variable.value()) {
@@ -599,6 +603,10 @@ protected:
     if (command.has_environment()) {
       foreach (const Environment::Variable& variable,
                command.environment().variables()) {
+        // Skip overwriting if the variable is unresolved secret.
+        if (variable.type() == Environment::Variable::SECRET) {
+          continue;
+        }
         const string& name = variable.name();
         if (environment.contains(name) &&
             environment[name].value() != variable.value()) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/e9e9e739/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 2c9cf38..b513e68 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -65,6 +65,7 @@
 
 #include "slave/containerizer/mesos/io/switchboard.hpp"
 
+#include "slave/containerizer/mesos/isolators/environment_secret.hpp"
 #include "slave/containerizer/mesos/isolators/filesystem/posix.hpp"
 #include "slave/containerizer/mesos/isolators/posix.hpp"
 #include "slave/containerizer/mesos/isolators/posix/disk.hpp"
@@ -221,6 +222,11 @@ Try<MesosContainerizer*> MesosContainerizer::create(
   }
 #endif // __linux__
 
+  // Always add environment secret isolator.
+  if (!strings::contains(flags_.isolation, "environment_secret")) {
+    flags_.isolation += ",environment_secret";
+  }
+
   LOG(INFO) << "Using isolation: " << flags_.isolation;
 
   // Create the launcher for the MesosContainerizer.
@@ -353,6 +359,11 @@ Try<MesosContainerizer*> MesosContainerizer::create(
 #if !defined(__WINDOWS__) && defined(ENABLE_PORT_MAPPING_ISOLATOR)
     {"network/port_mapping", &PortMappingIsolatorProcess::create},
 #endif
+
+    {"environment_secret",
+      [secretResolver] (const Flags& flags) -> Try<Isolator*> {
+        return EnvironmentSecretIsolatorProcess::create(flags, secretResolver);
+      }},
   };
 
   vector<string> tokens = strings::tokenize(flags_.isolation, ",");
@@ -1483,9 +1494,15 @@ Future<bool> MesosContainerizerProcess::_launch(
   containerEnvironment.MergeFrom(launchInfo.environment());
 
   // Include user specified environment.
+  // Skip over any secrets as they should have been resolved by the
+  // environment_secret isolator.
   if (container->config.command_info().has_environment()) {
-    containerEnvironment.MergeFrom(
-        container->config.command_info().environment());
+    foreach (const Environment::Variable& variable,
+             container->config.command_info().environment().variables()) {
+      if (variable.type() != Environment::Variable::SECRET) {
+        containerEnvironment.add_variables()->CopyFrom(variable);
+      }
+    }
   }
 
   // Set the aggregated environment of the launch command.

http://git-wip-us.apache.org/repos/asf/mesos/blob/e9e9e739/src/slave/containerizer/mesos/isolators/environment_secret.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/environment_secret.cpp b/src/slave/containerizer/mesos/isolators/environment_secret.cpp
new file mode 100644
index 0000000..5b0b2fc
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/environment_secret.cpp
@@ -0,0 +1,138 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "slave/containerizer/mesos/isolators/environment_secret.hpp"
+
+#include <list>
+
+#include <mesos/secret/resolver.hpp>
+
+#include <process/collect.hpp>
+#include <process/future.hpp>
+#include <process/id.hpp>
+#include <process/owned.hpp>
+
+#include <stout/foreach.hpp>
+
+#include "common/validation.hpp"
+
+using std::list;
+
+using process::collect;
+using process::Failure;
+using process::Future;
+using process::Owned;
+
+using mesos::slave::ContainerConfig;
+using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::ContainerState;
+using mesos::slave::Isolator;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+Try<Isolator*> EnvironmentSecretIsolatorProcess::create(
+    const Flags& flags,
+    SecretResolver* secretResolver)
+{
+  Owned<MesosIsolatorProcess> process(new EnvironmentSecretIsolatorProcess(
+      flags,
+      secretResolver));;
+
+  return new MesosIsolator(process);
+}
+
+
+EnvironmentSecretIsolatorProcess::EnvironmentSecretIsolatorProcess(
+    const Flags& _flags,
+    SecretResolver* _secretResolver)
+  : ProcessBase(process::ID::generate("environment-secret-isolator")),
+    flags(_flags),
+    secretResolver(_secretResolver) {}
+
+
+EnvironmentSecretIsolatorProcess::~EnvironmentSecretIsolatorProcess() {}
+
+
+bool EnvironmentSecretIsolatorProcess::supportsNesting()
+{
+  return true;
+}
+
+
+Future<Option<ContainerLaunchInfo>> EnvironmentSecretIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig)
+{
+  Option<Error> error = common::validation::validateEnvironment(
+      containerConfig.command_info().environment());
+  if (error.isSome()) {
+    return Failure( "Invalid environment specified: " + error->message);
+  }
+
+  Environment environment;
+
+  list<Future<Environment::Variable>> futures;
+  foreach (const Environment::Variable& variable,
+           containerConfig.command_info().environment().variables()) {
+    if (variable.type() != Environment::Variable::SECRET) {
+      continue;
+    }
+
+    const Secret& secret = variable.secret();
+
+    error = common::validation::validateSecret(secret);
+    if (error.isSome()) {
+      return Failure(
+          "Invalid secret specified in environment '" + variable.name() +
+          "': " + error->message);
+    }
+
+    if (secretResolver == nullptr) {
+      return Failure(
+          "Error: Environment variable '" + variable.name() +
+          "' contains secret but no secret resolver provided");
+    }
+
+    Future<Environment::Variable> future = secretResolver->resolve(secret)
+      .then([variable](const Secret::Value& secretValue)
+            -> Future<Environment::Variable> {
+          Environment::Variable result;
+          result.set_name(variable.name());
+          result.set_value(secretValue.data());
+          return result;
+        });
+
+    futures.push_back(future);
+  }
+
+  return collect(futures)
+    .then([](const list<Environment::Variable>& variables)
+        -> Future<Option<ContainerLaunchInfo>> {
+      ContainerLaunchInfo launchInfo;
+      Environment* environment = launchInfo.mutable_environment();
+      foreach (const Environment::Variable& variable, variables) {
+        environment->add_variables()->CopyFrom(variable);
+      }
+      launchInfo.mutable_task_environment()->CopyFrom(*environment);
+      return launchInfo;
+    });
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/e9e9e739/src/slave/containerizer/mesos/isolators/environment_secret.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/environment_secret.hpp b/src/slave/containerizer/mesos/isolators/environment_secret.hpp
new file mode 100644
index 0000000..b98e8fe
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/environment_secret.hpp
@@ -0,0 +1,60 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef __ENVIRONMENT_SECRET_ISOLATOR_HPP__
+#define __ENVIRONMENT_SECRET_ISOLATOR_HPP__
+
+#include <list>
+
+#include <mesos/secret/resolver.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class EnvironmentSecretIsolatorProcess : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(
+      const Flags& flags,
+      SecretResolver* secretResolver);
+
+  virtual ~EnvironmentSecretIsolatorProcess();
+
+  virtual bool supportsNesting();
+
+  virtual process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig);
+
+private:
+  EnvironmentSecretIsolatorProcess(
+      const Flags& flags,
+      SecretResolver* secretResolver);
+
+  const Flags flags;
+  SecretResolver* secretResolver;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __ENVIRONMENT_SECRET_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/e9e9e739/src/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt
index 9f2af9c..8cecb72 100644
--- a/src/tests/CMakeLists.txt
+++ b/src/tests/CMakeLists.txt
@@ -193,6 +193,7 @@ if (NOT WIN32)
     containerizer/docker_containerizer_tests.cpp
     containerizer/docker_spec_tests.cpp
     containerizer/docker_tests.cpp
+    containerizer/environment_secret_isolator_tests.cpp
     containerizer/io_switchboard_tests.cpp
     containerizer/isolator_tests.cpp
     containerizer/memory_isolator_tests.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/e9e9e739/src/tests/containerizer/environment_secret_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/environment_secret_isolator_tests.cpp b/src/tests/containerizer/environment_secret_isolator_tests.cpp
new file mode 100644
index 0000000..f8c7719
--- /dev/null
+++ b/src/tests/containerizer/environment_secret_isolator_tests.cpp
@@ -0,0 +1,130 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include <string>
+
+#include <mesos/secret/resolver.hpp>
+
+#include <process/future.hpp>
+#include <process/gtest.hpp>
+
+#include <stout/gtest.hpp>
+
+#include "tests/mesos.hpp"
+
+using process::Future;
+using process::Owned;
+
+using mesos::internal::slave::Fetcher;
+using mesos::internal::slave::MesosContainerizer;
+
+using mesos::master::detector::MasterDetector;
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+const char SECRET_VALUE[] = "password";
+const char SECRET_ENV_NAME[] = "My_SeCrEt";
+
+class EnvironmentSecretIsolatorTest : public MesosTest {};
+
+
+// This test verifies that the environment secrets are resolved when launching a
+// task.
+TEST_F(EnvironmentSecretIsolatorTest, ResolveSecret)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  mesos::internal::slave::Flags flags = CreateSlaveFlags();
+
+  Fetcher fetcher;
+  Try<SecretResolver*> secretResolver = SecretResolver::create();
+  EXPECT_SOME(secretResolver);
+
+  Try<MesosContainerizer*> containerizer =
+    MesosContainerizer::create(flags, false, &fetcher, secretResolver.get());
+  EXPECT_SOME(containerizer);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+  Try<Owned<cluster::Slave>> slave =
+    StartSlave(detector.get(), containerizer.get());
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  Future<std::vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  const string commandString = strings::format(
+      "env; test \"$%s\" = \"%s\"",
+      string(SECRET_ENV_NAME),
+      string(SECRET_VALUE));
+
+  CommandInfo command;
+  command.set_value(commandString);
+
+  // Request a secret.
+  // TODO(kapil): Update createEnvironment() to support secrets.
+  mesos::Environment::Variable *env =
+    command.mutable_environment()->add_variables();
+  env->set_name(SECRET_ENV_NAME);
+  env->set_type(mesos::Environment::Variable::SECRET);
+
+  mesos::Secret* secret = env->mutable_secret();
+  secret->set_type(Secret::VALUE);
+  secret->mutable_value()->set_data(SECRET_VALUE);
+
+  TaskInfo task = createTask(
+      offers.get()[0].slave_id(),
+      Resources::parse("cpus:0.1;mem:32").get(),
+      command);
+
+  // NOTE: Successful tasks will output two status updates.
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusFinished;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusFinished));
+
+  driver.launchTasks(offers.get()[0].id(), {task});
+
+  AWAIT_READY(statusRunning);
+  EXPECT_EQ(TASK_RUNNING, statusRunning.get().state());
+  AWAIT_READY(statusFinished);
+  EXPECT_EQ(TASK_FINISHED, statusFinished.get().state());
+
+  driver.stop();
+  driver.join();
+}
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {


[5/5] mesos git commit: Added volume secret isolator.

Posted by ka...@apache.org.
Added volume secret isolator.

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


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

Branch: refs/heads/master
Commit: 1c7ffbeb505b3f5ab759202195f0b946a20cb803
Parents: e9e9e73
Author: Kapil Arya <ka...@mesosphere.io>
Authored: Wed May 3 03:05:17 2017 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Mon May 22 16:36:22 2017 -0400

----------------------------------------------------------------------
 src/CMakeLists.txt                              |   1 +
 src/Makefile.am                                 |   5 +-
 src/slave/containerizer/mesos/containerizer.cpp |   6 +
 .../mesos/isolators/volume/secret.cpp           | 300 +++++++++++++++++++
 .../mesos/isolators/volume/secret.hpp           |  67 +++++
 src/tests/CMakeLists.txt                        |   1 +
 .../environment_secret_isolator_tests.cpp       |   4 +-
 src/tests/containerizer/rootfs.cpp              |   1 +
 .../volume_secret_isolator_tests.cpp            | 263 ++++++++++++++++
 9 files changed, 645 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index fd154e2..ca7d538 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -321,6 +321,7 @@ set(LINUX_SRC
   slave/containerizer/mesos/isolators/namespaces/pid.cpp
   slave/containerizer/mesos/isolators/network/cni/cni.cpp
   slave/containerizer/mesos/isolators/volume/image.cpp
+  slave/containerizer/mesos/isolators/volume/secret.cpp
   slave/containerizer/mesos/provisioner/backends/aufs.cpp
   slave/containerizer/mesos/provisioner/backends/bind.cpp
   slave/containerizer/mesos/provisioner/backends/overlay.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 78158a3..7e4ce85 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1204,6 +1204,7 @@ MESOS_LINUX_FILES =									\
   slave/containerizer/mesos/isolators/network/cni/cni.cpp				\
   slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.cpp	\
   slave/containerizer/mesos/isolators/volume/image.cpp					\
+  slave/containerizer/mesos/isolators/volume/secret.cpp					\
   slave/containerizer/mesos/provisioner/backends/aufs.cpp				\
   slave/containerizer/mesos/provisioner/backends/bind.cpp				\
   slave/containerizer/mesos/provisioner/backends/overlay.cpp
@@ -1250,6 +1251,7 @@ MESOS_LINUX_FILES +=									\
   slave/containerizer/mesos/isolators/network/cni/cni.hpp				\
   slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.hpp	\
   slave/containerizer/mesos/isolators/volume/image.hpp					\
+  slave/containerizer/mesos/isolators/volume/secret.hpp					\
   slave/containerizer/mesos/provisioner/backends/aufs.hpp				\
   slave/containerizer/mesos/provisioner/backends/bind.hpp				\
   slave/containerizer/mesos/provisioner/backends/overlay.hpp
@@ -2414,7 +2416,8 @@ mesos_tests_SOURCES +=						\
   tests/containerizer/runtime_isolator_tests.cpp		\
   tests/containerizer/sched_tests.cpp				\
   tests/containerizer/setns_test_helper.cpp			\
-  tests/containerizer/volume_image_isolator_tests.cpp
+  tests/containerizer/volume_image_isolator_tests.cpp		\
+  tests/containerizer/volume_secret_isolator_tests.cpp
 endif
 
 if ENABLE_PORT_MAPPING_ISOLATOR

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index b513e68..7695bce 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -71,6 +71,7 @@
 #include "slave/containerizer/mesos/isolators/posix/disk.hpp"
 #include "slave/containerizer/mesos/isolators/posix/rlimits.hpp"
 #include "slave/containerizer/mesos/isolators/volume/sandbox_path.hpp"
+#include "slave/containerizer/mesos/isolators/volume/secret.hpp"
 
 #include "slave/containerizer/mesos/provisioner/provisioner.hpp"
 
@@ -338,6 +339,11 @@ Try<MesosContainerizer*> MesosContainerizer::create(
         return VolumeImageIsolatorProcess::create(flags, provisioner);
       }},
 
+    {"volume/secret",
+      [secretResolver] (const Flags& flags) -> Try<Isolator*> {
+        return VolumeSecretIsolatorProcess::create(flags, secretResolver);
+      }},
+
     {"gpu/nvidia",
       [&nvidia] (const Flags& flags) -> Try<Isolator*> {
         if (!nvml::isAvailable()) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/slave/containerizer/mesos/isolators/volume/secret.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/volume/secret.cpp b/src/slave/containerizer/mesos/isolators/volume/secret.cpp
new file mode 100644
index 0000000..ee9366f
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/volume/secret.cpp
@@ -0,0 +1,300 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "slave/containerizer/mesos/isolators/volume/secret.hpp"
+
+#include <list>
+#include <string>
+
+#include <mesos/secret/resolver.hpp>
+
+#include <process/collect.hpp>
+#include <process/future.hpp>
+#include <process/id.hpp>
+#include <process/owned.hpp>
+
+#include <stout/foreach.hpp>
+#include <stout/stringify.hpp>
+#include <stout/strings.hpp>
+
+#include <stout/os/mkdir.hpp>
+#include <stout/os/write.hpp>
+
+#ifdef __linux__
+#include "linux/ns.hpp"
+#endif // __linux__
+
+#include "common/validation.hpp"
+
+using std::list;
+using std::string;
+
+using process::Failure;
+using process::Future;
+using process::Owned;
+
+using mesos::slave::ContainerClass;
+using mesos::slave::ContainerConfig;
+using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::ContainerState;
+using mesos::slave::Isolator;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+constexpr char SECRET_DIR[] = ".secret";
+
+
+Try<Isolator*> VolumeSecretIsolatorProcess::create(
+    const Flags& flags,
+    SecretResolver* secretResolver)
+{
+  if (flags.launcher != "linux" ||
+      !strings::contains(flags.isolation, "filesystem/linux")) {
+    return Error("Volume secret isolation requires filesystem/linux isolator.");
+  }
+
+  const string hostSecretTmpDir = path::join(flags.runtime_dir, SECRET_DIR);
+
+  Try<Nothing> mkdir = os::mkdir(hostSecretTmpDir);
+  if (mkdir.isError()) {
+    return Error("Failed to create secret directory on the host tmpfs:" +
+                 mkdir.error());
+  }
+
+  Owned<MesosIsolatorProcess> process(new VolumeSecretIsolatorProcess(
+      flags,
+      secretResolver));
+
+  return new MesosIsolator(process);
+}
+
+
+VolumeSecretIsolatorProcess::VolumeSecretIsolatorProcess(
+    const Flags& _flags,
+    SecretResolver* secretResolver)
+  : ProcessBase(process::ID::generate("volume-secret-isolator")),
+    flags(_flags),
+    secretResolver(secretResolver) {}
+
+
+bool VolumeSecretIsolatorProcess::supportsNesting()
+{
+  return true;
+}
+
+
+Future<Option<ContainerLaunchInfo>> VolumeSecretIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig)
+{
+  if (!containerConfig.has_container_info()) {
+    return None();
+  }
+
+  const ContainerInfo& containerInfo = containerConfig.container_info();
+
+  if (containerInfo.type() != ContainerInfo::MESOS) {
+    return Failure(
+        "Can only prepare the secret volume isolator for a MESOS container");
+  }
+
+  if (containerConfig.has_container_class() &&
+      containerConfig.container_class() == ContainerClass::DEBUG) {
+    return None();
+  }
+
+  ContainerLaunchInfo launchInfo;
+  launchInfo.add_clone_namespaces(CLONE_NEWNS);
+
+  const string sandboxSecretRootDir =
+    path::join(containerConfig.directory(),
+               SECRET_DIR + string("-") + stringify(UUID::random()));
+
+  // TODO(Kapil): Add some UUID suffix to the secret-root dir to avoid conflicts
+  // with user container_path.
+  Try<Nothing> mkdir = os::mkdir(sandboxSecretRootDir);
+  if (mkdir.isError()) {
+    return Failure("Failed to create sandbox secret root directory at '" +
+                   sandboxSecretRootDir + "': " + mkdir.error());
+  }
+
+  // Mount ramfs in the container.
+  CommandInfo* command = launchInfo.add_pre_exec_commands();
+  command->set_shell(false);
+  command->set_value("mount");
+  command->add_arguments("mount");
+  command->add_arguments("-n");
+  command->add_arguments("-t");
+  command->add_arguments("ramfs");
+  command->add_arguments("ramfs");
+  command->add_arguments(sandboxSecretRootDir);
+
+  list<Future<Nothing>> futures;
+  foreach (const Volume& volume, containerInfo.volumes()) {
+    if (!volume.has_source() ||
+        !volume.source().has_type() ||
+        volume.source().type() != Volume::Source::SECRET) {
+      continue;
+    }
+
+    if (!volume.source().has_secret()) {
+      return Failure("volume.source.secret is not specified");
+    }
+
+    if (secretResolver == nullptr) {
+      return Failure(
+          "Error: Volume has secret but no secret-resolver provided");
+    }
+
+    const Secret& secret = volume.source().secret();
+
+    Option<Error> error = common::validation::validateSecret(secret);
+    if (error.isSome()) {
+      return Failure("Invalid secret specified in volume: " + error->message);
+    }
+
+    string targetContainerPath;
+    if (path::absolute(volume.container_path())) {
+      if (containerConfig.has_rootfs()) {
+        targetContainerPath = path::join(
+            containerConfig.rootfs(),
+            volume.container_path());
+
+        Try<Nothing> mkdir = os::mkdir(Path(targetContainerPath).dirname());
+        if (mkdir.isError()) {
+          return Failure(
+              "Failed to create directory '" +
+              Path(targetContainerPath).dirname() + "' "
+              "for the target mount file: " + mkdir.error());
+        }
+
+        Try<Nothing> touch = os::touch(targetContainerPath);
+        if (touch.isError()) {
+          return Failure(
+              "Failed to create the target mount file at '" +
+              targetContainerPath + "': " + touch.error());
+        }
+      } else {
+        targetContainerPath = volume.container_path();
+
+        if (!os::exists(targetContainerPath)) {
+          return Failure(
+              "Absolute container path '" + targetContainerPath + "' "
+              "does not exist");
+        }
+      }
+    } else {
+      if (containerConfig.has_rootfs()) {
+        targetContainerPath = path::join(
+            containerConfig.rootfs(),
+            flags.sandbox_directory,
+            volume.container_path());
+      } else {
+        targetContainerPath = path::join(
+            containerConfig.directory(),
+            volume.container_path());
+      }
+
+      // Create the mount point if bind mount is used.
+      // NOTE: We cannot create the mount point at 'targetContainerPath' if
+      // container has rootfs defined. The bind mount of the sandbox
+      // will hide what's inside 'targetContainerPath'. So we should always
+      // create the mount point in the sandbox.
+      const string mountPoint = path::join(
+          containerConfig.directory(),
+          volume.container_path());
+
+      Try<Nothing> mkdir = os::mkdir(Path(mountPoint).dirname());
+      if (mkdir.isError()) {
+        return Failure(
+            "Failed to create the target mount file directory at '" +
+            Path(mountPoint).dirname() + "': " + mkdir.error());
+      }
+
+      Try<Nothing> touch = os::touch(mountPoint);
+      if (touch.isError()) {
+        return Failure(
+            "Failed to create the target mount file at '" +
+            targetContainerPath + "': " + touch.error());
+      }
+    }
+
+    const string hostSecretPath =
+      path::join(flags.runtime_dir, SECRET_DIR, stringify(UUID::random()));
+
+    const string sandboxSecretPath =
+      path::join(sandboxSecretRootDir,
+                 volume.container_path());
+
+    Try<Nothing> mkdir = os::mkdir(Path(sandboxSecretPath).dirname());
+    if (mkdir.isError()) {
+      return Failure(
+          "Failed to create the target mount file directory at '" +
+          Path(sandboxSecretPath).dirname() + "': " + mkdir.error());
+    }
+
+    // Create directory tree inside sandbox secret root dir.
+    command = launchInfo.add_pre_exec_commands();
+    command->set_shell(false);
+    command->set_value("mkdir");
+    command->add_arguments("mkdir");
+    command->add_arguments("-p");
+    command->add_arguments(Path(sandboxSecretPath).dirname());
+
+    // Move secret from hostSecretPath to sandboxSecretPath.
+    command = launchInfo.add_pre_exec_commands();
+    command->set_shell(false);
+    command->set_value("mv");
+    command->add_arguments("mv");
+    command->add_arguments("-f");
+    command->add_arguments(hostSecretPath);
+    command->add_arguments(sandboxSecretPath);
+
+    // Bind mount sandboxSecretPath to targetContainerPath
+    command = launchInfo.add_pre_exec_commands();
+    command->set_shell(false);
+    command->set_value("mount");
+    command->add_arguments("mount");
+    command->add_arguments("-n");
+    command->add_arguments("--rbind");
+    command->add_arguments(sandboxSecretPath);
+    command->add_arguments(targetContainerPath);
+
+    Future<Nothing> future = secretResolver->resolve(secret)
+      .then([hostSecretPath](const Secret::Value& value) -> Future<Nothing> {
+        Try<Nothing> writeSecret = os::write(hostSecretPath, value.data());
+        if (writeSecret.isError()) {
+          return Failure(
+              "Error writing secret to '" + hostSecretPath + "': " +
+              writeSecret.error());
+        }
+        return Nothing();
+      });
+
+    futures.push_back(future);
+  }
+
+  return collect(futures)
+    .then([launchInfo]() -> Future<Option<ContainerLaunchInfo>> {
+      return launchInfo;
+    });
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/slave/containerizer/mesos/isolators/volume/secret.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/volume/secret.hpp b/src/slave/containerizer/mesos/isolators/volume/secret.hpp
new file mode 100644
index 0000000..2680345
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/volume/secret.hpp
@@ -0,0 +1,67 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef __VOLUME_SECRET_ISOLATOR_HPP__
+#define __VOLUME_SECRET_ISOLATOR_HPP__
+
+#include <list>
+
+#include <mesos/secret/resolver.hpp>
+
+#include <mesos/slave/isolator.hpp>
+
+#include <process/future.hpp>
+
+#include <stout/option.hpp>
+#include <stout/try.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class VolumeSecretIsolatorProcess : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(
+      const Flags& flags,
+      SecretResolver* secretResolver);
+
+  virtual ~VolumeSecretIsolatorProcess() {}
+
+  virtual bool supportsNesting();
+
+  virtual process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig);
+
+private:
+  VolumeSecretIsolatorProcess(
+      const Flags& flags,
+      SecretResolver* secretResolver);
+
+  const Flags flags;
+  SecretResolver* secretResolver;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __VOLUME_SECRET_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt
index 8cecb72..29a46bd 100644
--- a/src/tests/CMakeLists.txt
+++ b/src/tests/CMakeLists.txt
@@ -231,6 +231,7 @@ if (LINUX)
     containerizer/runtime_isolator_tests.cpp
     containerizer/sched_tests.cpp
     containerizer/volume_image_isolator_tests.cpp
+    containerizer/volume_secret_isolator_tests.cpp
     )
 endif (LINUX)
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/tests/containerizer/environment_secret_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/environment_secret_isolator_tests.cpp b/src/tests/containerizer/environment_secret_isolator_tests.cpp
index f8c7719..6190040 100644
--- a/src/tests/containerizer/environment_secret_isolator_tests.cpp
+++ b/src/tests/containerizer/environment_secret_isolator_tests.cpp
@@ -85,8 +85,8 @@ TEST_F(EnvironmentSecretIsolatorTest, ResolveSecret)
 
   const string commandString = strings::format(
       "env; test \"$%s\" = \"%s\"",
-      string(SECRET_ENV_NAME),
-      string(SECRET_VALUE));
+      SECRET_ENV_NAME,
+      SECRET_VALUE).get();
 
   CommandInfo command;
   command.set_value(commandString);

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/tests/containerizer/rootfs.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/rootfs.cpp b/src/tests/containerizer/rootfs.cpp
index fdfecc6..fe9252d 100644
--- a/src/tests/containerizer/rootfs.cpp
+++ b/src/tests/containerizer/rootfs.cpp
@@ -126,6 +126,7 @@ Try<process::Owned<Rootfs>> LinuxRootfs::create(const string& root)
   }
 
   const vector<string> programs = {
+    "/bin/cat",
     "/bin/echo",
     "/bin/ls",
     "/bin/ping",

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/tests/containerizer/volume_secret_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/volume_secret_isolator_tests.cpp b/src/tests/containerizer/volume_secret_isolator_tests.cpp
new file mode 100644
index 0000000..073c392
--- /dev/null
+++ b/src/tests/containerizer/volume_secret_isolator_tests.cpp
@@ -0,0 +1,263 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include <map>
+#include <set>
+#include <string>
+
+#include <mesos/secret/resolver.hpp>
+
+#include <process/future.hpp>
+#include <process/gtest.hpp>
+
+#include <stout/gtest.hpp>
+
+#include "tests/mesos.hpp"
+
+#include "tests/containerizer/docker_archive.hpp"
+
+using process::Future;
+using process::Owned;
+
+using mesos::internal::slave::Fetcher;
+using mesos::internal::slave::MesosContainerizer;
+
+using mesos::internal::slave::state::SlaveState;
+
+using mesos::slave::ContainerTermination;
+
+using std::map;
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+const char SECRET_VALUE[] = "password";
+
+
+enum FS_TYPE {
+  WITH_ROOTFS,
+  WITHOUT_ROOTFS
+};
+
+
+enum CONTAINER_LAUNCH_STATUS {
+  CONTAINER_LAUNCH_FAILURE,
+  CONTAINER_LAUNCH_SUCCESS
+};
+
+
+class VolumeSecretIsolatorTest :
+  public MesosTest,
+  public ::testing::WithParamInterface<std::tr1::tuple<
+      const char*, const char*, enum FS_TYPE, enum CONTAINER_LAUNCH_STATUS>>
+
+{
+protected:
+  virtual void SetUp()
+  {
+    const char* prefix = std::tr1::get<0>(GetParam());
+    const char* path = std::tr1::get<1>(GetParam());
+    secretContainerPath = string(prefix) + string(path);
+
+    fsType = std::tr1::get<2>(GetParam());
+    expectedContainerLaunchStatus = std::tr1::get<3>(GetParam());
+
+    volume.set_mode(Volume::RW);
+    volume.set_container_path(secretContainerPath);
+
+    Volume::Source* source = volume.mutable_source();
+    source->set_type(Volume::Source::SECRET);
+
+    // Request a secret.
+    Secret* secret = source->mutable_secret();
+    secret->set_type(Secret::VALUE);
+    secret->mutable_value()->set_data(SECRET_VALUE);
+
+    MesosTest::SetUp();
+  }
+
+  string secretContainerPath;
+  bool expectedContainerLaunchStatus;
+  bool fsType;
+
+  Volume volume;
+};
+
+
+static const char* paths[] = {
+  "my_secret",
+  "some/my_secret",
+  "etc/my_secret",
+  "etc/some/my_secret"
+};
+
+
+INSTANTIATE_TEST_CASE_P(
+    SecretTestTypeWithoutRootFSRelativePath,
+    VolumeSecretIsolatorTest,
+    ::testing::Combine(::testing::Values(""),
+                       ::testing::ValuesIn(paths),
+                       ::testing::Values(WITHOUT_ROOTFS),
+                       ::testing::Values(CONTAINER_LAUNCH_SUCCESS)));
+
+
+INSTANTIATE_TEST_CASE_P(
+    SecretTestTypeWithoutRootFSNonExisitingAbsolutePath,
+    VolumeSecretIsolatorTest,
+    ::testing::Combine(::testing::Values("/"),
+                       ::testing::ValuesIn(paths),
+                       ::testing::Values(WITHOUT_ROOTFS),
+                       ::testing::Values(CONTAINER_LAUNCH_FAILURE)));
+
+
+INSTANTIATE_TEST_CASE_P(
+    SecretTestTypeWithoutRootFSExistingAbsolutePath,
+    VolumeSecretIsolatorTest,
+    ::testing::Combine(::testing::Values(""),
+                       ::testing::Values("/bin/touch"),
+                       ::testing::Values(WITHOUT_ROOTFS),
+                       ::testing::Values(CONTAINER_LAUNCH_SUCCESS)));
+
+
+INSTANTIATE_TEST_CASE_P(
+    SecretTestTypeWithRootFS,
+    VolumeSecretIsolatorTest,
+    ::testing::Combine(::testing::Values("", "/"),
+                       ::testing::ValuesIn(paths),
+                       ::testing::Values(WITH_ROOTFS),
+                       ::testing::Values(CONTAINER_LAUNCH_SUCCESS)));
+
+
+TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "filesystem/linux,volume/secret";
+
+  if (fsType == WITH_ROOTFS) {
+    const string registry = path::join(sandbox.get(), "registry");
+    AWAIT_READY(DockerArchive::create(registry, "test_image_rootfs"));
+    AWAIT_READY(DockerArchive::create(registry, "test_image_volume"));
+
+    flags.isolation += ",volume/image,docker/runtime";
+    flags.docker_registry = registry;
+    flags.docker_store_dir = path::join(sandbox.get(), "store");
+    flags.image_providers = "docker";
+  }
+
+  Fetcher fetcher;
+
+  Try<SecretResolver*> secretResolver = SecretResolver::create();
+  EXPECT_SOME(secretResolver);
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      true,
+      &fetcher,
+      secretResolver.get());
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  ContainerInfo containerInfo;
+  if (fsType == WITH_ROOTFS) {
+    containerInfo = createContainerInfo(
+        "test_image_rootfs",
+        {createVolumeFromDockerImage(
+            "rootfs", "test_image_volume", Volume::RW)});
+  } else {
+    containerInfo.set_type(ContainerInfo::MESOS);
+  }
+
+  containerInfo.add_volumes()->CopyFrom(volume);
+
+  CommandInfo command = createCommandInfo(
+      "secret=$(cat " + secretContainerPath + "); "
+      "test \"$secret\" = \"" + string(SECRET_VALUE) + "\" && sleep 1000");
+
+  ExecutorInfo executor = createExecutorInfo("test_executor", command);
+  executor.mutable_container()->CopyFrom(containerInfo);
+
+  string directory = path::join(flags.work_dir, "sandbox");
+  ASSERT_SOME(os::mkdir(directory));
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      executor,
+      directory,
+      None(),
+      SlaveID(),
+      map<string, string>(),
+      false);
+
+  if (expectedContainerLaunchStatus == CONTAINER_LAUNCH_FAILURE) {
+    AWAIT_FAILED(launch);
+    return;
+  }
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  // Now launch nested container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(UUID::random().toString());
+
+  CommandInfo nestedCommand = createCommandInfo(
+      "secret=$(cat " + secretContainerPath + "); "
+      "test \"$secret\" = \"" + string(SECRET_VALUE) + "\"");
+
+  launch = containerizer->launch(
+      nestedContainerId,
+      nestedCommand,
+      containerInfo,
+      None(),
+      state.id);
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  // Wait for nested container.
+  Future<Option<ContainerTermination>> wait = containerizer->wait(
+      nestedContainerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
+
+  // Now wait for parent container.
+  wait = containerizer->wait(containerId);
+  containerizer->destroy(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+}
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {


[2/5] mesos git commit: Added default secret resolver module.

Posted by ka...@apache.org.
Added default secret resolver module.

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


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

Branch: refs/heads/master
Commit: d284a9ed561ec4f8967114953febd5dd84c92006
Parents: 9430a3b
Author: Kapil Arya <ka...@mesosphere.io>
Authored: Wed Apr 26 10:59:01 2017 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Mon May 22 15:31:26 2017 -0400

----------------------------------------------------------------------
 src/CMakeLists.txt      |  5 +++
 src/Makefile.am         |  1 +
 src/secret/resolver.cpp | 85 ++++++++++++++++++++++++++++++++++++++++++++
 3 files changed, 91 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d284a9ed/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index a038c0b..d71f1c6 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -400,6 +400,10 @@ set(SCHEDULER_SRC
   scheduler/scheduler.cpp
   )
 
+set(SECRET_SRC
+  secret/resolver.cpp
+  )
+
 set(STATE_SRC
   ${STATE_SRC}
   state/in_memory.cpp
@@ -480,6 +484,7 @@ set(MESOS_SRC
   ${MODULE_SRC}
   ${OCI_SRC}
   ${SCHEDULER_SRC}
+  ${SECRET_SRC}
   ${STATE_SRC}
   ${URI_SRC}
   ${USAGE_SRC}

http://git-wip-us.apache.org/repos/asf/mesos/blob/d284a9ed/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index a122a8b..e1fdda3 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -932,6 +932,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   posix/rlimits.cpp							\
   sched/sched.cpp							\
   scheduler/scheduler.cpp						\
+  secret/resolver.cpp							\
   slave/constants.cpp							\
   slave/container_logger.cpp						\
   slave/flags.cpp							\

http://git-wip-us.apache.org/repos/asf/mesos/blob/d284a9ed/src/secret/resolver.cpp
----------------------------------------------------------------------
diff --git a/src/secret/resolver.cpp b/src/secret/resolver.cpp
new file mode 100644
index 0000000..13d45d0
--- /dev/null
+++ b/src/secret/resolver.cpp
@@ -0,0 +1,85 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include <string>
+
+#include <mesos/mesos.hpp>
+
+#include <mesos/module/secret_resolver.hpp>
+
+#include <mesos/secret/resolver.hpp>
+
+#include <process/future.hpp>
+
+#include <stout/try.hpp>
+
+#include "module/manager.hpp"
+
+using std::string;
+
+using process::Failure;
+using process::Future;
+using process::Shared;
+
+namespace mesos {
+namespace internal {
+
+// The default implementation verifies that the incoming secret
+// contains `value` but not `reference`. It then returns the value.
+class DefaultSecretResolver : public SecretResolver
+{
+public:
+  DefaultSecretResolver() {}
+
+  ~DefaultSecretResolver() {}
+
+  virtual process::Future<Secret::Value> resolve(const Secret& secret) const
+  {
+    if (secret.has_reference()) {
+      return Failure("Default secret resolver cannot resolve references");
+    }
+
+    if (!secret.has_value()) {
+      return Failure("Secret has no value");
+    }
+
+    return secret.value();
+  }
+};
+
+} // namespace internal {
+
+
+Try<SecretResolver*> SecretResolver::create(const Option<string>& moduleName)
+{
+  if (moduleName.isNone()) {
+    LOG(INFO) << "Creating default secret resolver";
+    return new internal::DefaultSecretResolver();
+  }
+
+  LOG(INFO) << "Creating secret resolver '" << moduleName.get() << "'";
+
+  Try<SecretResolver*> result =
+    modules::ModuleManager::create<SecretResolver>(moduleName.get());
+
+  if (result.isError()) {
+    return Error("Failed to initialize secret resolver: " + result.error());
+  }
+
+  return result;
+}
+
+} // namespace mesos {


[3/5] mesos git commit: Added --secret_resolver flag to agent.

Posted by ka...@apache.org.
Added --secret_resolver flag to agent.

Updated Containerizer to accept SecretResolver.

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


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

Branch: refs/heads/master
Commit: 5a630d82c1f59fc0156ffbe5356c3f1b3ac87489
Parents: d284a9e
Author: Kapil Arya <ka...@mesosphere.io>
Authored: Mon May 1 18:23:51 2017 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Mon May 22 15:36:36 2017 -0400

----------------------------------------------------------------------
 src/local/local.cpp                             | 19 +++++++++++++++++--
 src/slave/containerizer/containerizer.cpp       |  9 ++++++---
 src/slave/containerizer/containerizer.hpp       |  5 ++++-
 src/slave/containerizer/mesos/containerizer.cpp |  3 +++
 src/slave/containerizer/mesos/containerizer.hpp |  7 +++++--
 src/slave/flags.cpp                             |  7 +++++++
 src/slave/flags.hpp                             |  1 +
 src/slave/main.cpp                              | 17 +++++++++++++++--
 8 files changed, 58 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5a630d82/src/local/local.cpp
----------------------------------------------------------------------
diff --git a/src/local/local.cpp b/src/local/local.cpp
index e479809..3f4150b 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -29,6 +29,9 @@
 #include <mesos/module/authorizer.hpp>
 #include <mesos/module/contender.hpp>
 #include <mesos/module/detector.hpp>
+#include <mesos/module/secret_resolver.hpp>
+
+#include <mesos/secret/resolver.hpp>
 
 #include <mesos/slave/resource_estimator.hpp>
 
@@ -438,8 +441,20 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
       slaveFlags.launcher = "posix";
     }
 
-    Try<Containerizer*> containerizer =
-      Containerizer::create(slaveFlags, true, fetchers->back());
+    // Initialize SecretResolver.
+    Try<SecretResolver*> secretResolver =
+      mesos::SecretResolver::create(slaveFlags.secret_resolver);
+
+    if (secretResolver.isError()) {
+      EXIT(EXIT_FAILURE)
+        << "Failed to initialize secret resolver: " << secretResolver.error();
+    }
+
+    Try<Containerizer*> containerizer = Containerizer::create(
+        slaveFlags,
+        true,
+        fetchers->back(),
+        secretResolver.get());
 
     if (containerizer.isError()) {
       EXIT(EXIT_FAILURE)

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a630d82/src/slave/containerizer/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
index 9024371..15ae0b3 100644
--- a/src/slave/containerizer/containerizer.cpp
+++ b/src/slave/containerizer/containerizer.cpp
@@ -18,6 +18,8 @@
 #include <set>
 #include <vector>
 
+#include <mesos/secret/resolver.hpp>
+
 #include <process/dispatch.hpp>
 #include <process/owned.hpp>
 
@@ -208,7 +210,8 @@ Try<Resources> Containerizer::resources(const Flags& flags)
 Try<Containerizer*> Containerizer::create(
     const Flags& flags,
     bool local,
-    Fetcher* fetcher)
+    Fetcher* fetcher,
+    SecretResolver* secretResolver)
 {
   // Get the set of containerizer types.
   const vector<string> _types = strings::split(flags.containerizers, ",");
@@ -277,8 +280,8 @@ Try<Containerizer*> Containerizer::create(
 
   foreach (const string& type, containerizerTypes) {
     if (type == "mesos") {
-      Try<MesosContainerizer*> containerizer =
-        MesosContainerizer::create(flags, local, fetcher, nvidia);
+      Try<MesosContainerizer*> containerizer = MesosContainerizer::create(
+          flags, local, fetcher, secretResolver, nvidia);
       if (containerizer.isError()) {
         return Error("Could not create MesosContainerizer: " +
                      containerizer.error());

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a630d82/src/slave/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index 4c31a1f..f17e424 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -22,6 +22,8 @@
 #include <mesos/mesos.hpp>
 #include <mesos/resources.hpp>
 
+#include <mesos/secret/resolver.hpp>
+
 #include <mesos/slave/containerizer.hpp>
 
 #include <process/future.hpp>
@@ -60,7 +62,8 @@ public:
   static Try<Containerizer*> create(
       const Flags& flags,
       bool local,
-      Fetcher* fetcher);
+      Fetcher* fetcher,
+      SecretResolver* secretResolver = nullptr);
 
   // Determine slave resources from flags, probing the system or
   // querying a delegate.

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a630d82/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 50a63b5..2c9cf38 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -18,6 +18,8 @@
 
 #include <mesos/module/isolator.hpp>
 
+#include <mesos/secret/resolver.hpp>
+
 #include <mesos/slave/isolator.hpp>
 
 #include <process/collect.hpp>
@@ -145,6 +147,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     const Flags& flags,
     bool local,
     Fetcher* fetcher,
+    SecretResolver* secretResolver,
     const Option<NvidiaComponents>& nvidia)
 {
   // Modify `flags` based on the deprecated `isolation` flag (and then

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a630d82/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 04ab997..d767031 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -20,6 +20,10 @@
 #include <list>
 #include <vector>
 
+#include <mesos/secret/resolver.hpp>
+
+#include <mesos/slave/isolator.hpp>
+
 #include <process/id.hpp>
 #include <process/http.hpp>
 #include <process/sequence.hpp>
@@ -31,8 +35,6 @@
 #include <stout/multihashmap.hpp>
 #include <stout/os/int_fd.hpp>
 
-#include <mesos/slave/isolator.hpp>
-
 #include "slave/state.hpp"
 
 #include "slave/containerizer/containerizer.hpp"
@@ -60,6 +62,7 @@ public:
       const Flags& flags,
       bool local,
       Fetcher* fetcher,
+      SecretResolver* secretResolver = nullptr,
       const Option<NvidiaComponents>& nvidia = None());
 
   static Try<MesosContainerizer*> create(

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a630d82/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index e172aa5..c1e9568 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -958,6 +958,13 @@ mesos::internal::slave::Flags::Flags()
       "A comma-separated list of hook modules to be\n"
       "installed inside the agent.");
 
+  add(&Flags::secret_resolver,
+      "secret_resolver",
+      "The name of the secret resolver module to use for resolving\n"
+      "environment and file-based secrets. If this flag is not specified,\n"
+      "the default behavior is to resolve value-based secrets and error on\n"
+      "reference-based secrets.");
+
   add(&Flags::resource_estimator,
       "resource_estimator",
       "The name of the resource estimator to use for oversubscription.");

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a630d82/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 28f6482..b5dd841 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -154,6 +154,7 @@ public:
 #endif // USE_SSL_SOCKET
   Option<Path> http_credentials;
   Option<std::string> hooks;
+  Option<std::string> secret_resolver;
   Option<std::string> resource_estimator;
   Option<std::string> qos_controller;
   Duration qos_correction_interval_min;

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a630d82/src/slave/main.cpp
----------------------------------------------------------------------
diff --git a/src/slave/main.cpp b/src/slave/main.cpp
index cc83327..47b5576 100644
--- a/src/slave/main.cpp
+++ b/src/slave/main.cpp
@@ -27,6 +27,9 @@
 #include <mesos/mesos.hpp>
 
 #include <mesos/module/anonymous.hpp>
+#include <mesos/module/secret_resolver.hpp>
+
+#include <mesos/secret/resolver.hpp>
 
 #include <mesos/slave/resource_estimator.hpp>
 
@@ -84,6 +87,7 @@ using mesos::slave::QoSController;
 using mesos::slave::ResourceEstimator;
 
 using mesos::Authorizer;
+using mesos::SecretResolver;
 using mesos::SlaveInfo;
 
 using process::Owned;
@@ -233,7 +237,7 @@ int main(int argc, char** argv)
   //   contender/detector might depend upon anonymous modules.
   // * Hooks.
   // * Systemd support (if it exists).
-  // * Fetcher and Containerizer.
+  // * Fetcher, SecretResolver, and Containerizer.
   // * Master detector.
   // * Authorizer.
   // * Garbage collector.
@@ -446,8 +450,17 @@ int main(int argc, char** argv)
 
   Fetcher* fetcher = new Fetcher();
 
+  // Initialize SecretResolver.
+  Try<SecretResolver*> secretResolver =
+    mesos::SecretResolver::create(flags.secret_resolver);
+
+  if (secretResolver.isError()) {
+    EXIT(EXIT_FAILURE)
+        << "Failed to initialize secret resolver: " << secretResolver.error();
+  }
+
   Try<Containerizer*> containerizer =
-    Containerizer::create(flags, false, fetcher);
+    Containerizer::create(flags, false, fetcher, secretResolver.get());
 
   if (containerizer.isError()) {
     EXIT(EXIT_FAILURE)