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/02 23:56:41 UTC

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

Repository: mesos
Updated Branches:
  refs/heads/master b0a269691 -> 76c38f9d0


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/32fe3905
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/32fe3905
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/32fe3905

Branch: refs/heads/master
Commit: 32fe39055545a6511c1613be9907cbb3357d86a4
Parents: 8346ab0
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri Mar 2 15:38:59 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Fri Mar 2 15:40:30 2018 -0800

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


http://git-wip-us.apache.org/repos/asf/mesos/blob/32fe3905/src/docker/executor.cpp
----------------------------------------------------------------------
diff --git a/src/docker/executor.cpp b/src/docker/executor.cpp
index 93c3e1d..8fe8a7c 100644
--- a/src/docker/executor.cpp
+++ b/src/docker/executor.cpp
@@ -97,7 +97,6 @@ public:
       killed(false),
       terminated(false),
       killedByHealthCheck(false),
-      killingInProgress(false),
       launcherDir(launcherDir),
       docker(docker),
       containerName(containerName),
@@ -323,6 +322,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;
@@ -442,9 +448,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
@@ -453,6 +458,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;
+        });
     }
   }
 
@@ -463,9 +477,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
@@ -500,28 +512,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(
@@ -724,7 +751,6 @@ private:
   bool terminated;
 
   bool killedByHealthCheck;
-  bool killingInProgress; // Guard against simultaneous kill attempts.
 
   string launcherDir;
   Owned<Docker> docker;


[3/4] 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/84c3b4c4
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/84c3b4c4
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/84c3b4c4

Branch: refs/heads/master
Commit: 84c3b4c4b22b100644f46b561772732e942ceb49
Parents: 32fe390
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri Mar 2 15:39:05 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Fri Mar 2 15:40:31 2018 -0800

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


http://git-wip-us.apache.org/repos/asf/mesos/blob/84c3b4c4/src/docker/executor.cpp
----------------------------------------------------------------------
diff --git a/src/docker/executor.cpp b/src/docker/executor.cpp
index 8fe8a7c..1d67211 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>
@@ -215,13 +217,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;
 
@@ -322,13 +357,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;


[2/4] 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/8346ab0c
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/8346ab0c
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/8346ab0c

Branch: refs/heads/master
Commit: 8346ab0c812559ef73e1bbd30718f6c74a023079
Parents: b0a2696
Author: Greg Mann <gr...@mesosphere.io>
Authored: Fri Mar 2 15:39:58 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Fri Mar 2 15:40:30 2018 -0800

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


http://git-wip-us.apache.org/repos/asf/mesos/blob/8346ab0c/src/docker/docker.cpp
----------------------------------------------------------------------
diff --git a/src/docker/docker.cpp b/src/docker/docker.cpp
index 9dd4ceb..bb81c4d 100644
--- a/src/docker/docker.cpp
+++ b/src/docker/docker.cpp
@@ -1324,6 +1324,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);


[4/4] 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/76c38f9d
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/76c38f9d
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/76c38f9d

Branch: refs/heads/master
Commit: 76c38f9d03ee6854e6bcd00a959d697472e0ea58
Parents: 84c3b4c
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri Mar 2 15:39:09 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Fri Mar 2 15:40:31 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/76c38f9d/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/76c38f9d/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 1f4eeb4..31d64a7 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();