You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by an...@apache.org on 2017/08/16 00:49:41 UTC

[1/7] mesos git commit: Added `kill()` support to the test containerizer interface.

Repository: mesos
Updated Branches:
  refs/heads/master 8447d196a -> 6a1e4f7c6


Added `kill()` support to the test containerizer interface.

This is needed to make some tests pass that rely on the test
containerizer.

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


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

Branch: refs/heads/master
Commit: 6a1e4f7c6639ec4897c3a4a40cffea41ce302a94
Parents: 0eab58a
Author: Anand Mazumdar <an...@apache.org>
Authored: Tue Aug 15 17:41:27 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Tue Aug 15 17:48:39 2017 -0700

----------------------------------------------------------------------
 src/tests/containerizer.cpp | 20 ++++++++++++++++++++
 src/tests/containerizer.hpp |  8 ++++++++
 2 files changed, 28 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6a1e4f7c/src/tests/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.cpp b/src/tests/containerizer.cpp
index 1d2b639..cd140f4 100644
--- a/src/tests/containerizer.cpp
+++ b/src/tests/containerizer.cpp
@@ -303,6 +303,11 @@ public:
     return destroy(containerId.get());
   }
 
+  Future<bool> kill(const ContainerID& containerId, int /* signal */)
+  {
+    return destroy(containerId);
+  }
+
   Future<hashset<ContainerID>> containers()
   {
     return containers_.keys();
@@ -429,6 +434,9 @@ void TestContainerizer::setup()
 
   EXPECT_CALL(*this, destroy(_))
     .WillRepeatedly(Invoke(this, &TestContainerizer::_destroy));
+
+  EXPECT_CALL(*this, kill(_, _))
+    .WillRepeatedly(Invoke(this, &TestContainerizer::_kill));
 }
 
 
@@ -524,6 +532,18 @@ Future<bool> TestContainerizer::_destroy(
 }
 
 
+Future<bool> TestContainerizer::_kill(
+    const ContainerID& containerId,
+    int signal)
+{
+  return process::dispatch(
+      process.get(),
+      &TestContainerizerProcess::kill,
+      containerId,
+      signal);
+}
+
+
 Future<bool> TestContainerizer::destroy(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId)

http://git-wip-us.apache.org/repos/asf/mesos/blob/6a1e4f7c/src/tests/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.hpp b/src/tests/containerizer.hpp
index 4bd40c3..a778b85 100644
--- a/src/tests/containerizer.hpp
+++ b/src/tests/containerizer.hpp
@@ -118,6 +118,10 @@ public:
       destroy,
       process::Future<bool>(const ContainerID&));
 
+  MOCK_METHOD2(
+      kill,
+      process::Future<bool>(const ContainerID&, int));
+
   // Additional destroy method for testing because we won't know the
   // ContainerID created for each container.
   process::Future<bool> destroy(
@@ -159,6 +163,10 @@ private:
   process::Future<bool> _destroy(
       const ContainerID& containerId);
 
+  process::Future<bool> _kill(
+      const ContainerID& containerId,
+      int status);
+
   process::Owned<TestContainerizerProcess> process;
 };
 


[5/7] mesos git commit: Added `kill()` call to the composing containerizer.

Posted by an...@apache.org.
Added `kill()` call to the composing containerizer.

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


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

Branch: refs/heads/master
Commit: 8c8e2af56457346af586e94f071bfa14a92eb20a
Parents: 8b3fd24
Author: Anand Mazumdar <an...@apache.org>
Authored: Tue Aug 15 17:38:04 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Tue Aug 15 17:48:39 2017 -0700

----------------------------------------------------------------------
 src/slave/containerizer/composing.cpp | 25 +++++++++++++++++++++++++
 src/slave/containerizer/composing.hpp |  4 ++++
 2 files changed, 29 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8c8e2af5/src/slave/containerizer/composing.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.cpp b/src/slave/containerizer/composing.cpp
index a003e1b..f1a9c3d 100644
--- a/src/slave/containerizer/composing.cpp
+++ b/src/slave/containerizer/composing.cpp
@@ -89,6 +89,8 @@ public:
 
   Future<bool> destroy(const ContainerID& containerId);
 
+  Future<bool> kill(const ContainerID& containerId, int signal);
+
   Future<hashset<ContainerID>> containers();
 
   Future<Nothing> remove(const ContainerID& containerId);
@@ -232,6 +234,17 @@ Future<bool> ComposingContainerizer::destroy(const ContainerID& containerId)
 }
 
 
+Future<bool> ComposingContainerizer::kill(
+    const ContainerID& containerId,
+    int signal)
+{
+  return dispatch(process,
+                  &ComposingContainerizerProcess::kill,
+                  containerId,
+                  signal);
+}
+
+
 Future<hashset<ContainerID>> ComposingContainerizer::containers()
 {
   return dispatch(process, &ComposingContainerizerProcess::containers);
@@ -630,6 +643,18 @@ Future<bool> ComposingContainerizerProcess::destroy(
 }
 
 
+Future<bool> ComposingContainerizerProcess::kill(
+    const ContainerID& containerId,
+    int signal)
+{
+  if (!containers_.contains(containerId)) {
+    return false;
+  }
+
+  return containers_.at(containerId)->containerizer->kill(containerId, signal);
+}
+
+
 Future<hashset<ContainerID>> ComposingContainerizerProcess::containers()
 {
   return containers_.keys();

http://git-wip-us.apache.org/repos/asf/mesos/blob/8c8e2af5/src/slave/containerizer/composing.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.hpp b/src/slave/containerizer/composing.hpp
index bef6d88..06d68ee 100644
--- a/src/slave/containerizer/composing.hpp
+++ b/src/slave/containerizer/composing.hpp
@@ -78,6 +78,10 @@ public:
 
   virtual process::Future<bool> destroy(const ContainerID& containerId);
 
+  virtual process::Future<bool> kill(
+      const ContainerID& containerId,
+      int signal);
+
   virtual process::Future<hashset<ContainerID>> containers();
 
   virtual process::Future<Nothing> remove(const ContainerID& containerId);


[4/7] mesos git commit: Added the field `signal` to the `KillNestedContainer` call.

Posted by an...@apache.org.
Added the field `signal` to the `KillNestedContainer` call.

This would be used later for sending signals (SIGTERM, SIGKILL etc.)
to the running container. Previously, SIGKILL was used by default.

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


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

Branch: refs/heads/master
Commit: 093ccde55e68b705fe5ef6e851be1dc2b7e436da
Parents: 8447d19
Author: Anand Mazumdar <an...@apache.org>
Authored: Tue Aug 15 17:37:49 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Tue Aug 15 17:48:39 2017 -0700

----------------------------------------------------------------------
 include/mesos/agent/agent.proto    | 5 ++++-
 include/mesos/v1/agent/agent.proto | 5 ++++-
 2 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/093ccde5/include/mesos/agent/agent.proto
----------------------------------------------------------------------
diff --git a/include/mesos/agent/agent.proto b/include/mesos/agent/agent.proto
index 9bac954..7c8c8a7 100644
--- a/include/mesos/agent/agent.proto
+++ b/include/mesos/agent/agent.proto
@@ -121,9 +121,12 @@ message Call {
     required ContainerID container_id = 1;
   }
 
-  // Kills the nested container. Currently only supports SIGKILL.
+  // Kills the nested container. The signal (e.g., SIGTERM, SIGKILL, etc.)
+  // to be sent to the container can be specified in the 'signal' field.
+  // If 'signal' is not set, SIGKILL is used by default.
   message KillNestedContainer {
     required ContainerID container_id = 1;
+    optional int32 signal = 2;
   }
 
   // Removes a nested container and its artifacts (runtime and sandbox

http://git-wip-us.apache.org/repos/asf/mesos/blob/093ccde5/include/mesos/v1/agent/agent.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/agent/agent.proto b/include/mesos/v1/agent/agent.proto
index ea9282c..3e19912 100644
--- a/include/mesos/v1/agent/agent.proto
+++ b/include/mesos/v1/agent/agent.proto
@@ -121,9 +121,12 @@ message Call {
     required ContainerID container_id = 1;
   }
 
-  // Kills the nested container. Currently only supports SIGKILL.
+  // Kills the nested container. The signal (e.g., SIGTERM, SIGKILL, etc.)
+  // to be sent to the container can be specified in the 'signal' field.
+  // If 'signal' is not set, SIGKILL is used by default.
   message KillNestedContainer {
     required ContainerID container_id = 1;
+    optional int32 signal = 2;
   }
 
   // Removes a nested container and its artifacts (runtime and sandbox


[6/7] mesos git commit: Added support for kill policies to the default executor.

Posted by an...@apache.org.
Added support for kill policies to the default executor.

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


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

Branch: refs/heads/master
Commit: 0eab58a4790f22dfc1cec70367364bdedb6024f8
Parents: d1abe1e
Author: Anand Mazumdar <an...@apache.org>
Authored: Tue Aug 15 17:38:55 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Tue Aug 15 17:48:39 2017 -0700

----------------------------------------------------------------------
 src/launcher/default_executor.cpp | 62 +++++++++++++++++++++++++++-------
 1 file changed, 49 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0eab58a4/src/launcher/default_executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/default_executor.cpp b/src/launcher/default_executor.cpp
index a23097e..30bae5c 100644
--- a/src/launcher/default_executor.cpp
+++ b/src/launcher/default_executor.cpp
@@ -230,7 +230,11 @@ public:
       }
 
       case Event::KILL: {
-        killTask(event.kill().task_id());
+        Option<KillPolicy> killPolicy = event.kill().has_kill_policy()
+          ? Option<KillPolicy>(event.kill().kill_policy())
+          : None();
+
+        killTask(event.kill().task_id(), killPolicy);
         break;
       }
 
@@ -966,7 +970,9 @@ protected:
     terminate(self());
   }
 
-  Future<Nothing> kill(Owned<Container> container)
+  Future<Nothing> kill(
+      Owned<Container> container,
+      const Option<KillPolicy>& killPolicy = None())
   {
     CHECK_EQ(SUBSCRIBED, state);
 
@@ -993,27 +999,52 @@ protected:
       container->healthChecker = None();
     }
 
+    const TaskID& taskId = container->taskInfo.task_id();
+
     LOG(INFO)
-      << "Killing task " << container->taskInfo.task_id() << " running in child"
-      << " container " << container->containerId << " with SIGTERM signal";
+      << "Killing task " << taskId << " running in child container"
+      << " " << container->containerId << " with SIGTERM signal";
 
     // Default grace period is set to 3s.
-    //
-    // TODO(anand): Add support for handling kill policies.
-    const Duration GRACE_PERIOD = Seconds(3);
+    Duration gracePeriod = Seconds(3);
+
+    Option<KillPolicy> taskInfoKillPolicy;
+    if (container->taskInfo.has_kill_policy()) {
+      taskInfoKillPolicy = container->taskInfo.kill_policy();
+    }
+
+    // Kill policy provided in the `Kill` event takes precedence
+    // over kill policy specified when the task was launched.
+    if (killPolicy.isSome() && killPolicy->has_grace_period()) {
+      gracePeriod = Nanoseconds(killPolicy->grace_period().nanoseconds());
+    } else if (taskInfoKillPolicy.isSome() &&
+               taskInfoKillPolicy->has_grace_period()) {
+      gracePeriod =
+        Nanoseconds(taskInfoKillPolicy->grace_period().nanoseconds());
+    }
 
-    LOG(INFO) << "Scheduling escalation to SIGKILL in " << GRACE_PERIOD
+    LOG(INFO) << "Scheduling escalation to SIGKILL in " << gracePeriod
               << " from now";
 
     const ContainerID& containerId = container->containerId;
 
-    delay(GRACE_PERIOD,
+    delay(gracePeriod,
           self(),
           &Self::escalated,
           connectionId.get(),
           containerId,
           container->taskInfo.task_id(),
-          GRACE_PERIOD);
+          gracePeriod);
+
+    // Send a 'TASK_KILLING' update if the framework can handle it.
+    CHECK_SOME(frameworkInfo);
+
+    if (protobuf::frameworkHasCapability(
+            frameworkInfo.get(),
+            FrameworkInfo::Capability::TASK_KILLING_STATE)) {
+      TaskStatus status = createTaskStatus(taskId, TASK_KILLING);
+      forward(status);
+    }
 
     return kill(containerId, SIGTERM);
   }
@@ -1068,7 +1099,9 @@ protected:
     kill(containerId, SIGKILL);
   }
 
-  void killTask(const TaskID& taskId)
+  void killTask(
+      const TaskID& taskId,
+      const Option<KillPolicy>& killPolicy = None())
   {
     if (shuttingDown) {
       LOG(WARNING) << "Ignoring kill for task '" << taskId
@@ -1078,7 +1111,10 @@ protected:
 
     CHECK_EQ(SUBSCRIBED, state);
 
-    // TODO(anand): Add support for handling kill policies.
+    // TODO(anand): Add support for adjusting the remaining grace period if
+    // we receive another kill request while a task is being killed but has
+    // not terminated yet. See similar comments in the command executor
+    // for more context.
 
     LOG(INFO) << "Received kill for task '" << taskId << "'";
 
@@ -1095,7 +1131,7 @@ protected:
       return;
     }
 
-    kill(container);
+    kill(container, killPolicy);
   }
 
   void taskCheckUpdated(


[3/7] mesos git commit: Added the `kill()` function to the containerizer interface.

Posted by an...@apache.org.
Added the `kill()` function to the containerizer interface.

This would be used by now on for killing a container by sending
a signal to it similar to the linux equivalent `kill()` system call.

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


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

Branch: refs/heads/master
Commit: 8b3fd242f4af1e2e6d7af74e865b8f42c29af3be
Parents: 093ccde
Author: Anand Mazumdar <an...@apache.org>
Authored: Tue Aug 15 17:37:54 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Tue Aug 15 17:48:39 2017 -0700

----------------------------------------------------------------------
 src/slave/containerizer/containerizer.hpp       | 10 +++++
 src/slave/containerizer/mesos/containerizer.cpp | 46 ++++++++++++++++++++
 src/slave/containerizer/mesos/containerizer.hpp |  8 ++++
 3 files changed, 64 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8b3fd242/src/slave/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index 0954ed6..449bb5d 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -139,6 +139,16 @@ public:
   // so you should always call wait() before destroy().
   virtual process::Future<bool> destroy(const ContainerID& containerId) = 0;
 
+  // Sends a signal to a running container. Returns false when the container
+  // cannot be found. The future may be failed if an error occurs in sending
+  // the signal to the running container.
+  virtual process::Future<bool> kill(
+      const ContainerID& containerId,
+      int signal)
+  {
+    return process::Failure("Unsupported");
+  };
+
   virtual process::Future<hashset<ContainerID>> containers() = 0;
 
   // Remove a nested container, including its sandbox and runtime directories.

http://git-wip-us.apache.org/repos/asf/mesos/blob/8b3fd242/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index ff192bb..5772421 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -577,6 +577,17 @@ Future<bool> MesosContainerizer::destroy(const ContainerID& containerId)
 }
 
 
+Future<bool> MesosContainerizer::kill(
+    const ContainerID& containerId,
+    int signal)
+{
+  return dispatch(process.get(),
+                  &MesosContainerizerProcess::kill,
+                  containerId,
+                  signal);
+}
+
+
 Future<hashset<ContainerID>> MesosContainerizer::containers()
 {
   return dispatch(process.get(),
@@ -2428,6 +2439,41 @@ void MesosContainerizerProcess::______destroy(
 }
 
 
+Future<bool> MesosContainerizerProcess::kill(
+    const ContainerID& containerId,
+    int signal)
+{
+  if (!containers_.contains(containerId)) {
+    LOG(WARNING) << "Attempted to kill unknown container " << containerId;
+
+    return false;
+  }
+
+  const Owned<Container>& container = containers_.at(containerId);
+
+  // This can happen when we try to signal the container before it
+  // is launched. We destroy the container forcefully in this case.
+  //
+  // TODO(anand): Consider chaining this to the launch completion
+  // future instead.
+  if (container->pid.isNone()) {
+    LOG(WARNING) << "Unable to find the pid for container " << containerId
+                 << ", destroying it";
+
+    destroy(containerId);
+    return true;
+  }
+
+  int status = os::kill(container->pid.get(), signal);
+  if (status != 0) {
+    return Failure("Unable to send signal to container: "  +
+                   os::strerror(errno));
+  }
+
+  return true;
+}
+
+
 Future<Nothing> MesosContainerizerProcess::remove(
     const ContainerID& containerId)
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/8b3fd242/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index fd58630..cc23b4d 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -103,6 +103,10 @@ public:
   virtual process::Future<bool> destroy(
       const ContainerID& containerId);
 
+  virtual process::Future<bool> kill(
+      const ContainerID& containerId,
+      int signal);
+
   virtual process::Future<hashset<ContainerID>> containers();
 
   virtual process::Future<Nothing> remove(const ContainerID& containerId);
@@ -168,6 +172,10 @@ public:
   virtual process::Future<bool> destroy(
       const ContainerID& containerId);
 
+  virtual process::Future<bool> kill(
+      const ContainerID& containerId,
+      int signal);
+
   virtual process::Future<Nothing> remove(const ContainerID& containerId);
 
   virtual process::Future<hashset<ContainerID>> containers();


[2/7] mesos git commit: Made the default executor support signal escalation.

Posted by an...@apache.org.
Made the default executor support signal escalation.

This modifies the default executor to perform signal escalation
via the 'KILL_NESTED_CONTAINER' call i.e., send a SIGTERM followed
by a SIGKILL after a constant grace period.

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


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

Branch: refs/heads/master
Commit: d1abe1e9ae4fdd53a97e643df9b55fee969743db
Parents: 8fc0491
Author: Anand Mazumdar <an...@apache.org>
Authored: Tue Aug 15 17:38:14 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Tue Aug 15 17:48:39 2017 -0700

----------------------------------------------------------------------
 src/launcher/default_executor.cpp | 63 ++++++++++++++++++++++++++++++++--
 1 file changed, 61 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d1abe1e9/src/launcher/default_executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/default_executor.cpp b/src/launcher/default_executor.cpp
index c25cc94..a23097e 100644
--- a/src/launcher/default_executor.cpp
+++ b/src/launcher/default_executor.cpp
@@ -993,7 +993,34 @@ protected:
       container->healthChecker = None();
     }
 
-    LOG(INFO) << "Killing child container " << container->containerId;
+    LOG(INFO)
+      << "Killing task " << container->taskInfo.task_id() << " running in child"
+      << " container " << container->containerId << " with SIGTERM signal";
+
+    // Default grace period is set to 3s.
+    //
+    // TODO(anand): Add support for handling kill policies.
+    const Duration GRACE_PERIOD = Seconds(3);
+
+    LOG(INFO) << "Scheduling escalation to SIGKILL in " << GRACE_PERIOD
+              << " from now";
+
+    const ContainerID& containerId = container->containerId;
+
+    delay(GRACE_PERIOD,
+          self(),
+          &Self::escalated,
+          connectionId.get(),
+          containerId,
+          container->taskInfo.task_id(),
+          GRACE_PERIOD);
+
+    return kill(containerId, SIGTERM);
+  }
+
+  Future<Nothing> kill(const ContainerID& containerId, int signal)
+  {
+    CHECK_EQ(SUBSCRIBED, state);
 
     agent::Call call;
     call.set_type(agent::Call::KILL_NESTED_CONTAINER);
@@ -1001,7 +1028,8 @@ protected:
     agent::Call::KillNestedContainer* kill =
       call.mutable_kill_nested_container();
 
-    kill->mutable_container_id()->CopyFrom(container->containerId);
+    kill->mutable_container_id()->CopyFrom(containerId);
+    kill->set_signal(signal);
 
     return post(None(), call)
       .then([](const Response& /* response */) {
@@ -1009,6 +1037,37 @@ protected:
       });
   }
 
+  void escalated(
+      const UUID& _connectionId,
+      const ContainerID& containerId,
+      const TaskID& taskId,
+      const Duration& timeout)
+  {
+    if (connectionId != _connectionId) {
+      VLOG(1) << "Ignoring signal escalation timeout from a stale connection";
+      return;
+    }
+
+    CHECK_EQ(SUBSCRIBED, state);
+
+    // It might be possible that the container is already terminated.
+    // If that happens, don't bother escalating to SIGKILL.
+    if (!containers.contains(taskId)) {
+      LOG(WARNING)
+        << "Ignoring escalation to SIGKILL since the task '" << taskId
+        << "' running in child container " << containerId << " has"
+        << " already terminated";
+      return;
+    }
+
+    LOG(INFO)
+      << "Task '" << taskId << "' running in child container " << containerId
+      << " did not terminate after " << timeout << ", sending SIGKILL"
+      << " to the container";
+
+    kill(containerId, SIGKILL);
+  }
+
   void killTask(const TaskID& taskId)
   {
     if (shuttingDown) {


[7/7] mesos git commit: Made `killNestedContainer()` use `kill()` on the containerizer.

Posted by an...@apache.org.
Made `killNestedContainer()` use `kill()` on the containerizer.

Instead of invoking `destroy()` directly, `killNestedContainer()`
would invoke `kill()` to terminate the nested container.

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


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

Branch: refs/heads/master
Commit: 8fc0491be86a665bb80a0dd978d8799d6559be1f
Parents: 8c8e2af
Author: Anand Mazumdar <an...@apache.org>
Authored: Tue Aug 15 17:38:08 2017 -0700
Committer: Anand Mazumdar <an...@apache.org>
Committed: Tue Aug 15 17:48:39 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/mesos/blob/8fc0491b/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 2d33f0b..544a052 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -2497,6 +2497,12 @@ Future<Response> Http::killNestedContainer(
       const ContainerID& containerId =
         call.kill_nested_container().container_id();
 
+      // SIGKILL is used by default if a signal is not specified.
+      int signal = SIGKILL;
+      if (call.kill_nested_container().has_signal()) {
+        signal = call.kill_nested_container().signal();
+      }
+
       Executor* executor = slave->getExecutor(containerId);
       if (executor == nullptr) {
         return NotFound(
@@ -2518,9 +2524,9 @@ Future<Response> Http::killNestedContainer(
         return Forbidden();
       }
 
-      Future<bool> destroy = slave->containerizer->destroy(containerId);
+      Future<bool> kill = slave->containerizer->kill(containerId, signal);
 
-      return destroy
+      return kill
         .then([containerId](bool found) -> Response {
           if (!found) {
             return NotFound("Container '" + stringify(containerId) + "'"