You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2016/04/14 02:09:05 UTC

[1/3] mesos git commit: Added a test for slavePostFetchHook.

Repository: mesos
Updated Branches:
  refs/heads/master 124a05b4f -> f09ae5bcb


Added a test for slavePostFetchHook.

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


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

Branch: refs/heads/master
Commit: f09ae5bcb4acc88ad0a99eea76839b774f93bcaf
Parents: 0e00a83
Author: Jie Yu <yu...@gmail.com>
Authored: Wed Apr 13 17:07:58 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Apr 13 17:08:59 2016 -0700

----------------------------------------------------------------------
 src/examples/test_hook_module.cpp |  16 +++++
 src/tests/hook_tests.cpp          | 114 +++++++++++++++++++++++++++++++++
 2 files changed, 130 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f09ae5bc/src/examples/test_hook_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_hook_module.cpp b/src/examples/test_hook_module.cpp
index 0b1426b..4b97f84 100644
--- a/src/examples/test_hook_module.cpp
+++ b/src/examples/test_hook_module.cpp
@@ -198,6 +198,22 @@ public:
   }
 
 
+  virtual Try<Nothing> slavePostFetchHook(
+      const ContainerID& containerId,
+      const string& directory)
+  {
+    LOG(INFO) << "Executing 'slavePostFetchHook'";
+
+    const string path = path::join(directory, "post_fetch_hook");
+
+    if (os::exists(path)) {
+      return os::rm(path);
+    } else {
+      return Nothing();
+    }
+  }
+
+
   // This hook locates the file created by environment decorator hook
   // and deletes it.
   virtual Try<Nothing> slaveRemoveExecutorHook(

http://git-wip-us.apache.org/repos/asf/mesos/blob/f09ae5bc/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index 97ff55a..60d52c5 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -57,6 +57,7 @@ using mesos::internal::master::Master;
 
 using mesos::internal::protobuf::createLabel;
 
+using mesos::internal::slave::DockerContainerizer;
 using mesos::internal::slave::Fetcher;
 using mesos::internal::slave::MesosContainerizer;
 using mesos::internal::slave::Slave;
@@ -736,6 +737,119 @@ TEST_F(HookTest, ROOT_DOCKER_VerifySlavePreLaunchDockerHook)
   }
 }
 
+
+// Test that the slave post fetch hook is executed after fetching the
+// URIs but before the container is launched. We launch a command task
+// with a file URI (file name is "post_fetch_hook"). The test hook
+// will try to delete that file in the sandbox directory. We validate
+// the hook by verifying that "post_fetch_hook" file does not exist in
+// the sandbox when container is running.
+TEST_F(HookTest, ROOT_DOCKER_VerifySlavePostFetchHook)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Try<Owned<Docker>> _docker = Docker::create(
+      tests::flags.docker,
+      tests::flags.docker_socket);
+  ASSERT_SOME(_docker);
+
+  Shared<Docker> docker = _docker->share();
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  Fetcher fetcher;
+
+  Try<ContainerLogger*> logger =
+    ContainerLogger::create(flags.container_logger);
+  ASSERT_SOME(logger);
+
+  DockerContainerizer containerizer(
+      flags,
+      &fetcher,
+      Owned<ContainerLogger>(logger.get()),
+      docker);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(
+      detector.get(),
+      &containerizer,
+      flags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+
+  MesosSchedulerDriver driver(
+      &sched,
+      DEFAULT_FRAMEWORK_INFO,
+      master.get()->pid,
+      DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  ASSERT_NE(0u, offers->size());
+
+  TaskInfo task = createTask(
+      offers.get()[0].slave_id(),
+      Resources::parse("cpus:1;mem:128").get(),
+      "test ! -f " + path::join(flags.sandbox_directory, "post_fetch_hook"));
+
+  // Add a URI for a file on the host filesystem. This file will be
+  // fetched to the sandbox and will later be deleted by the hook.
+  const string file = path::join(sandbox.get(), "post_fetch_hook");
+  ASSERT_SOME(os::touch(file));
+
+  CommandInfo::URI* uri = task.mutable_command()->add_uris();
+  uri->set_value(file);
+
+  ContainerInfo* containerInfo = task.mutable_container();
+  containerInfo->set_type(ContainerInfo::DOCKER);
+
+  ContainerInfo::DockerInfo* dockerInfo = containerInfo->mutable_docker();
+  dockerInfo->set_image("alpine");
+
+  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_FOR(statusRunning, Seconds(60));
+  EXPECT_EQ(TASK_RUNNING, statusRunning.get().state());
+
+  AWAIT_READY_FOR(statusFinished, Seconds(60));
+  EXPECT_EQ(TASK_FINISHED, statusFinished.get().state());
+
+  driver.stop();
+  driver.join();
+
+  Future<list<Docker::Container>> containers =
+    docker->ps(true, slave::DOCKER_NAME_PREFIX);
+
+  AWAIT_READY(containers);
+
+  // Cleanup all mesos launched containers.
+  foreach (const Docker::Container& container, containers.get()) {
+    AWAIT_READY_FOR(docker->rm(container.id, true), Seconds(30));
+  }
+}
+
+
+// TODO(jieyu): Add a test for slavePostFetchHook using Mesos
+// containerizer.
+
+
 // Test that the changes made by the resources decorator hook are correctly
 // propagated to the resource offer.
 TEST_F(HookTest, VerifySlaveResourcesAndAttributesDecorator)


[3/3] mesos git commit: Added a slave post fetch hook.

Posted by ji...@apache.org.
Added a slave post fetch hook.

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


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

Branch: refs/heads/master
Commit: 80550c7d6105cf997def5930688485b70a66829f
Parents: 124a05b
Author: Jie Yu <yu...@gmail.com>
Authored: Wed Apr 13 15:28:19 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Apr 13 17:08:59 2016 -0700

----------------------------------------------------------------------
 include/mesos/hook.hpp                          | 11 ++++++++++
 src/hook/manager.cpp                            | 15 ++++++++++++++
 src/hook/manager.hpp                            |  4 ++++
 src/slave/containerizer/docker.cpp              | 21 ++++++++++++++++----
 src/slave/containerizer/mesos/containerizer.cpp | 10 +++++++++-
 5 files changed, 56 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/80550c7d/include/mesos/hook.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/hook.hpp b/include/mesos/hook.hpp
index 87e01a9..210ffba 100644
--- a/include/mesos/hook.hpp
+++ b/include/mesos/hook.hpp
@@ -101,6 +101,17 @@ public:
     return Nothing();
   }
 
+  // This hook is called from within slave after URIs and container
+  // image are fetched. A typical module implementing this hook will
+  // perform some operations on the fetched artifacts.
+  virtual Try<Nothing> slavePostFetchHook(
+      const ContainerID& containerId,
+      const std::string& directory)
+  {
+    return Nothing();
+  }
+
+
   // This hook is called from within slave when an executor is being
   // removed. A typical module implementing the hook will perform some
   // cleanup as required.

http://git-wip-us.apache.org/repos/asf/mesos/blob/80550c7d/src/hook/manager.cpp
----------------------------------------------------------------------
diff --git a/src/hook/manager.cpp b/src/hook/manager.cpp
index 17a42f8..692b9ea 100644
--- a/src/hook/manager.cpp
+++ b/src/hook/manager.cpp
@@ -232,6 +232,21 @@ void HookManager::slavePreLaunchDockerHook(
 }
 
 
+void HookManager::slavePostFetchHook(
+    const ContainerID& containerId,
+    const string& directory)
+{
+  foreach (const string& name, availableHooks.keys()) {
+    Hook* hook = availableHooks[name];
+    Try<Nothing> result = hook->slavePostFetchHook(containerId, directory);
+    if (result.isError()) {
+      LOG(WARNING) << "Slave post fetch hook failed for module "
+                   << "'" << name << "': " << result.error();
+    }
+  }
+}
+
+
 void HookManager::slaveRemoveExecutorHook(
     const FrameworkInfo& frameworkInfo,
     const ExecutorInfo& executorInfo)

http://git-wip-us.apache.org/repos/asf/mesos/blob/80550c7d/src/hook/manager.hpp
----------------------------------------------------------------------
diff --git a/src/hook/manager.hpp b/src/hook/manager.hpp
index 3af28a7..528674e 100644
--- a/src/hook/manager.hpp
+++ b/src/hook/manager.hpp
@@ -65,6 +65,10 @@ public:
       const Option<Resources>& resources,
       const Option<std::map<std::string, std::string>>& env);
 
+  static void slavePostFetchHook(
+      const ContainerID& containerId,
+      const std::string& directory);
+
   static void slaveRemoveExecutorHook(
       const FrameworkInfo& frameworkInfo,
       const ExecutorInfo& executorInfo);

http://git-wip-us.apache.org/repos/asf/mesos/blob/80550c7d/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 9c24227..30f85a1 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -1039,11 +1039,19 @@ Future<bool> DockerContainerizerProcess::launch(
     // creates the Docker container for the task. See more details in
     // the comments of r33174.
     return container.get()->launch = fetch(containerId, slaveId)
-      .then(defer(self(), [=]() { return pull(containerId); }))
       .then(defer(self(), [=]() {
+        return pull(containerId);
+      }))
+      .then(defer(self(), [=]() {
+        if (HookManager::hooksAvailable()) {
+          HookManager::slavePostFetchHook(containerId, directory);
+        }
+
         return mountPersistentVolumes(containerId);
       }))
-      .then(defer(self(), [=]() { return launchExecutorProcess(containerId); }))
+      .then(defer(self(), [=]() {
+        return launchExecutorProcess(containerId);
+      }))
       .then(defer(self(), [=](pid_t pid) {
         return reapExecutor(containerId, pid);
       }));
@@ -1063,8 +1071,14 @@ Future<bool> DockerContainerizerProcess::launch(
   // running in a container (via docker_mesos_image flag) we want the
   // executor to keep running when the slave container dies.
   return container.get()->launch = fetch(containerId, slaveId)
-    .then(defer(self(), [=]() { return pull(containerId); }))
     .then(defer(self(), [=]() {
+      return pull(containerId);
+    }))
+    .then(defer(self(), [=]() {
+      if (HookManager::hooksAvailable()) {
+        HookManager::slavePostFetchHook(containerId, directory);
+      }
+
       return mountPersistentVolumes(containerId);
     }))
     .then(defer(self(), [=]() {
@@ -2011,7 +2025,6 @@ void DockerContainerizerProcess::remove(
   }
 }
 
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/80550c7d/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index c25fa92..dfaffd7 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -38,6 +38,8 @@
 
 #include "common/protobuf_utils.hpp"
 
+#include "hook/manager.hpp"
+
 #include "module/manager.hpp"
 
 #include "slave/paths.hpp"
@@ -948,7 +950,13 @@ Future<Nothing> MesosContainerizerProcess::fetch(
       directory,
       user,
       slaveId,
-      flags);
+      flags)
+    .then([=]() -> Future<Nothing> {
+      if (HookManager::hooksAvailable()) {
+        HookManager::slavePostFetchHook(containerId, directory);
+      }
+      return Nothing();
+    });
 }
 
 


[2/3] mesos git commit: Used path::join in test hook for slavePreLaunchDockerHook.

Posted by ji...@apache.org.
Used path::join in test hook for slavePreLaunchDockerHook.


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

Branch: refs/heads/master
Commit: 0e00a83061a74a40aaad18ddc1bd7de720c36c6d
Parents: 80550c7
Author: Jie Yu <yu...@gmail.com>
Authored: Wed Apr 13 16:51:48 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Apr 13 17:08:59 2016 -0700

----------------------------------------------------------------------
 src/examples/test_hook_module.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0e00a830/src/examples/test_hook_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_hook_module.cpp b/src/examples/test_hook_module.cpp
index 3ff9fd7..0b1426b 100644
--- a/src/examples/test_hook_module.cpp
+++ b/src/examples/test_hook_module.cpp
@@ -194,7 +194,7 @@ public:
       const Option<map<string, string>>& env)
   {
     LOG(INFO) << "Executing 'slavePreLaunchDockerHook'";
-    return os::touch(sandboxDirectory + "/foo");
+    return os::touch(path::join(sandboxDirectory, "foo"));
   }