You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by vi...@apache.org on 2016/04/25 23:35:42 UTC

[01/48] mesos git commit: Fixed LogrotateContainerLogger's FD ownership.

Repository: mesos
Updated Branches:
  refs/heads/0.27.x [created] 3c9ec4a0f


Fixed LogrotateContainerLogger's FD ownership.

Changes the logrotate container logger to manually construct and deal
with pipes.  Specifically, both read and write ends of the pipe must
end up in the child processes (read -> logger executables, write ->
container).

If ownership is not transferred, the pipe's FDs may be closed (again)
when `Subprocess` is destructed, which may unexpectedly close random
FDs belonging to other threads.

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


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

Branch: refs/heads/0.27.x
Commit: bdd8815c250eacaee93c1fd31f8169237a222e25
Parents: 50556e8
Author: Joseph Wu <jo...@mesosphere.io>
Authored: Wed Jan 27 16:25:40 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 20:04:50 2016 -0800

----------------------------------------------------------------------
 src/slave/container_loggers/lib_logrotate.cpp | 61 ++++++++++++++++++++--
 1 file changed, 57 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/bdd8815c/src/slave/container_loggers/lib_logrotate.cpp
----------------------------------------------------------------------
diff --git a/src/slave/container_loggers/lib_logrotate.cpp b/src/slave/container_loggers/lib_logrotate.cpp
index bfc7cad..8d2f895 100644
--- a/src/slave/container_loggers/lib_logrotate.cpp
+++ b/src/slave/container_loggers/lib_logrotate.cpp
@@ -38,6 +38,7 @@
 #include <stout/path.hpp>
 
 #include <stout/os/environment.hpp>
+#include <stout/os/fcntl.hpp>
 #include <stout/os/killtree.hpp>
 
 #include "slave/container_loggers/logrotate.hpp"
@@ -88,6 +89,31 @@ public:
     environment.erase("LIBPROCESS_ADVERTISE_IP");
     environment.erase("LIBPROCESS_ADVERTISE_PORT");
 
+    // NOTE: We manually construct a pipe here instead of using
+    // `Subprocess::PIPE` so that the ownership of the FDs is properly
+    // represented.  The `Subprocess` spawned below owns the read-end
+    // of the pipe and will be solely responsible for closing that end.
+    // The ownership of the write-end will be passed to the caller
+    // of this function.
+    int pipefd[2];
+    if (::pipe(pipefd) == -1) {
+      return Failure(ErrnoError("Failed to create pipe").message);
+    }
+
+    Subprocess::IO::InputFileDescriptors outfds;
+    outfds.read = pipefd[0];
+    outfds.write = pipefd[1];
+
+    // NOTE: We need to `cloexec` this FD so that it will be closed when
+    // the child subprocess is spawned and so that the FD will not be
+    // inherited by the second child for stderr.
+    Try<Nothing> cloexec = os::cloexec(outfds.write.get());
+    if (cloexec.isError()) {
+      os::close(outfds.read);
+      os::close(outfds.write.get());
+      return Failure("Failed to cloexec: " + cloexec.error());
+    }
+
     // Spawn a process to handle stdout.
     mesos::internal::logger::rotate::Flags outFlags;
     outFlags.max_size = flags.max_stdout_size;
@@ -98,16 +124,40 @@ public:
     Try<Subprocess> outProcess = subprocess(
         path::join(flags.launcher_dir, mesos::internal::logger::rotate::NAME),
         {mesos::internal::logger::rotate::NAME},
-        Subprocess::PIPE(),
+        Subprocess::FD(outfds.read, Subprocess::IO::OWNED),
         Subprocess::PATH("/dev/null"),
         Subprocess::FD(STDERR_FILENO),
         outFlags,
         environment);
 
     if (outProcess.isError()) {
+      os::close(outfds.write.get());
       return Failure("Failed to create logger process: " + outProcess.error());
     }
 
+    // NOTE: We manually construct a pipe here to properly express
+    // ownership of the FDs.  See the NOTE above.
+    if (::pipe(pipefd) == -1) {
+      os::close(outfds.write.get());
+      os::killtree(outProcess.get().pid(), SIGKILL);
+      return Failure(ErrnoError("Failed to create pipe").message);
+    }
+
+    Subprocess::IO::InputFileDescriptors errfds;
+    errfds.read = pipefd[0];
+    errfds.write = pipefd[1];
+
+    // NOTE: We need to `cloexec` this FD so that it will be closed when
+    // the child subprocess is spawned.
+    cloexec = os::cloexec(errfds.write.get());
+    if (cloexec.isError()) {
+      os::close(outfds.write.get());
+      os::close(errfds.read);
+      os::close(errfds.write.get());
+      os::killtree(outProcess.get().pid(), SIGKILL);
+      return Failure("Failed to cloexec: " + cloexec.error());
+    }
+
     // Spawn a process to handle stderr.
     mesos::internal::logger::rotate::Flags errFlags;
     errFlags.max_size = flags.max_stderr_size;
@@ -118,20 +168,23 @@ public:
     Try<Subprocess> errProcess = subprocess(
         path::join(flags.launcher_dir, mesos::internal::logger::rotate::NAME),
         {mesos::internal::logger::rotate::NAME},
-        Subprocess::PIPE(),
+        Subprocess::FD(errfds.read, Subprocess::IO::OWNED),
         Subprocess::PATH("/dev/null"),
         Subprocess::FD(STDERR_FILENO),
         errFlags,
         environment);
 
     if (errProcess.isError()) {
+      os::close(outfds.write.get());
+      os::close(errfds.write.get());
       os::killtree(outProcess.get().pid(), SIGKILL);
       return Failure("Failed to create logger process: " + errProcess.error());
     }
 
+    // NOTE: The ownership of these FDs is given to the caller of this function.
     ContainerLogger::SubprocessInfo info;
-    info.out = SubprocessInfo::IO::FD(outProcess->in().get());
-    info.err = SubprocessInfo::IO::FD(errProcess->in().get());
+    info.out = SubprocessInfo::IO::FD(outfds.write.get());
+    info.err = SubprocessInfo::IO::FD(errfds.write.get());
     return info;
   }
 


[36/48] mesos git commit: Moved systemd executor slice initialization logic.

Posted by vi...@apache.org.
Moved systemd executor slice initialization logic.

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


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

Branch: refs/heads/0.27.x
Commit: bf6f1909e22b28fc410fcbd502f03985f183a7b5
Parents: 66d6a4d
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Thu Feb 4 18:13:19 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:10 2016 -0500

----------------------------------------------------------------------
 src/linux/systemd.cpp                           | 79 ++++++++++++++++
 src/linux/systemd.hpp                           | 35 ++++++++
 .../containerizer/mesos/linux_launcher.cpp      | 94 ++++----------------
 .../containerizer/mesos/linux_launcher.hpp      |  3 -
 src/slave/main.cpp                              | 21 +++++
 5 files changed, 153 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/bf6f1909/src/linux/systemd.cpp
----------------------------------------------------------------------
diff --git a/src/linux/systemd.cpp b/src/linux/systemd.cpp
index d3f4a63..69e8b39 100644
--- a/src/linux/systemd.cpp
+++ b/src/linux/systemd.cpp
@@ -25,6 +25,8 @@
 #include <stout/strings.hpp>
 #include <stout/try.hpp>
 
+#include "linux/cgroups.hpp"
+
 using process::Once;
 
 using std::string;
@@ -58,6 +60,16 @@ const Flags& flags()
 }
 
 
+namespace mesos {
+
+Try<Nothing> extendLifetime(pid_t child)
+{
+  // TODO(jmlvanre): Implement pid migration into systemd slice.
+}
+
+} // namespace mesos {
+
+
 Try<Nothing> initialize(const Flags& flags)
 {
   static Once* initialized = new Once();
@@ -66,6 +78,10 @@ Try<Nothing> initialize(const Flags& flags)
     return Nothing();
   }
 
+  if (!systemd::exists()) {
+    return Error("systemd does not exist on this system");
+  }
+
   systemd_flags = new Flags(flags);
 
   // If flags->runtime_directory doesn't exist, then we can't proceed.
@@ -74,6 +90,63 @@ Try<Nothing> initialize(const Flags& flags)
                  CHECK_NOTNULL(systemd_flags)->runtime_directory);
   }
 
+  // On systemd environments we currently migrate executor pids and processes
+  // that need to live alongside the executor into a separate executor slice.
+  // This allows the life-time of the process to be extended past the life-time
+  // of the slave. See MESOS-3352.
+  // This function takes responsibility for creating and starting this slice.
+  // We inject a `Subprocess::Hook` into the `subprocess` function that migrates
+  // pids into this slice if the `EXTEND_LIFETIME` option is set on the
+  // `subprocess` call.
+
+  // Ensure that the `MESOS_EXECUTORS_SLICE` exists and is running.
+  // TODO(jmlvanre): Prevent racing between multiple agents for this creation
+  // logic.
+
+  // Check whether the `MESOS_EXECUTORS_SLICE` already exists. Create it if
+  // it does not exist.
+  // We explicitly don't modify the file if it exists in case operators want
+  // to over-ride the settings for the slice that we provide when we create
+  // the `Unit` below.
+  const Path path(path::join(
+      systemd::runtimeDirectory(),
+      mesos::MESOS_EXECUTORS_SLICE));
+
+  if (!systemd::slices::exists(path)) {
+    // A simple systemd file to allow us to start a new slice.
+    string unit = "[Unit]\nDescription=Mesos Executors Slice\n";
+
+    Try<Nothing> create = systemd::slices::create(path, unit);
+
+    if (create.isError()) {
+      return Error("Failed to create systemd slice '" +
+                   stringify(mesos::MESOS_EXECUTORS_SLICE) +
+                   "': " + create.error());
+    }
+  }
+
+  // Regardless of whether we created the file or it existed already, we
+  // `start` the executor slice. It is safe (a no-op) to `start` an already
+  // running slice.
+  Try<Nothing> start = systemd::slices::start(mesos::MESOS_EXECUTORS_SLICE);
+
+  if (start.isError()) {
+    return Error("Failed to start '" +
+                 stringify(mesos::MESOS_EXECUTORS_SLICE) +
+                 "': " + start.error());
+  }
+
+  // Now the `MESOS_EXECUTORS_SLICE` is ready for us to assign any pids. We can
+  // verify that our cgroups assignments will work by testing the hierarchy.
+  Try<bool> exists = cgroups::exists(
+      systemd::hierarchy(),
+      mesos::MESOS_EXECUTORS_SLICE);
+
+  if (exists.isError() || !exists.get()) {
+    return Error("Failed to locate systemd cgroups hierarchy: " +
+                  (exists.isError() ? exists.error() : "does not exist"));
+  }
+
   initialized->done();
 
   return Nothing();
@@ -146,6 +219,12 @@ bool exists()
 }
 
 
+bool enabled()
+{
+  return exists() && systemd_flags != NULL;
+}
+
+
 Path runtimeDirectory()
 {
   return Path(flags().runtime_directory);

http://git-wip-us.apache.org/repos/asf/mesos/blob/bf6f1909/src/linux/systemd.hpp
----------------------------------------------------------------------
diff --git a/src/linux/systemd.hpp b/src/linux/systemd.hpp
index dc8605b..d868fda 100644
--- a/src/linux/systemd.hpp
+++ b/src/linux/systemd.hpp
@@ -17,6 +17,8 @@
 #ifndef __SYSTEMD_HPP__
 #define __SYSTEMD_HPP__
 
+#include <process/subprocess.hpp>
+
 #include <stout/flags.hpp>
 #include <stout/nothing.hpp>
 #include <stout/path.hpp>
@@ -24,6 +26,31 @@
 
 namespace systemd {
 
+// TODO(jmlvanre): Consider moving the generic systemd behaviour into
+// stout, and leaving the mesos specific behavior here.
+namespace mesos {
+
+/**
+ * The systemd slice which we use to extend the life of any process
+ * which we want to live together with the executor it is associated
+ * with, rather than the agent. This allows us to clean up the agent
+ * cgroup when the agent terminates without killing any critical
+ * components of the executor.
+ */
+// TODO(jmlvanre): We may want to allow this to be configured.
+static const char MESOS_EXECUTORS_SLICE[] = "mesos_executors.slice";
+
+
+/**
+ * A hook that is executed in the parent process. It migrates the pid
+ * of the child process into a the `MESOS_EXECUTORS_SLICE` in order to
+ * extend its life beyond that of the agent.
+ */
+Try<Nothing> extendLifetime(pid_t child);
+
+} // namespace mesos {
+
+
 /**
  * Flags to initialize systemd state.
  */
@@ -36,8 +63,10 @@ public:
   std::string cgroups_hierarchy;
 };
 
+
 const Flags& flags();
 
+
 /**
  * Initialized state for support of systemd functions in this file.
  *
@@ -62,6 +91,12 @@ bool exists();
 
 
 /**
+ * Check if systemd exists, and whether we have initialized it.
+ */
+bool enabled();
+
+
+/**
  * Returns the path to the runtime directory for systemd units.
  */
 Path runtimeDirectory();

http://git-wip-us.apache.org/repos/asf/mesos/blob/bf6f1909/src/slave/containerizer/mesos/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/linux_launcher.cpp b/src/slave/containerizer/mesos/linux_launcher.cpp
index d4541cd..90f8c73 100644
--- a/src/slave/containerizer/mesos/linux_launcher.cpp
+++ b/src/slave/containerizer/mesos/linux_launcher.cpp
@@ -108,65 +108,6 @@ Try<Launcher*> LinuxLauncher::create(const Flags& flags)
   // slice. It then migrates executor pids into this slice before it "unpauses"
   // the executor. This is the same pattern as the freezer.
 
-  // If this is a systemd environment, ensure that the
-  // `SYSTEMD_MESOS_EXECUTORS_SLICE` exists and is running.
-  // TODO(jmlvanre): Prevent racing between multiple agents for this creation
-  // logic.
-  if (systemd::exists()) {
-    systemd::Flags systemdFlags;
-    systemdFlags.runtime_directory = flags.systemd_runtime_directory;
-    systemdFlags.cgroups_hierarchy = flags.cgroups_hierarchy;
-    Try<Nothing> initialize = systemd::initialize(systemdFlags);
-    if (initialize.isError()) {
-      return Error("Failed to initialize systemd: " + initialize.error());
-    }
-
-    // Check whether the `SYSTEMD_MESOS_EXECUTORS_SLICE` already exists. Create
-    // it if it does not exist.
-    // We explicitly don't modify the file if it exists in case operators want
-    // to over-ride the settings for the slice that we provide when we create
-    // the `Unit` below.
-    const Path path(path::join(
-        systemd::runtimeDirectory(),
-        SYSTEMD_MESOS_EXECUTORS_SLICE));
-
-    if (!systemd::slices::exists(path)) {
-      // A simple systemd file to allow us to start a new slice.
-      string unit = "[Unit]\nDescription=Mesos Executors Slice\n";
-
-      Try<Nothing> create = systemd::slices::create(path, unit);
-
-      if (create.isError()) {
-        return Error("Failed to create systemd slice '" +
-                     stringify(SYSTEMD_MESOS_EXECUTORS_SLICE) + "': " +
-                     create.error());
-      }
-    }
-
-    // Regardless of whether we created the file or it existed already, we
-    // `start` the executor slice. It is safe (a no-op) to `start` an already
-    // running slice.
-    Try<Nothing> start = systemd::slices::start(SYSTEMD_MESOS_EXECUTORS_SLICE);
-
-    if (start.isError()) {
-      return Error("Failed to start '" +
-                   stringify(SYSTEMD_MESOS_EXECUTORS_SLICE) +
-                   "': " + start.error());
-    }
-
-    // Now the `SYSTEMD_MESOS_EXECUTORS_SLICE` is ready for us to assign any
-    // executors. We can verify that our cgroups assignments will work by
-    // testing the hierarchy.
-    Try<bool> exists = cgroups::exists(
-        systemd::hierarchy(),
-        SYSTEMD_MESOS_EXECUTORS_SLICE);
-
-    if (exists.isError() || !exists.get()) {
-      return Error("Failed to locate systemd cgroups hierarchy: " +
-                   (exists.isError() ? exists.error() : "does not exist"));
-    }
-  }
-
   return new LinuxLauncher(
       flags,
       freezerHierarchy.get(),
@@ -195,18 +136,19 @@ Future<hashset<ContainerID>> LinuxLauncher::recover(
   hashset<string> recovered;
 
   // On systemd environments, capture the pids under the
-  // `SYSTEMD_MESOS_EXECUTORS_SLICE` for validation during recovery.
+  // `MESOS_EXECUTORS_SLICE` for validation during recovery.
   Result<std::set<pid_t>> mesosExecutorSlicePids = None();
   if (systemdHierarchy.isSome()) {
-    mesosExecutorSlicePids =
-      cgroups::processes(systemdHierarchy.get(), SYSTEMD_MESOS_EXECUTORS_SLICE);
+    mesosExecutorSlicePids = cgroups::processes(
+        systemdHierarchy.get(),
+        systemd::mesos::MESOS_EXECUTORS_SLICE);
 
-    // If we error out trying to read the pids from the
-    // `SYSTEMD_MESOS_EXECUTORS_SLICE` we fail. This is a programmer error as we
-    // did not set up the slice correctly.
+    // If we error out trying to read the pids from the `MESOS_EXECUTORS_SLICE`
+    // we fail. This is a programmer error as we did not set up the slice
+    // correctly.
     if (mesosExecutorSlicePids.isError()) {
       return Failure("Failed to read pids from systemd '" +
-                     stringify(SYSTEMD_MESOS_EXECUTORS_SLICE) + "'");
+                     stringify(systemd::mesos::MESOS_EXECUTORS_SLICE) + "'");
     }
   }
 
@@ -245,18 +187,17 @@ Future<hashset<ContainerID>> LinuxLauncher::recover(
     }
 
     // If we are on a systemd environment, check that the pid is still in the
-    // `SYSTEMD_MESOS_EXECUTORS_SLICE`. If it is not, warn the operator that
-    // resource isolation may be invalidated.
+    // `MESOS_EXECUTORS_SLICE`. If it is not, warn the operator that resource
+    // isolation may be invalidated.
     // TODO(jmlvanre): Add a flag that enforces this matching (i.e. exits if a
     // pid was found in the freezer but not in the
-    // `SYSTEMD_MESOS_EXECUTORS_SLICE`. We need to flag to support the upgrade
-    // path.
+    // `MESOS_EXECUTORS_SLICE`. We need to flag to support the upgrade path.
     if (systemdHierarchy.isSome() && mesosExecutorSlicePids.isSome()) {
       if (mesosExecutorSlicePids.get().count(pid) <= 0) {
         LOG(WARNING)
           << "Couldn't find pid '" << pid << "' in '"
-          << SYSTEMD_MESOS_EXECUTORS_SLICE << "'. This can lead to lack of"
-          << " proper resource isolation";
+          << systemd::mesos::MESOS_EXECUTORS_SLICE << "'. This can lead to"
+          << " lack of proper resource isolation";
       }
     }
 
@@ -373,6 +314,7 @@ Try<pid_t> LinuxLauncher::fork(
       environment,
       lambda::bind(&childSetup, pipes, setup),
       lambda::bind(&os::clone, lambda::_1, cloneFlags),
+      // TODO(jmlvanre): Use systemd hook.
       Subprocess::Hook::None());
 
   if (child.isError()) {
@@ -401,12 +343,12 @@ Try<pid_t> LinuxLauncher::fork(
   }
 
   // If we are on systemd, then move the child into the
-  // `SYSTEMD_MESOS_EXECUTORS_SLICE`. As with the freezer, any grandchildren
-  // will also be contained in the slice.
+  // `MESOS_EXECUTORS_SLICE`. As with the freezer, any grandchildren will also
+  // be contained in the slice.
   if (systemdHierarchy.isSome()) {
     Try<Nothing> assign = cgroups::assign(
         systemdHierarchy.get(),
-        SYSTEMD_MESOS_EXECUTORS_SLICE,
+        systemd::mesos::MESOS_EXECUTORS_SLICE,
         child.get().pid());
 
     if (assign.isError()) {
@@ -419,7 +361,7 @@ Try<pid_t> LinuxLauncher::fork(
     }
 
     LOG(INFO) << "Assigned child process '" << child.get().pid() << "' to '"
-              << SYSTEMD_MESOS_EXECUTORS_SLICE << "'";
+              << systemd::mesos::MESOS_EXECUTORS_SLICE << "'";
   }
 
   // Now that we've contained the child we can signal it to continue

http://git-wip-us.apache.org/repos/asf/mesos/blob/bf6f1909/src/slave/containerizer/mesos/linux_launcher.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/linux_launcher.hpp b/src/slave/containerizer/mesos/linux_launcher.hpp
index b061981..1763d8d 100644
--- a/src/slave/containerizer/mesos/linux_launcher.hpp
+++ b/src/slave/containerizer/mesos/linux_launcher.hpp
@@ -23,9 +23,6 @@ namespace mesos {
 namespace internal {
 namespace slave {
 
-// TODO(jmlvanre): We may want to allow this to be configured.
-static const char SYSTEMD_MESOS_EXECUTORS_SLICE[] = "mesos_executors.slice";
-
 // Launcher for Linux systems with cgroups. Uses a freezer cgroup to
 // track pids.
 class LinuxLauncher : public Launcher

http://git-wip-us.apache.org/repos/asf/mesos/blob/bf6f1909/src/slave/main.cpp
----------------------------------------------------------------------
diff --git a/src/slave/main.cpp b/src/slave/main.cpp
index 9d48a08..ff406ac 100644
--- a/src/slave/main.cpp
+++ b/src/slave/main.cpp
@@ -39,6 +39,10 @@
 
 #include "hook/manager.hpp"
 
+#ifdef __linux__
+#include "linux/systemd.hpp"
+#endif // __linux__
+
 #include "logging/logging.hpp"
 
 #include "master/detector.hpp"
@@ -222,6 +226,23 @@ int main(int argc, char** argv)
 
   Fetcher fetcher;
 
+#ifdef __linux__
+  // Initialize systemd if it exists.
+  if (systemd::exists()) {
+    LOG(INFO) << "Inializing systemd state";
+
+    systemd::Flags systemdFlags;
+    systemdFlags.runtime_directory = flags.systemd_runtime_directory;
+    systemdFlags.cgroups_hierarchy = flags.cgroups_hierarchy;
+
+    Try<Nothing> initialize = systemd::initialize(systemdFlags);
+    if (initialize.isError()) {
+      EXIT(EXIT_FAILURE)
+        << "Failed to initialize systemd: " + initialize.error();
+    }
+  }
+#endif // __linux__
+
   Try<Containerizer*> containerizer =
     Containerizer::create(flags, false, &fetcher);
 


[28/48] mesos git commit: Extended life of process based posix exector on systemd.

Posted by vi...@apache.org.
Extended life of process based posix exector on systemd.

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


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

Branch: refs/heads/0.27.x
Commit: d04c2b3c4590c2cf731c42bd46a9a72aaa32eff8
Parents: 56a1edf
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Fri Feb 5 16:34:07 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:10 2016 -0500

----------------------------------------------------------------------
 src/slave/containerizer/mesos/launcher.cpp | 17 ++++++++++++++++-
 1 file changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d04c2b3c/src/slave/containerizer/mesos/launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launcher.cpp b/src/slave/containerizer/mesos/launcher.cpp
index d648324..c4ae1c5 100644
--- a/src/slave/containerizer/mesos/launcher.cpp
+++ b/src/slave/containerizer/mesos/launcher.cpp
@@ -25,6 +25,10 @@
 
 #include <stout/os/killtree.hpp>
 
+#ifdef __linux__
+#include "linux/systemd.hpp"
+#endif // __linux__
+
 #include "mesos/resources.hpp"
 
 #include "slave/containerizer/mesos/launcher.hpp"
@@ -115,6 +119,15 @@ Try<pid_t> PosixLauncher::fork(
                  stringify(containerId));
   }
 
+  // If we are on systemd, then extend the life of the child. Any
+  // grandchildren's lives will also be extended.
+  std::vector<Subprocess::Hook> parentHooks;
+#ifdef __linux__
+  if (systemd::enabled()) {
+    parentHooks.emplace_back(Subprocess::Hook(&systemd::mesos::extendLifetime));
+  }
+#endif // __linux__
+
   Try<Subprocess> child = subprocess(
       path,
       argv,
@@ -123,7 +136,9 @@ Try<pid_t> PosixLauncher::fork(
       err,
       flags,
       environment,
-      lambda::bind(&childSetup, setup));
+      lambda::bind(&childSetup, setup),
+      None(),
+      parentHooks);
 
   if (child.isError()) {
     return Error("Failed to fork a child process: " + child.error());


[22/48] mesos git commit: Updated Mesos version to 0.27.1.

Posted by vi...@apache.org.
Updated Mesos version to 0.27.1.


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

Branch: refs/heads/0.27.x
Commit: 46b4769a495e5401e7065d96f6cd6a001a10cf68
Parents: c29b3b6
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Mon Feb 15 16:14:32 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Mon Feb 15 16:15:23 2016 -0500

----------------------------------------------------------------------
 CMakeLists.txt | 2 +-
 configure.ac   | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/46b4769a/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 9b7044b..53a44c5 100755
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -21,7 +21,7 @@ cmake_minimum_required(VERSION 2.8)
 project(Mesos)
 set(MESOS_MAJOR_VERSION 0)
 set(MESOS_MINOR_VERSION 27)
-set(MESOS_PATCH_VERSION 0)
+set(MESOS_PATCH_VERSION 1)
 set(PACKAGE_VERSION
   ${MESOS_MAJOR_VERSION}.${MESOS_MINOR_VERSION}.${MESOS_PATCH_VERSION})
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/46b4769a/configure.ac
----------------------------------------------------------------------
diff --git a/configure.ac b/configure.ac
index 70d1687..259a4f0 100644
--- a/configure.ac
+++ b/configure.ac
@@ -18,7 +18,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.61])
-AC_INIT([mesos], [0.27.0])
+AC_INIT([mesos], [0.27.1])
 
 # Have autoconf setup some variables related to the system.
 AC_CANONICAL_HOST


[10/48] mesos git commit: Fixed a flaky test in disk quota tests.

Posted by vi...@apache.org.
Fixed a flaky test in disk quota tests.

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


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

Branch: refs/heads/0.27.x
Commit: 8186756cd019215a559c72a2492c391c9a7efa9e
Parents: 1a35d59
Author: Jie Yu <yu...@gmail.com>
Authored: Wed Jan 27 17:04:57 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 20:04:51 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/mesos/blob/8186756c/src/tests/disk_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/disk_quota_tests.cpp b/src/tests/disk_quota_tests.cpp
index eb93c2f..413e562 100644
--- a/src/tests/disk_quota_tests.cpp
+++ b/src/tests/disk_quota_tests.cpp
@@ -163,7 +163,7 @@ TEST_F(DiskUsageCollectorTest, ExcludeRelativePath)
   // path. Pattern matching is expected to fail causing exclude
   // to have no effect.
   Future<Bytes> usage2 = collector.usage(".", {file});
-  EXPECT_GT(usage2.get(), Kilobytes(128));
+  EXPECT_GE(usage2.get(), Kilobytes(128));
 }
 #endif
 


[35/48] mesos git commit: Mesos: Disambiguated call to 'subprocess' in linux launcher.

Posted by vi...@apache.org.
Mesos: Disambiguated call to 'subprocess' in linux launcher.

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


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

Branch: refs/heads/0.27.x
Commit: 66d6a4d69432b2a61d91b5e6c2c8e6e4571e0f9d
Parents: 4586166
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Thu Feb 4 15:27:54 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:10 2016 -0500

----------------------------------------------------------------------
 src/slave/containerizer/mesos/linux_launcher.cpp | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/66d6a4d6/src/slave/containerizer/mesos/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/linux_launcher.cpp b/src/slave/containerizer/mesos/linux_launcher.cpp
index 61801ff..d4541cd 100644
--- a/src/slave/containerizer/mesos/linux_launcher.cpp
+++ b/src/slave/containerizer/mesos/linux_launcher.cpp
@@ -372,7 +372,8 @@ Try<pid_t> LinuxLauncher::fork(
       flags,
       environment,
       lambda::bind(&childSetup, pipes, setup),
-      lambda::bind(&os::clone, lambda::_1, cloneFlags));
+      lambda::bind(&os::clone, lambda::_1, cloneFlags),
+      Subprocess::Hook::None());
 
   if (child.isError()) {
     return Error("Failed to clone child process: " + child.error());


[18/48] mesos git commit: Avoid unnecessary string copies in `json` for protobuf messages.

Posted by vi...@apache.org.
Avoid unnecessary string copies in `json` for protobuf messages.

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


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

Branch: refs/heads/0.27.x
Commit: c154d85e5408d6ae1fbdda1905815c304674ff17
Parents: b64851f
Author: Michael Park <mp...@apache.org>
Authored: Sun Jan 31 20:38:34 2016 -0800
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Mon Feb 15 16:15:23 2016 -0500

----------------------------------------------------------------------
 .../3rdparty/stout/include/stout/protobuf.hpp      | 17 ++++++++++-------
 1 file changed, 10 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c154d85e/3rdparty/libprocess/3rdparty/stout/include/stout/protobuf.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/protobuf.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/protobuf.hpp
index 6d7d033..eb5502c 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/protobuf.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/protobuf.hpp
@@ -696,12 +696,13 @@ inline void json(ObjectWriter* writer, const google::protobuf::Message& message)
                       reflection->GetRepeatedEnum(message, field, i)->name());
                   break;
                 case FieldDescriptor::CPPTYPE_STRING:
-                  std::string s =
-                    reflection->GetRepeatedString(message, field, i);
+                  const std::string& s = reflection->GetRepeatedStringReference(
+                      message, field, i, NULL);
                   if (field->type() == FieldDescriptor::TYPE_BYTES) {
-                    s = base64::encode(s);
+                    writer->element(base64::encode(s));
+                  } else {
+                    writer->element(s);
                   }
-                  writer->element(s);
                   break;
               }
             }
@@ -737,11 +738,13 @@ inline void json(ObjectWriter* writer, const google::protobuf::Message& message)
               field->name(), reflection->GetEnum(message, field)->name());
           break;
         case FieldDescriptor::CPPTYPE_STRING:
-          std::string str = reflection->GetString(message, field);
+          const std::string& s = reflection->GetStringReference(
+              message, field, NULL);
           if (field->type() == FieldDescriptor::TYPE_BYTES) {
-            str = base64::encode(str);
+            writer->field(field->name(), base64::encode(s));
+          } else {
+            writer->field(field->name(), s);
           }
-          writer->field(field->name(), str);
           break;
       }
     }


[19/48] mesos git commit: Changed ZooKeeper reconnection logic to retry more aggressively.

Posted by vi...@apache.org.
Changed ZooKeeper reconnection logic to retry more aggressively.

The previous implementation of `GroupProcess` tried to establish a
single ZooKeeper connection on startup, but didn't attempt to retry.
ZooKeeper will retry internally, but it only retries by attempting to
reconnect to a list of previously resolved IPs; it doesn't attempt to
re-resolve those IPs to pickup updates to DNS configuration. Because
DNS configuration can be quite dynamic, we now close the current Zk
handle and open a new one if we've seen a successful `zookeeper_init`
but haven't been connected within the ZooKeeper session timeout.

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


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

Branch: refs/heads/0.27.x
Commit: f8f75c2628fb6500c8c7f85b3fb99cb06da86536
Parents: 67e6bcc
Author: Neil Conway <ne...@gmail.com>
Authored: Sat Jan 30 20:02:15 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Mon Feb 15 16:15:23 2016 -0500

----------------------------------------------------------------------
 src/tests/group_tests.cpp   | 26 +++++++++++++
 src/zookeeper/group.cpp     | 80 +++++++++++++++++++++++++++-------------
 src/zookeeper/group.hpp     |  8 ++--
 src/zookeeper/zookeeper.cpp | 45 ++++++++++++++++------
 4 files changed, 119 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f8f75c26/src/tests/group_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/group_tests.cpp b/src/tests/group_tests.cpp
index 7734946..6344fad 100644
--- a/src/tests/group_tests.cpp
+++ b/src/tests/group_tests.cpp
@@ -34,6 +34,7 @@
 using zookeeper::Group;
 using zookeeper::GroupProcess;
 
+using process::Clock;
 using process::Future;
 
 using std::string;
@@ -433,6 +434,31 @@ TEST_F(GroupTest, LabelledGroup)
   ASSERT_TRUE(membership.get().cancelled().get());
 }
 
+
+// This test checks that the `expired` event is invoked even if we
+// have not ever established a connection to ZooKeeper (MESOS-4546).
+TEST_F(GroupTest, ConnectTimer)
+{
+  const Duration sessionTimeout = Seconds(10);
+
+  Clock::pause();
+
+  // Ensure that we won't be able to establish a connection to ZooKeeper.
+  server->shutdownNetwork();
+
+  Future<Nothing> expired = FUTURE_DISPATCH(_, &GroupProcess::expired);
+
+  Group group(server->connectString(), sessionTimeout, "/test/");
+
+  // Advance the clock to ensure that we forcibly expire the current
+  // ZooKeeper connection attempt and try to reconnect.
+  Clock::advance(sessionTimeout);
+
+  AWAIT_READY(expired);
+
+  Clock::resume();
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f8f75c26/src/zookeeper/group.cpp
----------------------------------------------------------------------
diff --git a/src/zookeeper/group.cpp b/src/zookeeper/group.cpp
index add01a3..ded1458 100644
--- a/src/zookeeper/group.cpp
+++ b/src/zookeeper/group.cpp
@@ -125,6 +125,9 @@ GroupProcess::GroupProcess(
 {}
 
 
+// NB: The `retry` and `connect` timers might still be active. However,
+// we don't need to clean them up -- when the timers fire, they will
+// attempt to dispatch to a no-longer-valid PID, which is a no-op.
 GroupProcess::~GroupProcess()
 {
   discard(&pending.joins);
@@ -141,11 +144,28 @@ void GroupProcess::initialize()
 {
   // Doing initialization here allows to avoid the race between
   // instantiating the ZooKeeper instance and being spawned ourself.
+  startConnection();
+}
+
+
+void GroupProcess::startConnection()
+{
   watcher = new ProcessWatcher<GroupProcess>(self());
   zk = new ZooKeeper(servers, sessionTimeout, watcher);
   state = CONNECTING;
-}
 
+  // If the connection is not established within the session timeout,
+  // close the ZooKeeper handle and create a new one. This is
+  // important because the ZooKeeper 3.4 client libraries don't try to
+  // re-resolve the list of hostnames, so we create a new ZooKeeper
+  // handle to ensure we observe DNS changes. See MESOS-4546 and
+  // `ZooKeeperProcess::initialize` for more information.
+  CHECK_NONE(connectTimer);
+  connectTimer = delay(zk->getSessionTimeout(),
+                       self(),
+                       &Self::timedout,
+                       zk->getSessionId());
+}
 
 Future<Group::Membership> GroupProcess::join(
     const string& data,
@@ -346,11 +366,17 @@ void GroupProcess::connected(int64_t sessionId, bool reconnect)
       << state;
   }
 
-  // Cancel and cleanup the reconnect timer (if necessary).
-  if (timer.isSome()) {
-    Clock::cancel(timer.get());
-    timer = None();
-  }
+  // Cancel and cleanup the connect timer. The timer should always be
+  // set, because it is set before making the initial connection
+  // attempt and whenever a reconnection attempt is made.
+  CHECK_SOME(connectTimer);
+
+  // Now that we are connected, we'll learn about a subsequent
+  // disconnection event via the `reconnecting` callback. At that
+  // point we'll also restart the `connectTimer` to ensure we retry
+  // the reconnection attempt.
+  Clock::cancel(connectTimer.get());
+  connectTimer = None();
 
   // Sync group operations (and set up the group on ZK).
   Try<bool> synced = sync();
@@ -446,13 +472,18 @@ void GroupProcess::reconnecting(int64_t sessionId)
   // we create a local timer and "expire" our session prematurely if
   // we haven't reconnected within the session expiration time out.
   // The timer can be reset if the connection is restored.
-  CHECK_NONE(timer);
 
-  // Use the negotiated session timeout for the reconnect timer.
-  timer = delay(zk->getSessionTimeout(),
-                self(),
-                &Self::timedout,
-                zk->getSessionId());
+  // We expect to see exactly one `reconnecting` event when our
+  // session is disconnected, even if we're disconnected for an
+  // extended period. Since we clear the `connectTimer` when the
+  // connection is established, it should still be unset here.
+  CHECK_NONE(connectTimer);
+
+  // Use the negotiated session timeout for the connect timer.
+  connectTimer = delay(zk->getSessionTimeout(),
+                       self(),
+                       &Self::timedout,
+                       zk->getSessionId());
 }
 
 
@@ -464,13 +495,13 @@ void GroupProcess::timedout(int64_t sessionId)
 
   CHECK_NOTNULL(zk);
 
-  if (timer.isSome() &&
-      timer.get().timeout().expired() &&
+  // The connect timer can be reset or replaced and `zk`
+  // can be replaced since this method was dispatched.
+  if (connectTimer.isSome() &&
+      connectTimer.get().timeout().expired() &&
       zk->getSessionId() == sessionId) {
-    // The timer can be reset or replaced and 'zk' can be replaced
-    // since this method was dispatched.
-    LOG(WARNING) << "Timed out waiting to reconnect to ZooKeeper."
-                 << " Forcing ZooKeeper session "
+    LOG(WARNING) << "Timed out waiting to connect to ZooKeeper. "
+                 << "Forcing ZooKeeper session "
                  << "(sessionId=" << std::hex << sessionId << ") expiration";
 
     // Locally determine that the current session has expired.
@@ -490,10 +521,10 @@ void GroupProcess::expired(int64_t sessionId)
   // Cancel the retries. Group will sync() after it reconnects to ZK.
   retrying = false;
 
-  // Cancel and cleanup the reconnect timer (if necessary).
-  if (timer.isSome()) {
-    Clock::cancel(timer.get());
-    timer = None();
+  // Cancel and cleanup the connect timer (if necessary).
+  if (connectTimer.isSome()) {
+    Clock::cancel(connectTimer.get());
+    connectTimer = None();
   }
 
   // From the group's local perspective all the memberships are
@@ -529,10 +560,7 @@ void GroupProcess::expired(int64_t sessionId)
 
   delete CHECK_NOTNULL(zk);
   delete CHECK_NOTNULL(watcher);
-  watcher = new ProcessWatcher<GroupProcess>(self());
-  zk = new ZooKeeper(servers, sessionTimeout, watcher);
-
-  state = CONNECTING;
+  startConnection();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f8f75c26/src/zookeeper/group.hpp
----------------------------------------------------------------------
diff --git a/src/zookeeper/group.hpp b/src/zookeeper/group.hpp
index ed5d0a0..db8a120 100644
--- a/src/zookeeper/group.hpp
+++ b/src/zookeeper/group.hpp
@@ -208,6 +208,8 @@ public:
   void deleted(int64_t sessionId, const std::string& path);
 
 private:
+  void startConnection();
+
   Result<Group::Membership> doJoin(
       const std::string& data,
       const Option<std::string>& label);
@@ -339,9 +341,9 @@ private:
   // cache and 'Some' represents a valid cache.
   Option<std::set<Group::Membership> > memberships;
 
-  // The timer that determines whether we should quit waiting for the
-  // connection to be restored.
-  Option<process::Timer> timer;
+  // A timer that controls when we should give up on waiting for the
+  // current connection attempt to succeed and try to reconnect.
+  Option<process::Timer> connectTimer;
 };
 
 } // namespace zookeeper {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f8f75c26/src/zookeeper/zookeeper.cpp
----------------------------------------------------------------------
diff --git a/src/zookeeper/zookeeper.cpp b/src/zookeeper/zookeeper.cpp
index 790bd15..02fa158 100644
--- a/src/zookeeper/zookeeper.cpp
+++ b/src/zookeeper/zookeeper.cpp
@@ -70,17 +70,40 @@ public:
 
   virtual void initialize()
   {
-    // We retry zookeeper_init until the timeout elapses because we've
-    // seen cases where temporary DNS outages cause the slave to abort
-    // here. See MESOS-1326 for more information.
-    // ZooKeeper masks EAI_AGAIN as EINVAL and a name resolution timeout
-    // may be upwards of 30 seconds. As such, a 10 second timeout is not
-    // enough. Hard code this to 10 minutes to be sure we're trying again
-    // in the face of temporary name resolution failures. See MESOS-1523
-    // for more information.
-    const Timeout timeout_ = Timeout::in(Minutes(10));
-
-    while (!timeout_.expired()) {
+    // There are two different timeouts here:
+    //
+    // (1) `sessionTimeout` is the client's proposed value for the
+    // ZooKeeper session timeout.
+    //
+    // (2) `initLoopTimeout` is how long we are prepared to wait,
+    // calling `zookeeper_init` in a loop, until a call succeeds.
+    //
+    // `sessionTimeout` is used to determine the liveness of our
+    // ZooKeeper session. `initLoopTimeout` determines how long to
+    // retry erroneous calls to `zookeeper_init`, because there are
+    // cases when temporary DNS outages cause `zookeeper_init` to
+    // return failure. ZooKeeper masks EAI_AGAIN as EINVAL and a name
+    // resolution timeout may be upwards of 30 seconds. As such, a 10
+    // second timeout (the default `sessionTimeout`) is not enough. We
+    // hardcode `initLoopTimeout` to 10 minutes ensure we're trying
+    // again in the face of temporary name resolution failures. See
+    // MESOS-1523 for more information.
+    //
+    // Note that there are cases where `zookeeper_init` returns
+    // success but we don't see a subsequent ZooKeeper event
+    // indicating that our connection has been established. A common
+    // cause for this situation is that the ZK hostname list resolves
+    // to unreachable IP addresses. ZooKeeper will continue looping,
+    // trying to connect to the list of IPs but never attempting to
+    // re-resolve the input hostnames. Since DNS may have changed, we
+    // close the ZK handle and create a new handle to ensure that ZK
+    // will try to re-resolve the configured list of hostnames.
+    // However, since we can't easily check if the `connected` ZK
+    // event has been fired for this session yet, we implement this
+    // timeout in `GroupProcess`. See MESOS-4546 for more information.
+    const Timeout initLoopTimeout = Timeout::in(Minutes(10));
+
+    while (!initLoopTimeout.expired()) {
       zh = zookeeper_init(
           servers.c_str(),
           event,


[07/48] mesos git commit: Disallowed non-`const` iteration over `Resources`.

Posted by vi...@apache.org.
Disallowed non-`const` iteration over `Resources`.

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


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

Branch: refs/heads/0.27.x
Commit: 96acace1df51ca3ea23a1dea45b9988e76c85018
Parents: bdd8815
Author: Michael Park <mp...@apache.org>
Authored: Wed Jan 27 14:04:09 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 20:04:51 2016 -0800

----------------------------------------------------------------------
 include/mesos/resources.hpp                 | 18 +++++++++++++++---
 include/mesos/v1/resources.hpp              | 18 +++++++++++++++---
 src/master/allocator/mesos/hierarchical.cpp |  7 ++++---
 src/master/http.cpp                         |  5 +++--
 src/slave/resource_estimators/fixed.cpp     |  4 ++--
 src/tests/resources_tests.cpp               |  4 ++--
 6 files changed, 41 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/96acace1/include/mesos/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/resources.hpp b/include/mesos/resources.hpp
index cc8fef9..6bfac2e 100644
--- a/include/mesos/resources.hpp
+++ b/include/mesos/resources.hpp
@@ -317,14 +317,26 @@ public:
   // which holds the ephemeral ports allocation logic.
   Option<Value::Ranges> ephemeral_ports() const;
 
-  typedef google::protobuf::RepeatedPtrField<Resource>::iterator
+  // NOTE: Non-`const` `iterator`, `begin()` and `end()` are __intentionally__
+  // defined with `const` semantics in order to prevent mutable access to the
+  // `Resource` objects within `resources`.
+  typedef google::protobuf::RepeatedPtrField<Resource>::const_iterator
   iterator;
 
   typedef google::protobuf::RepeatedPtrField<Resource>::const_iterator
   const_iterator;
 
-  iterator begin() { return resources.begin(); }
-  iterator end() { return resources.end(); }
+  const_iterator begin()
+  {
+    using google::protobuf::RepeatedPtrField;
+    return static_cast<const RepeatedPtrField<Resource>&>(resources).begin();
+  }
+
+  const_iterator end()
+  {
+    using google::protobuf::RepeatedPtrField;
+    return static_cast<const RepeatedPtrField<Resource>&>(resources).end();
+  }
 
   const_iterator begin() const { return resources.begin(); }
   const_iterator end() const { return resources.end(); }

http://git-wip-us.apache.org/repos/asf/mesos/blob/96acace1/include/mesos/v1/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/v1/resources.hpp b/include/mesos/v1/resources.hpp
index f489297..5a88c07 100644
--- a/include/mesos/v1/resources.hpp
+++ b/include/mesos/v1/resources.hpp
@@ -317,14 +317,26 @@ public:
   // which holds the ephemeral ports allocation logic.
   Option<Value::Ranges> ephemeral_ports() const;
 
-  typedef google::protobuf::RepeatedPtrField<Resource>::iterator
+  // NOTE: Non-`const` `iterator`, `begin()` and `end()` are __intentionally__
+  // defined with `const` semantics in order to prevent mutable access to the
+  // `Resource` objects within `resources`.
+  typedef google::protobuf::RepeatedPtrField<Resource>::const_iterator
   iterator;
 
   typedef google::protobuf::RepeatedPtrField<Resource>::const_iterator
   const_iterator;
 
-  iterator begin() { return resources.begin(); }
-  iterator end() { return resources.end(); }
+  const_iterator begin()
+  {
+    using google::protobuf::RepeatedPtrField;
+    return static_cast<const RepeatedPtrField<Resource>&>(resources).begin();
+  }
+
+  const_iterator end()
+  {
+    using google::protobuf::RepeatedPtrField;
+    return static_cast<const RepeatedPtrField<Resource>&>(resources).end();
+  }
 
   const_iterator begin() const { return resources.begin(); }
   const_iterator end() const { return resources.end(); }

http://git-wip-us.apache.org/repos/asf/mesos/blob/96acace1/src/master/allocator/mesos/hierarchical.cpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.cpp b/src/master/allocator/mesos/hierarchical.cpp
index dbc77a2..1a07d69 100644
--- a/src/master/allocator/mesos/hierarchical.cpp
+++ b/src/master/allocator/mesos/hierarchical.cpp
@@ -1160,13 +1160,14 @@ void HierarchicalAllocatorProcess::allocate(
   auto getQuotaRoleAllocatedResources = [this](const std::string& role) {
     CHECK(quotas.contains(role));
 
-    Resources resources = quotaRoleSorter->allocationScalars(role);
-
     // Strip the reservation and persistent volume info.
-    foreach (Resource& resource, resources) {
+    Resources resources;
+
+    foreach (Resource resource, quotaRoleSorter->allocationScalars(role)) {
       resource.set_role("*");
       resource.clear_reservation();
       resource.clear_disk();
+      resources += resource;
     }
 
     return resources;

http://git-wip-us.apache.org/repos/asf/mesos/blob/96acace1/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index 12c1fe5..54264f6 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -708,10 +708,11 @@ string Master::Http::CREATE_VOLUMES_HELP()
 
 static Resources removeDiskInfos(const Resources& resources)
 {
-  Resources result = resources;
+  Resources result;
 
-  foreach (Resource& resource, result) {
+  foreach (Resource resource, resources) {
     resource.clear_disk();
+    result += resource;
   }
 
   return result;

http://git-wip-us.apache.org/repos/asf/mesos/blob/96acace1/src/slave/resource_estimators/fixed.cpp
----------------------------------------------------------------------
diff --git a/src/slave/resource_estimators/fixed.cpp b/src/slave/resource_estimators/fixed.cpp
index aedd08d..c858a48 100644
--- a/src/slave/resource_estimators/fixed.cpp
+++ b/src/slave/resource_estimators/fixed.cpp
@@ -68,11 +68,11 @@ class FixedResourceEstimator : public ResourceEstimator
 {
 public:
   FixedResourceEstimator(const Resources& _totalRevocable)
-    : totalRevocable(_totalRevocable)
   {
     // Mark all resources as revocable.
-    foreach (Resource& resource, totalRevocable) {
+    foreach (Resource resource, _totalRevocable) {
       resource.mutable_revocable();
+      totalRevocable += resource;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/96acace1/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index 54a4fa8..4b25e82 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -351,7 +351,7 @@ TEST(ResourcesTest, ParsingFromJSONWithRoles)
   EXPECT_TRUE(resources.contains(Resources(*cpus)));
   EXPECT_EQ(145.54, resources.cpus().get());
 
-  foreach (Resource& resource, resources) {
+  foreach (const Resource& resource, resources) {
     if (resource.role() == "role1") {
       EXPECT_EQ(91.1, resource.scalar().value());
     } else {
@@ -396,7 +396,7 @@ TEST(ResourcesTest, ParsingFromJSONWithRoles)
 
   resources = resourcesTry.get();
 
-  foreach (Resource& resource, resources) {
+  foreach (const Resource& resource, resources) {
     if (resource.role() == "role1") {
       EXPECT_EQ(Value::RANGES, resource.type());
       EXPECT_EQ(2, resource.ranges().range_size());


[24/48] mesos git commit: Fixed the stout Makefile.am.

Posted by vi...@apache.org.
Fixed the stout Makefile.am.


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

Branch: refs/heads/0.27.x
Commit: dce4c7b081045365def7b8714f42f0f2fabdf012
Parents: 2da5d5f
Author: Jie Yu <yu...@gmail.com>
Authored: Thu Feb 4 11:46:44 2016 -0800
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:09 2016 -0500

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/stout/include/Makefile.am | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/dce4c7b0/3rdparty/libprocess/3rdparty/stout/include/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/Makefile.am b/3rdparty/libprocess/3rdparty/stout/include/Makefile.am
index f3a7b4b..3dc2221 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/stout/include/Makefile.am
@@ -33,7 +33,6 @@ nobase_include_HEADERS =		\
   stout/foreach.hpp			\
   stout/format.hpp			\
   stout/fs.hpp				\
-  stout/freebsd.hpp			\
   stout/gtest.hpp			\
   stout/gzip.hpp			\
   stout/hashmap.hpp			\
@@ -68,6 +67,7 @@ nobase_include_HEADERS =		\
   stout/os/fcntl.hpp			\
   stout/os/find.hpp			\
   stout/os/fork.hpp			\
+  stout/os/freebsd.hpp			\
   stout/os/ftruncate.hpp		\
   stout/os/getcwd.hpp			\
   stout/os/killtree.hpp			\


[26/48] mesos git commit: Don't remove IP from the logger's environment.

Posted by vi...@apache.org.
Don't remove IP from the logger's environment.

Unsetting the `LIBPROCESS_IP` was originally unnecessary, as this
variable does not result in port conflicts.  Unsetting this environment
variable may cause the subprocess to exit if DNS cannot resolve the
agent's hostname.

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


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

Branch: refs/heads/0.27.x
Commit: 9f4fa0ea52788fa1c7938caab059f4d3843becda
Parents: dce4c7b
Author: Joseph Wu <jo...@mesosphere.io>
Authored: Fri Feb 5 03:54:24 2016 +0100
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:09 2016 -0500

----------------------------------------------------------------------
 src/slave/container_loggers/lib_logrotate.cpp | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9f4fa0ea/src/slave/container_loggers/lib_logrotate.cpp
----------------------------------------------------------------------
diff --git a/src/slave/container_loggers/lib_logrotate.cpp b/src/slave/container_loggers/lib_logrotate.cpp
index 8d2f895..01a3ff0 100644
--- a/src/slave/container_loggers/lib_logrotate.cpp
+++ b/src/slave/container_loggers/lib_logrotate.cpp
@@ -81,12 +81,10 @@ public:
     // Inherit most, but not all of the agent's environment.
     // Since the subprocess links to libmesos, it will need some of the
     // same environment used to launch the agent (also uses libmesos).
-    // The libprocess IP and port are explicitly removed because these
+    // The libprocess port is explicitly removed because this
     // will conflict with the already-running agent.
     std::map<std::string, std::string> environment = os::environment();
-    environment.erase("LIBPROCESS_IP");
     environment.erase("LIBPROCESS_PORT");
-    environment.erase("LIBPROCESS_ADVERTISE_IP");
     environment.erase("LIBPROCESS_ADVERTISE_PORT");
 
     // NOTE: We manually construct a pipe here instead of using


[14/48] mesos git commit: Updated `Master::Http::stateSummary` to use `jsonify`.

Posted by vi...@apache.org.
Updated `Master::Http::stateSummary` to use `jsonify`.

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


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

Branch: refs/heads/0.27.x
Commit: 00441c1b305ca49fe35768009461be112aa88314
Parents: 94c8e5e
Author: Michael Park <mp...@apache.org>
Authored: Tue Jan 19 20:10:02 2016 -0800
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Mon Feb 15 16:15:23 2016 -0500

----------------------------------------------------------------------
 src/master/http.cpp | 184 +++++++++++++++++++++++------------------------
 1 file changed, 90 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/00441c1b/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index 54264f6..9593373 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -1570,109 +1570,105 @@ string Master::Http::STATESUMMARY_HELP()
 
 Future<Response> Master::Http::stateSummary(const Request& request) const
 {
-  JSON::Object object;
-
-  object.values["hostname"] = master->info().hostname();
-
-  if (master->flags.cluster.isSome()) {
-    object.values["cluster"] = master->flags.cluster.get();
-  }
-
-  // We use the tasks in the 'Frameworks' struct to compute summaries
-  // for this endpoint. This is done 1) for consistency between the
-  // 'slaves' and 'frameworks' subsections below 2) because we want to
-  // provide summary information for frameworks that are currently
-  // registered 3) the frameworks keep a circular buffer of completed
-  // tasks that we can use to keep a limited view on the history of
-  // recent completed / failed tasks.
-
-  // Generate mappings from 'slave' to 'framework' and reverse.
-  SlaveFrameworkMapping slaveFrameworkMapping(master->frameworks.registered);
-
-  // Generate 'TaskState' summaries for all framework and slave ids.
-  TaskStateSummaries taskStateSummaries(master->frameworks.registered);
-
-  // Model all of the slaves.
-  {
-    JSON::Array array;
-    array.values.reserve(master->slaves.registered.size()); // MESOS-2353.
-
-    foreachvalue (Slave* slave, master->slaves.registered) {
-      JSON::Object json = summarize(*slave);
+  auto stateSummary = [this](JSON::ObjectWriter* writer) {
+    writer->field("hostname", master->info().hostname());
 
-      // Add the 'TaskState' summary for this slave.
-      const TaskStateSummary& summary = taskStateSummaries.slave(slave->id);
+    if (master->flags.cluster.isSome()) {
+      writer->field("cluster", master->flags.cluster.get());
+    }
 
-      json.values["TASK_STAGING"] = summary.staging;
-      json.values["TASK_STARTING"] = summary.starting;
-      json.values["TASK_RUNNING"] = summary.running;
-      json.values["TASK_FINISHED"] = summary.finished;
-      json.values["TASK_KILLED"] = summary.killed;
-      json.values["TASK_FAILED"] = summary.failed;
-      json.values["TASK_LOST"] = summary.lost;
-      json.values["TASK_ERROR"] = summary.error;
+    // We use the tasks in the 'Frameworks' struct to compute summaries
+    // for this endpoint. This is done 1) for consistency between the
+    // 'slaves' and 'frameworks' subsections below 2) because we want to
+    // provide summary information for frameworks that are currently
+    // registered 3) the frameworks keep a circular buffer of completed
+    // tasks that we can use to keep a limited view on the history of
+    // recent completed / failed tasks.
 
-      // Add the ids of all the frameworks running on this slave.
-      const hashset<FrameworkID>& frameworks =
-        slaveFrameworkMapping.frameworks(slave->id);
+    // Generate mappings from 'slave' to 'framework' and reverse.
+    SlaveFrameworkMapping slaveFrameworkMapping(master->frameworks.registered);
 
-      JSON::Array frameworkIdArray;
-      frameworkIdArray.values.reserve(frameworks.size()); // MESOS-2353.
+    // Generate 'TaskState' summaries for all framework and slave ids.
+    TaskStateSummaries taskStateSummaries(master->frameworks.registered);
 
-      foreach (const FrameworkID& frameworkId, frameworks) {
-        frameworkIdArray.values.push_back(frameworkId.value());
+    // Model all of the slaves.
+    writer->field("slaves", [this,
+                             &slaveFrameworkMapping,
+                             &taskStateSummaries](JSON::ArrayWriter* writer) {
+      foreachvalue (Slave* slave, master->slaves.registered) {
+        writer->element([&slave,
+                         &slaveFrameworkMapping,
+                         &taskStateSummaries](JSON::ObjectWriter* writer) {
+          json(writer, Summary<Slave>(*slave));
+
+          // Add the 'TaskState' summary for this slave.
+          const TaskStateSummary& summary = taskStateSummaries.slave(slave->id);
+
+          writer->field("TASK_STAGING", summary.staging);
+          writer->field("TASK_STARTING", summary.starting);
+          writer->field("TASK_RUNNING", summary.running);
+          writer->field("TASK_FINISHED", summary.finished);
+          writer->field("TASK_KILLED", summary.killed);
+          writer->field("TASK_FAILED", summary.failed);
+          writer->field("TASK_LOST", summary.lost);
+          writer->field("TASK_ERROR", summary.error);
+
+          // Add the ids of all the frameworks running on this slave.
+          const hashset<FrameworkID>& frameworks =
+            slaveFrameworkMapping.frameworks(slave->id);
+
+          writer->field("framework_ids",
+                        [&frameworks](JSON::ArrayWriter* writer) {
+            foreach (const FrameworkID& frameworkId, frameworks) {
+              writer->element(frameworkId.value());
+            }
+          });
+        });
       }
+    });
 
-      json.values["framework_ids"] = std::move(frameworkIdArray);
-
-      array.values.push_back(std::move(json));
-    }
-
-    object.values["slaves"] = std::move(array);
-  }
-
-  // Model all of the frameworks.
-  {
-    JSON::Array array;
-    array.values.reserve(master->frameworks.registered.size()); // MESOS-2353.
-
-    foreachpair (const FrameworkID& frameworkId,
-                 Framework* framework,
-                 master->frameworks.registered) {
-      JSON::Object json = summarize(*framework);
-
-      // Add the 'TaskState' summary for this framework.
-      const TaskStateSummary& summary =
-        taskStateSummaries.framework(frameworkId);
-      json.values["TASK_STAGING"] = summary.staging;
-      json.values["TASK_STARTING"] = summary.starting;
-      json.values["TASK_RUNNING"] = summary.running;
-      json.values["TASK_FINISHED"] = summary.finished;
-      json.values["TASK_KILLED"] = summary.killed;
-      json.values["TASK_FAILED"] = summary.failed;
-      json.values["TASK_LOST"] = summary.lost;
-      json.values["TASK_ERROR"] = summary.error;
-
-      // Add the ids of all the slaves running this framework.
-      const hashset<SlaveID>& slaves =
-        slaveFrameworkMapping.slaves(frameworkId);
-
-      JSON::Array slaveIdArray;
-      slaveIdArray.values.reserve(slaves.size()); // MESOS-2353.
-
-      foreach (const SlaveID& slaveId, slaves) {
-        slaveIdArray.values.push_back(slaveId.value());
+    // Model all of the frameworks.
+    writer->field("frameworks",
+                  [this,
+                   &slaveFrameworkMapping,
+                   &taskStateSummaries](JSON::ArrayWriter* writer) {
+      foreachpair (const FrameworkID& frameworkId,
+                   Framework* framework,
+                   master->frameworks.registered) {
+        writer->element([&frameworkId,
+                         &framework,
+                         &slaveFrameworkMapping,
+                         &taskStateSummaries](JSON::ObjectWriter* writer) {
+          json(writer, Summary<Framework>(*framework));
+
+          // Add the 'TaskState' summary for this framework.
+          const TaskStateSummary& summary =
+            taskStateSummaries.framework(frameworkId);
+
+          writer->field("TASK_STAGING", summary.staging);
+          writer->field("TASK_STARTING", summary.starting);
+          writer->field("TASK_RUNNING", summary.running);
+          writer->field("TASK_FINISHED", summary.finished);
+          writer->field("TASK_KILLED", summary.killed);
+          writer->field("TASK_FAILED", summary.failed);
+          writer->field("TASK_LOST", summary.lost);
+          writer->field("TASK_ERROR", summary.error);
+
+          // Add the ids of all the slaves running this framework.
+          const hashset<SlaveID>& slaves =
+            slaveFrameworkMapping.slaves(frameworkId);
+
+          writer->field("slave_ids", [&slaves](JSON::ArrayWriter* writer) {
+            foreach (const SlaveID& slaveId, slaves) {
+              writer->element(slaveId.value());
+            }
+          });
+        });
       }
+    });
+  };
 
-      json.values["slave_ids"] = std::move(slaveIdArray);
-
-      array.values.push_back(std::move(json));
-    }
-
-    object.values["frameworks"] = std::move(array);
-  }
-
-  return OK(object, request.url.query.get("jsonp"));
+  return OK(jsonify(stateSummary), request.url.query.get("jsonp"));
 }
 
 


[38/48] mesos git commit: Added documentation for multiple-disk support.

Posted by vi...@apache.org.
Added documentation for multiple-disk support.

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


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

Branch: refs/heads/0.27.x
Commit: efe8a2c6d25be863ae305d155ad6ca935d1a9d42
Parents: 8afbcc3
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Mon Feb 15 14:40:54 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:31 2016 -0500

----------------------------------------------------------------------
 docs/home.md              |   1 +
 docs/multiple-disk.md     | 141 +++++++++++++++++++++++++++++++++++++++++
 docs/persistent-volume.md |   3 +
 3 files changed, 145 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/efe8a2c6/docs/home.md
----------------------------------------------------------------------
diff --git a/docs/home.md b/docs/home.md
index dea6ec2..2780eea 100644
--- a/docs/home.md
+++ b/docs/home.md
@@ -41,6 +41,7 @@ layout: documentation
 * [Networking for Mesos-managed Containers](networking-for-mesos-managed-containers.md)
 * [Oversubscription](oversubscription.md) for how to configure Mesos to take advantage of unused resources to launch "best-effort" tasks.
 * [Persistent Volume](persistent-volume.md) for how to allow tasks to access persistent storage resources.
+* [Multiple Disks](multiple-disk.md) for how to to allow tasks to use multiple isolated disk resources.
 * [Quota](quota.md) for how to configure Mesos to provide guaranteed resource allocations for use by a role.
 * [Reservation](reservation.md) for how operators and frameworks can reserve resources on individual agents for use by a role.
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/efe8a2c6/docs/multiple-disk.md
----------------------------------------------------------------------
diff --git a/docs/multiple-disk.md b/docs/multiple-disk.md
new file mode 100644
index 0000000..4fc5327
--- /dev/null
+++ b/docs/multiple-disk.md
@@ -0,0 +1,141 @@
+---
+layout: documentation
+---
+
+# Multiple Disks
+
+Mesos provides a mechanism for operators to expose multiple disk resources. When
+creating [persistent volumes](persistent-volume.md) frameworks can decide
+whether to use specific disks by examining the `source` field on the disk
+resources offered.
+
+`Disk` resources come in three forms:
+
+* A `Root` disk is presented by not having the `source` set in `DiskInfo`.
+* A `Path` disk is presented by having the `PATH` enum set for `source` in
+  `DiskInfo`. It also has a `root` which the operator uses to specify the
+  directory to be used to store data.
+* A `Mount` disk is presented by having the `MOUNT` enum set for `source` in
+  `DiskInfo`. It also has a `root` which the operator uses to specify the
+  mount point used to store data.
+
+Operators can use the JSON-formated `--resources` option on the agent to provide
+these different kind of disk resources on agent start-up.
+
+#### `Root` disk
+
+A `Root` disk is the basic disk resource in Mesos. It usually maps to the
+storage on the main operating system drive that the operator has presented to
+the agent. Data is mapped into the `work_dir` of the agent.
+
+```
+{
+  "resources" : [
+    {
+      "name" : "disk",
+      "type" : "SCALAR",
+      "scalar" : { "value" : 2048 },
+      "role" : <framework_role>
+    }
+  ]
+}
+```
+
+#### `Path` disks
+
+A `Path` disk is an auxiliary disk resource provided by the operator. This can
+can be carved up into smaller chunks by accepting less than the total amount as
+frameworks see fit. Common uses for this kind of disk are extra logging space,
+file archives or caches, or other non performance-critical applications.
+Operators can present extra disks on their agents as `Path` disks simply by
+creating a directory and making that the `root` of the `Path` in `DiskInfo`'s
+`source`.
+
+`Path` disks are also useful for mocking up a multiple-disk environment by
+creating some directories on the operating system drive. This should only be
+done in a testing or staging environment.
+
+```
+{
+  "resources" : [
+    {
+      "name" : "disk",
+      "type" : "SCALAR",
+      "scalar" : { "value" : 2048 },
+      "role" : <framework_role>,
+      "disk" : {
+        "source" : {
+          "type" : "PATH",
+          "path" : { "root" : "/mnt/data" }
+        }
+      }
+    }
+  ]
+}
+```
+
+#### `Mount` disks
+
+A `Mount` disk is an auxiliary disk resource provided by the operator. This
+__cannot__ be carved up into smaller chunks by frameworks. This lack of
+flexibility allows operators to provide assurances to frameworks that they will
+have exclusive access to the disk device. Common uses for this kind of disk
+include database storage, write-ahead logs, or other performance-critical
+applications.
+
+Another requirement of `Mount` disks is that (on Linux) they map to a `mount`
+point in the `/proc/mounts` table. Operators should mount a physical disk with
+their preferred file system and provide the mount point as the `root` of the
+`Mount` in `DiskInfo`'s `source`.
+
+Aside from the performance advantages of `Mount` disks, applications running on
+them should be able to rely on disk errors when they attempt to exceed the
+capacity of the volume. This holds true as long as the file system in use
+correctly propagates these errors. Due to this expectation, the `posix/disk`
+quota enforcement is disabled for `Mount` disks.
+
+```
+{
+  "resources" : [
+    {
+      "name" : "disk",
+      "type" : "SCALAR",
+      "scalar" : { "value" : 2048 },
+      "role" : <framework_role>,
+      "disk" : {
+        "source" : {
+          "type" : "MOUNT",
+          "mount" : { "root" : "/mnt/data" }
+        }
+      }
+    }
+  ]
+}
+```
+
+#### `Block` disks
+
+Mesos currently does not allow operators to expose raw block devices. It may do
+so in the future, but there are security and flexibility concerns that need to
+be addressed in a design document first.
+
+### Storage Management
+
+Mesos currently does not clean up or destroy data when persistent volumes are
+destroyed. It may do so in the future; however, the expectation is currently
+upon the framework, executor, and application to delete their data before
+destroying their persistent volumes. This is strongly encouraged for both
+security and ensuring that future users of the underlying disk resource are not
+penalized for prior consumption of the disk capacity.
+
+### Implementation
+
+A `Path` disk will have sub-directories created within the `root` which will be
+used to differentiate the different volumes that are created on it.
+
+A `Mount` disk will __not__ have sub-directories created, allowing applications
+to use the full file system mounted on the device. This provides operators a
+construct through which to enable data ingestion.
+
+Operators should be aware of these distinctions when inspecting or cleaning up
+remnant data.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/mesos/blob/efe8a2c6/docs/persistent-volume.md
----------------------------------------------------------------------
diff --git a/docs/persistent-volume.md b/docs/persistent-volume.md
index 3a3e370..2c6308f 100644
--- a/docs/persistent-volume.md
+++ b/docs/persistent-volume.md
@@ -25,6 +25,9 @@ cluster.
 Please refer to the [Reservation](reservation.md) documentation for details
 regarding reservation mechanisms available in Mesos.
 
+Persistent volumes can also be created on isolated and auxiliary disks by
+reserving [Multiple Disk resources](multiple-disk.md).
+
 Persistent volumes can be created by __operators__ and authorized
 __frameworks__. We require a `principal` from the operator or framework in order
 to authenticate/authorize the operations. Permissions are specified via the


[47/48] mesos git commit: Fixed variable shadowing in HookManager::slavePreLaunchDockerHook.

Posted by vi...@apache.org.
Fixed variable shadowing in HookManager::slavePreLaunchDockerHook.

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


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

Branch: refs/heads/0.27.x
Commit: c04f33c6fdd198dec974e61cf943bd8650b23f30
Parents: 6744167
Author: Kevin Devroede <ke...@gmail.com>
Authored: Fri Feb 26 23:28:17 2016 -0500
Committer: Michael Park <mp...@apache.org>
Committed: Fri Feb 26 20:36:24 2016 -0800

----------------------------------------------------------------------
 src/hook/manager.cpp | 4 ++--
 src/hook/manager.hpp | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c04f33c6/src/hook/manager.cpp
----------------------------------------------------------------------
diff --git a/src/hook/manager.cpp b/src/hook/manager.cpp
index 6ee9303..92be376 100644
--- a/src/hook/manager.cpp
+++ b/src/hook/manager.cpp
@@ -205,7 +205,7 @@ void HookManager::slavePreLaunchDockerHook(
     const CommandInfo& commandInfo,
     const Option<TaskInfo>& taskInfo,
     const ExecutorInfo& executorInfo,
-    const string& name,
+    const string& containerName,
     const string& sandboxDirectory,
     const string& mappedDirectory,
     const Option<Resources>& resources,
@@ -219,7 +219,7 @@ void HookManager::slavePreLaunchDockerHook(
           commandInfo,
           taskInfo,
           executorInfo,
-          name,
+          containerName,
           sandboxDirectory,
           mappedDirectory,
           resources,

http://git-wip-us.apache.org/repos/asf/mesos/blob/c04f33c6/src/hook/manager.hpp
----------------------------------------------------------------------
diff --git a/src/hook/manager.hpp b/src/hook/manager.hpp
index 8ab331a..3af28a7 100644
--- a/src/hook/manager.hpp
+++ b/src/hook/manager.hpp
@@ -59,7 +59,7 @@ public:
       const CommandInfo& commandInfo,
       const Option<TaskInfo>& taskInfo,
       const ExecutorInfo& executorInfo,
-      const std::string& name,
+      const std::string& containerName,
       const std::string& sandboxDirectory,
       const std::string& mappedDirectory,
       const Option<Resources>& resources,


[45/48] mesos git commit: Removed race condition from libevent based poll implementation.

Posted by vi...@apache.org.
Removed race condition from libevent based poll implementation.

Under certains circumstances, the future returned by poll is discarded
right after the event is triggered, this causes the event callback to be
called before the discard callback which results in an abort signal
being raised by the libevent library.

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


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

Branch: refs/heads/0.27.x
Commit: 859b4a3d711d71a048d5c1e6ef07a123007fa7fb
Parents: 5f15fef
Author: Alexander Rojas <al...@mesosphere.io>
Authored: Fri Feb 26 17:17:50 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Fri Feb 26 18:57:17 2016 -0800

----------------------------------------------------------------------
 3rdparty/libprocess/src/libevent_poll.cpp | 40 +++++++++++++++++++++-----
 1 file changed, 33 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/859b4a3d/3rdparty/libprocess/src/libevent_poll.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/libevent_poll.cpp b/3rdparty/libprocess/src/libevent_poll.cpp
index 461624c..43a1abb 100644
--- a/3rdparty/libprocess/src/libevent_poll.cpp
+++ b/3rdparty/libprocess/src/libevent_poll.cpp
@@ -12,6 +12,8 @@
 
 #include <event2/event.h>
 
+#include <memory>
+
 #include <process/future.hpp>
 #include <process/io.hpp>
 #include <process/process.hpp> // For process::initialize.
@@ -26,7 +28,7 @@ namespace internal {
 struct Poll
 {
   Promise<short> promise;
-  event* ev;
+  std::shared_ptr<event> ev;
 };
 
 
@@ -45,14 +47,26 @@ void pollCallback(evutil_socket_t, short what, void* arg)
     poll->promise.set(events);
   }
 
-  event_free(poll->ev);
+  // Deleting the `poll` also destructs `ev` and hence triggers `event_free`,
+  // which makes the event non-pending.
   delete poll;
 }
 
 
-void pollDiscard(event* ev)
+void pollDiscard(const std::weak_ptr<event>& ev, short events)
 {
-  event_active(ev, EV_READ, 0);
+  // Discarding inside the event loop prevents `pollCallback()` from being
+  // called twice if the future is discarded.
+  run_in_event_loop([=]() {
+    std::shared_ptr<event> shared = ev.lock();
+    // If `ev` cannot be locked `pollCallback` already ran. If it was locked
+    // but not pending, `pollCallback` is scheduled to be executed.
+    if (static_cast<bool>(shared) &&
+        event_pending(shared.get(), events, NULL)) {
+      // `event_active` will trigger the `pollCallback` to be executed.
+      event_active(shared.get(), EV_READ, 0);
+    }
+  });
 }
 
 } // namespace internal {
@@ -71,15 +85,27 @@ Future<short> poll(int fd, short events)
   short what =
     ((events & io::READ) ? EV_READ : 0) | ((events & io::WRITE) ? EV_WRITE : 0);
 
-  poll->ev = event_new(base, fd, what, &internal::pollCallback, poll);
+  // Bind `event_free` to the destructor of the `ev` shared pointer
+  // guaranteeing that the event will be freed only once.
+  poll->ev.reset(
+      event_new(base, fd, what, &internal::pollCallback, poll),
+      event_free);
+
   if (poll->ev == NULL) {
     LOG(FATAL) << "Failed to poll, event_new";
   }
 
-  event_add(poll->ev, NULL);
+  // Using a `weak_ptr` prevents `ev` to become a dangling pointer if
+  // the returned future is discarded after the event is triggered.
+  // The `weak_ptr` needs to be created before `event_add` in case
+  // the event is ready and the callback is executed before creating
+  // `ev`.
+  std::weak_ptr<event> ev(poll->ev);
+
+  event_add(poll->ev.get(), NULL);
 
   return future
-    .onDiscard(lambda::bind(&internal::pollDiscard, poll->ev));
+    .onDiscard(lambda::bind(&internal::pollDiscard, ev, what));
 }
 
 } // namespace io {


[25/48] mesos git commit: Added a missing file (`freebsd.hpp`) to stout's Makefile.am.

Posted by vi...@apache.org.
Added a missing file (`freebsd.hpp`) to stout's Makefile.am.

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


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

Branch: refs/heads/0.27.x
Commit: 2da5d5fba520c825e95f9a514105932c7113a70e
Parents: c154d85
Author: Neil Conway <ne...@gmail.com>
Authored: Thu Feb 4 11:11:59 2016 -0800
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:09 2016 -0500

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/stout/include/Makefile.am | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2da5d5fb/3rdparty/libprocess/3rdparty/stout/include/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/Makefile.am b/3rdparty/libprocess/3rdparty/stout/include/Makefile.am
index 1c452fb..f3a7b4b 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/stout/include/Makefile.am
@@ -33,6 +33,7 @@ nobase_include_HEADERS =		\
   stout/foreach.hpp			\
   stout/format.hpp			\
   stout/fs.hpp				\
+  stout/freebsd.hpp			\
   stout/gtest.hpp			\
   stout/gzip.hpp			\
   stout/hashmap.hpp			\
@@ -40,8 +41,8 @@ nobase_include_HEADERS =		\
   stout/internal/windows/dirent.hpp	\
   stout/internal/windows/grp.hpp	\
   stout/internal/windows/pwd.hpp	\
-  stout/internal/windows/symlink.hpp	\
   stout/internal/windows/reparsepoint.hpp	\
+  stout/internal/windows/symlink.hpp	\
   stout/interval.hpp			\
   stout/ip.hpp				\
   stout/json.hpp			\


[08/48] mesos git commit: Relaxed the subsystem check for cgroups net_cls isolator.

Posted by vi...@apache.org.
Relaxed the subsystem check for cgroups net_cls isolator.

The check ensures that only the net_cls and net_prio subsystem can be
mounted in the same hierarchy.

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


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

Branch: refs/heads/0.27.x
Commit: 502c2d35bb744db8c542d5f72429f9ebd024a0ce
Parents: 96acace
Author: Avinash sridharan <av...@mesosphere.io>
Authored: Wed Jan 27 17:22:59 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 20:04:51 2016 -0800

----------------------------------------------------------------------
 .../mesos/isolators/cgroups/net_cls.cpp         | 22 ++++++++++++++------
 1 file changed, 16 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/502c2d35/src/slave/containerizer/mesos/isolators/cgroups/net_cls.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/net_cls.cpp b/src/slave/containerizer/mesos/isolators/cgroups/net_cls.cpp
index 03a488e..ddc1bf0 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/net_cls.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/net_cls.cpp
@@ -72,16 +72,26 @@ Try<Isolator*> CgroupsNetClsIsolatorProcess::create(const Flags& flags)
     return Error("Failed to create net_cls cgroup: " + hierarchy.error());
   }
 
-  // Ensure that no other subsystem is attached to the hierarchy.
+  // Ensure that no unexpected subsystem is attached to the hierarchy.
   Try<set<string>> subsystems = cgroups::subsystems(hierarchy.get());
   if (subsystems.isError()) {
     return Error(
-        "Failed to get the list of attached subsystems for hierarchy " +
-        hierarchy.get());
+        "Failed to get the list of attached subsystems for hierarchy "
+        "'" + hierarchy.get() + "'");
   } else if (subsystems.get().size() != 1) {
-    return Error(
-        "Unexpected subsystems found attached to the hierarchy " +
-        hierarchy.get());
+    // Some Linux distributions mount net_cls and net_prio subsystems
+    // to the same hierarchy.
+    // TODO(jieyu): If we ever introduce a cgroups net_prio isolator,
+    // we need to make sure it will not conflict with this isolator if
+    // two subsystems are co-mounted into the same hierarchy. For
+    // instance, we should not remove a cgroup twice.
+    foreach (const string& subsystem, subsystems.get()) {
+      if (subsystem != "net_cls" && subsystem != "net_prio") {
+        return Error(
+            "Unexpected subsystems found attached to hierarchy "
+            "'" + hierarchy.get() + "'");
+      }
+    }
   }
 
   process::Owned<MesosIsolatorProcess> process(


[39/48] mesos git commit: Improved Multiple Disk documentation.

Posted by vi...@apache.org.
Improved Multiple Disk documentation.

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


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

Branch: refs/heads/0.27.x
Commit: 6c6195739a3d08851528912fd3a60ada31a14e93
Parents: efe8a2c
Author: Joerg Schad <jo...@mesosphere.io>
Authored: Tue Feb 16 18:02:48 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:40 2016 -0500

----------------------------------------------------------------------
 docs/multiple-disk.md     | 100 ++++++++++++++++++++---------------------
 docs/persistent-volume.md |   2 +-
 2 files changed, 49 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6c619573/docs/multiple-disk.md
----------------------------------------------------------------------
diff --git a/docs/multiple-disk.md b/docs/multiple-disk.md
index 4fc5327..ab2e7e8 100644
--- a/docs/multiple-disk.md
+++ b/docs/multiple-disk.md
@@ -28,18 +28,16 @@ A `Root` disk is the basic disk resource in Mesos. It usually maps to the
 storage on the main operating system drive that the operator has presented to
 the agent. Data is mapped into the `work_dir` of the agent.
 
-```
-{
-  "resources" : [
-    {
-      "name" : "disk",
-      "type" : "SCALAR",
-      "scalar" : { "value" : 2048 },
-      "role" : <framework_role>
-    }
-  ]
-}
-```
+        {
+          "resources" : [
+            {
+              "name" : "disk",
+              "type" : "SCALAR",
+              "scalar" : { "value" : 2048 },
+              "role" : <framework_role>
+            }
+          ]
+        }
 
 #### `Path` disks
 
@@ -51,28 +49,26 @@ Operators can present extra disks on their agents as `Path` disks simply by
 creating a directory and making that the `root` of the `Path` in `DiskInfo`'s
 `source`.
 
-`Path` disks are also useful for mocking up a multiple-disk environment by
+`Path` disks are also useful for mocking up a multiple disk environment by
 creating some directories on the operating system drive. This should only be
 done in a testing or staging environment.
 
-```
-{
-  "resources" : [
-    {
-      "name" : "disk",
-      "type" : "SCALAR",
-      "scalar" : { "value" : 2048 },
-      "role" : <framework_role>,
-      "disk" : {
-        "source" : {
-          "type" : "PATH",
-          "path" : { "root" : "/mnt/data" }
+        {
+          "resources" : [
+            {
+              "name" : "disk",
+              "type" : "SCALAR",
+              "scalar" : { "value" : 2048 },
+              "role" : <framework_role>,
+              "disk" : {
+                "source" : {
+                  "type" : "PATH",
+                  "path" : { "root" : "/mnt/data" }
+                }
+              }
+            }
+          ]
         }
-      }
-    }
-  ]
-}
-```
 
 #### `Mount` disks
 
@@ -92,26 +88,24 @@ Aside from the performance advantages of `Mount` disks, applications running on
 them should be able to rely on disk errors when they attempt to exceed the
 capacity of the volume. This holds true as long as the file system in use
 correctly propagates these errors. Due to this expectation, the `posix/disk`
-quota enforcement is disabled for `Mount` disks.
-
-```
-{
-  "resources" : [
-    {
-      "name" : "disk",
-      "type" : "SCALAR",
-      "scalar" : { "value" : 2048 },
-      "role" : <framework_role>,
-      "disk" : {
-        "source" : {
-          "type" : "MOUNT",
-          "mount" : { "root" : "/mnt/data" }
+isolation is disabled for `Mount` disks.
+
+        {
+          "resources" : [
+            {
+              "name" : "disk",
+              "type" : "SCALAR",
+              "scalar" : { "value" : 2048 },
+              "role" : <framework_role>,
+              "disk" : {
+                "source" : {
+                  "type" : "MOUNT",
+                  "mount" : { "root" : "/mnt/data" }
+                }
+              }
+            }
+          ]
         }
-      }
-    }
-  ]
-}
-```
 
 #### `Block` disks
 
@@ -126,7 +120,7 @@ destroyed. It may do so in the future; however, the expectation is currently
 upon the framework, executor, and application to delete their data before
 destroying their persistent volumes. This is strongly encouraged for both
 security and ensuring that future users of the underlying disk resource are not
-penalized for prior consumption of the disk capacity.
+penalized due to prior consumption of the disk capacity.
 
 ### Implementation
 
@@ -134,8 +128,10 @@ A `Path` disk will have sub-directories created within the `root` which will be
 used to differentiate the different volumes that are created on it.
 
 A `Mount` disk will __not__ have sub-directories created, allowing applications
-to use the full file system mounted on the device. This provides operators a
-construct through which to enable data ingestion.
+to use the full file system mounted on the device. This construct allows Mesos
+tasks to access volumes that contain pre-existing directory structures. This can
+be useful to simplify ingesting data such as a pre-existing Postgres database or
+HDFS data directory.
 
 Operators should be aware of these distinctions when inspecting or cleaning up
 remnant data.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/mesos/blob/6c619573/docs/persistent-volume.md
----------------------------------------------------------------------
diff --git a/docs/persistent-volume.md b/docs/persistent-volume.md
index 2c6308f..c1ecc82 100644
--- a/docs/persistent-volume.md
+++ b/docs/persistent-volume.md
@@ -26,7 +26,7 @@ Please refer to the [Reservation](reservation.md) documentation for details
 regarding reservation mechanisms available in Mesos.
 
 Persistent volumes can also be created on isolated and auxiliary disks by
-reserving [Multiple Disk resources](multiple-disk.md).
+reserving [multiple disk resources](multiple-disk.md).
 
 Persistent volumes can be created by __operators__ and authorized
 __frameworks__. We require a `principal` from the operator or framework in order


[27/48] mesos git commit: Extended life of logrotate processes on systemd.

Posted by vi...@apache.org.
Extended life of logrotate processes on systemd.

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


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

Branch: refs/heads/0.27.x
Commit: 56a1edf05e0f1b450e9e2461bb15246ef4a0ae66
Parents: 60dc7ab
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Fri Feb 5 14:52:18 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:10 2016 -0500

----------------------------------------------------------------------
 src/slave/container_loggers/lib_logrotate.cpp | 25 ++++++++++++++++++++--
 1 file changed, 23 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/56a1edf0/src/slave/container_loggers/lib_logrotate.cpp
----------------------------------------------------------------------
diff --git a/src/slave/container_loggers/lib_logrotate.cpp b/src/slave/container_loggers/lib_logrotate.cpp
index 01a3ff0..cf5f238 100644
--- a/src/slave/container_loggers/lib_logrotate.cpp
+++ b/src/slave/container_loggers/lib_logrotate.cpp
@@ -41,6 +41,10 @@
 #include <stout/os/fcntl.hpp>
 #include <stout/os/killtree.hpp>
 
+#ifdef __linux__
+#include "linux/systemd.hpp"
+#endif // __linux__
+
 #include "slave/container_loggers/logrotate.hpp"
 #include "slave/container_loggers/lib_logrotate.hpp"
 
@@ -119,6 +123,17 @@ public:
     outFlags.log_filename = path::join(sandboxDirectory, "stdout");
     outFlags.logrotate_path = flags.logrotate_path;
 
+    // If we are on systemd, then extend the life of the process as we
+    // do with the executor. Any grandchildren's lives will also be
+    // extended.
+    std::vector<Subprocess::Hook> parentHooks;
+#ifdef __linux__
+    if (systemd::enabled()) {
+      parentHooks.emplace_back(Subprocess::Hook(
+          &systemd::mesos::extendLifetime));
+    }
+#endif // __linux__
+
     Try<Subprocess> outProcess = subprocess(
         path::join(flags.launcher_dir, mesos::internal::logger::rotate::NAME),
         {mesos::internal::logger::rotate::NAME},
@@ -126,7 +141,10 @@ public:
         Subprocess::PATH("/dev/null"),
         Subprocess::FD(STDERR_FILENO),
         outFlags,
-        environment);
+        environment,
+        None(),
+        None(),
+        parentHooks);
 
     if (outProcess.isError()) {
       os::close(outfds.write.get());
@@ -170,7 +188,10 @@ public:
         Subprocess::PATH("/dev/null"),
         Subprocess::FD(STDERR_FILENO),
         errFlags,
-        environment);
+        environment,
+        None(),
+        None(),
+        parentHooks);
 
     if (errProcess.isError()) {
       os::close(outfds.write.get());


[15/48] mesos git commit: Updated `json` to handle integral types comprehensively.

Posted by vi...@apache.org.
Updated `json` to handle integral types comprehensively.

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


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

Branch: refs/heads/0.27.x
Commit: 94c8e5e2bee0052e57e54e2555d85e895e526b9a
Parents: 1ec0516
Author: Michael Park <mp...@apache.org>
Authored: Tue Jan 19 20:08:53 2016 -0800
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Mon Feb 15 16:15:23 2016 -0500

----------------------------------------------------------------------
 .../3rdparty/stout/include/stout/jsonify.hpp    | 103 +++++++++++++++----
 1 file changed, 83 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/94c8e5e2/3rdparty/libprocess/3rdparty/stout/include/stout/jsonify.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/jsonify.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/jsonify.hpp
index addec8e..f9d7224 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/jsonify.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/jsonify.hpp
@@ -128,7 +128,7 @@ class NumberWriter
 {
 public:
   NumberWriter(std::ostream* stream)
-    : stream_(stream), type_(INT64), int64_(0) {}
+    : stream_(stream), type_(INT), int_(0) {}
 
   NumberWriter(const NumberWriter&) = delete;
   NumberWriter(NumberWriter&&) = delete;
@@ -136,12 +136,12 @@ public:
   ~NumberWriter()
   {
     switch (type_) {
-      case INT64: {
-        *stream_ << int64_;
+      case INT: {
+        *stream_ << int_;
         break;
       }
-      case UINT64: {
-        *stream_ << uint64_;
+      case UINT: {
+        *stream_ << uint_;
         break;
       }
       case DOUBLE: {
@@ -170,20 +170,52 @@ public:
   NumberWriter& operator=(const NumberWriter&) = delete;
   NumberWriter& operator=(NumberWriter&&) = delete;
 
-  void set(int32_t value) { set(static_cast<int64_t>(value)); }
+  // NOTE 1: We enumerate overloads for all of the integral types here to avoid
+  // ambiguities between signed and unsigned conversions. If we were to only
+  // overload for `long long int` and `unsigned long long int`, passing an
+  // argument of `0` would be ambiguous since `0` has type `int`, and cost of
+  // conversion to `long long int` or `unsigned long long int` is equivalent.
 
-  void set(int64_t value)
+  // NOTE 2: We use the various modifiers on `int` as opposed to fixed size
+  // types such as `int32_t` and `int64_t` because these types do not cover all
+  // of the integral types. For example, `uint32_t` may map to `unsigned int`,
+  // and `uint64_t` to `unsigned long long int`. If `size_t` maps to `unsigned
+  // long int`, it is ambiguous to pass an instance of `size_t`. defining an
+  // overload for `size_t` would solve the problem on a specific platform, but
+  // we can run into issues again on another platform if `size_t` maps to
+  // `unsigned long long int`, since we would get a redefinition error.
+
+  void set(short int value) { set(static_cast<long long int>(value)); }
+
+  void set(int value) { set(static_cast<long long int>(value)); }
+
+  void set(long int value) { set(static_cast<long long int>(value)); }
+
+  void set(long long int value)
   {
-    type_ = INT64;
-    int64_ = value;
+    type_ = INT;
+    int_ = value;
   }
 
-  void set(uint32_t value) { set(static_cast<uint64_t>(value)); }
+  void set(unsigned short int value)
+  {
+    set(static_cast<unsigned long long int>(value));
+  }
+
+  void set(unsigned int value)
+  {
+    set(static_cast<unsigned long long int>(value));
+  }
 
-  void set(uint64_t value)
+  void set(unsigned long int value)
   {
-    type_ = UINT64;
-    uint64_ = value;
+    set(static_cast<unsigned long long int>(value));
+  }
+
+  void set(unsigned long long int value)
+  {
+    type_ = UINT;
+    uint_ = value;
   }
 
   void set(float value) { set(static_cast<double>(value)); }
@@ -197,12 +229,12 @@ public:
 private:
   std::ostream* stream_;
 
-  enum { INT64, UINT64, DOUBLE } type_;
+  enum { INT, UINT, DOUBLE } type_;
 
   union
   {
-    int64_t int64_;
-    uint64_t uint64_;
+    long long int int_;
+    unsigned long long int uint_;
     double double_;
   };
 };
@@ -336,10 +368,41 @@ inline void json(BooleanWriter* writer, bool value) { writer->set(value); }
 
 
 // `json` functions for numbers.
-inline void json(NumberWriter* writer, int32_t value) { writer->set(value); }
-inline void json(NumberWriter* writer, int64_t value) { writer->set(value); }
-inline void json(NumberWriter* writer, uint32_t value) { writer->set(value); }
-inline void json(NumberWriter* writer, uint64_t value) { writer->set(value); }
+inline void json(NumberWriter* writer, short int value) { writer->set(value); }
+inline void json(NumberWriter* writer, int value) { writer->set(value); }
+inline void json(NumberWriter* writer, long int value) { writer->set(value); }
+
+
+inline void json(NumberWriter* writer, long long int value)
+{
+  writer->set(value);
+}
+
+
+inline void json(NumberWriter* writer, unsigned short int value)
+{
+  writer->set(value);
+}
+
+
+inline void json(NumberWriter* writer, unsigned int value)
+{
+  writer->set(value);
+}
+
+
+inline void json(NumberWriter* writer, unsigned long int value)
+{
+  writer->set(value);
+}
+
+
+inline void json(NumberWriter* writer, unsigned long long int value)
+{
+  writer->set(value);
+}
+
+
 inline void json(NumberWriter* writer, float value) { writer->set(value); }
 inline void json(NumberWriter* writer, double value) { writer->set(value); }
 


[48/48] mesos git commit: Updated CHANGELOG for 0.27.2-rc1 release.

Posted by vi...@apache.org.
Updated CHANGELOG for 0.27.2-rc1 release.


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

Branch: refs/heads/0.27.x
Commit: 3c9ec4a0f34420b7803848af597de00fedefe0e2
Parents: c04f33c
Author: Michael Park <mp...@apache.org>
Authored: Fri Feb 26 19:07:20 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Fri Feb 26 20:37:15 2016 -0800

----------------------------------------------------------------------
 CHANGELOG | 13 +++++++++++++
 1 file changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3c9ec4a0/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 5dae7c4..e95e53e 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,3 +1,16 @@
+Release Notes - Mesos - Version 0.27.2
+--------------------------------------
+* This is a bug fix release.
+
+** Bug
+  * [MESOS-4693] - Variable shadowing in HookManager::slavePreLaunchDockerHook.
+  * [MESOS-4711] - Race condition in libevent poll implementation causes crash.
+  * [MESOS-4754] - The "executors" field is exposed under a backwards incompatible schema.
+
+** Improvement
+  * [MESOS-4687] - Implement reliable floating point for scalar resources.
+
+
 Release Notes - Mesos - Version 0.27.1
 --------------------------------------
 * This is a bug fix release.


[29/48] mesos git commit: Fix CGROUPS_ROOT_* tests on systemd platforms.

Posted by vi...@apache.org.
Fix CGROUPS_ROOT_* tests on systemd platforms.

Tests do not run with systemd configured, so any dependency on systemd
will fail some checks.

This fixes the `LinuxLauncher` to use the correct systemd-guard function.

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


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

Branch: refs/heads/0.27.x
Commit: 531cb60ebb538f7c4b7951170c324afe12f43f13
Parents: d04c2b3
Author: Joseph Wu <jo...@mesosphere.io>
Authored: Wed Feb 10 16:40:02 2016 -0800
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:10 2016 -0500

----------------------------------------------------------------------
 src/slave/containerizer/mesos/linux_launcher.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/531cb60e/src/slave/containerizer/mesos/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/linux_launcher.cpp b/src/slave/containerizer/mesos/linux_launcher.cpp
index a5546c7..caae2ec 100644
--- a/src/slave/containerizer/mesos/linux_launcher.cpp
+++ b/src/slave/containerizer/mesos/linux_launcher.cpp
@@ -111,7 +111,7 @@ Try<Launcher*> LinuxLauncher::create(const Flags& flags)
   return new LinuxLauncher(
       flags,
       freezerHierarchy.get(),
-      systemd::exists() ?
+      systemd::enabled() ?
         Some(systemd::hierarchy()) :
         Option<std::string>::none());
 }


[13/48] mesos git commit: Fixed missing include for V1 `Resources.cpp`.

Posted by vi...@apache.org.
Fixed missing include for V1 `Resources.cpp`.


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

Branch: refs/heads/0.27.x
Commit: c29b3b6801846cda270aab17a670542ec5d238f5
Parents: b0b978c
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Wed Jan 27 22:46:40 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 22:47:32 2016 -0800

----------------------------------------------------------------------
 src/v1/resources.cpp | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c29b3b68/src/v1/resources.cpp
----------------------------------------------------------------------
diff --git a/src/v1/resources.cpp b/src/v1/resources.cpp
index 969c0a6..be4a5d1 100644
--- a/src/v1/resources.cpp
+++ b/src/v1/resources.cpp
@@ -25,6 +25,8 @@
 
 #include <google/protobuf/repeated_field.h>
 
+#include <mesos/roles.hpp>
+
 #include <mesos/v1/mesos.hpp>
 #include <mesos/v1/resources.hpp>
 #include <mesos/v1/values.hpp>


[05/48] mesos git commit: Allocator Performance: Simplified Sorter's 'CalculateShare'.

Posted by vi...@apache.org.
Allocator Performance: Simplified Sorter's 'CalculateShare'.

Used existing functions to aggregate Scalars in the 'Resources' object.

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


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

Branch: refs/heads/0.27.x
Commit: 0869c2db6f32c462656402bda48aa678f303e017
Parents: b956e2c
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Wed Jan 27 19:14:21 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 20:04:51 2016 -0800

----------------------------------------------------------------------
 src/master/allocator/sorter/drf/sorter.cpp | 24 ++++++++++++++----------
 1 file changed, 14 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0869c2db/src/master/allocator/sorter/drf/sorter.cpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/sorter/drf/sorter.cpp b/src/master/allocator/sorter/drf/sorter.cpp
index 1f39f6d..db47d64 100644
--- a/src/master/allocator/sorter/drf/sorter.cpp
+++ b/src/master/allocator/sorter/drf/sorter.cpp
@@ -346,24 +346,28 @@ double DRFSorter::calculateShare(const string& name)
   // scalars.
 
   foreach (const string& scalar, total_.scalars.names()) {
-    double _total = 0.0;
-
+    // We collect the scalar accumulated total value from the
+    // `Resources` object.
+    //
     // NOTE: Scalar resources may be spread across multiple
     // 'Resource' objects. E.g. persistent volumes.
-    foreach (const Resource& resource, total_.scalars.get(scalar)) {
-      CHECK_EQ(resource.type(), Value::SCALAR);
-      _total += resource.scalar().value();
-    }
+    Option<Value::Scalar> __total = total_.scalars.get<Value::Scalar>(scalar);
+    CHECK_SOME(__total);
+    const double _total = __total.get().value();
 
     if (_total > 0.0) {
       double allocation = 0.0;
 
+      // We collect the scalar accumulated allocation value from the
+      // `Resources` object.
+      //
       // NOTE: Scalar resources may be spread across multiple
       // 'Resource' objects. E.g. persistent volumes.
-      foreach (const Resource& resource,
-               allocations[name].scalars.get(scalar)) {
-        CHECK_EQ(resource.type(), Value::SCALAR);
-        allocation += resource.scalar().value();
+      Option<Value::Scalar> _allocation =
+        allocations[name].scalars.get<Value::Scalar>(scalar);
+
+      if (_allocation.isSome()) {
+        allocation = _allocation.get().value();
       }
 
       share = std::max(share, allocation / _total);


[46/48] mesos git commit: Changed scalar resources to use fixed-point internally.

Posted by vi...@apache.org.
Changed scalar resources to use fixed-point internally.

Scalar resource values are represented using floating point. As a
result, users could see unexpected results when accepting offers and
making reservations for fractional resources: values like "0.1" cannot
be precisely represented using standard floating point, and the resource
values returned to frameworks might contain an unpredictable amount of
roundoff error.

This commit adjusts the master to use fixed-point when doing internal
computations on scalar resource values. The fixed-point format only
supports three decimal digits of precision: that is, fractional resource
values like "0.001" will be supported, but "0.0001" will not be.

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


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

Branch: refs/heads/0.27.x
Commit: 6744167423df77dae98b1f3cd599275f2579fc9a
Parents: 859b4a3
Author: Neil Conway <ne...@gmail.com>
Authored: Fri Feb 26 18:09:12 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Fri Feb 26 19:02:36 2016 -0800

----------------------------------------------------------------------
 docs/attributes-resources.md                |   4 +-
 docs/upgrades.md                            |   4 +
 include/mesos/mesos.proto                   |   8 ++
 include/mesos/v1/mesos.proto                |   8 ++
 src/common/resources.cpp                    |  23 +----
 src/common/values.cpp                       |  51 +++++++++--
 src/master/allocator/mesos/hierarchical.cpp |   4 +-
 src/tests/resources_tests.cpp               | 105 +++++++++++++++++++++--
 src/v1/resources.cpp                        |  23 +----
 src/v1/values.cpp                           |  51 +++++++++--
 10 files changed, 216 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/67441674/docs/attributes-resources.md
----------------------------------------------------------------------
diff --git a/docs/attributes-resources.md b/docs/attributes-resources.md
index 53a27f4..a264275 100644
--- a/docs/attributes-resources.md
+++ b/docs/attributes-resources.md
@@ -36,7 +36,9 @@ Attributes are key-value pairs (where value is optional) that Mesos passes along
 
 ## Resources
 
-The Mesos system can manage 3 different *types* of resources: scalars, ranges, and sets.  These are used to represent the different resources that a Mesos slave has to offer.  For example, a scalar resource type could be used to represent the amount of memory on a slave.  Resources can be specified either with a JSON array or a semicolon-delimited string of key:value pairs.  If, after examining the examples below, you have questions about the format of the JSON, inspect the `Resource` protobuf message definition in `include/mesos/mesos.proto`.
+Mesos can manage three different *types* of resources: scalars, ranges, and sets.  These are used to represent the different resources that a Mesos slave has to offer.  For example, a scalar resource type could be used to represent the amount of memory on a slave. Scalar resources are represented using floating point numbers to allow fractional values to be specified (e.g., "1.5 CPUs"). Mesos only supports three decimal digits of precision for scalar resources (e.g., reserving "1.5123 CPUs" is considered equivalent to reserving "1.512 CPUs").
+
+Resources can be specified either with a JSON array or a semicolon-delimited string of key-value pairs.  If, after examining the examples below, you have questions about the format of the JSON, inspect the `Resource` protobuf message definition in `include/mesos/mesos.proto`.
 
 As JSON:
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/67441674/docs/upgrades.md
----------------------------------------------------------------------
diff --git a/docs/upgrades.md b/docs/upgrades.md
index 4f30d72..57ee540 100644
--- a/docs/upgrades.md
+++ b/docs/upgrades.md
@@ -6,6 +6,10 @@ layout: documentation
 
 This document serves as a guide for users who wish to upgrade an existing Mesos cluster. Some versions require particular upgrade techniques when upgrading a running cluster. Some upgrades will have incompatible changes.
 
+## Upgrading from 0.26.0 or 0.27.x to 0.27.2 ##
+
+* Mesos 0.27.2 only supports three decimal digits of precision for scalar resource values. For example, frameworks can reserve "0.001" CPUs but more fine-grained reservations (e.g., "0.0001" CPUs) are no longer supported (although they did not work reliably in prior versions of Mesos anyway). Internally, resource math is now done using a fixed-point format that supports three decimal digits of precision, and then converted to/from floating point for input and output, respectively. Frameworks that do their own resource math and manipulate fractional resources may observe differences in roundoff error and numerical precision.
+
 ## Upgrading from 0.26.x to 0.27.x ##
 
 * Mesos 0.27 introduces the concept of _implicit roles_. In previous releases, configuring roles required specifying a static whitelist of valid role names on master startup (via the `--roles` flag). In Mesos 0.27, if `--roles` is omitted, _any_ role name can be used; controlling which principals are allowed to register as which roles should be done using [ACLs](authorization.md). The role whitelist functionality is still supported but is deprecated.

http://git-wip-us.apache.org/repos/asf/mesos/blob/67441674/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 96b911f..370531e 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -498,6 +498,14 @@ message Value {
   }
 
   message Scalar {
+    // Scalar values are represented using floating point. To reduce
+    // the chance of unpredictable floating point behavior due to
+    // roundoff error, Mesos only supports three decimal digits of
+    // precision for scalar resource values. That is, floating point
+    // values are converted to a fixed point format that supports
+    // three decimal digits of precision, and then converted back to
+    // floating point on output. Any additional precision in scalar
+    // resource values is discarded (via rounding).
     required double value = 1;
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/67441674/include/mesos/v1/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/mesos.proto b/include/mesos/v1/mesos.proto
index 0501dfa..9474cea 100644
--- a/include/mesos/v1/mesos.proto
+++ b/include/mesos/v1/mesos.proto
@@ -497,6 +497,14 @@ message Value {
   }
 
   message Scalar {
+    // Scalar values are represented using floating point. To reduce
+    // the chance of unpredictable floating point behavior due to
+    // roundoff error, Mesos only supports three decimal digits of
+    // precision for scalar resource values. That is, floating point
+    // values are converted to a fixed point format that supports
+    // three decimal digits of precision, and then converted back to
+    // floating point on output. Any additional precision in scalar
+    // resource values is discarded (via rounding).
     required double value = 1;
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/67441674/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index 588a279..7d0d7b1 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -37,11 +37,6 @@
 #include <stout/protobuf.hpp>
 #include <stout/strings.hpp>
 
-// TODO(bernd-mesos): Remove this interim dependency in the course of
-// solving MESOS-3997.
-#include <master/constants.hpp>
-
-
 using std::map;
 using std::ostream;
 using std::set;
@@ -50,9 +45,6 @@ using std::vector;
 
 using google::protobuf::RepeatedPtrField;
 
-using mesos::internal::master::MIN_CPUS;
-
-
 namespace mesos {
 
 /////////////////////////////////////////////////
@@ -1078,17 +1070,10 @@ Try<Resources> Resources::apply(const Offer::Operation& operation) const
   // TODO(jieyu): Currently, we only check known resource types like
   // cpus, mem, disk, ports, etc. We should generalize this.
 
-  CHECK(result.mem() == mem() &&
-        result.disk() == disk() &&
-        result.ports() == ports());
-
-  // This comparison is an interim fix - see MESOS-3552. We are making it
-  // reasonably certain that almost equal values are correctly regarded as
-  // equal. Small, usually acceptable, differences occur due to numeric
-  // operations such as unparsing and then parsing a floating point number.
-  // TODO(bernd-mesos): Of course, they might also accumulate, so we need a
-  // better long-term fix. Apply one here when solving MESOS-3997.
-  CHECK_NEAR(result.cpus().getOrElse(0.0), cpus().getOrElse(0.0), MIN_CPUS);
+  CHECK(result.cpus() == cpus());
+  CHECK(result.mem() == mem());
+  CHECK(result.disk() == disk());
+  CHECK(result.ports() == ports());
 
   return result;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/67441674/src/common/values.cpp
----------------------------------------------------------------------
diff --git a/src/common/values.cpp b/src/common/values.cpp
index c64407b..9f3c0b9 100644
--- a/src/common/values.cpp
+++ b/src/common/values.cpp
@@ -17,6 +17,7 @@
 #include <stdint.h>
 
 #include <algorithm>
+#include <cmath>
 #include <initializer_list>
 #include <ostream>
 #include <string>
@@ -42,50 +43,82 @@ using std::vector;
 
 namespace mesos {
 
+// We manipulate scalar values by converting them from floating point to a
+// fixed point representation, doing a calculation, and then converting
+// the result back to floating point. We deliberately only preserve three
+// decimal digits of precision in the fixed point representation. This
+// ensures that client applications see predictable numerical behavior, at
+// the expense of sacrificing some precision.
+
+static long long convertToFixed(double floatValue)
+{
+  return std::llround(floatValue * 1000);
+}
+
+
+static double convertToFloating(long long fixedValue)
+{
+  // NOTE: We do the conversion from fixed point via integer division
+  // and then modulus, rather than a single floating point division.
+  // This ensures that we only apply floating point division to inputs
+  // in the range [0,999], which is easier to check for correctness.
+  double quotient = static_cast<double>(fixedValue / 1000);
+  double remainder = static_cast<double>(fixedValue % 1000) / 1000.0;
+
+  return quotient + remainder;
+}
+
+
 ostream& operator<<(ostream& stream, const Value::Scalar& scalar)
 {
-  return stream << scalar.value();
+  // We discard any additional precision from scalar resources before
+  // writing them to an ostream. This is redundant when the scalar is
+  // obtained from one of the operators below, but user-specified
+  // resource values might contain additional precision.
+  return stream << convertToFloating(convertToFixed(scalar.value()));
 }
 
 
 bool operator==(const Value::Scalar& left, const Value::Scalar& right)
 {
-  return left.value() == right.value();
+  return convertToFixed(left.value()) == convertToFixed(right.value());
 }
 
 
 bool operator<=(const Value::Scalar& left, const Value::Scalar& right)
 {
-  return left.value() <= right.value();
+  return convertToFixed(left.value()) <= convertToFixed(right.value());
 }
 
 
 Value::Scalar operator+(const Value::Scalar& left, const Value::Scalar& right)
 {
-  Value::Scalar result;
-  result.set_value(left.value() + right.value());
+  Value::Scalar result = left;
+  result += right;
   return result;
 }
 
 
 Value::Scalar operator-(const Value::Scalar& left, const Value::Scalar& right)
 {
-  Value::Scalar result;
-  result.set_value(left.value() - right.value());
+  Value::Scalar result = left;
+  result -= right;
   return result;
 }
 
 
 Value::Scalar& operator+=(Value::Scalar& left, const Value::Scalar& right)
 {
-  left.set_value(left.value() + right.value());
+  long long sum = convertToFixed(left.value()) + convertToFixed(right.value());
+  left.set_value(convertToFloating(sum));
   return left;
 }
 
 
 Value::Scalar& operator-=(Value::Scalar& left, const Value::Scalar& right)
 {
-  left.set_value(left.value() - right.value());
+  long long diff = convertToFixed(left.value()) - convertToFixed(right.value());
+  left.set_value(convertToFloating(diff));
   return left;
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/67441674/src/master/allocator/mesos/hierarchical.cpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.cpp b/src/master/allocator/mesos/hierarchical.cpp
index 1a07d69..16b1945 100644
--- a/src/master/allocator/mesos/hierarchical.cpp
+++ b/src/master/allocator/mesos/hierarchical.cpp
@@ -883,9 +883,7 @@ void HierarchicalAllocatorProcess::recoverResources(
   // which it might not in the event that we dispatched Master::offer
   // before we received Allocator::removeSlave).
   if (slaves.contains(slaveId)) {
-    // NOTE: We cannot add the following CHECK due to the double
-    // precision errors. See MESOS-1187 for details.
-    // CHECK(slaves[slaveId].allocated.contains(resources));
+    CHECK(slaves[slaveId].allocated.contains(resources));
 
     slaves[slaveId].allocated -= resources;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/67441674/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index 4b25e82..00fa3a5 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -834,6 +834,37 @@ TEST(ResourcesTest, PrintingExtendedAttributes)
 }
 
 
+TEST(ResourcesTest, PrintingScalarPrecision)
+{
+  Resource scalar;
+  scalar.set_name("cpus");
+  scalar.set_type(Value::SCALAR);
+  scalar.mutable_scalar()->set_value(1.234);
+
+  // Three decimal digits of precision are supported.
+  ostringstream stream;
+  stream << scalar;
+  EXPECT_EQ("cpus(*):1.234", stream.str());
+
+  // Additional precision is discarded via rounding.
+  scalar.mutable_scalar()->set_value(1.2345);
+  stream.str("");
+  stream << scalar;
+  EXPECT_EQ("cpus(*):1.235", stream.str());
+
+  scalar.mutable_scalar()->set_value(1.2344);
+  stream.str("");
+  stream << scalar;
+  EXPECT_EQ("cpus(*):1.234", stream.str());
+
+  // Trailing zeroes are not printed.
+  scalar.mutable_scalar()->set_value(1.1);
+  stream.str("");
+  stream << scalar;
+  EXPECT_EQ("cpus(*):1.1", stream.str());
+}
+
+
 TEST(ResourcesTest, InitializedIsEmpty)
 {
   Resources r;
@@ -1525,15 +1556,79 @@ TEST(ResourcesTest, Types)
 }
 
 
-// NOTE: This is disabled due to MESOS-1187.
-TEST(ResourcesTest, DISABLED_Precision)
+TEST(ResourcesTest, PrecisionSimple)
 {
-  Resources cpu = Resources::parse("cpus:0.1").get();
+  Resources cpu = Resources::parse("cpus:1.001").get();
+  EXPECT_EQ(1.001, cpu.cpus().get());
 
   Resources r1 = cpu + cpu + cpu - cpu - cpu;
-  Resources r2 = cpu;
 
-  EXPECT_EQ(r1, r2);
+  EXPECT_EQ(cpu, r1);
+  EXPECT_EQ(1.001, r1.cpus().get());
+
+  Resources zero = Resources::parse("cpus:0").get();
+
+  EXPECT_EQ(cpu, cpu - zero);
+  EXPECT_EQ(cpu, cpu + zero);
+}
+
+
+TEST(ResourcesTest, PrecisionManyOps)
+{
+  Resources start = Resources::parse("cpus:1.001").get();
+  Resources current = start;
+  Resources next;
+
+  for (int i = 0; i < 2500; i++) {
+    next = current + current + current - current - current;
+    EXPECT_EQ(1.001, next.cpus().get());
+    EXPECT_EQ(current, next);
+    EXPECT_EQ(start, next);
+    current = next;
+  }
+}
+
+
+TEST(ResourcesTest, PrecisionManyConsecutiveOps)
+{
+  Resources start = Resources::parse("cpus:1.001").get();
+  Resources increment = start;
+  Resources current = start;
+
+  for (int i = 0; i < 100000; i++) {
+    current += increment;
+  }
+
+  for (int i = 0; i < 100000; i++) {
+    current -= increment;
+  }
+
+  EXPECT_EQ(start, current);
+}
+
+
+TEST(ResourcesTest, PrecisionLost)
+{
+  Resources cpu = Resources::parse("cpus:1.5011").get();
+  EXPECT_EQ(1.501, cpu.cpus().get());
+
+  Resources r1 = cpu + cpu + cpu - cpu - cpu;
+
+  EXPECT_EQ(cpu, r1);
+  EXPECT_EQ(1.501, r1.cpus().get());
+}
+
+
+TEST(ResourcesTest, PrecisionRounding)
+{
+  // Round up (away from zero) at the half-way point.
+  Resources cpu = Resources::parse("cpus:1.5015").get();
+  EXPECT_EQ(1.502, cpu.cpus().get());
+
+  Resources r1 = cpu + cpu + cpu - cpu - cpu;
+
+  EXPECT_EQ(cpu, r1);
+  EXPECT_EQ(1.502, r1.cpus().get());
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/67441674/src/v1/resources.cpp
----------------------------------------------------------------------
diff --git a/src/v1/resources.cpp b/src/v1/resources.cpp
index be4a5d1..92a288d 100644
--- a/src/v1/resources.cpp
+++ b/src/v1/resources.cpp
@@ -38,11 +38,6 @@
 #include <stout/protobuf.hpp>
 #include <stout/strings.hpp>
 
-// TODO(bernd-mesos): Remove this interim dependency in the course of
-// solving MESOS-3997.
-#include <master/constants.hpp>
-
-
 using std::map;
 using std::ostream;
 using std::set;
@@ -51,9 +46,6 @@ using std::vector;
 
 using google::protobuf::RepeatedPtrField;
 
-using mesos::internal::master::MIN_CPUS;
-
-
 namespace mesos {
 namespace v1 {
 
@@ -1081,17 +1073,10 @@ Try<Resources> Resources::apply(const Offer::Operation& operation) const
   // TODO(jieyu): Currently, we only check known resource types like
   // cpus, mem, disk, ports, etc. We should generalize this.
 
-  CHECK(result.mem() == mem() &&
-        result.disk() == disk() &&
-        result.ports() == ports());
-
-  // This comparison is an interim fix - see MESOS-3552. We are making it
-  // reasonably certain that almost equal values are correctly regarded as
-  // equal. Small, usually acceptable, differences occur due to numeric
-  // operations such as unparsing and then parsing a floating point number.
-  // TODO(bernd-mesos): Of course, they might also accumulate, so we need a
-  // better long-term fix. Apply one here when solving MESOS-3997.
-  CHECK_NEAR(result.cpus().getOrElse(0.0), cpus().getOrElse(0.0), MIN_CPUS);
+  CHECK(result.cpus() == cpus());
+  CHECK(result.mem() == mem());
+  CHECK(result.disk() == disk());
+  CHECK(result.ports() == ports());
 
   return result;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/67441674/src/v1/values.cpp
----------------------------------------------------------------------
diff --git a/src/v1/values.cpp b/src/v1/values.cpp
index 86094b3..28f7e40 100644
--- a/src/v1/values.cpp
+++ b/src/v1/values.cpp
@@ -17,6 +17,7 @@
 #include <stdint.h>
 
 #include <algorithm>
+#include <cmath>
 #include <initializer_list>
 #include <ostream>
 #include <string>
@@ -43,50 +44,82 @@ using std::vector;
 namespace mesos {
 namespace v1 {
 
+// We manipulate scalar values by converting them from floating point to a
+// fixed point representation, doing a calculation, and then converting
+// the result back to floating point. We deliberately only preserve three
+// decimal digits of precision in the fixed point representation. This
+// ensures that client applications see predictable numerical behavior, at
+// the expense of sacrificing some precision.
+
+static long long convertToFixed(double floatValue)
+{
+  return std::llround(floatValue * 1000);
+}
+
+
+static double convertToFloating(long long fixedValue)
+{
+  // NOTE: We do the conversion from fixed point via integer division
+  // and then modulus, rather than a single floating point division.
+  // This ensures that we only apply floating point division to inputs
+  // in the range [0,999], which is easier to check for correctness.
+  double quotient = static_cast<double>(fixedValue / 1000);
+  double remainder = static_cast<double>(fixedValue % 1000) / 1000.0;
+
+  return quotient + remainder;
+}
+
+
 ostream& operator<<(ostream& stream, const Value::Scalar& scalar)
 {
-  return stream << scalar.value();
+  // We discard any additional precision from scalar resources before
+  // writing them to an ostream. This is redundant when the scalar is
+  // obtained from one of the operators below, but user-specified
+  // resource values might contain additional precision.
+  return stream << convertToFloating(convertToFixed(scalar.value()));
 }
 
 
 bool operator==(const Value::Scalar& left, const Value::Scalar& right)
 {
-  return left.value() == right.value();
+  return convertToFixed(left.value()) == convertToFixed(right.value());
 }
 
 
 bool operator<=(const Value::Scalar& left, const Value::Scalar& right)
 {
-  return left.value() <= right.value();
+  return convertToFixed(left.value()) <= convertToFixed(right.value());
 }
 
 
 Value::Scalar operator+(const Value::Scalar& left, const Value::Scalar& right)
 {
-  Value::Scalar result;
-  result.set_value(left.value() + right.value());
+  Value::Scalar result = left;
+  result += right;
   return result;
 }
 
 
 Value::Scalar operator-(const Value::Scalar& left, const Value::Scalar& right)
 {
-  Value::Scalar result;
-  result.set_value(left.value() - right.value());
+  Value::Scalar result = left;
+  result -= right;
   return result;
 }
 
 
 Value::Scalar& operator+=(Value::Scalar& left, const Value::Scalar& right)
 {
-  left.set_value(left.value() + right.value());
+  long long sum = convertToFixed(left.value()) + convertToFixed(right.value());
+  left.set_value(convertToFloating(sum));
   return left;
 }
 
 
 Value::Scalar& operator-=(Value::Scalar& left, const Value::Scalar& right)
 {
-  left.set_value(left.value() - right.value());
+  long long diff = convertToFixed(left.value()) - convertToFixed(right.value());
+  left.set_value(convertToFloating(diff));
   return left;
 }
 


[20/48] mesos git commit: Renamed a parameter for the sake of clarity.

Posted by vi...@apache.org.
Renamed a parameter for the sake of clarity.

We use several different "timeouts" in this code and the
"Zk session timeout" has a very specific meaning in this context.

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


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

Branch: refs/heads/0.27.x
Commit: 67e6bcce2c8f8d858e48bd455e9d4c4b8493a5c4
Parents: 4ecca83
Author: Neil Conway <ne...@gmail.com>
Authored: Sat Jan 30 20:01:59 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Mon Feb 15 16:15:23 2016 -0500

----------------------------------------------------------------------
 src/zookeeper/group.cpp     | 20 ++++++++++----------
 src/zookeeper/group.hpp     | 12 ++++++------
 src/zookeeper/zookeeper.cpp | 12 ++++++------
 src/zookeeper/zookeeper.hpp |  6 ++++--
 4 files changed, 26 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/67e6bcce/src/zookeeper/group.cpp
----------------------------------------------------------------------
diff --git a/src/zookeeper/group.cpp b/src/zookeeper/group.cpp
index 2ae3193..add01a3 100644
--- a/src/zookeeper/group.cpp
+++ b/src/zookeeper/group.cpp
@@ -87,12 +87,12 @@ void discard(queue<T*>* queue)
 
 GroupProcess::GroupProcess(
     const string& _servers,
-    const Duration& _timeout,
+    const Duration& _sessionTimeout,
     const string& _znode,
     const Option<Authentication>& _auth)
   : ProcessBase(ID::generate("group")),
     servers(_servers),
-    timeout(_timeout),
+    sessionTimeout(_sessionTimeout),
     znode(strings::remove(_znode, "/", strings::SUFFIX)),
     auth(_auth),
     acl(_auth.isSome()
@@ -109,10 +109,10 @@ GroupProcess::GroupProcess(
 // C++ 11.
 GroupProcess::GroupProcess(
     const URL& url,
-    const Duration& _timeout)
+    const Duration& _sessionTimeout)
   : ProcessBase(ID::generate("group")),
     servers(url.servers),
-    timeout(_timeout),
+    sessionTimeout(_sessionTimeout),
     znode(strings::remove(url.path, "/", strings::SUFFIX)),
     auth(url.authentication),
     acl(url.authentication.isSome()
@@ -142,7 +142,7 @@ void GroupProcess::initialize()
   // Doing initialization here allows to avoid the race between
   // instantiating the ZooKeeper instance and being spawned ourself.
   watcher = new ProcessWatcher<GroupProcess>(self());
-  zk = new ZooKeeper(servers, timeout, watcher);
+  zk = new ZooKeeper(servers, sessionTimeout, watcher);
   state = CONNECTING;
 }
 
@@ -530,7 +530,7 @@ void GroupProcess::expired(int64_t sessionId)
   delete CHECK_NOTNULL(zk);
   delete CHECK_NOTNULL(watcher);
   watcher = new ProcessWatcher<GroupProcess>(self());
-  zk = new ZooKeeper(servers, timeout, watcher);
+  zk = new ZooKeeper(servers, sessionTimeout, watcher);
 
   state = CONNECTING;
 }
@@ -969,19 +969,19 @@ string GroupProcess::zkBasename(const Group::Membership& membership)
 
 
 Group::Group(const string& servers,
-             const Duration& timeout,
+             const Duration& sessionTimeout,
              const string& znode,
              const Option<Authentication>& auth)
 {
-  process = new GroupProcess(servers, timeout, znode, auth);
+  process = new GroupProcess(servers, sessionTimeout, znode, auth);
   spawn(process);
 }
 
 
 Group::Group(const URL& url,
-             const Duration& timeout)
+             const Duration& sessionTimeout)
 {
-  process = new GroupProcess(url, timeout);
+  process = new GroupProcess(url, sessionTimeout);
   spawn(process);
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/67e6bcce/src/zookeeper/group.hpp
----------------------------------------------------------------------
diff --git a/src/zookeeper/group.hpp b/src/zookeeper/group.hpp
index cf82fec..ed5d0a0 100644
--- a/src/zookeeper/group.hpp
+++ b/src/zookeeper/group.hpp
@@ -121,13 +121,13 @@ public:
   };
 
   // Constructs this group using the specified ZooKeeper servers (list
-  // of host:port) with the given timeout at the specified znode.
+  // of host:port) with the given session timeout at the specified znode.
   Group(const std::string& servers,
-        const Duration& timeout,
+        const Duration& sessionTimeout,
         const std::string& znode,
         const Option<Authentication>& auth = None());
   Group(const URL& url,
-        const Duration& timeout);
+        const Duration& sessionTimeout);
 
   ~Group();
 
@@ -170,12 +170,12 @@ class GroupProcess : public process::Process<GroupProcess>
 {
 public:
   GroupProcess(const std::string& servers,
-               const Duration& timeout,
+               const Duration& sessionTimeout,
                const std::string& znode,
                const Option<Authentication>& auth);
 
   GroupProcess(const URL& url,
-               const Duration& timeout);
+               const Duration& sessionTimeout);
 
   virtual ~GroupProcess();
 
@@ -256,7 +256,7 @@ private:
   const std::string servers;
 
   // The session timeout requested by the client.
-  const Duration timeout;
+  const Duration sessionTimeout;
 
   const std::string znode;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/67e6bcce/src/zookeeper/zookeeper.cpp
----------------------------------------------------------------------
diff --git a/src/zookeeper/zookeeper.cpp b/src/zookeeper/zookeeper.cpp
index 3c4fdad..790bd15 100644
--- a/src/zookeeper/zookeeper.cpp
+++ b/src/zookeeper/zookeeper.cpp
@@ -50,11 +50,11 @@ public:
   ZooKeeperProcess(
       ZooKeeper* zk,
       const string& servers,
-      const Duration& timeout,
+      const Duration& sessionTimeout,
       Watcher* watcher)
     : ProcessBase(ID::generate("zookeeper")),
       servers(servers),
-      timeout(timeout),
+      sessionTimeout(sessionTimeout),
       zh(NULL)
   {
     // We bind the Watcher::process callback so we can pass it to the
@@ -84,7 +84,7 @@ public:
       zh = zookeeper_init(
           servers.c_str(),
           event,
-          static_cast<int>(timeout.ms()),
+          static_cast<int>(sessionTimeout.ms()),
           NULL,
           &callback,
           0);
@@ -511,7 +511,7 @@ private:
   friend class ZooKeeper;
 
   const string servers; // ZooKeeper host:port pairs.
-  const Duration timeout; // ZooKeeper session timeout;
+  const Duration sessionTimeout; // ZooKeeper session timeout.
 
   zhandle_t* zh; // ZooKeeper connection handle.
 
@@ -523,10 +523,10 @@ private:
 
 ZooKeeper::ZooKeeper(
     const string& servers,
-    const Duration& timeout,
+    const Duration& sessionTimeout,
     Watcher* watcher)
 {
-  process = new ZooKeeperProcess(this, servers, timeout, watcher);
+  process = new ZooKeeperProcess(this, servers, sessionTimeout, watcher);
   spawn(process);
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/67e6bcce/src/zookeeper/zookeeper.hpp
----------------------------------------------------------------------
diff --git a/src/zookeeper/zookeeper.hpp b/src/zookeeper/zookeeper.hpp
index 573ff5b..a29a004 100644
--- a/src/zookeeper/zookeeper.hpp
+++ b/src/zookeeper/zookeeper.hpp
@@ -122,7 +122,7 @@ public:
    *    method will be invoked.
    */
   ZooKeeper(const std::string& servers,
-            const Duration& timeout,
+            const Duration& sessionTimeout,
             Watcher* watcher);
 
   ~ZooKeeper();
@@ -145,7 +145,9 @@ public:
    * \brief get the current session timeout.
    *
    * The session timeout requested by the client or the negotiated
-   * session timeout after the session is established with ZooKeeper.
+   * session timeout after the session is established with
+   * ZooKeeper. Note that this might differ from the initial
+   * `sessionTimeout` specified when this instance was constructed.
    */
   Duration getSessionTimeout() const;
 


[43/48] mesos git commit: Updated Mesos version to 0.27.2.

Posted by vi...@apache.org.
Updated Mesos version to 0.27.2.


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

Branch: refs/heads/0.27.x
Commit: c832e4abceadd0ac4e52f1090a8ea4a574a84b11
Parents: 864fe8e
Author: Michael Park <mp...@apache.org>
Authored: Fri Feb 26 18:56:14 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Fri Feb 26 18:56:14 2016 -0800

----------------------------------------------------------------------
 CMakeLists.txt | 2 +-
 configure.ac   | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c832e4ab/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 53a44c5..3c86e27 100755
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -21,7 +21,7 @@ cmake_minimum_required(VERSION 2.8)
 project(Mesos)
 set(MESOS_MAJOR_VERSION 0)
 set(MESOS_MINOR_VERSION 27)
-set(MESOS_PATCH_VERSION 1)
+set(MESOS_PATCH_VERSION 2)
 set(PACKAGE_VERSION
   ${MESOS_MAJOR_VERSION}.${MESOS_MINOR_VERSION}.${MESOS_PATCH_VERSION})
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c832e4ab/configure.ac
----------------------------------------------------------------------
diff --git a/configure.ac b/configure.ac
index 259a4f0..9ed181f 100644
--- a/configure.ac
+++ b/configure.ac
@@ -18,7 +18,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.61])
-AC_INIT([mesos], [0.27.1])
+AC_INIT([mesos], [0.27.2])
 
 # Have autoconf setup some variables related to the system.
 AC_CANONICAL_HOST


[34/48] mesos git commit: Added flag to disable systemd support.

Posted by vi...@apache.org.
Added flag to disable systemd support.

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


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

Branch: refs/heads/0.27.x
Commit: 8afbcc39d55a1e4525298098c0b74a260f5b8755
Parents: 531cb60
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Mon Feb 15 12:30:27 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:10 2016 -0500

----------------------------------------------------------------------
 docs/configuration.md | 12 ++++++++++++
 src/linux/systemd.cpp | 15 ++++++++++++++-
 src/linux/systemd.hpp |  1 +
 src/slave/flags.cpp   |  8 ++++++++
 src/slave/flags.hpp   |  1 +
 src/slave/main.cpp    |  3 ++-
 6 files changed, 38 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8afbcc39/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index b1ef131..29a6b65 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -866,6 +866,18 @@ file:///path/to/file (where file contains one of the above)</code></pre>
   </tr>
   <tr>
     <td>
+      --[no-]systemd_enable_support
+    </td>
+    <td>
+      Top level control of systemd support. When enabled, features such as
+      executor life-time extension are enabled unless there is an explicit
+      flag to disable these (see other flags). This should be enabled when
+      the agent is launched as a systemd unit.
+      (default: true)
+    </td>
+  </tr>
+  <tr>
+    <td>
       --systemd_runtime_directory=VALUE
     </td>
     <td>

http://git-wip-us.apache.org/repos/asf/mesos/blob/8afbcc39/src/linux/systemd.cpp
----------------------------------------------------------------------
diff --git a/src/linux/systemd.cpp b/src/linux/systemd.cpp
index 0142caf..0a9baa7 100644
--- a/src/linux/systemd.cpp
+++ b/src/linux/systemd.cpp
@@ -39,6 +39,13 @@ int DELEGATE_MINIMUM_VERSION = 218;
 
 Flags::Flags()
 {
+  add(&Flags::enabled,
+      "enabled",
+      "Top level control of systemd support. When enabled, features such as\n"
+      "processes life-time extension are enabled unless there is an explicit\n"
+      "flag to disable these (see other flags).",
+      true);
+
   add(&Flags::runtime_directory,
       "runtime_directory",
       "The path to the systemd system run time directory\n",
@@ -108,6 +115,12 @@ Try<Nothing> initialize(const Flags& flags)
 
   systemd_flags = new Flags(flags);
 
+  // Do not initialize any state if we do not have systemd support enabled.
+  if (!systemd_flags->enabled) {
+    initialized->done();
+    return Nothing();
+  }
+
   // If flags->runtime_directory doesn't exist, then we can't proceed.
   if (!os::exists(CHECK_NOTNULL(systemd_flags)->runtime_directory)) {
     return Error("Failed to locate systemd runtime directory: " +
@@ -245,7 +258,7 @@ bool exists()
 
 bool enabled()
 {
-  return exists() && systemd_flags != NULL;
+  return systemd_flags != NULL && flags().enabled && exists();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/8afbcc39/src/linux/systemd.hpp
----------------------------------------------------------------------
diff --git a/src/linux/systemd.hpp b/src/linux/systemd.hpp
index d868fda..6b240b9 100644
--- a/src/linux/systemd.hpp
+++ b/src/linux/systemd.hpp
@@ -59,6 +59,7 @@ class Flags : public virtual flags::FlagsBase
 public:
   Flags();
 
+  bool enabled;
   std::string runtime_directory;
   std::string cgroups_hierarchy;
 };

http://git-wip-us.apache.org/repos/asf/mesos/blob/8afbcc39/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index 75d7429..2e88d74 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -386,6 +386,14 @@ mesos::internal::slave::Flags::Flags()
       "supported by the cgroups/cpu isolator.",
       true);
 
+  add(&Flags::systemd_enable_support,
+      "systemd_enable_support",
+      "Top level control of systemd support. When enabled, features such as\n"
+      "executor life-time extension are enabled unless there is an explicit\n"
+      "flag to disable these (see other flags). This should be enabled when\n"
+      "the agent is launched as a systemd unit.",
+      true);
+
   add(&Flags::systemd_runtime_directory,
       "systemd_runtime_directory",
       "The path to the systemd system run time directory\n",

http://git-wip-us.apache.org/repos/asf/mesos/blob/8afbcc39/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 23ec158..1293397 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -92,6 +92,7 @@ public:
   Duration perf_interval;
   Duration perf_duration;
   bool revocable_cpu_low_priority;
+  bool systemd_enable_support;
   std::string systemd_runtime_directory;
 #endif
   Option<Firewall> firewall_rules;

http://git-wip-us.apache.org/repos/asf/mesos/blob/8afbcc39/src/slave/main.cpp
----------------------------------------------------------------------
diff --git a/src/slave/main.cpp b/src/slave/main.cpp
index ff406ac..a5d5976 100644
--- a/src/slave/main.cpp
+++ b/src/slave/main.cpp
@@ -228,10 +228,11 @@ int main(int argc, char** argv)
 
 #ifdef __linux__
   // Initialize systemd if it exists.
-  if (systemd::exists()) {
+  if (systemd::exists() && flags.systemd_enable_support) {
     LOG(INFO) << "Inializing systemd state";
 
     systemd::Flags systemdFlags;
+    systemdFlags.enabled = flags.systemd_enable_support;
     systemdFlags.runtime_directory = flags.systemd_runtime_directory;
     systemdFlags.cgroups_hierarchy = flags.cgroups_hierarchy;
 


[11/48] mesos git commit: Synchronized V1 API for `resources.cpp`.

Posted by vi...@apache.org.
Synchronized V1 API for `resources.cpp`.


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

Branch: refs/heads/0.27.x
Commit: d0e81458a2ecf40b159eee2856ad05ab68c545fc
Parents: 45be8ef
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Wed Jan 27 22:31:03 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 22:42:52 2016 -0800

----------------------------------------------------------------------
 src/v1/resources.cpp | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d0e81458/src/v1/resources.cpp
----------------------------------------------------------------------
diff --git a/src/v1/resources.cpp b/src/v1/resources.cpp
index fc7c885..969c0a6 100644
--- a/src/v1/resources.cpp
+++ b/src/v1/resources.cpp
@@ -707,6 +707,12 @@ Option<Error> Resources::validate(const Resource& resource)
         "Invalid reservation: role \"*\" cannot be dynamically reserved");
   }
 
+  // Check role name.
+  Option<Error> error = roles::validate(resource.role());
+  if (error.isSome()) {
+    return error;
+  }
+
   return None();
 }
 


[03/48] mesos git commit: Updated CHANGELOG for 0.27.0-rc2 release.

Posted by vi...@apache.org.
Updated CHANGELOG for 0.27.0-rc2 release.


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

Branch: refs/heads/0.27.x
Commit: 45be8ef26e3cba868fd6901d1421078d8f034373
Parents: 12d0cf4
Author: Michael Park <mp...@apache.org>
Authored: Wed Jan 27 20:02:21 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 20:04:51 2016 -0800

----------------------------------------------------------------------
 CHANGELOG | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/45be8ef2/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 78e7796..82c1be6 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -80,6 +80,11 @@ Release Notes - Mesos - Version 0.27.0
   * [MESOS-4449] - SegFault on agent during executor startup
   * [MESOS-4507] - Replace busybox image with alpine in Docker tests
   * [MESOS-4515] - ContainerLoggerTest.LOGROTATE_RotateInSandbox breaks when running on Centos6.
+  * [MESOS-4530] - NetClsIsolatorTest.ROOT_CGROUPS_NetClsIsolate is flaky
+  * [MESOS-4533] - DiskUsageCollectorTest.ExcludeRelativePath fails on Linux
+  * [MESOS-4534] - Resources object can be mutated through the public API
+  * [MESOS-4535] - Logrotate ContainerLogger may not handle FD ownership correctly
+  * [MESOS-4539] - Exclude paths in Posix disk isolator should be absolute paths.
 
 ** Documentation
   * [MESOS-3581] - License headers show up all over doxygen documentation.


[21/48] mesos git commit: Ignored empty NetworkSettings.IPAddress from docker inspect.

Posted by vi...@apache.org.
Ignored empty NetworkSettings.IPAddress from docker inspect.

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


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

Branch: refs/heads/0.27.x
Commit: 4ecca83ce6ad6ec55b912dbaa06867f773b5a1e9
Parents: 46b4769
Author: Jie Yu <yu...@gmail.com>
Authored: Fri Jan 29 17:24:14 2016 -0800
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Mon Feb 15 16:15:23 2016 -0500

----------------------------------------------------------------------
 src/docker/docker.cpp | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/4ecca83c/src/docker/docker.cpp
----------------------------------------------------------------------
diff --git a/src/docker/docker.cpp b/src/docker/docker.cpp
index 4d2f1fa..a831726 100755
--- a/src/docker/docker.cpp
+++ b/src/docker/docker.cpp
@@ -311,7 +311,10 @@ Try<Docker::Container> Docker::Container::create(const string& output)
         ipAddressValue.error());
   }
 
-  string ipAddress = ipAddressValue.get().value;
+  Option<string> ipAddress;
+  if (!ipAddressValue->value.empty()) {
+    ipAddress = ipAddressValue->value;
+  }
 
   return Docker::Container(output, id, name, optionalPid, started, ipAddress);
 }


[37/48] mesos git commit: Migrated linux launcher systemd executor logic into subprocess hook.

Posted by vi...@apache.org.
Migrated linux launcher systemd executor logic into subprocess hook.

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


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

Branch: refs/heads/0.27.x
Commit: e83e63adcade67d3aad97986cac575c259976947
Parents: bf6f190
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Fri Feb 5 10:32:48 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:10 2016 -0500

----------------------------------------------------------------------
 src/linux/systemd.cpp                           | 26 +++++++++++++++-
 .../containerizer/mesos/linux_launcher.cpp      | 32 +++++---------------
 2 files changed, 33 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e83e63ad/src/linux/systemd.cpp
----------------------------------------------------------------------
diff --git a/src/linux/systemd.cpp b/src/linux/systemd.cpp
index 69e8b39..0142caf 100644
--- a/src/linux/systemd.cpp
+++ b/src/linux/systemd.cpp
@@ -64,7 +64,31 @@ namespace mesos {
 
 Try<Nothing> extendLifetime(pid_t child)
 {
-  // TODO(jmlvanre): Implement pid migration into systemd slice.
+  if (!systemd::exists()) {
+    return Error("systemd does not exist on this system");
+  }
+
+  if (!systemd::enabled()) {
+    return Error("systemd is not enabled on this system");
+  }
+
+  Try<Nothing> assign = cgroups::assign(
+      hierarchy(),
+      systemd::mesos::MESOS_EXECUTORS_SLICE,
+      child);
+
+  if (assign.isError()) {
+    LOG(ERROR) << "Failed to assign process " << child
+                << " to its systemd executor slice: " << assign.error();
+
+    ::kill(child, SIGKILL);
+    return Error("Failed to contain process on systemd");
+  }
+
+  LOG(INFO) << "Assigned child process '" << child << "' to '"
+            << systemd::mesos::MESOS_EXECUTORS_SLICE << "'";
+
+  return Nothing();
 }
 
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/e83e63ad/src/slave/containerizer/mesos/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/linux_launcher.cpp b/src/slave/containerizer/mesos/linux_launcher.cpp
index 90f8c73..a5546c7 100644
--- a/src/slave/containerizer/mesos/linux_launcher.cpp
+++ b/src/slave/containerizer/mesos/linux_launcher.cpp
@@ -304,6 +304,13 @@ Try<pid_t> LinuxLauncher::fork(
   LOG(INFO) << "Cloning child process with flags = "
             << ns::stringify(cloneFlags);
 
+  // If we are on systemd, then extend the life of the child. As with the
+  // freezer, any grandchildren will also be contained in the slice.
+  std::vector<Subprocess::Hook> parentHooks;
+  if (systemdHierarchy.isSome()) {
+    parentHooks.emplace_back(Subprocess::Hook(&systemd::mesos::extendLifetime));
+  }
+
   Try<Subprocess> child = subprocess(
       path,
       argv,
@@ -314,8 +321,7 @@ Try<pid_t> LinuxLauncher::fork(
       environment,
       lambda::bind(&childSetup, pipes, setup),
       lambda::bind(&os::clone, lambda::_1, cloneFlags),
-      // TODO(jmlvanre): Use systemd hook.
-      Subprocess::Hook::None());
+      parentHooks);
 
   if (child.isError()) {
     return Error("Failed to clone child process: " + child.error());
@@ -342,28 +348,6 @@ Try<pid_t> LinuxLauncher::fork(
     return Error("Failed to contain process");
   }
 
-  // If we are on systemd, then move the child into the
-  // `MESOS_EXECUTORS_SLICE`. As with the freezer, any grandchildren will also
-  // be contained in the slice.
-  if (systemdHierarchy.isSome()) {
-    Try<Nothing> assign = cgroups::assign(
-        systemdHierarchy.get(),
-        systemd::mesos::MESOS_EXECUTORS_SLICE,
-        child.get().pid());
-
-    if (assign.isError()) {
-      LOG(ERROR) << "Failed to assign process " << child.get().pid()
-                  << " of container '" << containerId << "'"
-                  << " to its systemd executor slice: " << assign.error();
-
-      ::kill(child.get().pid(), SIGKILL);
-      return Error("Failed to contain process on systemd");
-    }
-
-    LOG(INFO) << "Assigned child process '" << child.get().pid() << "' to '"
-              << systemd::mesos::MESOS_EXECUTORS_SLICE << "'";
-  }
-
   // Now that we've contained the child we can signal it to continue
   // by writing to the pipe.
   char dummy;


[44/48] mesos git commit: Added missing `json` declaration for `ExecutorInfo`.

Posted by vi...@apache.org.
Added missing `json` declaration for `ExecutorInfo`.

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


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

Branch: refs/heads/0.27.x
Commit: 5f15fef76a57aa0eb854e4bc88fac494452a1d3e
Parents: c832e4a
Author: Michael Park <mp...@apache.org>
Authored: Wed Feb 24 22:35:39 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Fri Feb 26 18:57:10 2016 -0800

----------------------------------------------------------------------
 src/common/http.hpp | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5f15fef7/src/common/http.hpp
----------------------------------------------------------------------
diff --git a/src/common/http.hpp b/src/common/http.hpp
index f88590f..61c63a0 100644
--- a/src/common/http.hpp
+++ b/src/common/http.hpp
@@ -95,6 +95,7 @@ void json(JSON::ObjectWriter* writer, const Task& task);
 } // namespace internal {
 
 void json(JSON::ObjectWriter* writer, const Attributes& attributes);
+void json(JSON::ObjectWriter* writer, const ExecutorInfo& executorInfo);
 void json(JSON::ArrayWriter* writer, const Labels& labels);
 void json(JSON::ObjectWriter* writer, const Resources& resources);
 void json(JSON::ObjectWriter* writer, const TaskStatus& status);


[31/48] mesos git commit: Libprocess: Added 'parent_hooks' as an argument to 'subprocess'.

Posted by vi...@apache.org.
Libprocess: Added 'parent_hooks' as an argument to 'subprocess'.

In the future we will remove the default argument for `parent_hooks`
to force the caller to think about what kind of subprocess they want
to launch.

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


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

Branch: refs/heads/0.27.x
Commit: 4586166396a3043c6c44a1da1c6c75b7ee49da3c
Parents: cc876f1
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Thu Feb 4 15:01:12 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:10 2016 -0500

----------------------------------------------------------------------
 .../libprocess/include/process/subprocess.hpp   |  28 ++++-
 3rdparty/libprocess/src/subprocess.cpp          | 103 ++++++++++++++++++-
 2 files changed, 123 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/45861663/3rdparty/libprocess/include/process/subprocess.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/subprocess.hpp b/3rdparty/libprocess/include/process/subprocess.hpp
index ae0fa14..d7d5519 100644
--- a/3rdparty/libprocess/include/process/subprocess.hpp
+++ b/3rdparty/libprocess/include/process/subprocess.hpp
@@ -46,6 +46,9 @@ namespace process {
 class Subprocess
 {
 public:
+  // Forward declarations.
+  struct Hook;
+
   /**
    * Describes how the I/O is redirected for stdin/stdout/stderr.
    * One of the following three modes are supported:
@@ -123,7 +126,8 @@ public:
         const Option<std::map<std::string, std::string>>& environment,
         const Option<lambda::function<int()>>& setup,
         const Option<lambda::function<
-            pid_t(const lambda::function<int()>&)>>& clone);
+            pid_t(const lambda::function<int()>&)>>& clone,
+        const std::vector<Subprocess::Hook>& parent_hooks);
 
     IO(const lambda::function<Try<InputFileDescriptors>()>& _input,
        const lambda::function<Try<OutputFileDescriptors>()>& _output)
@@ -223,7 +227,8 @@ private:
       const Option<std::map<std::string, std::string>>& environment,
       const Option<lambda::function<int()>>& setup,
       const Option<lambda::function<
-          pid_t(const lambda::function<int()>&)>>& clone);
+          pid_t(const lambda::function<int()>&)>>& clone,
+      const std::vector<Subprocess::Hook>& parent_hooks);
 
   struct Data
   {
@@ -277,8 +282,12 @@ private:
  *     async unsafe code in the body of this function.
  * @param clone Function to be invoked in order to fork/clone the
  *     subprocess.
+ * @param parent_hooks Hooks that will be executed in the parent
+ *     before the child execs.
  * @return The subprocess or an error if one occured.
  */
+// TODO(jmlvanre): Consider removing default argument for
+// `parent_hooks` to force the caller to think about setting them.
 Try<Subprocess> subprocess(
     const std::string& path,
     std::vector<std::string> argv,
@@ -289,7 +298,9 @@ Try<Subprocess> subprocess(
     const Option<std::map<std::string, std::string>>& environment = None(),
     const Option<lambda::function<int()>>& setup = None(),
     const Option<lambda::function<
-        pid_t(const lambda::function<int()>&)>>& clone = None());
+        pid_t(const lambda::function<int()>&)>>& clone = None(),
+    const std::vector<Subprocess::Hook>& parent_hooks =
+      Subprocess::Hook::None());
 
 
 /**
@@ -311,8 +322,12 @@ Try<Subprocess> subprocess(
  *     async unsafe code in the body of this function.
  * @param clone Function to be invoked in order to fork/clone the
  *     subprocess.
+ * @param parent_hooks Hooks that will be executed in the parent
+ *     before the child execs.
  * @return The subprocess or an error if one occured.
  */
+// TODO(jmlvanre): Consider removing default argument for
+// `parent_hooks` to force the caller to think about setting them.
 inline Try<Subprocess> subprocess(
     const std::string& command,
     const Subprocess::IO& in = Subprocess::FD(STDIN_FILENO),
@@ -321,7 +336,9 @@ inline Try<Subprocess> subprocess(
     const Option<std::map<std::string, std::string>>& environment = None(),
     const Option<lambda::function<int()>>& setup = None(),
     const Option<lambda::function<
-        pid_t(const lambda::function<int()>&)>>& clone = None())
+        pid_t(const lambda::function<int()>&)>>& clone = None(),
+    const std::vector<Subprocess::Hook>& parent_hooks =
+      Subprocess::Hook::None())
 {
   std::vector<std::string> argv = {"sh", "-c", command};
 
@@ -334,7 +351,8 @@ inline Try<Subprocess> subprocess(
       None(),
       environment,
       setup,
-      clone);
+      clone,
+      parent_hooks);
 }
 
 } // namespace process {

http://git-wip-us.apache.org/repos/asf/mesos/blob/45861663/3rdparty/libprocess/src/subprocess.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/subprocess.cpp b/3rdparty/libprocess/src/subprocess.cpp
index 30e6efb..ae73a58 100644
--- a/3rdparty/libprocess/src/subprocess.cpp
+++ b/3rdparty/libprocess/src/subprocess.cpp
@@ -247,7 +247,9 @@ static int childMain(
     const Option<lambda::function<int()>>& setup,
     const InputFileDescriptors& stdinfds,
     const OutputFileDescriptors& stdoutfds,
-    const OutputFileDescriptors& stderrfds)
+    const OutputFileDescriptors& stderrfds,
+    bool blocking,
+    int pipes[2])
 {
   // Close parent's end of the pipes.
   if (stdinfds.write.isSome()) {
@@ -260,6 +262,12 @@ static int childMain(
     ::close(stderrfds.read.get());
   }
 
+  // Currently we will block the child's execution of the new process
+  // until all the parent hooks (if any) have executed.
+  if (blocking) {
+    ::close(pipes[1]);
+  }
+
   // Redirect I/O for stdin/stdout/stderr.
   while (::dup2(stdinfds.read, STDIN_FILENO) == -1 && errno == EINTR);
   while (::dup2(stdoutfds.write, STDOUT_FILENO) == -1 && errno == EINTR);
@@ -286,6 +294,22 @@ static int childMain(
     ::close(stderrfds.write);
   }
 
+  if (blocking) {
+    // Do a blocking read on the pipe until the parent signals us to
+    // continue.
+    char dummy;
+    ssize_t length;
+    while ((length = ::read(pipes[0], &dummy, sizeof(dummy))) == -1 &&
+          errno == EINTR);
+
+    if (length != sizeof(dummy)) {
+      ABORT("Failed to synchronize with parent");
+    }
+
+    // Now close the pipe as we don't need it anymore.
+    ::close(pipes[0]);
+  }
+
   if (setup.isSome()) {
     int status = setup.get()();
     if (status != 0) {
@@ -309,7 +333,8 @@ Try<Subprocess> subprocess(
     const Option<map<string, string>>& environment,
     const Option<lambda::function<int()>>& setup,
     const Option<lambda::function<
-        pid_t(const lambda::function<int()>&)>>& _clone)
+        pid_t(const lambda::function<int()>&)>>& _clone,
+    const std::vector<Subprocess::Hook>& parent_hooks)
 {
   // File descriptors for redirecting stdin/stdout/stderr.
   // These file descriptors are used for different purposes depending
@@ -397,6 +422,17 @@ Try<Subprocess> subprocess(
   lambda::function<pid_t(const lambda::function<int()>&)> clone =
     (_clone.isSome() ? _clone.get() : defaultClone);
 
+  // Currently we will block the child's execution of the new process
+  // until all the `parent_hooks` (if any) have executed.
+  int pipes[2];
+  const bool blocking = !parent_hooks.empty();
+
+  if (blocking) {
+    // We assume this should not fail under reasonable conditions so we
+    // use CHECK.
+    CHECK_EQ(0, ::pipe(pipes));
+  }
+
   // Now, clone the child process.
   pid_t pid = clone(lambda::bind(
       &childMain,
@@ -406,7 +442,9 @@ Try<Subprocess> subprocess(
       setup,
       stdinfds,
       stdoutfds,
-      stderrfds));
+      stderrfds,
+      blocking,
+      pipes));
 
   delete[] _argv;
 
@@ -421,9 +459,68 @@ Try<Subprocess> subprocess(
     // Save the errno as 'close' below might overwrite it.
     ErrnoError error("Failed to clone");
     internal::close(stdinfds, stdoutfds, stderrfds);
+
+    if (blocking) {
+      os::close(pipes[0]);
+      os::close(pipes[1]);
+    }
+
     return error;
   }
 
+  if (blocking) {
+    os::close(pipes[0]);
+
+    // Run the parent hooks.
+    foreach (const Subprocess::Hook& hook, parent_hooks) {
+      Try<Nothing> callback = hook.parent_callback(pid);
+
+      // If the hook callback fails, we shouldn't proceed with the
+      // execution.
+      if (callback.isError()) {
+        LOG(WARNING)
+          << "Failed to execute Subprocess::Hook in parent for child '"
+          << pid << "': " << callback.error();
+
+        os::close(pipes[1]);
+
+        // Close the child-ends of the file descriptors that are created
+        // by this function.
+        os::close(stdinfds.read);
+        os::close(stdoutfds.write);
+        os::close(stderrfds.write);
+
+        // Ensure the child is killed.
+        ::kill(pid, SIGKILL);
+
+        return Error(
+            "Failed to execute Subprocess::Hook in parent for child '" +
+            stringify(pid) + "': " + callback.error());
+      }
+    }
+
+    // Now that we've executed the parent hooks, we can signal the child to
+    // continue by writing to the pipe.
+    char dummy;
+    ssize_t length;
+    while ((length = ::write(pipes[1], &dummy, sizeof(dummy))) == -1 &&
+           errno == EINTR);
+
+    os::close(pipes[1]);
+
+    if (length != sizeof(dummy)) {
+      // Ensure the child is killed.
+      ::kill(pid, SIGKILL);
+
+      // Close the child-ends of the file descriptors that are created
+      // by this function.
+      os::close(stdinfds.read);
+      os::close(stdoutfds.write);
+      os::close(stderrfds.write);
+      return Error("Failed to synchronize child process");
+    }
+  }
+
   // Parent.
   Subprocess process;
   process.data->pid = pid;


[40/48] mesos git commit: Removed the duplicate "active" field in json schema of `Framework`.

Posted by vi...@apache.org.
Removed the duplicate "active" field in json schema of `Framework`.

The new `jsonify` library is a writer-based approach, and does not keep
track of the fields that have been written out so far. The previous
version of `summarize(framework)` and `model(framework)` had a duplicate
`"active"` field which was de-duplicated since they simply get inserted
to a `std::map`, overriding the previous value.

In the `jsonify` case, this pattern results in duplicate key in the JSON
output. Although the presence of duplicate keys is technically not
__invalid__ according to the JSON specification, some JSON libraries
disallow them. As such, we should generate JSON outputs without
duplicate keys.

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


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

Branch: refs/heads/0.27.x
Commit: 57c80bfcf76a4e7f34881756fea7b546f6d5dee5
Parents: 6c61957
Author: Michael Park <mp...@apache.org>
Authored: Tue Feb 16 15:02:16 2016 -0800
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:29:21 2016 -0500

----------------------------------------------------------------------
 src/master/http.cpp | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/57c80bfc/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index 9593373..d67f0da 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -219,7 +219,6 @@ void json(JSON::ObjectWriter* writer, const Full<Framework>& full)
   writer->field("role", framework.info.role());
   writer->field("registered_time", framework.registeredTime.secs());
   writer->field("unregistered_time", framework.unregisteredTime.secs());
-  writer->field("active", framework.active);
 
   if (framework.info.has_principal()) {
     writer->field("principal", framework.info.principal());
@@ -368,7 +367,6 @@ JSON::Object model(const Framework& framework)
   object.values["role"] = framework.info.role();
   object.values["registered_time"] = framework.registeredTime.secs();
   object.values["unregistered_time"] = framework.unregisteredTime.secs();
-  object.values["active"] = framework.active;
 
   if (framework.info.has_principal()) {
     object.values["principal"] = framework.info.principal();


[42/48] mesos git commit: Updated CHANGELOG for 0.27.1-rc1 release.

Posted by vi...@apache.org.
Updated CHANGELOG for 0.27.1-rc1 release.


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

Branch: refs/heads/0.27.x
Commit: 864fe8eabd4a83b78ce9140c501908ee3cb90beb
Parents: 4d475af
Author: Michael Park <mp...@apache.org>
Authored: Tue Feb 16 16:02:36 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Tue Feb 16 16:06:31 2016 -0800

----------------------------------------------------------------------
 CHANGELOG | 26 ++++++++++++++++++++++++++
 1 file changed, 26 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/864fe8ea/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 13ae949..5dae7c4 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,3 +1,29 @@
+Release Notes - Mesos - Version 0.27.1
+--------------------------------------
+* This is a bug fix release.
+
+** Bug
+  * [MESOS-4546] - Mesos Agents needs to re-resolve hosts in zk string on leader change / failure to connect.
+  * [MESOS-4563] - Docker::Container::Create should handle NetworkSettings.IPAddress being an empty string.
+  * [MESOS-4582] - state.json serving duplicate "active" fields.
+  * [MESOS-4585] - mesos-fetcher LIBPROCESS_PORT set to 5051 URI fetch failure.
+  * [MESOS-4587] - Docker environment variables must be able to contain the equal sign.
+  * [MESOS-4597] - `freebsd.hpp` is missing from the release tarball.
+  * [MESOS-4598] - Logrotate ContainerLogger should not remove IP from environment.
+  * [MESOS-4637] - Docker process executor can die with agent unit on systemd.
+  * [MESOS-4639] - Posix process executor can die with agent unit on systemd.
+  * [MESOS-4640] - Logrotate container logger can die with agent unit on systemd.
+  * [MESOS-4675] - Can not disable systemd support.
+
+** Improvement
+  * [MESOS-4566] - Avoid unnecessary temporary `std::string` constructions and copies in `jsonify`.
+  * [MESOS-4636] - Add parent hook to subprocess.
+
+** Task
+  * [MESOS-4435] - Update `Master::Http::stateSummary` to use `jsonify`.
+  * [MESOS-4531] - Document multi-disk support.
+
+
 Release Notes - Mesos - Version 0.27.0
 --------------------------------------------
 


[04/48] mesos git commit: Updated Maven README file for 0.27.0 release.

Posted by vi...@apache.org.
Updated Maven README file for 0.27.0 release.


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

Branch: refs/heads/0.27.x
Commit: 12d0cf4c70ce754a2daf507e86c4ec91fd942ef0
Parents: 0869c2d
Author: Michael Park <mp...@apache.org>
Authored: Wed Jan 27 19:52:11 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 20:04:51 2016 -0800

----------------------------------------------------------------------
 src/java/MESOS-MAVEN-README | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/12d0cf4c/src/java/MESOS-MAVEN-README
----------------------------------------------------------------------
diff --git a/src/java/MESOS-MAVEN-README b/src/java/MESOS-MAVEN-README
index c39befd..2812fb1 100644
--- a/src/java/MESOS-MAVEN-README
+++ b/src/java/MESOS-MAVEN-README
@@ -8,11 +8,11 @@ it will be copied into Maven Central.
 
 NOTE: Publishing artifacts to Maven as described below requires the Mesos Java
 classes to be generated from the Protobuf file. The easiest way to do this is
-to run `make src/java/target/mesos-0.19.0.jar`. In addition, Maven will create
+to run `make src/java/target/mesos-0.27.0.jar`. In addition, Maven will create
 a shaded jar that includes the protobuf classes
-(src/java/target/mesos-shaded-protobuf-0.19.0.jar), a jar containing the Java
-source files (src/java/target/mesos-0.19.0-sources.jar) and the Javadoc
-(src/java/target/mesos-0.19.0-javadoc.jar), and all four of these files, as
+(src/java/target/mesos-shaded-protobuf-0.27.0.jar), a jar containing the Java
+source files (src/java/target/mesos-0.27.0-sources.jar) and the Javadoc
+(src/java/target/mesos-0.27.0-javadoc.jar), and all four of these files, as
 well as an .md5 and .sha1 for each of the jars, and finally an additional *.asc
 signature file for each of the above files (all in src/java/target).
 


[33/48] mesos git commit: Libprocess: Introduced Hooks for Subprocess.

Posted by vi...@apache.org.
Libprocess: Introduced Hooks for Subprocess.

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


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

Branch: refs/heads/0.27.x
Commit: cc876f1b6e578ef2e992d27cdf2e0dc5eb9e1ebd
Parents: a0d0e0c
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Thu Feb 4 14:54:27 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:10 2016 -0500

----------------------------------------------------------------------
 .../libprocess/include/process/subprocess.hpp   | 27 ++++++++++++++++++++
 3rdparty/libprocess/src/subprocess.cpp          |  5 ++++
 2 files changed, 32 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/cc876f1b/3rdparty/libprocess/include/process/subprocess.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/subprocess.hpp b/3rdparty/libprocess/include/process/subprocess.hpp
index 482ea23..ae0fa14 100644
--- a/3rdparty/libprocess/include/process/subprocess.hpp
+++ b/3rdparty/libprocess/include/process/subprocess.hpp
@@ -21,6 +21,7 @@
 
 #include <map>
 #include <memory>
+#include <mutex>
 #include <string>
 #include <vector>
 
@@ -140,6 +141,32 @@ public:
     lambda::function<Try<OutputFileDescriptors>()> output;
   };
 
+  /**
+   * A hook can be passed to a `subprocess` call. It provides a way to
+   * inject dynamic implementation behavior between the clone and exec
+   * calls in the implementation of `subprocess`.
+   */
+  struct Hook
+  {
+    /**
+     * Returns an empty list of hooks.
+     */
+    static std::vector<Hook> None() { return std::vector<Hook>(); }
+
+    Hook(const lambda::function<Try<Nothing>(pid_t)>& _parent_callback);
+
+    /**
+     * The callback that must be sepcified for execution after the
+     * child has been cloned, but before it start executing the new
+     * process. This provides access to the child pid after its
+     * initialization to add tracking or modify execution state of
+     * the child before it executes the new process.
+     */
+    const lambda::function<Try<Nothing>(pid_t)> parent_callback;
+
+    friend class Subprocess;
+  };
+
   // Some syntactic sugar to create an IO::PIPE redirector.
   static IO PIPE();
   static IO PATH(const std::string& path);

http://git-wip-us.apache.org/repos/asf/mesos/blob/cc876f1b/3rdparty/libprocess/src/subprocess.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/subprocess.cpp b/3rdparty/libprocess/src/subprocess.cpp
index ef05687..30e6efb 100644
--- a/3rdparty/libprocess/src/subprocess.cpp
+++ b/3rdparty/libprocess/src/subprocess.cpp
@@ -42,6 +42,11 @@ namespace process {
 using InputFileDescriptors = Subprocess::IO::InputFileDescriptors;
 using OutputFileDescriptors = Subprocess::IO::OutputFileDescriptors;
 
+
+Subprocess::Hook::Hook(
+    const lambda::function<Try<Nothing>(pid_t)>& _parent_callback)
+  : parent_callback(_parent_callback) {}
+
 namespace internal {
 
 // See the comment below as to why subprocess is passed to cleanup.


[17/48] mesos git commit: Avoid construction of temporary strings in `NumberWriter` for doubles.

Posted by vi...@apache.org.
Avoid construction of temporary strings in `NumberWriter` for doubles.

With this + https://reviews.apache.org/r/43024/, the number of calls to
`operator new` and `operator delete` were reduced by roughly 1/3.

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


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

Branch: refs/heads/0.27.x
Commit: b64851f22e459232eec1f11eef15ad8f8610ccdd
Parents: 00441c1
Author: Michael Park <mp...@apache.org>
Authored: Sun Jan 31 20:37:42 2016 -0800
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Mon Feb 15 16:15:23 2016 -0500

----------------------------------------------------------------------
 .../3rdparty/stout/include/stout/jsonify.hpp       | 17 +++++++++++++----
 1 file changed, 13 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b64851f2/3rdparty/libprocess/3rdparty/stout/include/stout/jsonify.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/jsonify.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/jsonify.hpp
index f9d7224..8220d00 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/jsonify.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/jsonify.hpp
@@ -148,8 +148,8 @@ public:
         // Prints a floating point value, with the specified precision, see:
         // http://www.open-std.org/jtc1/sc22/wg21/docs/papers/2006/n2005.pdf
         // Additionally ensures that a decimal point is in the output.
-        char buffer[50] = {}; // More than enough for the specified precision.
-        snprintf(
+        char buffer[50]; // More than enough for the specified precision.
+        const int size = snprintf(
             buffer,
             sizeof(buffer),
             "%#.*g",
@@ -158,10 +158,19 @@ public:
 
         // Get rid of excess trailing zeroes before outputting.
         // Otherwise, printing 1.0 would result in "1.00000000000000".
-        std::string trimmed = strings::trim(buffer, strings::SUFFIX, "0");
+        //
+        // NOTE: We intentionally do not use `strings::trim` here in order to
+        // avoid construction of temporary strings.
+        int back = size - 1;
+        for (; back > 0; --back) {
+          if (buffer[back] != '0') {
+            break;
+          }
+          buffer[back] = '\0';
+        }
 
         // NOTE: valid JSON numbers cannot end with a '.'.
-        *stream_ << trimmed << (trimmed.back() == '.' ? "0" : "");
+        *stream_ << buffer << (buffer[back] == '.' ? "0" : "");
         break;
       }
     }


[16/48] mesos git commit: Fixed webui task counts for STARTING/RUNNING tasks.

Posted by vi...@apache.org.
Fixed webui task counts for STARTING/RUNNING tasks.

The "Starting" and "Running" counts show the number of tasks in
"TASK_STARTING" and "TASK_RUNNING" state respectively.

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


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

Branch: refs/heads/0.27.x
Commit: 1ec0516720562480102a207323da25041b68abc0
Parents: f8f75c2
Author: Kapil Arya <ka...@mesosphere.io>
Authored: Sun Jan 31 17:02:21 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Mon Feb 15 16:15:23 2016 -0500

----------------------------------------------------------------------
 src/webui/master/static/home.html         | 12 ++++++++----
 src/webui/master/static/js/controllers.js | 10 ++++++----
 src/webui/master/static/slave.html        | 12 ++++++++----
 3 files changed, 22 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/1ec05167/src/webui/master/static/home.html
----------------------------------------------------------------------
diff --git a/src/webui/master/static/home.html b/src/webui/master/static/home.html
index f6d2f7d..d6cde1e 100644
--- a/src/webui/master/static/home.html
+++ b/src/webui/master/static/home.html
@@ -60,12 +60,16 @@
 
         <tbody>
           <tr>
-            <td>Staged</td>
-            <td class="text-right">{{staged_tasks | number}}</td>
+            <td>Staging</td>
+            <td class="text-right">{{staging_tasks | number}}</td>
           </tr>
           <tr>
-            <td>Started</td>
-            <td class="text-right">{{started_tasks | number}}</td>
+            <td>Starting</td>
+            <td class="text-right">{{starting_tasks | number}}</td>
+          </tr>
+          <tr>
+            <td>Running</td>
+            <td class="text-right">{{running_tasks | number}}</td>
           </tr>
           <tr>
             <td>Finished</td>

http://git-wip-us.apache.org/repos/asf/mesos/blob/1ec05167/src/webui/master/static/js/controllers.js
----------------------------------------------------------------------
diff --git a/src/webui/master/static/js/controllers.js b/src/webui/master/static/js/controllers.js
index 36865f1..ff73a46 100644
--- a/src/webui/master/static/js/controllers.js
+++ b/src/webui/master/static/js/controllers.js
@@ -222,8 +222,9 @@
   // Update the outermost scope with the metrics/snapshot endpoint.
   function updateMetrics($scope, $timeout, data) {
     var metrics = JSON.parse(data);
-    $scope.staged_tasks = metrics['master/tasks_staging'];
-    $scope.started_tasks = metrics['master/tasks_starting'];
+    $scope.staging_tasks = metrics['master/tasks_staging'];
+    $scope.starting_tasks = metrics['master/tasks_starting'];
+    $scope.running_tasks = metrics['master/tasks_running'];
     $scope.finished_tasks = metrics['master/tasks_finished'];
     $scope.killed_tasks = metrics['master/tasks_killed'];
     $scope.failed_tasks = metrics['master/tasks_failed'];
@@ -525,8 +526,9 @@
             $scope.state = {};
           }
 
-          $scope.state.staged_tasks = response['slave/tasks_staging'];
-          $scope.state.started_tasks = response['slave/tasks_starting'];
+          $scope.state.staging_tasks = response['slave/tasks_staging'];
+          $scope.state.starting_tasks = response['slave/tasks_starting'];
+          $scope.state.running_tasks = response['slave/tasks_running'];
           $scope.state.finished_tasks = response['slave/tasks_finished'];
           $scope.state.killed_tasks = response['slave/tasks_killed'];
           $scope.state.failed_tasks = response['slave/tasks_failed'];

http://git-wip-us.apache.org/repos/asf/mesos/blob/1ec05167/src/webui/master/static/slave.html
----------------------------------------------------------------------
diff --git a/src/webui/master/static/slave.html b/src/webui/master/static/slave.html
index ad37ee2..bc46885 100644
--- a/src/webui/master/static/slave.html
+++ b/src/webui/master/static/slave.html
@@ -48,12 +48,16 @@
       <table class="table table-condensed">
         <tbody>
           <tr>
-            <td>Staged</td>
-            <td class="text-right">{{state.staged_tasks | number}}</td>
+            <td>Staging</td>
+            <td class="text-right">{{state.staging_tasks | number}}</td>
           </tr>
           <tr>
-            <td>Started</td>
-            <td class="text-right">{{state.started_tasks | number}}</td>
+            <td>Starting</td>
+            <td class="text-right">{{state.starting_tasks | number}}</td>
+          </tr>
+          <tr>
+            <td>Running</td>
+            <td class="text-right">{{state.running_tasks | number}}</td>
           </tr>
           <tr>
             <td>Finished</td>


[02/48] mesos git commit: Add test for LogrotateContainerLogger's FD management.

Posted by vi...@apache.org.
Add test for LogrotateContainerLogger's FD management.

Adds a test which checks for erroneous calls to `os::close` by the
LogrotateContainerLogger.  This may happen by accident if the
container logger module uses `Subprocess::PIPE` when launching child
processes; as libprocess will track these FDs and close them (possibly
even if they've already been closed) when the child processes exit.

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


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

Branch: refs/heads/0.27.x
Commit: 50556e88f6a493ae1e5e92cd3d19fd3d1a23dc49
Parents: 14b9717
Author: Joseph Wu <jo...@mesosphere.io>
Authored: Wed Jan 27 16:25:28 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 20:04:50 2016 -0800

----------------------------------------------------------------------
 src/tests/container_logger_tests.cpp | 99 +++++++++++++++++++++++++++++++
 1 file changed, 99 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/50556e88/src/tests/container_logger_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/container_logger_tests.cpp b/src/tests/container_logger_tests.cpp
index 5fe9cce..e161fd6 100644
--- a/src/tests/container_logger_tests.cpp
+++ b/src/tests/container_logger_tests.cpp
@@ -31,6 +31,7 @@
 #include <stout/try.hpp>
 
 #include <stout/os/exists.hpp>
+#include <stout/os/killtree.hpp>
 #include <stout/os/mkdir.hpp>
 #include <stout/os/pstree.hpp>
 #include <stout/os/stat.hpp>
@@ -484,6 +485,104 @@ TEST_F(ContainerLoggerTest, LOGROTATE_RotateInSandbox)
   }
 }
 
+
+// Tests that the logrotate container logger only closes FDs when it
+// is supposed to and does not interfere with other FDs on the agent.
+TEST_F(ContainerLoggerTest, LOGROTATE_ModuleFDOwnership)
+{
+  // Create a master, agent, and framework.
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Future<SlaveRegisteredMessage> slaveRegisteredMessage =
+    FUTURE_PROTOBUF(SlaveRegisteredMessage(), _, _);
+
+  // We'll need access to these flags later.
+  slave::Flags flags = CreateSlaveFlags();
+
+  // Use the non-default container logger that rotates logs.
+  flags.container_logger = LOGROTATE_CONTAINER_LOGGER_NAME;
+
+  Fetcher fetcher;
+
+  // We use an actual containerizer + executor since we want something to run.
+  Try<MesosContainerizer*> containerizer =
+    MesosContainerizer::create(flags, false, &fetcher);
+  CHECK_SOME(containerizer);
+
+  Try<PID<Slave>> slave = StartSlave(containerizer.get(), flags);
+  ASSERT_SOME(slave);
+
+  AWAIT_READY(slaveRegisteredMessage);
+  SlaveID slaveId = slaveRegisteredMessage.get().slave_id();
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+
+  Future<FrameworkID> frameworkId;
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .WillOnce(FutureArg<1>(&frameworkId));
+
+  // Wait for an offer, and start a task.
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+  AWAIT_READY(frameworkId);
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  // Start a task that will keep running until the end of the test.
+  TaskInfo task = createTask(offers.get()[0], "sleep 100");
+
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusKilled;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusKilled))
+    .WillRepeatedly(Return());       // Ignore subsequent updates.
+
+  driver.launchTasks(offers.get()[0].id(), {task});
+
+  AWAIT_READY(statusRunning);
+  EXPECT_EQ(TASK_RUNNING, statusRunning.get().state());
+
+  // Open multiple files, so that we're fairly certain we've opened
+  // the same FDs (integers) opened by the container logger.
+  vector<int> fds;
+  for (int i = 0; i < 50; i++) {
+    Try<int> fd = os::open("/dev/null", O_RDONLY);
+    ASSERT_SOME(fd);
+
+    fds.push_back(fd.get());
+  }
+
+  // Kill the task, which also kills the executor.
+  driver.killTask(statusRunning.get().task_id());
+
+  AWAIT_READY(statusKilled);
+  EXPECT_EQ(TASK_KILLED, statusKilled.get().state());
+
+  Future<Nothing> executorTerminated =
+    FUTURE_DISPATCH(_, &Slave::executorTerminated);
+
+  AWAIT_READY(executorTerminated);
+
+  // Close all the FDs we opened.  Every `close` should succeed.
+  foreach (int fd, fds) {
+    ASSERT_SOME(os::close(fd));
+  }
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[06/48] mesos git commit: Fixed the NetClsIsolatorTest to correctly learn the net_cls hierarchy.

Posted by vi...@apache.org.
Fixed the NetClsIsolatorTest to correctly learn the net_cls hierarchy.

The test relied on the flags.cgroups_hierarchy to learn the net_cls
hierarchy, instead it should be making a call to cgroups::hierarchy to
learn the hierarchy.

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


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

Branch: refs/heads/0.27.x
Commit: 1a35d59e9933e43328a92ba72160f9e0f28e5ee0
Parents: 502c2d3
Author: Avinash sridharan <av...@mesosphere.io>
Authored: Wed Jan 27 17:31:14 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 20:04:51 2016 -0800

----------------------------------------------------------------------
 src/tests/containerizer/isolator_tests.cpp | 16 +++++++++-------
 1 file changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/1a35d59e/src/tests/containerizer/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index 7033ac1..8d101df 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -867,6 +867,7 @@ TEST_F(NetClsIsolatorTest, ROOT_CGROUPS_NetClsIsolate)
 
   Try<MesosContainerizer*> containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
+
   ASSERT_SOME(containerizer);
 
   Try<PID<Slave>> slave = StartSlave(containerizer.get(), flags);
@@ -914,15 +915,16 @@ TEST_F(NetClsIsolatorTest, ROOT_CGROUPS_NetClsIsolate)
 
   const ContainerID& containerID = *(containers.get().begin());
 
-  // Check if the net_cls cgroup for this container exists, by checking for the
-  // processes associated with this cgroup.
-  string container_cgroup = path::join(
+  Result<string> hierarchy = cgroups::hierarchy("net_cls");
+  ASSERT_SOME(hierarchy);
+
+  // Check if the net_cls cgroup for this container exists, by
+  // checking for the processes associated with this cgroup.
+  string cgroup = path::join(
       flags.cgroups_root,
       containerID.value());
 
-  Try<set<pid_t>> pids = cgroups::processes(
-        path::join(flags.cgroups_hierarchy, "net_cls"),
-        container_cgroup);
+  Try<set<pid_t>> pids = cgroups::processes(hierarchy.get(), cgroup);
   ASSERT_SOME(pids);
 
   // There should be at least one TGID associated with this cgroup.
@@ -945,7 +947,7 @@ TEST_F(NetClsIsolatorTest, ROOT_CGROUPS_NetClsIsolate)
 
   // If the cleanup is successful the net_cls cgroup for this container should
   // not exist.
-  ASSERT_FALSE(os::exists(container_cgroup));
+  ASSERT_FALSE(os::exists(cgroup));
 
   driver.stop();
   driver.join();


[41/48] mesos git commit: Added missing JIRA ticket in CHANGELOG for 0.27.0.

Posted by vi...@apache.org.
Added missing JIRA ticket in CHANGELOG for 0.27.0.


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

Branch: refs/heads/0.27.x
Commit: 4d475af882a45f32dbde38e9c111b4a2d458870f
Parents: 57c80bf
Author: Michael Park <mp...@apache.org>
Authored: Tue Feb 16 15:54:32 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Tue Feb 16 15:56:44 2016 -0800

----------------------------------------------------------------------
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/4d475af8/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 82c1be6..13ae949 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -128,6 +128,7 @@ Release Notes - Mesos - Version 0.27.0
   * [MESOS-4351] - Remove logic around checkpointing in the slave
   * [MESOS-4410] - Introduce protobuf for quota set request.
   * [MESOS-4505] - Hierarchical allocator performance is slow due to Quota
+  * [MESOS-4578] - docker run -c is deprecated
 
 ** Task
   * [MESOS-2079] - IO.Write test is flaky on OS X 10.10.


[09/48] mesos git commit: Used absolute paths for excludes paths in posix disk isolator.

Posted by vi...@apache.org.
Used absolute paths for excludes paths in posix disk isolator.

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


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

Branch: refs/heads/0.27.x
Commit: b956e2c81b6ddf119791acabeab9829620ea6645
Parents: 8186756
Author: Jie Yu <yu...@gmail.com>
Authored: Wed Jan 27 17:05:48 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 20:04:51 2016 -0800

----------------------------------------------------------------------
 .../mesos/isolators/posix/disk.cpp              | 110 +++++++++++--------
 .../mesos/isolators/posix/disk.hpp              |   4 +
 2 files changed, 68 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b956e2c8/src/slave/containerizer/mesos/isolators/posix/disk.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/posix/disk.cpp b/src/slave/containerizer/mesos/isolators/posix/disk.cpp
index d5bf8c1..fd9b3ab 100644
--- a/src/slave/containerizer/mesos/isolators/posix/disk.cpp
+++ b/src/slave/containerizer/mesos/isolators/posix/disk.cpp
@@ -42,18 +42,35 @@
 
 #include <stout/os/exists.hpp>
 #include <stout/os/killtree.hpp>
+#include <stout/os/stat.hpp>
 
 #include "common/protobuf_utils.hpp"
 
 #include "slave/containerizer/mesos/isolators/posix/disk.hpp"
 
-using namespace process;
+namespace io = process::io;
 
 using std::deque;
 using std::list;
 using std::string;
 using std::vector;
 
+using process::Failure;
+using process::Future;
+using process::Owned;
+using process::PID;
+using process::Process;
+using process::Promise;
+using process::Subprocess;
+
+using process::await;
+using process::defer;
+using process::delay;
+using process::dispatch;
+using process::spawn;
+using process::subprocess;
+using process::terminate;
+
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
 using mesos::slave::ContainerLimitation;
@@ -157,9 +174,6 @@ Future<Nothing> PosixDiskIsolatorProcess::update(
   // This stores the updated quotas.
   hashmap<string, Resources> quotas;
 
-  // Volume paths to exclude from sandbox disk usage calculation.
-  vector<string> excludes;
-
   foreach (const Resource& resource, resources) {
     if (resource.name() != "disk") {
       continue;
@@ -171,23 +185,20 @@ Future<Nothing> PosixDiskIsolatorProcess::update(
     // NOTE: We do not allow the case where has_disk() is true but
     // with nothing set inside DiskInfo. The master will enforce it.
     if (!resource.has_disk() || !resource.disk().has_volume()) {
-      // If either DiskInfo or DiskInfo.Volume are not set we're dealing
-      // with the working directory of the executor (aka the sanbox).
+      // If either DiskInfo or DiskInfo.Volume are not set we're
+      // dealing with the working directory of the executor (aka the
+      // sanbox).
       path = info->directory;
     } else {
-      // Otherwise it is a disk resource (such as a persistent volume) and
-      // we extract the path from the protobuf.
+      // Otherwise it is a disk resource (such as a persistent volume)
+      // and we extract the path from the protobuf.
       path = resource.disk().volume().container_path();
 
-      excludes.push_back(path);
-
-      // In case the path in the protobuf is not an absolute path it is
-      // relative to the working directory of the executor. We always store
-      // the absolute path.
+      // In case the path in the protobuf is not an absolute path it
+      // is relative to the working directory of the executor. We
+      // always store the absolute path.
       if (!path::absolute(path)) {
-        // We prepend "/" at the end to make sure that 'du' runs on actual
-        // directory pointed by the symlink (and not the symlink itself).
-        path = path::join(info->directory, path, "");
+        path = path::join(info->directory, path);
       }
     }
 
@@ -202,15 +213,7 @@ Future<Nothing> PosixDiskIsolatorProcess::update(
   // the disk usage collection.
   foreachpair (const string& path, const Resources& quota, quotas) {
     if (!info->paths.contains(path)) {
-      info->paths[path].usage = collector.usage(
-          path,
-          (path == info->directory) ? excludes : vector<string>())
-        .onAny(defer(
-            PID<PosixDiskIsolatorProcess>(this),
-            &PosixDiskIsolatorProcess::_collect,
-            containerId,
-            path,
-            lambda::_1));
+      info->paths[path].usage = collect(containerId, path);
     }
 
     info->paths[path].quota = quota;
@@ -227,6 +230,41 @@ Future<Nothing> PosixDiskIsolatorProcess::update(
 }
 
 
+Future<Bytes> PosixDiskIsolatorProcess::collect(
+    const ContainerID& containerId,
+    const string& path)
+{
+  CHECK(infos.contains(containerId));
+
+  const Owned<Info>& info = infos[containerId];
+
+  // Volume paths to exclude from sandbox disk usage calculation.
+  vector<string> excludes;
+  if (path == info->directory) {
+    foreachkey (const string& exclude, info->paths) {
+      if (exclude != info->directory) {
+        excludes.push_back(exclude);
+      }
+    }
+  }
+
+  // We append "/" at the end to make sure that 'du' runs on actual
+  // directory pointed by the symlink (and not the symlink itself).
+  string _path = path;
+  if (path != info->directory && os::stat::islink(path)) {
+    _path = path::join(path, "");
+  }
+
+  return collector.usage(_path, excludes)
+    .onAny(defer(
+        PID<PosixDiskIsolatorProcess>(this),
+        &PosixDiskIsolatorProcess::_collect,
+        containerId,
+        path,
+        lambda::_1));
+}
+
+
 void PosixDiskIsolatorProcess::_collect(
     const ContainerID& containerId,
     const string& path,
@@ -289,27 +327,7 @@ void PosixDiskIsolatorProcess::_collect(
     }
   }
 
-  // Build excludes array if the current path is the sandbox.
-  vector<string> excludes;
-  if (path == info->directory) {
-    foreachkey (const string& exclude, info->paths) {
-      if (exclude != path) {
-        // `du --exclude` uses pattern matching so we strip both
-        // prefix (sandbox path) and suffix ('/') from volume path.
-        string relative = strings::remove(exclude, path + "/", strings::PREFIX);
-        relative = strings::remove(relative, "/", strings::SUFFIX);
-        excludes.push_back(relative);
-      }
-    }
-  }
-
-  info->paths[path].usage = collector.usage(path, excludes)
-      .onAny(defer(
-          PID<PosixDiskIsolatorProcess>(this),
-          &PosixDiskIsolatorProcess::_collect,
-          containerId,
-          path,
-          lambda::_1));
+  info->paths[path].usage = collect(containerId, path);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/b956e2c8/src/slave/containerizer/mesos/isolators/posix/disk.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/posix/disk.hpp b/src/slave/containerizer/mesos/isolators/posix/disk.hpp
index 5607c0f..3998251 100644
--- a/src/slave/containerizer/mesos/isolators/posix/disk.hpp
+++ b/src/slave/containerizer/mesos/isolators/posix/disk.hpp
@@ -106,6 +106,10 @@ public:
 private:
   PosixDiskIsolatorProcess(const Flags& flags);
 
+  process::Future<Bytes> collect(
+      const ContainerID& containerId,
+      const std::string& path);
+
   void _collect(
       const ContainerID& containerId,
       const std::string& path,


[23/48] mesos git commit: Added LIBPROCESS_PORT environment removal for mesos-fetcher.

Posted by vi...@apache.org.
Added LIBPROCESS_PORT environment removal for mesos-fetcher.

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


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

Branch: refs/heads/0.27.x
Commit: a1a36053dcedd7208de543f62db54968cb7c8bd1
Parents: 9f4fa0e
Author: Shuai Lin <li...@gmail.com>
Authored: Fri Feb 5 04:38:28 2016 +0100
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:09 2016 -0500

----------------------------------------------------------------------
 src/slave/containerizer/fetcher.cpp | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a1a36053/src/slave/containerizer/fetcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/fetcher.cpp b/src/slave/containerizer/fetcher.cpp
index f7e3f7d..1b67d1a 100644
--- a/src/slave/containerizer/fetcher.cpp
+++ b/src/slave/containerizer/fetcher.cpp
@@ -752,6 +752,11 @@ Future<Nothing> FetcherProcess::run(
   // environment variable.
   map<string, string> environment = os::environment();
 
+  // The libprocess port is explicitly removed because this will conflict
+  // with the already-running agent.
+  environment.erase("LIBPROCESS_PORT");
+  environment.erase("LIBPROCESS_ADVERTISE_PORT");
+
   environment["MESOS_FETCHER_INFO"] = stringify(JSON::protobuf(info));
 
   if (!flags.hadoop_home.empty()) {


[32/48] mesos git commit: Extended life of process based docker executor on systemd.

Posted by vi...@apache.org.
Extended life of process based docker executor on systemd.

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


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

Branch: refs/heads/0.27.x
Commit: 60dc7ab347d014d700daed7944f3823bb5cd6c1a
Parents: e83e63a
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Fri Feb 5 14:00:01 2016 -0500
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:10 2016 -0500

----------------------------------------------------------------------
 src/slave/containerizer/docker.cpp | 15 ++++++++++++++-
 1 file changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/60dc7ab3/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 2887cb4..3849e96 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -40,6 +40,7 @@
 
 #ifdef __linux__
 #include "linux/cgroups.hpp"
+#include "linux/systemd.hpp"
 #endif // __linux__
 
 #include "slave/paths.hpp"
@@ -954,6 +955,16 @@ Future<pid_t> DockerContainerizerProcess::launchExecutorProcess(
         self(),
         [=](const ContainerLogger::SubprocessInfo& subprocessInfo)
           -> Future<pid_t> {
+    // If we are on systemd, then extend the life of the executor. Any
+    // grandchildren's lives will also be extended.
+    std::vector<Subprocess::Hook> parentHooks;
+#ifdef __linux__
+    if (systemd::enabled()) {
+      parentHooks.emplace_back(Subprocess::Hook(
+          &systemd::mesos::extendLifetime));
+    }
+#endif // __linux__
+
     // Construct the mesos-docker-executor using the "name" we gave the
     // container (to distinguish it from Docker containers not created
     // by Mesos).
@@ -965,7 +976,9 @@ Future<pid_t> DockerContainerizerProcess::launchExecutorProcess(
         subprocessInfo.err,
         dockerFlags(flags, container->name(), container->directory),
         environment,
-        lambda::bind(&setup, container->directory));
+        lambda::bind(&setup, container->directory),
+        None(),
+        parentHooks);
 
     if (s.isError()) {
       return Failure("Failed to fork executor: " + s.error());


[30/48] mesos git commit: Fixed parsing docker image env vars.

Posted by vi...@apache.org.
Fixed parsing docker image env vars.

Allow equal character in the value of environemnt variables.

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


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

Branch: refs/heads/0.27.x
Commit: a0d0e0cc56b8adb0ac135ba52948cf774afed4c5
Parents: a1a3605
Author: Shuai Lin <li...@gmail.com>
Authored: Fri Feb 5 09:22:09 2016 -0800
Committer: Joris Van Remoortere <jo...@gmail.com>
Committed: Tue Feb 16 18:21:10 2016 -0500

----------------------------------------------------------------------
 src/docker/docker.cpp                    | 4 ++--
 src/tests/containerizer/docker_tests.cpp | 6 ++++++
 2 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a0d0e0cc/src/docker/docker.cpp
----------------------------------------------------------------------
diff --git a/src/docker/docker.cpp b/src/docker/docker.cpp
index a831726..5272870 100755
--- a/src/docker/docker.cpp
+++ b/src/docker/docker.cpp
@@ -382,8 +382,8 @@ Try<Docker::Image> Docker::Image::create(const JSON::Object& json)
           return Error("Expecting environment value to be type string");
         }
 
-        const std::vector<std::string> tokens =
-            strings::tokenize(value.as<JSON::String>().value, "=");
+        const vector<string> tokens =
+          strings::split(value.as<JSON::String>().value, "=", 2);
 
         if (tokens.size() != 2) {
           return Error("Unexpected Env format for 'ContainerConfig.Env'");

http://git-wip-us.apache.org/repos/asf/mesos/blob/a0d0e0cc/src/tests/containerizer/docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/docker_tests.cpp b/src/tests/containerizer/docker_tests.cpp
index f0ad20e..6208193 100644
--- a/src/tests/containerizer/docker_tests.cpp
+++ b/src/tests/containerizer/docker_tests.cpp
@@ -504,6 +504,7 @@ TEST_F(DockerImageTest, ParseInspectonImage)
     "            \"LANG=C.UTF-8\","
     "            \"JAVA_VERSION=8u66\","
     "            \"JAVA_DEBIAN_VERSION=8u66-b01-1~bpo8+1\","
+    "            \"SPARK_OPTS=--driver-java-options=-Xms1024M --driver-java-options=-Xmx4096M --driver-java-options=-Dlog4j.logLevel=info\"," // NOLINT(whitespace/line_length)
     "            \"CA_CERTIFICATES_JAVA_VERSION=20140324\""
     "        ],"
     "        \"Cmd\": ["
@@ -544,6 +545,7 @@ TEST_F(DockerImageTest, ParseInspectonImage)
     "            \"LANG=C.UTF-8\","
     "            \"JAVA_VERSION=8u66\","
     "            \"JAVA_DEBIAN_VERSION=8u66-b01-1~bpo8+1\","
+    "            \"SPARK_OPTS=--driver-java-options=-Xms1024M --driver-java-options=-Xmx4096M --driver-java-options=-Dlog4j.logLevel=info\"," // NOLINT(whitespace/line_length)
     "            \"CA_CERTIFICATES_JAVA_VERSION=20140324\""
     "        ],"
     "        \"Cmd\": null,"
@@ -577,6 +579,10 @@ TEST_F(DockerImageTest, ParseInspectonImage)
   EXPECT_EQ("8u66", image.get().environment.get().at("JAVA_VERSION"));
   EXPECT_EQ("8u66-b01-1~bpo8+1",
             image.get().environment.get().at("JAVA_DEBIAN_VERSION"));
+  EXPECT_EQ("--driver-java-options=-Xms1024M "
+            "--driver-java-options=-Xmx4096M "
+            "--driver-java-options=-Dlog4j.logLevel=info",
+            image.get().environment.get().at("SPARK_OPTS"));
   EXPECT_EQ("20140324",
             image.get().environment.get().at("CA_CERTIFICATES_JAVA_VERSION"));
 }


[12/48] mesos git commit: Synchronized V1 API for `type_utils.cpp`.

Posted by vi...@apache.org.
Synchronized V1 API for `type_utils.cpp`.


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

Branch: refs/heads/0.27.x
Commit: b0b978ce435130e23bcfbabe51f39b371947e6ab
Parents: d0e8145
Author: Joris Van Remoortere <jo...@gmail.com>
Authored: Wed Jan 27 22:37:02 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Wed Jan 27 22:42:58 2016 -0800

----------------------------------------------------------------------
 src/common/type_utils.cpp | 2 +-
 src/v1/mesos.cpp          | 3 ++-
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b0b978ce/src/common/type_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/type_utils.cpp b/src/common/type_utils.cpp
index 76f48f6..42e3061 100644
--- a/src/common/type_utils.cpp
+++ b/src/common/type_utils.cpp
@@ -391,7 +391,7 @@ ostream& operator<<(ostream& stream, const ExecutorInfo& executor)
 }
 
 
-ostream& operator<<(std::ostream& stream, const FrameworkID& frameworkId)
+ostream& operator<<(ostream& stream, const FrameworkID& frameworkId)
 {
   return stream << frameworkId.value();
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0b978ce/src/v1/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/v1/mesos.cpp b/src/v1/mesos.cpp
index 9264752..21b14cf 100644
--- a/src/v1/mesos.cpp
+++ b/src/v1/mesos.cpp
@@ -227,7 +227,8 @@ bool operator==(const Port& left, const Port& right)
 {
   return left.number() == right.number() &&
     left.name() == right.name() &&
-    left.protocol() == right.protocol();
+    left.protocol() == right.protocol() &&
+    left.visibility() == right.visibility();
 }