You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by me...@apache.org on 2015/04/21 00:32:01 UTC

[1/6] mesos git commit: Enabled label decorator to override.

Repository: mesos
Updated Branches:
  refs/heads/master 11105f7c0 -> 01ee1e504


Enabled label decorator to override.

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


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

Branch: refs/heads/master
Commit: a090e9dc7698ecfe5da51826c1aa2e5163966272
Parents: 11105f7
Author: Niklas Nielsen <ni...@qni.dk>
Authored: Mon Apr 20 14:36:14 2015 -0700
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Apr 20 14:36:14 2015 -0700

----------------------------------------------------------------------
 include/mesos/hook.hpp            |  6 +++---
 src/examples/test_hook_module.cpp | 16 +++++++++++++---
 src/hook/manager.cpp              | 15 +++++++++++----
 src/master/master.cpp             |  4 ++--
 src/tests/hook_tests.cpp          | 25 +++++++++++++++++--------
 5 files changed, 46 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a090e9dc/include/mesos/hook.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/hook.hpp b/include/mesos/hook.hpp
index 9ae8b94..f2b8259 100644
--- a/include/mesos/hook.hpp
+++ b/include/mesos/hook.hpp
@@ -34,9 +34,9 @@ public:
   virtual ~Hook() {};
 
   // This label decorator hook is called from within master during
-  // the launchTask routine.  A module implementing the hook creates
-  // and returns a set of labels.  These labels are then merged with
-  // the task labels and passed on to the slave/executor.
+  // the launchTask routine. A module implementing the hook creates
+  // and returns a set of labels. These labels overwrite the existing
+  // labels on the task info.
   virtual Result<Labels> masterLaunchTaskLabelDecorator(
       const TaskInfo& taskInfo,
       const FrameworkInfo& frameworkInfo,

http://git-wip-us.apache.org/repos/asf/mesos/blob/a090e9dc/src/examples/test_hook_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_hook_module.cpp b/src/examples/test_hook_module.cpp
index 2f2da1c..4fc7013 100644
--- a/src/examples/test_hook_module.cpp
+++ b/src/examples/test_hook_module.cpp
@@ -32,6 +32,7 @@ using namespace mesos;
 // tests/hook_tests.cpp.
 const char* testLabelKey = "MESOS_Test_Label";
 const char* testLabelValue = "ApacheMesos";
+const char* testRemoveLabelKey = "MESOS_Test_Remove_Label";
 
 
 class TestHook : public Hook
@@ -45,9 +46,18 @@ public:
     LOG(INFO) << "Executing 'masterLaunchTaskLabelDecorator' hook";
 
     Labels labels;
-    Label *label = labels.add_labels();
-    label->set_key(testLabelKey);
-    label->set_value(testLabelValue);
+
+    // Set one known label.
+    Label* newLabel = labels.add_labels();
+    newLabel->set_key(testLabelKey);
+    newLabel->set_value(testLabelValue);
+
+    // Remove the 'testRemoveLabelKey' label which was set by the test.
+    foreach (const Label& oldLabel, taskInfo.labels().labels()) {
+      if (oldLabel.key() != testRemoveLabelKey) {
+        labels.add_labels()->CopyFrom(oldLabel);
+      }
+    }
 
     return labels;
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/a090e9dc/src/hook/manager.cpp
----------------------------------------------------------------------
diff --git a/src/hook/manager.cpp b/src/hook/manager.cpp
index 7a4cb09..0d79ba2 100644
--- a/src/hook/manager.cpp
+++ b/src/hook/manager.cpp
@@ -96,20 +96,27 @@ Labels HookManager::masterLaunchTaskLabelDecorator(
     const SlaveInfo& slaveInfo)
 {
   Lock lock(&mutex);
-  Labels labels;
+
+  // We need a mutable copy of the task info and set the new
+  // labels after each hook invocation. Otherwise, the last hook
+  // will be the only effective hook setting the labels.
+  TaskInfo taskInfo_ = taskInfo;
 
   foreachpair (const string& name, Hook* hook, availableHooks) {
     const Result<Labels>& result =
-      hook->masterLaunchTaskLabelDecorator(taskInfo, frameworkInfo, slaveInfo);
+      hook->masterLaunchTaskLabelDecorator(taskInfo_, frameworkInfo, slaveInfo);
+
+    // NOTE: If the hook returns None(), the task labels won't be
+    // changed.
     if (result.isSome()) {
-      labels.MergeFrom(result.get());
+      taskInfo_.mutable_labels()->CopyFrom(result.get());
     } else if (result.isError()) {
       LOG(WARNING) << "Master label decorator hook failed for module '"
                    << name << "': " << result.error();
     }
   }
 
-  return labels;
+  return taskInfo_.labels();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a090e9dc/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index e30b951..b1093bb 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -2619,8 +2619,8 @@ void Master::_accept(
             message.set_pid(framework->pid);
             message.mutable_task()->MergeFrom(task);
 
-            // Merge labels retrieved from label-decorator hooks.
-            message.mutable_task()->mutable_labels()->MergeFrom(
+            // Set labels retrieved from label-decorator hooks.
+            message.mutable_task()->mutable_labels()->CopyFrom(
                 HookManager::masterLaunchTaskLabelDecorator(
                     task,
                     framework->info,

http://git-wip-us.apache.org/repos/asf/mesos/blob/a090e9dc/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index bb9de25..fdfcd38 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -74,6 +74,8 @@ const char* HOOK_MODULE_NAME = "org_apache_mesos_TestHook";
 // examples/test_hook_module.cpp.
 const char* testLabelKey = "MESOS_Test_Label";
 const char* testLabelValue = "ApacheMesos";
+const char* testRemoveLabelKey = "MESOS_Test_Remove_Label";
+const char* testRemoveLabelValue = "FooBar";
 const char* testEnvironmentVariableName = "MESOS_TEST_ENVIRONMENT_VARIABLE";
 
 class HookTest : public MesosTest
@@ -162,6 +164,12 @@ TEST_F(HookTest, VerifyMasterLaunchTaskHook)
   task.mutable_resources()->CopyFrom(offers.get()[0].resources());
   task.mutable_command()->CopyFrom(command);
 
+  // Add label which will be removed by the hook.
+  Labels* labels = task.mutable_labels();
+  Label* label = labels->add_labels();
+  label->set_key(testRemoveLabelKey);
+  label->set_value(testRemoveLabelValue);
+
   vector<TaskInfo> tasks;
   tasks.push_back(task);
 
@@ -175,14 +183,15 @@ TEST_F(HookTest, VerifyMasterLaunchTaskHook)
   AWAIT_READY(taskInfo);
 
   // At launchTasks, the label decorator hook inside should have been
-  // executed and we should see the labels now.
-  Option<string> labelValue;
-  foreach (const Label& label, taskInfo.get().labels().labels()) {
-    if (label.key() == testLabelKey) {
-      labelValue = label.value();
-    }
-  }
-  EXPECT_SOME_EQ(testLabelValue, labelValue);
+  // executed and we should see the labels now. Also, verify that the
+  // hook module has stripped the first 'testRemoveLabelKey' label.
+  // We do this by ensuring that only one label is present and that it
+  // is the new 'testLabelKey' label.
+  const Labels &labels_ = taskInfo.get().labels();
+  ASSERT_EQ(1, labels_.labels_size());
+
+  EXPECT_EQ(labels_.labels().Get(0).key(), testLabelKey);
+  EXPECT_EQ(labels_.labels().Get(0).value(), testLabelValue);
 
   driver.stop();
   driver.join();


[5/6] mesos git commit: Refactored VerifyMasterLaunchTaskHook to _not_ use command executor.

Posted by me...@apache.org.
Refactored VerifyMasterLaunchTaskHook to _not_ use command executor.

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


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

Branch: refs/heads/master
Commit: 66cebd3fd7666747eb5d7c7ffd31208c94d5b7da
Parents: e2d6c86
Author: Niklas Nielsen <ni...@qni.dk>
Authored: Mon Apr 20 14:37:37 2015 -0700
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Apr 20 14:37:37 2015 -0700

----------------------------------------------------------------------
 src/tests/hook_tests.cpp | 42 ++++++++++++++++++++++--------------------
 1 file changed, 22 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/66cebd3f/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index 9681905..a65c0ab 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -129,20 +129,19 @@ TEST_F(HookTest, VerifyMasterLaunchTaskHook)
   Try<PID<Master>> master = StartMaster(CreateMasterFlags());
   ASSERT_SOME(master);
 
-  TestContainerizer containerizer;
+  MockExecutor exec(DEFAULT_EXECUTOR_ID);
 
-  StandaloneMasterDetector detector(master.get());
+  TestContainerizer containerizer(&exec);
 
   // Start a mock slave since we aren't testing the slave hooks yet.
-  MockSlave slave(CreateSlaveFlags(), &detector, &containerizer);
-  process::spawn(slave);
+  Try<PID<Slave>> slave = StartSlave(&containerizer);
+  ASSERT_SOME(slave);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(sched, registered(&driver, _, _))
-    .Times(1);
+  EXPECT_CALL(sched, registered(&driver, _, _));
 
   Future<vector<Offer>> offers;
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -154,16 +153,12 @@ TEST_F(HookTest, VerifyMasterLaunchTaskHook)
   AWAIT_READY(offers);
   EXPECT_NE(0u, offers.get().size());
 
-  CommandInfo command;
-  command.set_value("sleep 10");
-
-  // Launch a task with the command executor.
   TaskInfo task;
   task.set_name("");
   task.mutable_task_id()->set_value("1");
   task.mutable_slave_id()->CopyFrom(offers.get()[0].slave_id());
   task.mutable_resources()->CopyFrom(offers.get()[0].resources());
-  task.mutable_command()->CopyFrom(command);
+  task.mutable_executor()->CopyFrom(DEFAULT_EXECUTOR_INFO);
 
   // Add label which will be removed by the hook.
   Labels* labels = task.mutable_labels();
@@ -174,21 +169,31 @@ TEST_F(HookTest, VerifyMasterLaunchTaskHook)
   vector<TaskInfo> tasks;
   tasks.push_back(task);
 
-  Future<TaskInfo> taskInfo;
-  EXPECT_CALL(slave, runTask(_, _, _, _, _))
-    .Times(1)
-    .WillOnce(FutureArg<4>(&taskInfo));
+  Future<RunTaskMessage> runTaskMessage =
+    FUTURE_PROTOBUF(RunTaskMessage(), _, _);
+
+  EXPECT_CALL(exec, registered(_, _, _, _));
+
+  EXPECT_CALL(exec, launchTask(_, _))
+    .WillOnce(SendStatusUpdateFromTask(TASK_RUNNING));
+
+  Future<TaskStatus> status;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&status))
+    .WillRepeatedly(Return());
 
   driver.launchTasks(offers.get()[0].id(), tasks);
 
-  AWAIT_READY(taskInfo);
+  AWAIT_READY(runTaskMessage);
+
+  AWAIT_READY(status);
 
   // At launchTasks, the label decorator hook inside should have been
   // executed and we should see the labels now. Also, verify that the
   // hook module has stripped the first 'testRemoveLabelKey' label.
   // We do this by ensuring that only one label is present and that it
   // is the new 'testLabelKey' label.
-  const Labels &labels_ = taskInfo.get().labels();
+  const Labels &labels_ = runTaskMessage.get().task().labels();
   ASSERT_EQ(1, labels_.labels_size());
 
   EXPECT_EQ(labels_.labels().Get(0).key(), testLabelKey);
@@ -197,9 +202,6 @@ TEST_F(HookTest, VerifyMasterLaunchTaskHook)
   driver.stop();
   driver.join();
 
-  process::terminate(slave);
-  process::wait(slave);
-
   Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
 


[3/6] mesos git commit: Added slave run task decorator.

Posted by me...@apache.org.
Added slave run task decorator.

Added decorator which gets invoked on start of runTask() sequence in the
slave.

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


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

Branch: refs/heads/master
Commit: 8dd1bb1213ac3f69fdcd5cbc35599fea49272b14
Parents: a454fc5
Author: Niklas Nielsen <ni...@qni.dk>
Authored: Mon Apr 20 14:36:40 2015 -0700
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Apr 20 14:36:40 2015 -0700

----------------------------------------------------------------------
 include/mesos/hook.hpp | 12 ++++++++++++
 src/hook/manager.cpp   | 27 +++++++++++++++++++++++++++
 src/hook/manager.hpp   |  5 +++++
 src/slave/slave.cpp    |  6 +++++-
 src/slave/slave.hpp    |  2 +-
 src/tests/mesos.cpp    |  2 +-
 src/tests/mesos.hpp    |  4 ++--
 7 files changed, 53 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8dd1bb12/include/mesos/hook.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/hook.hpp b/include/mesos/hook.hpp
index f2b8259..a9dbc10 100644
--- a/include/mesos/hook.hpp
+++ b/include/mesos/hook.hpp
@@ -45,6 +45,18 @@ public:
     return None();
   }
 
+  // This label decorator hook is called from within the slave when
+  // receiving a run task request from the master. A module
+  // implementing the hook creates and returns a set of labels. These
+  // labels overwrite the existing labels on the task info.
+  virtual Result<Labels> slaveRunTaskLabelDecorator(
+      const TaskInfo& taskInfo,
+      const FrameworkInfo& frameworkInfo,
+      const SlaveInfo& slaveInfo)
+  {
+    return None();
+  }
+
   // This environment decorator hook is called from within slave when
   // launching a new executor. A module implementing the hook creates
   // and returns a set of environment variables. These environment

http://git-wip-us.apache.org/repos/asf/mesos/blob/8dd1bb12/src/hook/manager.cpp
----------------------------------------------------------------------
diff --git a/src/hook/manager.cpp b/src/hook/manager.cpp
index 28d5c58..54b0d34 100644
--- a/src/hook/manager.cpp
+++ b/src/hook/manager.cpp
@@ -120,6 +120,33 @@ Labels HookManager::masterLaunchTaskLabelDecorator(
 }
 
 
+Labels HookManager::slaveRunTaskLabelDecorator(
+    const TaskInfo& taskInfo,
+    const FrameworkInfo& frameworkInfo,
+    const SlaveInfo& slaveInfo)
+{
+  Lock lock(&mutex);
+
+  TaskInfo taskInfo_ = taskInfo;
+
+  foreachpair (const string& name, Hook* hook, availableHooks) {
+    const Result<Labels>& result =
+      hook->slaveRunTaskLabelDecorator(taskInfo_, frameworkInfo, slaveInfo);
+
+    // NOTE: If the hook returns None(), the task labels won't be
+    // changed.
+    if (result.isSome()) {
+      taskInfo_.mutable_labels()->CopyFrom(result.get());
+    } else if (result.isError()) {
+      LOG(WARNING) << "Slave label decorator hook failed for module '"
+                   << name << "': " << result.error();
+    }
+  }
+
+  return taskInfo_.labels();
+}
+
+
 Environment HookManager::slaveExecutorEnvironmentDecorator(
     ExecutorInfo executorInfo)
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/8dd1bb12/src/hook/manager.hpp
----------------------------------------------------------------------
diff --git a/src/hook/manager.hpp b/src/hook/manager.hpp
index da81349..638e19f 100644
--- a/src/hook/manager.hpp
+++ b/src/hook/manager.hpp
@@ -44,6 +44,11 @@ public:
       const FrameworkInfo& frameworkInfo,
       const SlaveInfo& slaveInfo);
 
+  static Labels slaveRunTaskLabelDecorator(
+      const TaskInfo& taskInfo,
+      const FrameworkInfo& frameworkInfo,
+      const SlaveInfo& slaveInfo);
+
   static Environment slaveExecutorEnvironmentDecorator(
       ExecutorInfo executorInfo);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/8dd1bb12/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 8ec80ed..60345ec 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -1107,7 +1107,7 @@ void Slave::runTask(
     const FrameworkInfo& frameworkInfo_,
     const FrameworkID& frameworkId_,
     const string& pid,
-    const TaskInfo& task)
+    TaskInfo task)
 {
   if (master != from) {
     LOG(WARNING) << "Ignoring run task message from " << from
@@ -1149,6 +1149,10 @@ void Slave::runTask(
     return;
   }
 
+  // Set task labels from run task label decorator.
+  task.mutable_labels()->CopyFrom(
+      HookManager::slaveRunTaskLabelDecorator(task, frameworkInfo, info));
+
   Future<bool> unschedule = true;
 
   // If we are about to create a new framework, unschedule the work

http://git-wip-us.apache.org/repos/asf/mesos/blob/8dd1bb12/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 9495c70..d214ddb 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -108,7 +108,7 @@ public:
       const FrameworkInfo& frameworkInfo,
       const FrameworkID& frameworkId,
       const std::string& pid,
-      const TaskInfo& task);
+      TaskInfo task);
 
   // Made 'virtual' for Slave mocking.
   virtual void _runTask(

http://git-wip-us.apache.org/repos/asf/mesos/blob/8dd1bb12/src/tests/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index fc534e9..42a4015 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -394,7 +394,7 @@ void MockSlave::unmocked_runTask(
     const FrameworkInfo& frameworkInfo,
     const FrameworkID& frameworkId,
     const std::string& pid,
-    const TaskInfo& task)
+    TaskInfo task)
 {
   slave::Slave::runTask(from, frameworkInfo, frameworkId, pid, task);
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/8dd1bb12/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 7744df5..4edb33b 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -679,14 +679,14 @@ public:
       const FrameworkInfo& frameworkInfo,
       const FrameworkID& frameworkId,
       const std::string& pid,
-      const TaskInfo& task));
+      TaskInfo task));
 
   void unmocked_runTask(
       const process::UPID& from,
       const FrameworkInfo& frameworkInfo,
       const FrameworkID& frameworkId,
       const std::string& pid,
-      const TaskInfo& task);
+      TaskInfo task);
 
   MOCK_METHOD4(_runTask, void(
       const process::Future<bool>& future,


[2/6] mesos git commit: Enabled environment decorator to override.

Posted by me...@apache.org.
Enabled environment decorator to override.

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


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

Branch: refs/heads/master
Commit: a454fc50a8304453d436021ff36742a6af6d8798
Parents: a090e9d
Author: Niklas Nielsen <ni...@qni.dk>
Authored: Mon Apr 20 14:36:28 2015 -0700
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Apr 20 14:36:28 2015 -0700

----------------------------------------------------------------------
 src/examples/test_hook_module.cpp | 5 +++++
 src/hook/manager.cpp              | 8 ++++----
 2 files changed, 9 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a454fc50/src/examples/test_hook_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_hook_module.cpp b/src/examples/test_hook_module.cpp
index 4fc7013..64fb503 100644
--- a/src/examples/test_hook_module.cpp
+++ b/src/examples/test_hook_module.cpp
@@ -71,6 +71,11 @@ public:
     LOG(INFO) << "Executing 'slaveExecutorEnvironmentDecorator' hook";
 
     Environment environment;
+
+    if (executorInfo.command().has_environment()) {
+      environment.CopyFrom(executorInfo.command().environment());
+    }
+
     Environment::Variable* variable = environment.add_variables();
     variable->set_name("FOO");
     variable->set_value("bar");

http://git-wip-us.apache.org/repos/asf/mesos/blob/a454fc50/src/hook/manager.cpp
----------------------------------------------------------------------
diff --git a/src/hook/manager.cpp b/src/hook/manager.cpp
index 0d79ba2..28d5c58 100644
--- a/src/hook/manager.cpp
+++ b/src/hook/manager.cpp
@@ -128,11 +128,11 @@ Environment HookManager::slaveExecutorEnvironmentDecorator(
   foreachpair (const string& name, Hook* hook, availableHooks) {
     const Result<Environment>& result =
       hook->slaveExecutorEnvironmentDecorator(executorInfo);
+
+    // NOTE: If the hook returns None(), the environment won't be
+    // changed.
     if (result.isSome()) {
-      // Update executorInfo to include newer environment variables
-      // so that the next hook module can extend the environment
-      // variables instead of simply overwriting them.
-      executorInfo.mutable_command()->mutable_environment()->MergeFrom(
+      executorInfo.mutable_command()->mutable_environment()->CopyFrom(
           result.get());
     } else if (result.isError()) {
       LOG(WARNING) << "Slave environment decorator hook failed for module '"


[6/6] mesos git commit: Fixed comment for remove executor hook.

Posted by me...@apache.org.
Fixed comment for remove executor hook.

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


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

Branch: refs/heads/master
Commit: 01ee1e504dec5cb65d2ae208fb01f28b2c93719f
Parents: 66cebd3
Author: Niklas Nielsen <ni...@qni.dk>
Authored: Mon Apr 20 14:38:33 2015 -0700
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Apr 20 14:38:33 2015 -0700

----------------------------------------------------------------------
 include/mesos/hook.hpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/01ee1e50/include/mesos/hook.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/hook.hpp b/include/mesos/hook.hpp
index a9dbc10..0995c24 100644
--- a/include/mesos/hook.hpp
+++ b/include/mesos/hook.hpp
@@ -71,8 +71,8 @@ public:
   }
 
   // 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.
+  // removed. A typical module implementing the hook will perform some
+  // cleanup as required.
   virtual Try<Nothing> slaveRemoveExecutorHook(
       const FrameworkInfo& frameworkInfo,
       const ExecutorInfo& executorInfo)


[4/6] mesos git commit: Added slave run task hook tests.

Posted by me...@apache.org.
Added slave run task hook tests.

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


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

Branch: refs/heads/master
Commit: e2d6c86b8af2ff28a8591d9c8a2dbb2568c779d3
Parents: 8dd1bb1
Author: Niklas Nielsen <ni...@qni.dk>
Authored: Mon Apr 20 14:37:19 2015 -0700
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Apr 20 14:37:19 2015 -0700

----------------------------------------------------------------------
 src/examples/test_hook_module.cpp | 25 +++++++++
 src/tests/hook_tests.cpp          | 96 ++++++++++++++++++++++++++++++++++
 2 files changed, 121 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e2d6c86b/src/examples/test_hook_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_hook_module.cpp b/src/examples/test_hook_module.cpp
index 64fb503..b25830a 100644
--- a/src/examples/test_hook_module.cpp
+++ b/src/examples/test_hook_module.cpp
@@ -62,6 +62,31 @@ public:
     return labels;
   }
 
+  // TODO(nnielsen): Split hook tests into multiple modules to avoid
+  // interference.
+  virtual Result<Labels> slaveRunTaskLabelDecorator(
+      const TaskInfo& taskInfo,
+      const FrameworkInfo& frameworkInfo,
+      const SlaveInfo& slaveInfo)
+  {
+    LOG(INFO) << "Executing 'slaveRunTaskLabelDecorator' hook";
+
+    Labels labels;
+
+    // Set one known label.
+    Label* newLabel = labels.add_labels();
+    newLabel->set_key("baz");
+    newLabel->set_value("qux");
+
+    // Remove label which was set by test.
+    foreach (const Label& oldLabel, taskInfo.labels().labels()) {
+      if (oldLabel.key() != "foo") {
+        labels.add_labels()->CopyFrom(oldLabel);
+      }
+    }
+
+    return labels;
+  }
 
   // In this hook, we create a new environment variable "FOO" and set
   // it's value to "bar".

http://git-wip-us.apache.org/repos/asf/mesos/blob/e2d6c86b/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index fdfcd38..9681905 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -61,6 +61,7 @@ using std::string;
 using std::vector;
 
 using testing::_;
+using testing::DoAll;
 using testing::Return;
 
 namespace mesos {
@@ -333,6 +334,101 @@ TEST_F(HookTest, DISABLED_VerifySlaveLaunchExecutorHook)
   EXPECT_FALSE(os::stat::isfile(path.get()));
 }
 
+
+// This test verifies that the slave run task label decorator can add
+// and remove labels from a task during the launch sequence. A task
+// with two labels ("foo":"bar" and "bar":"baz") is launched and will
+// get modified by the slave hook to strip the "foo":"bar" pair and
+// add a new "baz":"qux" pair.
+TEST_F(HookTest, VerifySlaveRunTaskHook)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  MockExecutor exec(DEFAULT_EXECUTOR_ID);
+
+  TestContainerizer containerizer(&exec);
+
+  Try<PID<Slave>> slave = StartSlave(&containerizer);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get(), 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_EQ(1u, offers.get().size());
+
+  TaskInfo task;
+  task.set_name("");
+  task.mutable_task_id()->set_value("1");
+  task.mutable_slave_id()->CopyFrom(offers.get()[0].slave_id());
+  task.mutable_resources()->CopyFrom(offers.get()[0].resources());
+  task.mutable_executor()->CopyFrom(DEFAULT_EXECUTOR_INFO);
+
+  // Add two labels: (1) will be removed by the hook to ensure that
+  // runTaskHook can remove labels (2) will be preserved to ensure
+  // that the framework can add labels to the task and have those be
+  // available by the end of the launch task sequence when hooks are
+  // used (to protect against hooks removing labels completely).
+  Labels* labels = task.mutable_labels();
+  Label* label1 = labels->add_labels();
+  label1->set_key("foo");
+  label1->set_value("bar");
+
+  Label* label2 = labels->add_labels();
+  label2->set_key("bar");
+  label2->set_value("baz");
+
+  vector<TaskInfo> tasks;
+  tasks.push_back(task);
+
+  EXPECT_CALL(exec, registered(_, _, _, _));
+
+  Future<TaskInfo> taskInfo;
+  EXPECT_CALL(exec, launchTask(_, _))
+    .WillOnce(DoAll(
+        FutureArg<1>(&taskInfo),
+        SendStatusUpdateFromTask(TASK_RUNNING)));
+
+  driver.launchTasks(offers.get()[0].id(), tasks);
+
+  AWAIT_READY(taskInfo);
+
+  // The master hook will hang an extra label off.
+  const Labels& labels_ = taskInfo.get().labels();
+
+  ASSERT_EQ(3, labels_.labels_size());
+
+  // The slave run task hook will prepend a new "baz":"qux" label.
+  EXPECT_EQ(labels_.labels(0).key(), "baz");
+  EXPECT_EQ(labels_.labels(0).value(), "qux");
+
+  // Master launch task hook will still hang off test label.
+  EXPECT_EQ(labels_.labels(1).key(), testLabelKey);
+  EXPECT_EQ(labels_.labels(1).value(), testLabelValue);
+
+  // And lastly, we only expect the "foo":"bar" pair to be stripped by
+  // the module. The last pair should be the original "bar":"baz"
+  // pair set by the test.
+  EXPECT_EQ(labels_.labels(2).key(), "bar");
+  EXPECT_EQ(labels_.labels(2).value(), "baz");
+
+  driver.stop();
+  driver.join();
+
+  Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {