You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by gi...@apache.org on 2018/03/07 09:09:58 UTC

[01/10] mesos git commit: Handled hanging docker `stop`, `inspect` commands in docker executor.

Repository: mesos
Updated Branches:
  refs/heads/1.5.x 0de1cce2e -> c8dd080af


Handled hanging docker `stop`, `inspect` commands in docker executor.

Previosly, if `docker inspect` command hanged, the docker container
ended up in an unkillable state. This patch adds a timeout for inspect
command after receiving `killTask` analogically to `reaped` handler.
In addition we've added a timeout for `docker stop` command. If docker
`stop` or `inspect` command times out, we discard the related future,
thus the docker library kills previously spawned docker cli subprocess.
As a result, a scheduler can retry `killTask` operation to handle
nasty docker bugs that lead to hanging docker cli.

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


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

Branch: refs/heads/1.5.x
Commit: 65b6cbbad2eb0a8f381816f6a0e620726215e4b4
Parents: d323bbd
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri Mar 2 15:38:59 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Wed Mar 7 01:08:12 2018 -0800

----------------------------------------------------------------------
 src/docker/executor.cpp | 68 ++++++++++++++++++++++++++++++--------------
 1 file changed, 47 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/65b6cbba/src/docker/executor.cpp
----------------------------------------------------------------------
diff --git a/src/docker/executor.cpp b/src/docker/executor.cpp
index ea47b44..1fb5b53 100644
--- a/src/docker/executor.cpp
+++ b/src/docker/executor.cpp
@@ -96,7 +96,6 @@ public:
       killed(false),
       terminated(false),
       killedByHealthCheck(false),
-      killingInProgress(false),
       launcherDir(launcherDir),
       docker(docker),
       containerName(containerName),
@@ -322,6 +321,13 @@ public:
         return Nothing();
       }));
 
+    inspect
+      .after(DOCKER_INSPECT_TIMEOUT, [=](const Future<Nothing>&) {
+        LOG(WARNING) << "Docker inspect has not finished after "
+                     << DOCKER_INSPECT_TIMEOUT;
+        return inspect;
+      });
+
     inspect.onFailed(defer(self(), [=](const string& failure) {
       LOG(ERROR) << "Failed to inspect container '" << containerName << "'"
                  << ": " << failure;
@@ -441,9 +447,8 @@ private:
     // TODO(alexr): If a kill is in progress, consider adjusting
     // the grace period if a new one is provided.
 
-    // Issue the kill signal if the container is running
-    // and kill attempt is not in progress.
-    if (run.isSome() && !killingInProgress) {
+    // Issue the kill signal if there was an attempt to launch the container.
+    if (run.isSome()) {
       // We have to issue the kill after 'docker inspect' has
       // completed, otherwise we may race with 'docker run'
       // and docker may not know about the container. Note
@@ -452,6 +457,15 @@ private:
       // issued the kill.
       inspect
         .onAny(defer(self(), &Self::_killTask, _taskId, gracePeriod));
+
+      // If the inspect takes too long we discard it to ensure we
+      // don't wait forever, however in this case there may be no
+      // TASK_RUNNING update.
+      inspect
+        .after(DOCKER_INSPECT_TIMEOUT, [=](const Future<Nothing>&) {
+          inspect.discard();
+          return inspect;
+        });
     }
   }
 
@@ -462,9 +476,7 @@ private:
     CHECK_SOME(taskId);
     CHECK_EQ(taskId_, taskId.get());
 
-    if (!terminated && !killingInProgress) {
-      killingInProgress = true;
-
+    if (!terminated) {
       // Once the task has been transitioned to `killed`,
       // there is no way back, even if the kill attempt
       // failed. This also allows us to send TASK_KILLING
@@ -499,28 +511,43 @@ private:
         }
       }
 
+      // If a previous attempt to stop a Docker container is still in progress,
+      // we need to kill the hanging Docker CLI subprocess. Discarding this
+      // future triggers a callback in the Docker library that kills the
+      // subprocess.
+      if (stop.isPending()) {
+        LOG(WARNING) << "Previous docker stop has not terminated yet"
+                     << " for container '" << containerName << "'";
+        stop.discard();
+      }
+
       // TODO(bmahler): Replace this with 'docker kill' so
       // that we can adjust the grace period in the case of
       // a `KillPolicy` override.
+      //
+      // NOTE: `docker stop` may or may not finish. Our behaviour is to give
+      // the subprocess a chance to finish until next time `_killtask` is
+      // invoked. Also, invoking `docker stop` might be unsuccessful, in which
+      // case the container most probably does not receive the signal.
+      // In any case we should allow schedulers to retry the kill operation or,
+      // if the kill was initiated by a failing health check, retry ourselves.
+      // We do not bail out nor stop retrying to avoid sending a terminal
+      // status update while the container might still be running.
       stop = docker->stop(containerName, gracePeriod);
 
-      // Invoking `docker stop` might be unsuccessful, in which case the
-      // container most probably does not receive the signal. In this case we
-      // should allow schedulers to retry the kill operation or, if the kill
-      // was initiated by a failing health check, retry ourselves. We do not
-      // bail out nor stop retrying to avoid sending a terminal status update
-      // while the container might still be running.
-      //
-      // NOTE: `docker stop` might also hang. We do not address this for now,
-      // because there is no evidence that in this case docker daemon might
-      // function properly, i.e., it's only the docker cli command that hangs,
-      // and hence there is not so much we can do. See MESOS-6743.
+      if (killedByHealthCheck) {
+        stop
+          .after(KILL_RETRY_INTERVAL, defer(self(), [=](Future<Nothing>) {
+            LOG(INFO) << "Retrying to kill task";
+            _killTask(taskId_, gracePeriod);
+            return stop;
+          }));
+      }
+
       stop.onFailed(defer(self(), [=](const string& failure) {
         LOG(ERROR) << "Failed to stop container '" << containerName << "'"
                    << ": " << failure;
 
-        killingInProgress = false;
-
         if (killedByHealthCheck) {
           LOG(INFO) << "Retrying to kill task in " << KILL_RETRY_INTERVAL;
           delay(
@@ -743,7 +770,6 @@ private:
   bool terminated;
 
   bool killedByHealthCheck;
-  bool killingInProgress; // Guard against simultaneous kill attempts.
 
   string launcherDir;
   Owned<Docker> docker;


[05/10] mesos git commit: Windows: Fixed flaky Docker command health check test.

Posted by gi...@apache.org.
Windows: Fixed flaky Docker command health check test.

The `DockerContainerizerHealthCheckTest.ROOT_DOCKER_
DockerHealthStatusChange` test was flaky on Windows, because
the Docker executor manually reaps the container exit code in
case that `docker run` fails to get the exit code. This logic
doesn't work on Windows, since the process might not be visible to
the container host machine, causing `TASK_FAILED` to get sent. By
removing the reaping logic on Windows, the test is much more reliable.

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


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

Branch: refs/heads/1.5.x
Commit: e524ac337b6fb53c6754f3f1636df720f81f96b0
Parents: a56b182
Author: Akash Gupta <ak...@hotmail.com>
Authored: Sun Feb 25 13:37:42 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Wed Mar 7 01:08:12 2018 -0800

----------------------------------------------------------------------
 src/docker/executor.cpp | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e524ac33/src/docker/executor.cpp
----------------------------------------------------------------------
diff --git a/src/docker/executor.cpp b/src/docker/executor.cpp
index 80e2d81..ea47b44 100644
--- a/src/docker/executor.cpp
+++ b/src/docker/executor.cpp
@@ -277,6 +277,13 @@ public:
         // executor never returning although the container has already exited.
         // To workaround this issue, here we reap the container process directly
         // so we will be notified when the container exits.
+        //
+        // The issue has only been reported on Linux, so it's not clear if
+        // Windows also has this issue. Regardless, we don't use this workaround
+        // for Windows, because the pid legitimately might not exist. For
+        // example, if the container is running in Hyper-V isolation, the pid
+        // will only exist in the guest OS.
+#ifndef __WINDOWS__
         if (container.pid.isSome()) {
           process::reap(container.pid.get())
             .then(defer(self(), [=](const Option<int>& status) {
@@ -310,6 +317,7 @@ public:
               &Self::reapedContainer,
               None());
         }
+#endif // __WINDOWS__
 
         return Nothing();
       }));


[10/10] mesos git commit: Added inspect retries to the docker containerizer in `update` method.

Posted by gi...@apache.org.
Added inspect retries to the docker containerizer in `update` method.

This patch fixes the bug when a terminal status update is never sent
after termination of the docker executor, when the Docker daemon hangs
for `inspect` command. A terminal status update is postponed until
containerizer `update` completes that uses `inspect` command to get
a PID of the docker container. To address the issue, we retry `inspect`
in the loop.

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


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

Branch: refs/heads/1.5.x
Commit: c8dd080af92274eeab4a0d1937a9ff985000d61c
Parents: 811c5aa
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri Mar 2 15:39:09 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Wed Mar 7 01:08:13 2018 -0800

----------------------------------------------------------------------
 src/slave/constants.hpp            |  3 +++
 src/slave/containerizer/docker.cpp | 42 ++++++++++++++++++++++++++++++++-
 2 files changed, 44 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c8dd080a/src/slave/constants.hpp
----------------------------------------------------------------------
diff --git a/src/slave/constants.hpp b/src/slave/constants.hpp
index 030fb05..b79c084 100644
--- a/src/slave/constants.hpp
+++ b/src/slave/constants.hpp
@@ -135,6 +135,9 @@ constexpr Duration DOCKER_REMOVE_DELAY = Hours(6);
 // container.
 constexpr Duration DOCKER_INSPECT_DELAY = Seconds(1);
 
+// Default duration to wait for `inspect` command completion.
+constexpr Duration DOCKER_INSPECT_TIMEOUT = Seconds(5);
+
 // Default maximum number of docker inspect calls docker ps will invoke
 // in parallel to prevent hitting system's open file descriptor limit.
 constexpr size_t DOCKER_PS_MAX_INSPECT_CALLS = 100;

http://git-wip-us.apache.org/repos/asf/mesos/blob/c8dd080a/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index e589c7e..9d425e0 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -1711,7 +1711,47 @@ Future<Nothing> DockerContainerizerProcess::update(
     return __update(containerId, _resources, container->pid.get());
   }
 
-  return docker->inspect(containers_.at(containerId)->containerName)
+  string containerName = containers_.at(containerId)->containerName;
+
+  // Since the Docker daemon might hang, we have to retry the inspect command.
+  //
+  // NOTE: This code is duplicated from the built-in docker executor, but
+  // the retry interval is not passed to `inspect`, because the container might
+  // be terminated.
+  // TODO(abudnik): Consider using a class helper for retrying docker commands.
+  auto inspectLoop = loop(
+      self(),
+      [=]() {
+        return await(
+            docker->inspect(containerName)
+              .after(
+                  slave::DOCKER_INSPECT_TIMEOUT,
+                  [=](Future<Docker::Container> future) {
+                    LOG(WARNING) << "Docker inspect timed out after "
+                                 << slave::DOCKER_INSPECT_TIMEOUT
+                                 << " for container "
+                                 << "'" << containerName << "'";
+
+                    // We need to clean up the hanging Docker CLI process.
+                    // Discarding the inspect future triggers a callback in
+                    // the Docker library that kills the subprocess and
+                    // transitions the future.
+                    future.discard();
+                    return future;
+                  }));
+      },
+      [](const Future<Docker::Container>& future)
+          -> Future<ControlFlow<Docker::Container>> {
+        if (future.isReady()) {
+          return Break(future.get());
+        }
+        if (future.isFailed()) {
+          return Failure(future.failure());
+        }
+        return Continue();
+      });
+
+  return inspectLoop
     .then(defer(self(), &Self::_update, containerId, _resources, lambda::_1));
 #else
   return Nothing();


[06/10] mesos git commit: Avoided orphan subprocess in the Docker library.

Posted by gi...@apache.org.
Avoided orphan subprocess in the Docker library.

This patch ensures that `Docker::inspect` will not leave orphan
subprocesses behind.

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


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

Branch: refs/heads/1.5.x
Commit: d323bbd0836d7f8055d54cee699d4ecfdefb734e
Parents: e72def4
Author: Greg Mann <gr...@mesosphere.io>
Authored: Fri Mar 2 15:39:58 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Wed Mar 7 01:08:12 2018 -0800

----------------------------------------------------------------------
 src/docker/docker.cpp | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d323bbd0/src/docker/docker.cpp
----------------------------------------------------------------------
diff --git a/src/docker/docker.cpp b/src/docker/docker.cpp
index 3e3b8d5..1daa13f 100644
--- a/src/docker/docker.cpp
+++ b/src/docker/docker.cpp
@@ -1293,6 +1293,13 @@ void Docker::_inspect(
   // Set the `onDiscard` callback which will clean up the subprocess if the
   // caller discards the `Future` that we returned.
   synchronized (callback->second) {
+    // It's possible that the caller has discarded their future while we were
+    // creating a new subprocess, so we clean up here if necessary.
+    if (promise->future().hasDiscard()) {
+      commandDiscarded(s.get(), cmd);
+      return;
+    }
+
     callback->first = [promise, s, cmd]() {
       promise->discard();
       CHECK_SOME(s);


[04/10] mesos git commit: Reaped the container process directly in Docker executor.

Posted by gi...@apache.org.
Reaped the container process directly in Docker executor.

Due to a Docker issue (https://github.com/moby/moby/issues/33820),
Docker daemon can fail to catch a container exit, i.e., the container
process has already exited but the command `docker ps` shows the
container still running, this will lead to the "docker run" command
that we execute in Docker executor never returning, and it will also
cause the `docker stop` command takes no effect, i.e., it will return
without error but `docker ps` shows the container still running, so
the task will stuck in `TASK_KILLING` state.

To workaround this Docker issue, in this patch we made Docker executor
reaps the container process directly so Docker executor will be notified
once the container process exits.

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


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

Branch: refs/heads/1.5.x
Commit: a56b182edf31a138826a8cebfaf83858d69da70c
Parents: 0de1cce
Author: Qian Zhang <zh...@gmail.com>
Authored: Mon Feb 5 20:42:07 2018 +0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Wed Mar 7 01:08:12 2018 -0800

----------------------------------------------------------------------
 src/docker/executor.cpp | 60 ++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 60 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a56b182e/src/docker/executor.cpp
----------------------------------------------------------------------
diff --git a/src/docker/executor.cpp b/src/docker/executor.cpp
index e4c53d5..80e2d81 100644
--- a/src/docker/executor.cpp
+++ b/src/docker/executor.cpp
@@ -270,6 +270,47 @@ public:
           driver->sendStatusUpdate(status);
         }
 
+        // This is a workaround for the Docker issue below:
+        // https://github.com/moby/moby/issues/33820
+        // Due to this issue, Docker daemon can fail to catch a container exit,
+        // which will lead to the `docker run` command that we execute in this
+        // executor never returning although the container has already exited.
+        // To workaround this issue, here we reap the container process directly
+        // so we will be notified when the container exits.
+        if (container.pid.isSome()) {
+          process::reap(container.pid.get())
+            .then(defer(self(), [=](const Option<int>& status) {
+              // We cannot get the actual exit status of the container
+              // process since it is not a child process of this executor,
+              // so here `status` must be `None`.
+              CHECK_NONE(status);
+
+              // There will be a race between the method `reaped` and this
+              // lambda; ideally when the Docker issue mentioned above does
+              // not occur, `reaped` will be invoked (i.e., the `docker run`
+              // command returns) to get the actual exit status of the
+              // container, so here we wait a few seconds for `reaped` to be
+              // invoked. If `reaped` is not invoked within the timeout, that
+              // means we hit that Docker issue.
+              delay(
+                  Seconds(3),
+                  self(),
+                  &Self::reapedContainer,
+                  container.pid.get());
+
+              return Nothing();
+            }));
+        } else {
+          // This is the case that the container process has already exited,
+          // Similar to the above case, let's wait a few seconds for `reaped`
+          // to be invoked.
+          delay(
+              Seconds(3),
+              self(),
+              &Self::reapedContainer,
+              None());
+        }
+
         return Nothing();
       }));
 
@@ -485,8 +526,27 @@ private:
     }
   }
 
+  void reapedContainer(Option<pid_t> pid)
+  {
+    // Do nothing if the method `reaped` has already been invoked.
+    if (terminated) {
+      return;
+    }
+
+    // This means the Docker issue mentioned in `launchTask` occurs.
+    LOG(WARNING) << "The container process"
+                 << (pid.isSome() ? " (pid: " + stringify(pid.get()) + ")" : "")
+                 << " has exited, but Docker daemon failed to catch it.";
+
+    reaped(None());
+  }
+
   void reaped(const Future<Option<int>>& run)
   {
+    if (terminated) {
+      return;
+    }
+
     terminated = true;
 
     // Stop health checking the task.


[03/10] mesos git commit: Updated discard handling for Docker 'stop' and 'pull' commands.

Posted by gi...@apache.org.
Updated discard handling for Docker 'stop' and 'pull' commands.

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


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

Branch: refs/heads/1.5.x
Commit: 02e32c9ea5a5de25235f15df9a26cffa61297e8c
Parents: 42ab3f6
Author: Greg Mann <gr...@mesosphere.io>
Authored: Fri Feb 23 16:41:46 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Wed Mar 7 01:08:12 2018 -0800

----------------------------------------------------------------------
 src/docker/docker.cpp | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/02e32c9e/src/docker/docker.cpp
----------------------------------------------------------------------
diff --git a/src/docker/docker.cpp b/src/docker/docker.cpp
index 3f20900..6013707 100644
--- a/src/docker/docker.cpp
+++ b/src/docker/docker.cpp
@@ -1163,7 +1163,8 @@ Future<Nothing> Docker::stop(
         containerName,
         cmd,
         s.get(),
-        remove));
+        remove))
+    .onDiscard(lambda::bind(&commandDiscarded, s.get(), cmd));
 }
 
 
@@ -1581,7 +1582,8 @@ Future<Docker::Image> Docker::pull(
         path,
         socket,
         config,
-        output));
+        output))
+    .onDiscard(lambda::bind(&commandDiscarded, s.get(), cmd));
 }
 
 


[07/10] mesos git commit: Updated discard handling in `Docker::inspect()`.

Posted by gi...@apache.org.
Updated discard handling in `Docker::inspect()`.

Previously, discards of the `Future` returned by `Docker::inspect()`
were only handled at the beginning of each asynchronous continuation
in the library function's call chain. This meant that if a Docker
CLI command became stuck in between async calls, discarding the
`Future` would have no effect.

This patch adds an `onDiscard` callback to the `Future` to ensure
that any discards have the desired effect: cleanup of any spawned
subprocess, and a transition of the `Future` to the discarded state.

Since the Docker library is not a libprocess process, we must
implement this with a `shared_ptr` and a mutex, to protect against
concurrent access to the `onDiscard` callback, which must be updated
when retries are performed.

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


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

Branch: refs/heads/1.5.x
Commit: e72def4926049d45641bbeb761faeb2613494441
Parents: 93fbdbe
Author: Greg Mann <gr...@mesosphere.io>
Authored: Wed Feb 28 15:24:17 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Wed Mar 7 01:08:12 2018 -0800

----------------------------------------------------------------------
 src/docker/docker.cpp | 51 ++++++++++++++++++++++++++++++++++------------
 src/docker/docker.hpp | 14 ++++++++++---
 2 files changed, 49 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e72def49/src/docker/docker.cpp
----------------------------------------------------------------------
diff --git a/src/docker/docker.cpp b/src/docker/docker.cpp
index 6013707..3e3b8d5 100644
--- a/src/docker/docker.cpp
+++ b/src/docker/docker.cpp
@@ -15,6 +15,8 @@
 // limitations under the License.
 
 #include <map>
+#include <mutex>
+#include <utility>
 #include <vector>
 
 #include <stout/error.hpp>
@@ -56,6 +58,9 @@ using namespace process;
 
 using std::list;
 using std::map;
+using std::mutex;
+using std::pair;
+using std::shared_ptr;
 using std::string;
 using std::vector;
 
@@ -1246,20 +1251,29 @@ Future<Docker::Container> Docker::inspect(
 {
   Owned<Promise<Docker::Container>> promise(new Promise<Docker::Container>());
 
+  // Holds a callback used for cleanup in case this call to 'docker inspect' is
+  // discarded, and a mutex to control access to the callback.
+  auto callback = std::make_shared<pair<lambda::function<void()>, mutex>>();
+
   const string cmd = path + " -H " + socket + " inspect " + containerName;
-  _inspect(cmd, promise, retryInterval);
+  _inspect(cmd, promise, retryInterval, callback);
 
-  return promise->future();
+  return promise->future()
+    .onDiscard([callback]() {
+      synchronized (callback->second) {
+        callback->first();
+      }
+    });
 }
 
 
 void Docker::_inspect(
     const string& cmd,
     const Owned<Promise<Docker::Container>>& promise,
-    const Option<Duration>& retryInterval)
+    const Option<Duration>& retryInterval,
+    shared_ptr<pair<lambda::function<void()>, mutex>> callback)
 {
   if (promise->future().hasDiscard()) {
-    promise->discard();
     return;
   }
 
@@ -1276,13 +1290,25 @@ void Docker::_inspect(
     return;
   }
 
+  // Set the `onDiscard` callback which will clean up the subprocess if the
+  // caller discards the `Future` that we returned.
+  synchronized (callback->second) {
+    callback->first = [promise, s, cmd]() {
+      promise->discard();
+      CHECK_SOME(s);
+      commandDiscarded(s.get(), cmd);
+    };
+  }
+
   // Start reading from stdout so writing to the pipe won't block
   // to handle cases where the output is larger than the pipe
   // capacity.
   const Future<string> output = io::read(s.get().out().get());
 
   s.get().status()
-    .onAny([=]() { __inspect(cmd, promise, retryInterval, output, s.get()); });
+    .onAny([=]() {
+      __inspect(cmd, promise, retryInterval, output, s.get(), callback);
+    });
 }
 
 
@@ -1291,11 +1317,10 @@ void Docker::__inspect(
     const Owned<Promise<Docker::Container>>& promise,
     const Option<Duration>& retryInterval,
     Future<string> output,
-    const Subprocess& s)
+    const Subprocess& s,
+    shared_ptr<pair<lambda::function<void()>, mutex>> callback)
 {
   if (promise->future().hasDiscard()) {
-    promise->discard();
-    output.discard();
     return;
   }
 
@@ -1313,7 +1338,7 @@ void Docker::__inspect(
       VLOG(1) << "Retrying inspect with non-zero status code. cmd: '"
               << cmd << "', interval: " << stringify(retryInterval.get());
       Clock::timer(retryInterval.get(),
-                   [=]() { _inspect(cmd, promise, retryInterval); } );
+                   [=]() { _inspect(cmd, promise, retryInterval, callback); });
       return;
     }
 
@@ -1335,7 +1360,7 @@ void Docker::__inspect(
   CHECK_SOME(s.out());
   output
     .onAny([=](const Future<string>& output) {
-      ___inspect(cmd, promise, retryInterval, output);
+      ___inspect(cmd, promise, retryInterval, output, callback);
     });
 }
 
@@ -1344,10 +1369,10 @@ void Docker::___inspect(
     const string& cmd,
     const Owned<Promise<Docker::Container>>& promise,
     const Option<Duration>& retryInterval,
-    const Future<string>& output)
+    const Future<string>& output,
+    shared_ptr<pair<lambda::function<void()>, mutex>> callback)
 {
   if (promise->future().hasDiscard()) {
-    promise->discard();
     return;
   }
 
@@ -1368,7 +1393,7 @@ void Docker::___inspect(
     VLOG(1) << "Retrying inspect since container not yet started. cmd: '"
             << cmd << "', interval: " << stringify(retryInterval.get());
     Clock::timer(retryInterval.get(),
-                 [=]() { _inspect(cmd, promise, retryInterval); } );
+                 [=]() { _inspect(cmd, promise, retryInterval, callback); } );
     return;
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/e72def49/src/docker/docker.hpp
----------------------------------------------------------------------
diff --git a/src/docker/docker.hpp b/src/docker/docker.hpp
index d9e71f8..f5e4a70 100644
--- a/src/docker/docker.hpp
+++ b/src/docker/docker.hpp
@@ -19,7 +19,9 @@
 
 #include <list>
 #include <map>
+#include <mutex>
 #include <string>
+#include <utility>
 
 #include <process/future.hpp>
 #include <process/owned.hpp>
@@ -340,20 +342,26 @@ private:
   static void _inspect(
       const std::string& cmd,
       const process::Owned<process::Promise<Container>>& promise,
-      const Option<Duration>& retryInterval);
+      const Option<Duration>& retryInterval,
+      std::shared_ptr<std::pair<lambda::function<void()>, std::mutex>>
+        callback);
 
   static void __inspect(
       const std::string& cmd,
       const process::Owned<process::Promise<Container>>& promise,
       const Option<Duration>& retryInterval,
       process::Future<std::string> output,
-      const process::Subprocess& s);
+      const process::Subprocess& s,
+      std::shared_ptr<std::pair<lambda::function<void()>, std::mutex>>
+        callback);
 
   static void ___inspect(
       const std::string& cmd,
       const process::Owned<process::Promise<Container>>& promise,
       const Option<Duration>& retryInterval,
-      const process::Future<std::string>& output);
+      const process::Future<std::string>& output,
+      std::shared_ptr<std::pair<lambda::function<void()>, std::mutex>>
+        callback);
 
   static process::Future<std::list<Container>> _ps(
       const Docker& docker,


[08/10] mesos git commit: Prevented Docker library from terminating incorrect processes.

Posted by gi...@apache.org.
Prevented Docker library from terminating incorrect processes.

Previously, the Docker library might call `os::killtree()` on a
PID after the associated subprocess had already terminated, which
could lead to an unknown process being incorrectly killed.

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


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

Branch: refs/heads/1.5.x
Commit: 42ab3f6417d07fce3fcfb3d713cdcb7d04d64039
Parents: e524ac3
Author: Greg Mann <gr...@mesosphere.io>
Authored: Fri Feb 23 16:41:44 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Wed Mar 7 01:08:12 2018 -0800

----------------------------------------------------------------------
 src/docker/docker.cpp | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/42ab3f64/src/docker/docker.cpp
----------------------------------------------------------------------
diff --git a/src/docker/docker.cpp b/src/docker/docker.cpp
index c2feebe..3f20900 100644
--- a/src/docker/docker.cpp
+++ b/src/docker/docker.cpp
@@ -148,8 +148,10 @@ Try<Owned<Docker>> Docker::create(
 
 void commandDiscarded(const Subprocess& s, const string& cmd)
 {
-  VLOG(1) << "'" << cmd << "' is being discarded";
-  os::killtree(s.pid(), SIGKILL);
+  if (s.status().isPending()) {
+    VLOG(1) << "'" << cmd << "' is being discarded";
+    os::killtree(s.pid(), SIGKILL);
+  }
 }
 
 


[09/10] mesos git commit: Added inspect retries to the Docker executor.

Posted by gi...@apache.org.
Added inspect retries to the Docker executor.

This patch adds retries for `inspect` command to workaround docker
daemon hangs. We assume that the docker daemon can be temporarily
unresponsive. If it's unresponsive, then any started docker cli
command hangs. To address the issue, we retry `inspect` in the loop.

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


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

Branch: refs/heads/1.5.x
Commit: 811c5aae799bde94ea6bceeb06303bb3e0b52d67
Parents: 65b6cbb
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri Mar 2 15:39:05 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Wed Mar 7 01:08:13 2018 -0800

----------------------------------------------------------------------
 src/docker/executor.cpp | 46 +++++++++++++++++++++++++++++++++++---------
 1 file changed, 37 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/811c5aae/src/docker/executor.cpp
----------------------------------------------------------------------
diff --git a/src/docker/executor.cpp b/src/docker/executor.cpp
index 1fb5b53..de462d9 100644
--- a/src/docker/executor.cpp
+++ b/src/docker/executor.cpp
@@ -22,8 +22,10 @@
 #include <mesos/executor.hpp>
 #include <mesos/mesos.hpp>
 
+#include <process/collect.hpp>
 #include <process/delay.hpp>
 #include <process/id.hpp>
+#include <process/loop.hpp>
 #include <process/owned.hpp>
 #include <process/process.hpp>
 #include <process/protobuf.hpp>
@@ -214,13 +216,46 @@ public:
 
     run->onAny(defer(self(), &Self::reaped, lambda::_1));
 
+    // Since the Docker daemon might hang, we have to retry the inspect command.
+    auto inspectLoop = loop(
+        self(),
+        [=]() {
+          return await(
+              docker->inspect(containerName, DOCKER_INSPECT_DELAY)
+                .after(
+                    DOCKER_INSPECT_TIMEOUT,
+                    [=](Future<Docker::Container> future) {
+                      LOG(WARNING) << "Docker inspect timed out after "
+                                   << DOCKER_INSPECT_TIMEOUT
+                                   << " for container "
+                                   << "'" << containerName << "'";
+
+                      // We need to clean up the hanging Docker CLI process.
+                      // Discarding the inspect future triggers a callback in
+                      // the Docker library that kills the subprocess and
+                      // transitions the future.
+                      future.discard();
+                      return future;
+                    }));
+        },
+        [](const Future<Docker::Container>& future)
+            -> Future<ControlFlow<Docker::Container>> {
+          if (future.isReady()) {
+            return Break(future.get());
+          }
+          if (future.isFailed()) {
+            return Failure(future.failure());
+          }
+          return Continue();
+        });
+
     // Delay sending TASK_RUNNING status update until we receive
     // inspect output. Note that we store a future that completes
     // after the sending of the running update. This allows us to
     // ensure that the terminal update is sent after the running
     // update (see `reaped()`).
-    inspect = docker->inspect(containerName, DOCKER_INSPECT_DELAY)
-      .then(defer(self(), [=](const Docker::Container& container) {
+    inspect =
+      inspectLoop.then(defer(self(), [=](const Docker::Container& container) {
         if (!killed) {
           containerPid = container.pid;
 
@@ -321,13 +356,6 @@ public:
         return Nothing();
       }));
 
-    inspect
-      .after(DOCKER_INSPECT_TIMEOUT, [=](const Future<Nothing>&) {
-        LOG(WARNING) << "Docker inspect has not finished after "
-                     << DOCKER_INSPECT_TIMEOUT;
-        return inspect;
-      });
-
     inspect.onFailed(defer(self(), [=](const string& failure) {
       LOG(ERROR) << "Failed to inspect container '" << containerName << "'"
                  << ": " << failure;


[02/10] mesos git commit: Ensured that Docker containerizer returns a failed Future in one case.

Posted by gi...@apache.org.
Ensured that Docker containerizer returns a failed Future in one case.

Because the Docker library did not immediately transition the
Future returned by `inspect()` to DISCARDED, it was safe for the
Docker containerizer to discard this Future before failing the
Promise associated with the Future returned by `launch()`.

However, the introduction of an `onDiscard` callback in
`Docker::inspect()` makes this assumption invalid. This patch
addresses this by failing the Promise before discarding the
Future.

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


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

Branch: refs/heads/1.5.x
Commit: 93fbdbe83ee588cb92f9ad1cf05f84957ff33de7
Parents: 02e32c9
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Feb 27 16:42:10 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Wed Mar 7 01:08:12 2018 -0800

----------------------------------------------------------------------
 src/slave/containerizer/docker.cpp | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/93fbdbe8/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 7585178..e589c7e 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -1442,15 +1442,15 @@ Future<Docker::Container> DockerContainerizerProcess::launchExecutorContainer(
 
     run.onAny([=]() mutable {
       if (!run.isReady()) {
-        inspect.discard();
         promise->fail(run.isFailed() ? run.failure() : "discarded");
-      } else if (run->isNone()) {
         inspect.discard();
+      } else if (run->isNone()) {
         promise->fail("Failed to obtain exit status of container");
+        inspect.discard();
       } else {
         if (!WSUCCEEDED(run->get())) {
-          inspect.discard();
           promise->fail("Container " + WSTRINGIFY(run->get()));
+          inspect.discard();
         }
 
         // TODO(bmahler): Handle the case where the 'run' exits