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 2014/08/13 08:28:56 UTC

[1/6] git commit: Updated command executor to use the new CommandInfo.

Repository: mesos
Updated Branches:
  refs/heads/master 13d846034 -> 4aa3ec22c


Updated command executor to use the new CommandInfo.

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


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

Branch: refs/heads/master
Commit: 4aa3ec22cfde53bb6aaf078c7c24ea351e3b41ed
Parents: 955c64a
Author: Jie Yu <yu...@gmail.com>
Authored: Tue Aug 12 22:13:03 2014 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Aug 12 23:28:29 2014 -0700

----------------------------------------------------------------------
 src/launcher/executor.cpp | 55 ++++++++++++++++++++++++++++++++++++++----
 src/master/master.cpp     |  2 ++
 src/slave/slave.cpp       | 30 +++++++++++++++++++----
 3 files changed, 77 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/4aa3ec22/src/launcher/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.cpp b/src/launcher/executor.cpp
index b096b68..64a4175 100644
--- a/src/launcher/executor.cpp
+++ b/src/launcher/executor.cpp
@@ -24,6 +24,7 @@
 #include <iostream>
 #include <list>
 #include <string>
+#include <vector>
 
 #include <mesos/executor.hpp>
 
@@ -117,9 +118,25 @@ public:
       return;
     }
 
+    // Sanity checks.
     CHECK(task.has_command()) << "Expecting task " << task.task_id()
                               << " to have a command!";
 
+    // TODO(jieyu): For now, we just fail the executor if the task's
+    // CommandInfo is not valid. The framework will receive
+    // TASK_FAILED for the task, and will most likely find out the
+    // cause with some debugging. This is a temporary solution. A more
+    // correct solution is to perform this validation at master side.
+    if (task.command().shell()) {
+      CHECK(task.command().has_value())
+        << "Shell command of task " << task.task_id()
+        << " is not specified!";
+    } else {
+      CHECK(task.command().has_value())
+        << "Executable of task " << task.task_id()
+        << " is not specified!";
+    }
+
     cout << "Starting task " << task.task_id() << endl;
 
     // TODO(benh): Clean this up with the new 'Fork' abstraction.
@@ -145,12 +162,30 @@ public:
       abort();
     }
 
+    // Prepare the argv before fork as it's not async signal safe.
+    char **argv = new char*[task.command().argv_size() + 1];
+    for (int i = 0; i < task.command().argv_size(); i++) {
+      argv[i] = (char*) task.command().argv(i).c_str();
+    }
+    argv[task.command().argv_size()] = NULL;
+
+    // Prepare the messages before fork as it's not async signal safe.
+    string command;
+    if (task.command().shell()) {
+      command = "sh -c '" + task.command().value() + "'";
+    } else {
+      command =
+        "[" + task.command().value() + ", " +
+        strings::join(", ", task.command().argv()) + "]";
+    }
+
     if ((pid = fork()) == -1) {
-      cerr << "Failed to fork to run '" << task.command().value() << "': "
+      cerr << "Failed to fork to run " << command << ": "
            << strerror(errno) << endl;
       abort();
     }
 
+    // TODO(jieyu): Make the child process async signal safe.
     if (pid == 0) {
       // In child process, we make cleanup easier by putting process
       // into it's own session.
@@ -183,11 +218,19 @@ public:
       os::close(pipes[1]);
 
       // The child has successfully setsid, now run the command.
-
       if (override.isNone()) {
-        cout << "sh -c '" << task.command().value() << "'" << endl;
-        execl("/bin/sh", "sh", "-c",
-              task.command().value().c_str(), (char*) NULL);
+        cout << command << endl;
+
+        if (task.command().shell()) {
+          execl(
+              "/bin/sh",
+              "sh",
+              "-c",
+              task.command().value().c_str(),
+              (char*) NULL);
+        } else {
+          execvp(task.command().value().c_str(), argv);
+        }
       } else {
         char** argv = override.get();
 
@@ -205,6 +248,8 @@ public:
       abort();
     }
 
+    delete[] argv;
+
     // In parent process.
     os::close(pipes[1]);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/4aa3ec22/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index d53d6c2..a8cf9ba 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -2249,6 +2249,8 @@ Future<Option<Error> > Master::validateTask(
 
   // TODO(benh): Add a HealthCheckChecker visitor.
 
+  // TODO(jieyu): Add a CommandInfoCheck visitor.
+
   // Invoke each visitor.
   Option<Error> error = None();
   foreach (TaskInfoVisitor* visitor, taskVisitors) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/4aa3ec22/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 787bd05..59477d5 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -2391,13 +2391,33 @@ ExecutorInfo Slave::getExecutorInfo(
 
     // Prepare an executor name which includes information on the
     // command being launched.
-    string name =
-      "(Task: " + task.task_id().value() + ") " + "(Command: sh -c '";
+    string name = "(Task: " + task.task_id().value() + ") ";
 
-    if (task.command().value().length() > 15) {
-      name += task.command().value().substr(0, 12) + "...')";
+    if (task.command().shell()) {
+      if (!task.command().has_value()) {
+        name += "(Command: NO COMMAND)";
+      } else {
+        name += "(Command: sh -c '";
+        if (task.command().value().length() > 15) {
+          name += task.command().value().substr(0, 12) + "...')";
+        } else {
+          name += task.command().value() + "')";
+        }
+      }
     } else {
-      name += task.command().value() + "')";
+      if (!task.command().has_value()) {
+        name += "(Command: NO EXECUTABLE)";
+      } else {
+        string args =
+          task.command().value() + ", " +
+          strings::join(", ", task.command().argv());
+
+        if (args.length() > 15) {
+          name += "(Command: [" + args.substr(0, 12) + "...])";
+        } else {
+          name += "(Command: [" + args + "])";
+        }
+      }
     }
 
     executor.set_name("Command Executor " + name);


[5/6] git commit: Updated slave http to use the new CommmandInfo.

Posted by ji...@apache.org.
Updated slave http to use the new CommmandInfo.

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


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

Branch: refs/heads/master
Commit: 955c64a1b91a01a0e0984d2ab48b0a61767f7a82
Parents: 1c48805
Author: Jie Yu <yu...@gmail.com>
Authored: Tue Aug 12 22:11:48 2014 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Aug 12 23:28:29 2014 -0700

----------------------------------------------------------------------
 src/slave/http.cpp | 17 +++++++++++++++--
 1 file changed, 15 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/955c64a1/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 98bdab6..92eb348 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -83,13 +83,26 @@ using process::http::Request;
 JSON::Object model(const CommandInfo& command)
 {
   JSON::Object object;
-  object.values["value"] = command.value();
+
+  if (command.has_shell()) {
+    object.values["shell"] = command.shell();
+  }
+
+  if (command.has_value()) {
+    object.values["value"] = command.value();
+  }
+
+  JSON::Array argv;
+  foreach (const string& arg, command.argv()) {
+    argv.values.push_back(arg);
+  }
+  object.values["argv"] = argv;
 
   if (command.has_environment()) {
     JSON::Object environment;
     JSON::Array variables;
     foreach(const Environment_Variable& variable,
-        command.environment().variables()) {
+            command.environment().variables()) {
       JSON::Object variableObject;
       variableObject.values["name"] = variable.name();
       variableObject.values["value"] = variable.value();


[3/6] git commit: Made value in CommandInfo optional and added argv.

Posted by ji...@apache.org.
Made value in CommandInfo optional and added argv.

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


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

Branch: refs/heads/master
Commit: 038d83371c99a1741edca93f005573ade6041845
Parents: 13d8460
Author: Jie Yu <yu...@gmail.com>
Authored: Tue Aug 12 16:18:46 2014 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Aug 12 23:28:29 2014 -0700

----------------------------------------------------------------------
 include/mesos/mesos.proto | 16 ++++++++++++++--
 1 file changed, 14 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/038d8337/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 226ddb4..dc781d1 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -231,8 +231,20 @@ message CommandInfo {
 
   optional Environment environment = 2;
 
-  // Actual command (i.e., 'echo hello world').
-  required string value = 3;
+  // Whether the command will be executed via shell (i.e., /bin/sh -c)
+  // or not. If it is set to true, the 'value' below will be used as
+  // the shell command. Otherwise, the command will be executed via
+  // execve using the specified 'argv' below.
+  optional bool shell = 6 [default = true];
+
+  // The shell command (i.e., 'echo hello world').
+  // NOTE: This field is changed from 'required' to 'optional' in
+  // 0.20.0. It will only cause issues if a new framework is
+  // connecting to an old master.
+  optional string value = 3;
+
+  // The argument list.
+  repeated string argv = 7;
 
   // Enables executor and tasks to run as a specific user. If the user
   // field is present both in FrameworkInfo and here, the CommandInfo


[2/6] git commit: Updated mesos containerizer launcher to use the new CommandInfo.

Posted by ji...@apache.org.
Updated mesos containerizer launcher to use the new CommandInfo.

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


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

Branch: refs/heads/master
Commit: 1c48805f6fc32787b8e549ee88ebbf3e72afecdb
Parents: afc65bf
Author: Jie Yu <yu...@gmail.com>
Authored: Tue Aug 12 20:45:11 2014 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Aug 12 23:28:29 2014 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/launch.cpp | 55 ++++++++++++++++++++++-----
 1 file changed, 46 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/1c48805f/src/slave/containerizer/mesos/launch.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index f8324bb..4e727a1 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+#include <string.h>
 #include <unistd.h>
 
 #include <iostream>
@@ -105,6 +106,19 @@ int MesosContainerizerLaunch::execute()
     return 1;
   }
 
+  // Validate the command.
+  if (command.get().shell()) {
+    if (!command.get().has_value()) {
+      cerr << "Shell command is not specified" << endl;
+      return 1;
+    }
+  } else {
+    if (!command.get().has_value()) {
+      cerr << "Executable path is not specified" << endl;
+      return 1;
+    }
+  }
+
   Try<Nothing> close = os::close(flags.pipe_write.get());
   if (close.isError()) {
     cerr << "Failed to close pipe[1]: " << close.error() << endl;
@@ -162,10 +176,22 @@ int MesosContainerizerLaunch::execute()
         return 1;
       }
 
+      // TODO(jieyu): Currently, we only accept shell commands for the
+      // preparation commands.
+      if (!parse.get().shell()) {
+        cerr << "Preparation commands need to be shell commands" << endl;
+        return 1;
+      }
+
+      if (!parse.get().has_value()) {
+        cerr << "The 'value' of a preparation command is not specified" << endl;
+        return 1;
+      }
+
       // Block until the command completes.
       int status = os::system(parse.get().value());
       if (!WIFEXITED(status) || (WEXITSTATUS(status) != 0)) {
-        cerr << "Failed to execute a preparation command" << endl;
+        cerr << "Failed to execute a preparation shell command" << endl;
         return 1;
       }
     }
@@ -196,14 +222,25 @@ int MesosContainerizerLaunch::execute()
   map<string, string> env;
   os::ExecEnv envp(env);
 
-  // Execute the command (via '/bin/sh -c command') with its environment.
-  execle(
-      "/bin/sh",
-      "sh",
-      "-c",
-      command.get().value().c_str(),
-      (char*) NULL,
-      envp());
+  if (command.get().shell()) {
+    // Execute the command using shell.
+    execle(
+        "/bin/sh",
+        "sh",
+        "-c",
+        command.get().value().c_str(),
+        (char*) NULL,
+        envp());
+  } else {
+    // Use execve to launch the command.
+    char** argv = new char*[command.get().argv_size() + 1];
+    for (int i = 0; i < command.get().argv_size(); i++) {
+      argv[i] = strdup(command.get().argv(i).c_str());
+    }
+    argv[command.get().argv_size()] = NULL;
+
+    execve(command.get().value().c_str(), argv, envp());
+  }
 
   // If we get here, the execle call failed.
   cerr << "Failed to execute command" << endl;


[6/6] git commit: Updated health check to use the new CommandInfo.

Posted by ji...@apache.org.
Updated health check to use the new CommandInfo.

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


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

Branch: refs/heads/master
Commit: afc65bf004d342e9ef122b614a15ff643b417748
Parents: 2d4eb0e
Author: Jie Yu <yu...@gmail.com>
Authored: Tue Aug 12 19:17:29 2014 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Aug 12 23:28:29 2014 -0700

----------------------------------------------------------------------
 src/health-check/main.cpp        | 137 ++++++++++++++++++++++------------
 src/tests/health_check_tests.cpp | 103 ++++++++++++++++++++++---
 2 files changed, 182 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/afc65bf0/src/health-check/main.cpp
----------------------------------------------------------------------
diff --git a/src/health-check/main.cpp b/src/health-check/main.cpp
index 472bffc..6849947 100644
--- a/src/health-check/main.cpp
+++ b/src/health-check/main.cpp
@@ -18,28 +18,31 @@
 
 #include <signal.h>
 #include <stdio.h>
-#include <iostream>
-#include <string>
 #include <string.h>
 #include <unistd.h>
 
+#include <iostream>
+#include <string>
+#include <vector>
+
 #include <mesos/mesos.hpp>
 
-#include <process/pid.hpp>
 #include <process/defer.hpp>
 #include <process/delay.hpp>
 #include <process/future.hpp>
 #include <process/io.hpp>
+#include <process/pid.hpp>
 #include <process/process.hpp>
 #include <process/protobuf.hpp>
 #include <process/subprocess.hpp>
 
 #include <stout/duration.hpp>
-#include <stout/os.hpp>
-#include <stout/option.hpp>
 #include <stout/flags.hpp>
-#include <stout/protobuf.hpp>
 #include <stout/json.hpp>
+#include <stout/option.hpp>
+#include <stout/os.hpp>
+#include <stout/protobuf.hpp>
+#include <stout/strings.hpp>
 
 #include "common/status_utils.hpp"
 
@@ -50,8 +53,9 @@ using namespace mesos;
 using std::cout;
 using std::cerr;
 using std::endl;
-using std::string;
 using std::map;
+using std::string;
+using std::vector;
 
 using process::UPID;
 
@@ -137,61 +141,96 @@ private:
   {
     if (check.has_http()) {
       promise.fail("HTTP health check is not supported");
-    } else if (check.has_command()) {
-      const CommandInfo& command = check.command();
+      return;
+    }
 
-      map<string, string> environment;
+    if (!check.has_command()) {
+      promise.fail("No check found in health check");
+      return;
+    }
+
+    const CommandInfo& command = check.command();
+
+    map<string, string> environment;
+    foreach (const Environment_Variable& variable,
+             command.environment().variables()) {
+      environment[variable.name()] = variable.value();
+    }
 
-      foreach (const Environment_Variable& variable,
-               command.environment().variables()) {
-        environment[variable.name()] = variable.value();
+    // Launch the subprocess.
+    Option<Try<Subprocess> > external;
+
+    if (command.shell()) {
+      // Use the shell variant.
+      if (!command.has_value()) {
+        promise.fail("Shell command is not specified");
+        return;
+      }
+
+      VLOG(2) << "Launching health command '" << command.value() << "'";
+
+      external = process::subprocess(
+          command.value(),
+          Subprocess::PATH("/dev/null"),
+          Subprocess::FD(STDERR_FILENO),
+          Subprocess::FD(STDERR_FILENO),
+          environment);
+    } else {
+      // Use the execve variant.
+      if (!command.has_value()) {
+        promise.fail("Executable path is not specified");
+        return;
+      }
+
+      vector<string> argv;
+      foreach (const string& arg, command.argv()) {
+        argv.push_back(arg);
       }
 
-      VLOG(2) << "Launching health command: " << command.value();
+      VLOG(2) << "Launching health command [" << command.value() << ", "
+              << strings::join(", ", argv) << "]";
 
-      Try<Subprocess> external =
-        process::subprocess(
+      external = process::subprocess(
           command.value(),
-          // Reading from STDIN instead of PIPE because scripts
-          // seeing an open STDIN pipe might behave differently
-          // and we do not expect to pass any value from STDIN
-          // or PIPE.
-          Subprocess::FD(STDIN_FILENO),
+          argv,
+          Subprocess::PATH("/dev/null"),
           Subprocess::FD(STDERR_FILENO),
           Subprocess::FD(STDERR_FILENO),
+          None(),
           environment);
+    }
+
+    CHECK_SOME(external);
+
+    if (external.get().isError()) {
+      promise.fail("Error creating subprocess for healthcheck");
+      return;
+    }
+
+    Future<Option<int> > status = external.get().get().status();
+    status.await(Seconds(check.timeout_seconds()));
 
-      if (external.isError()) {
-        promise.fail("Error creating subprocess for healthcheck");
+    if (!status.isReady()) {
+      string msg = "Command check failed with reason: ";
+      if (status.isFailed()) {
+        msg += "failed with error: " + status.failure();
+      } else if (status.isDiscarded()) {
+        msg += "status future discarded";
       } else {
-        Future<Option<int> > status = external.get().status();
-        status.await(Seconds(check.timeout_seconds()));
-
-        if (!status.isReady()) {
-          string msg = "Shell command check failed with reason: ";
-          if (status.isFailed()) {
-            msg += "failed with error: " + status.failure();
-          } else if (status.isDiscarded()) {
-            msg += "status future discarded";
-          } else {
-            msg += "status still pending after timeout " +
-                   stringify(Seconds(check.timeout_seconds()));
-          }
-
-          promise.fail(msg);
-          return;
-        }
-
-        int statusCode = status.get().get();
-        if (statusCode != 0) {
-          string message = "Health command check " + WSTRINGIFY(statusCode);
-          failure(message);
-        } else {
-          success();
-        }
+        msg += "status still pending after timeout " +
+               stringify(Seconds(check.timeout_seconds()));
       }
+
+      promise.fail(msg);
+      return;
+    }
+
+    int statusCode = status.get().get();
+    if (statusCode != 0) {
+      string message = "Health command check " + WSTRINGIFY(statusCode);
+      failure(message);
     } else {
-      promise.fail("No check found in health check");
+      success();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/afc65bf0/src/tests/health_check_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/health_check_tests.cpp b/src/tests/health_check_tests.cpp
index 731d944..64fbf62 100644
--- a/src/tests/health_check_tests.cpp
+++ b/src/tests/health_check_tests.cpp
@@ -60,12 +60,32 @@ class HealthCheckTest : public MesosTest
 {
 public:
   vector<TaskInfo> populateTasks(
-    const string& cmd,
-    const string& healthCmd,
-    const Offer& offer,
-    const int gracePeriodSeconds = 0,
-    const Option<int>& consecutiveFailures = None(),
-    const Option<map<string, string> >& env = None())
+      const string& cmd,
+      const string& healthCmd,
+      const Offer& offer,
+      int gracePeriodSeconds = 0,
+      const Option<int>& consecutiveFailures = None(),
+      const Option<map<string, string> >& env = None())
+  {
+    CommandInfo healthCommand;
+    healthCommand.set_value(healthCmd);
+
+    return populateTasks(
+        cmd,
+        healthCommand,
+        offer,
+        gracePeriodSeconds,
+        consecutiveFailures,
+        env);
+  }
+
+  vector<TaskInfo> populateTasks(
+      const string& cmd,
+      CommandInfo healthCommand,
+      const Offer& offer,
+      int gracePeriodSeconds = 0,
+      const Option<int>& consecutiveFailures = None(),
+      const Option<map<string, string> >& env = None())
   {
     TaskInfo task;
     task.set_name("");
@@ -75,8 +95,10 @@ public:
 
     CommandInfo command;
     command.set_value(cmd);
+
     Environment::Variable* variable =
       command.mutable_environment()->add_variables();
+
     // We need to set the correct directory to launch health check process
     // instead of the default for tests.
     variable->set_name("MESOS_LAUNCHER_DIR");
@@ -86,9 +108,6 @@ public:
 
     HealthCheck healthCheck;
 
-    CommandInfo healthCommand;
-    healthCommand.set_value(healthCmd);
-
     if (env.isSome()) {
       foreachpair (const string& name, const string value, env.get()) {
         Environment::Variable* variable =
@@ -175,6 +194,71 @@ TEST_F(HealthCheckTest, HealthyTask)
   Shutdown();
 }
 
+
+// Same as above, but use the non-shell version of the health command.
+TEST_F(HealthCheckTest, HealthyTaskNonShell)
+{
+  Try<PID<Master> > master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "posix/cpu,posix/mem";
+
+  Try<MesosContainerizer*> containerizer =
+    MesosContainerizer::create(flags, false);
+  CHECK_SOME(containerizer);
+
+  Try<PID<Slave> > slave = StartSlave(containerizer.get());
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+    &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .Times(1);
+
+  Future<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());
+
+  CommandInfo command;
+  command.set_shell(false);
+  command.set_value("/bin/true");
+  command.add_argv("true");
+
+  vector<TaskInfo> tasks =
+    populateTasks("sleep 120", command, offers.get()[0]);
+
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusHealth;
+
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusHealth));
+
+  driver.launchTasks(offers.get()[0].id(), tasks);
+
+  AWAIT_READY(statusRunning);
+  EXPECT_EQ(TASK_RUNNING, statusRunning.get().state());
+
+  AWAIT_READY(statusHealth);
+  EXPECT_EQ(TASK_RUNNING, statusHealth.get().state());
+  EXPECT_TRUE(statusHealth.get().healthy());
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
+
 // Testing health status change reporting to scheduler.
 TEST_F(HealthCheckTest, HealthStatusChange)
 {
@@ -263,6 +347,7 @@ TEST_F(HealthCheckTest, HealthStatusChange)
   Shutdown();
 }
 
+
 // Testing killing task after number of consecutive failures.
 // Temporarily disabled due to MESOS-1613.
 TEST_F(HealthCheckTest, DISABLED_ConsecutiveFailures)


[4/6] git commit: Refactored the protobuf message comparison logic.

Posted by ji...@apache.org.
Refactored the protobuf message comparison logic.

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


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

Branch: refs/heads/master
Commit: 2d4eb0ef3c162502595f135408b0fe42cf29ebfa
Parents: 038d833
Author: Jie Yu <yu...@gmail.com>
Authored: Tue Aug 12 17:15:39 2014 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Aug 12 23:28:29 2014 -0700

----------------------------------------------------------------------
 src/common/type_utils.cpp | 88 ++++++++++++------------------------------
 1 file changed, 25 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2d4eb0ef/src/common/type_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/type_utils.cpp b/src/common/type_utils.cpp
index c8fc7b3..af0e3a6 100644
--- a/src/common/type_utils.cpp
+++ b/src/common/type_utils.cpp
@@ -16,79 +16,54 @@
  * limitations under the License.
  */
 
+#include <string>
+
 #include <mesos/mesos.hpp>
 #include <mesos/resources.hpp>
 
 #include "common/attributes.hpp"
 #include "common/type_utils.hpp"
 
+using std::string;
+
 namespace mesos {
 
-bool operator == (const Environment& left, const Environment& right)
+static bool equals(
+    const google::protobuf::Message& left,
+    const google::protobuf::Message& right)
 {
-  if (left.variables().size() != right.variables().size()) {
+  string _left;
+  string _right;
+
+  // NOTE: If either of the two messages is not initialized, we will
+  // treat them as not equal.
+  if (!left.SerializeToString(&_left)) {
     return false;
   }
 
-  for (int i = 0; i < left.variables().size(); i++) {
-    const std::string& name = left.variables().Get(i).name();
-    const std::string& value = left.variables().Get(i).value();
-    bool found = false;
-    for (int j = 0; j < right.variables().size(); j++) {
-      if (name == right.variables().Get(j).name() &&
-          value == right.variables().Get(j).value()) {
-        found = true;
-        break;
-      }
-    }
-    if (!found) {
-      return false;
-    }
+  if (!right.SerializeToString(&_right)) {
+    return false;
   }
 
-  return true;
+  return _left == _right;
 }
 
 
-bool operator == (const CommandInfo& left, const CommandInfo& right)
+bool operator == (const Environment& left, const Environment& right)
 {
-  if (left.uris().size() != right.uris().size()) {
-    return false;
-  }
+  return equals(left, right);
+}
 
-  for (int i=0; i<left.uris().size(); i++) {
-    bool found = false;
-    for (int j=0; j<right.uris().size(); j++) {
-      if (left.uris().Get(i) == right.uris().Get(j)) {
-        found = true;
-        break;
-      }
-    }
-    if (!found) {
-      return false;
-    }
-  }
 
-  return left.has_environment() == right.has_environment() &&
-    (!left.has_environment() || (left.environment() == right.environment())) &&
-    left.value() == right.value();
+bool operator == (const CommandInfo& left, const CommandInfo& right)
+{
+  return equals(left, right);
 }
 
 
 bool operator == (const ExecutorInfo& left, const ExecutorInfo& right)
 {
-  return left.executor_id() == right.executor_id() &&
-    left.has_framework_id() == right.has_framework_id() &&
-    (!left.has_framework_id() ||
-    (left.framework_id() == right.framework_id())) &&
-    left.command() == right.command() &&
-    Resources(left.resources()) == Resources(right.resources()) &&
-    left.has_name() == right.has_name() &&
-    (!left.has_name() || (left.name() == right.name())) &&
-    left.has_source() == right.has_source() &&
-    (!left.has_source() || (left.source() == right.source())) &&
-    left.has_data() == right.has_data() &&
-    (!left.has_data() || (left.data() == right.data()));
+  return equals(left, right);
 }
 
 
@@ -109,13 +84,7 @@ bool operator == (const SlaveInfo& left, const SlaveInfo& right)
 
 bool operator == (const MasterInfo& left, const MasterInfo& right)
 {
-  return left.id() == right.id() &&
-    left.ip() == right.ip() &&
-    left.port() == right.port() &&
-    left.has_pid() == right.has_pid() &&
-    (!left.has_pid() || (left.pid() == right.pid())) &&
-    left.has_hostname() == right.has_hostname() &&
-    (!left.has_hostname() || (left.hostname() == right.hostname()));
+  return equals(left, right);
 }
 
 
@@ -123,14 +92,7 @@ namespace internal {
 
 bool operator == (const Task& left, const Task& right)
 {
-  return left.name() == right.name() &&
-    left.task_id() == right.task_id() &&
-    left.framework_id() == right.framework_id() &&
-    left.slave_id() == right.slave_id() &&
-    left.state() == right.state() &&
-    Resources(left.resources()) == Resources(right.resources()) &&
-    left.has_executor_id() == right.has_executor_id() &&
-    (!left.has_executor_id() || (left.executor_id() == right.executor_id()));
+  return equals(left, right);
 }