You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by nn...@apache.org on 2015/02/13 20:17:52 UTC

mesos git commit: Reverted configurable graceful shutdown period patch set.

Repository: mesos
Updated Branches:
  refs/heads/master b3385f361 -> 9175d8273


Reverted configurable graceful shutdown period patch set.

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


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

Branch: refs/heads/master
Commit: 9175d8273cc50543298bf665b42942b0bd7a3ddc
Parents: b3385f3
Author: Niklas Nielsen <ni...@qni.dk>
Authored: Fri Feb 13 10:59:16 2015 -0800
Committer: Niklas Q. Nielsen <ni...@mesosphere.io>
Committed: Fri Feb 13 10:59:18 2015 -0800

----------------------------------------------------------------------
 include/mesos/mesos.proto                 |   4 -
 src/Makefile.am                           |   2 -
 src/exec/exec.cpp                         |  50 +------
 src/launcher/executor.cpp                 |  52 ++-----
 src/slave/constants.cpp                   |   4 +-
 src/slave/constants.hpp                   |  18 +--
 src/slave/containerizer/containerizer.cpp |  13 --
 src/slave/flags.hpp                       |   6 +-
 src/slave/graceful_shutdown.cpp           |  74 ---------
 src/slave/graceful_shutdown.hpp           |  66 --------
 src/slave/slave.cpp                       |  38 +----
 src/tests/gc_tests.cpp                    |   4 +-
 src/tests/slave_tests.cpp                 | 199 +------------------------
 13 files changed, 28 insertions(+), 502 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 3a2921d..6ece3ea 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -252,10 +252,6 @@ message CommandInfo {
   optional string value = 3;
   repeated string arguments = 7;
 
-  // Shutdown grace period for the command (in seconds). Any changes
-  // made by frameworks are currently ignored.
-  optional double grace_period_seconds = 8;
-
   // Enables executor and tasks to run as a specific user. If the user
   // field is present both in FrameworkInfo and here, the CommandInfo
   // user value takes precedence.

http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 05ee76d..d372404 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -324,7 +324,6 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	scheduler/scheduler.cpp						\
 	slave/constants.cpp						\
 	slave/gc.cpp							\
-	slave/graceful_shutdown.cpp					\
 	slave/http.cpp							\
 	slave/metrics.cpp						\
 	slave/monitor.cpp						\
@@ -520,7 +519,6 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/constants.hpp						\
 	slave/flags.hpp							\
 	slave/gc.hpp							\
-	slave/graceful_shutdown.hpp					\
 	slave/metrics.hpp						\
 	slave/monitor.hpp						\
 	slave/paths.hpp							\

http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/src/exec/exec.cpp
----------------------------------------------------------------------
diff --git a/src/exec/exec.cpp b/src/exec/exec.cpp
index 3492943..8250bd4 100644
--- a/src/exec/exec.cpp
+++ b/src/exec/exec.cpp
@@ -54,7 +54,6 @@
 #include "messages/messages.hpp"
 
 #include "slave/constants.hpp"
-#include "slave/graceful_shutdown.hpp"
 #include "slave/state.hpp"
 
 using namespace mesos;
@@ -69,24 +68,15 @@ using process::wait; // Necessary on some OS's to disambiguate.
 
 namespace mesos {
 
-
-// A custom executor can be non-cooperative as it can block the
-// shutdown callback and take over the actor thread. As a result,
-// libprocess process may exit (e.g. a Java executor can be garbage
-// collected) before a delayed shutdown callback is invoked. Therefore
-// we need a separate libprocess process to ensure clean-up. However,
-// if the executor shuts down and calls os::exit() in another
-// libprocess process, the ShutdownProcess::kill() won't be called.
 class ShutdownProcess : public Process<ShutdownProcess>
 {
-public:
-  explicit ShutdownProcess(const Duration& timeout) : timeout(timeout) {}
-
 protected:
   virtual void initialize()
   {
     VLOG(1) << "Scheduling shutdown of the executor";
-    delay(timeout, self(), &Self::kill);
+    // TODO(benh): Pass the shutdown timeout with ExecutorRegistered
+    // since it might have gotten configured on the command line.
+    delay(slave::EXECUTOR_SHUTDOWN_GRACE_PERIOD, self(), &Self::kill);
   }
 
   void kill()
@@ -102,9 +92,6 @@ protected:
     os::sleep(Seconds(5));
     exit(-1);
   }
-
-private:
-  const Duration timeout;
 };
 
 
@@ -120,8 +107,7 @@ public:
                   bool _local,
                   const string& _directory,
                   bool _checkpoint,
-                  const Duration& _recoveryTimeout,
-                  const Duration& _shutdownTimeout,
+                  Duration _recoveryTimeout,
                   pthread_mutex_t* _mutex,
                   pthread_cond_t* _cond)
     : ProcessBase(ID::generate("executor")),
@@ -139,8 +125,7 @@ public:
       cond(_cond),
       directory(_directory),
       checkpoint(_checkpoint),
-      recoveryTimeout(_recoveryTimeout),
-      shutdownTimeout(_shutdownTimeout)
+      recoveryTimeout(_recoveryTimeout)
   {
     LOG(INFO) << "Version: " << MESOS_VERSION;
 
@@ -393,7 +378,7 @@ protected:
 
     if (!local) {
       // Start the Shutdown Process.
-      spawn(new ShutdownProcess(shutdownTimeout), true);
+      spawn(new ShutdownProcess(), true);
     }
 
     Stopwatch stopwatch;
@@ -475,7 +460,7 @@ protected:
 
     if (!local) {
       // Start the Shutdown Process.
-      spawn(new ShutdownProcess(shutdownTimeout), true);
+      spawn(new ShutdownProcess(), true);
     }
 
     Stopwatch stopwatch;
@@ -571,7 +556,6 @@ private:
   const string directory;
   bool checkpoint;
   Duration recoveryTimeout;
-  Duration shutdownTimeout;
 
   LinkedHashMap<UUID, StatusUpdate> updates; // Unacknowledged updates.
 
@@ -721,25 +705,6 @@ Status MesosExecutorDriver::start()
     }
   }
 
-  // Get the appropriate shutdown grace period.
-  Duration shutdownTimeout = slave::EXECUTOR_SHUTDOWN_GRACE_PERIOD;
-  value = os::getenv("MESOS_SHUTDOWN_GRACE_PERIOD", false);
-  if (!value.empty()) {
-    Try<Duration> parse = Duration::parse(value);
-    if (parse.isSome()) {
-      shutdownTimeout = parse.get();
-    } else {
-      LOG(WARNING) << "Cannot parse MESOS_SHUTDOWN_GRACE_PERIOD '"
-                   << value << "': " << parse.error();
-    }
-  } else {
-    LOG(WARNING) << "Environment variable MESOS_SHUTDOWN_GRACE_PERIOD is not "
-                 << "set, using default value: " << shutdownTimeout;
-  }
-
-  shutdownTimeout = slave::getExecGracePeriod(shutdownTimeout);
-  VLOG(2) << "Shutdown timeout is set to " << shutdownTimeout;
-
   CHECK(process == NULL);
 
   process = new ExecutorProcess(
@@ -753,7 +718,6 @@ Status MesosExecutorDriver::start()
       workDirectory,
       checkpoint,
       recoveryTimeout,
-      shutdownTimeout,
       &mutex,
       &cond);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/src/launcher/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.cpp b/src/launcher/executor.cpp
index 8f47f0e..f5db28c 100644
--- a/src/launcher/executor.cpp
+++ b/src/launcher/executor.cpp
@@ -56,10 +56,8 @@
 #include "messages/messages.hpp"
 
 #include "slave/constants.hpp"
-#include "slave/graceful_shutdown.hpp"
 
 using process::wait; // Necessary on some OS's to disambiguate.
-using namespace mesos::slave;
 
 using std::cout;
 using std::cerr;
@@ -74,18 +72,15 @@ using namespace process;
 class CommandExecutorProcess : public ProtobufProcess<CommandExecutorProcess>
 {
 public:
-  CommandExecutorProcess(
-      Option<char**> override,
-      const string& healthCheckDir,
-      const Duration& shutdownTimeout)
+  CommandExecutorProcess(Option<char**> override, const string& _healthCheckDir)
     : launched(false),
       killed(false),
       killedByHealthCheck(false),
       pid(-1),
       healthPid(-1),
-      shutdownTimeout(shutdownTimeout),
+      escalationTimeout(slave::EXECUTOR_SIGNAL_ESCALATION_TIMEOUT),
       driver(None()),
-      healthCheckDir(healthCheckDir),
+      healthCheckDir(_healthCheckDir),
       override(override) {}
 
   virtual ~CommandExecutorProcess() {}
@@ -327,7 +322,7 @@ public:
       // TODO(nnielsen): Make escalationTimeout configurable through
       // slave flags and/or per-framework/executor.
       escalationTimer = delay(
-          shutdownTimeout,
+          escalationTimeout,
           self(),
           &Self::escalated);
 
@@ -429,12 +424,8 @@ private:
 
   void escalated()
   {
-    // TODO(alex): If the escalation timeout is too small, the process
-    // may have already exited, but not yet reaped. If this is the
-    // case, do not kill the process, since its OS pid could have been
-    // already reused.
     cout << "Process " << pid << " did not terminate after "
-         << shutdownTimeout << ", sending SIGKILL to "
+         << escalationTimeout << ", sending SIGKILL to "
          << "process tree at " << pid << endl;
 
     // TODO(nnielsen): Sending SIGTERM in the first stage of the
@@ -501,7 +492,7 @@ private:
   bool killedByHealthCheck;
   pid_t pid;
   pid_t healthPid;
-  Duration shutdownTimeout;
+  Duration escalationTimeout;
   Timer escalationTimer;
   Option<ExecutorDriver*> driver;
   string healthCheckDir;
@@ -512,15 +503,9 @@ private:
 class CommandExecutor: public Executor
 {
 public:
-  CommandExecutor(
-      Option<char**> override,
-      const string& healthCheckDir,
-      const Duration& shutdownTimeout)
+  CommandExecutor(Option<char**> override, string healthCheckDir)
   {
-    process = new CommandExecutorProcess(
-        override,
-        healthCheckDir,
-        shutdownTimeout);
+    process = new CommandExecutorProcess(override, healthCheckDir);
     spawn(process);
   }
 
@@ -632,25 +617,6 @@ int main(int argc, char** argv)
             "Prints this help message",
             false);
 
-  // Get the appropriate shutdown grace period.
-  Duration shutdownTimeout = EXECUTOR_SHUTDOWN_GRACE_PERIOD;
-  string value = os::getenv("MESOS_SHUTDOWN_GRACE_PERIOD", false);
-  if (!value.empty()) {
-    Try<Duration> parse = Duration::parse(value);
-    if (parse.isSome()) {
-      shutdownTimeout = parse.get();
-    } else {
-      cerr << "Cannot parse MESOS_SHUTDOWN_GRACE_PERIOD '" << value << "': "
-           << parse.error() << endl;
-    }
-  } else {
-    cout << "Environment variable MESOS_SHUTDOWN_GRACE_PERIOD is not set, "
-         << "using default value: " << shutdownTimeout << endl;
-  }
-
-  shutdownTimeout = getExecutorGracePeriod(shutdownTimeout);
-  cout << "Shutdown timeout is set to " << shutdownTimeout;
-
   // Load flags from command line.
   Try<Nothing> load = flags.load(None(), &argc, &argv);
 
@@ -681,7 +647,7 @@ int main(int argc, char** argv)
   if (path.empty()) {
     path = os::realpath(dirname(argv[0])).get();
   }
-  mesos::CommandExecutor executor(override, path, shutdownTimeout);
+  mesos::CommandExecutor executor(override, path);
   mesos::MesosExecutorDriver driver(&executor);
   return driver.run() == mesos::DRIVER_STOPPED ? 0 : 1;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/src/slave/constants.cpp
----------------------------------------------------------------------
diff --git a/src/slave/constants.cpp b/src/slave/constants.cpp
index cdf5c46..7868bef 100644
--- a/src/slave/constants.cpp
+++ b/src/slave/constants.cpp
@@ -26,9 +26,9 @@ namespace mesos {
 namespace slave {
 
 const Duration EXECUTOR_REGISTRATION_TIMEOUT = Minutes(1);
+const Duration EXECUTOR_SHUTDOWN_GRACE_PERIOD = Seconds(5);
 const Duration EXECUTOR_REREGISTER_TIMEOUT = Seconds(2);
-const Duration EXECUTOR_SHUTDOWN_GRACE_PERIOD = Seconds(3);
-const Duration GRACE_PERIOD_DELTA = Seconds(1);
+const Duration EXECUTOR_SIGNAL_ESCALATION_TIMEOUT = Seconds(3);
 const Duration STATUS_UPDATE_RETRY_INTERVAL_MIN = Seconds(10);
 const Duration STATUS_UPDATE_RETRY_INTERVAL_MAX = Minutes(10);
 const Duration REGISTRATION_BACKOFF_FACTOR = Seconds(1);

http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/src/slave/constants.hpp
----------------------------------------------------------------------
diff --git a/src/slave/constants.hpp b/src/slave/constants.hpp
index 7717abd..12d6e92 100644
--- a/src/slave/constants.hpp
+++ b/src/slave/constants.hpp
@@ -37,7 +37,9 @@ namespace slave {
 // details in MESOS-1023.
 
 extern const Duration EXECUTOR_REGISTRATION_TIMEOUT;
+extern const Duration EXECUTOR_SHUTDOWN_GRACE_PERIOD;
 extern const Duration EXECUTOR_REREGISTER_TIMEOUT;
+extern const Duration EXECUTOR_SIGNAL_ESCALATION_TIMEOUT;
 extern const Duration RECOVERY_TIMEOUT;
 extern const Duration STATUS_UPDATE_RETRY_INTERVAL_MIN;
 extern const Duration STATUS_UPDATE_RETRY_INTERVAL_MAX;
@@ -45,22 +47,6 @@ extern const Duration GC_DELAY;
 extern const Duration DISK_WATCH_INTERVAL;
 extern const Duration RESOURCE_MONITORING_INTERVAL;
 
-// Default parameters for graceful shutdown mechanism for executor. We
-// control the shutdown on several levels, e.g.:
-//   [Containerizer [ExecutorProcess [CommandExecutorProcess [Task]]]]
-// Only the innermost shutdown grace period (aka shutdown timeout) is
-// exposed to the user through a slave flag, other timeouts should be
-// calculated based on it. Each nested timeout is somewhat shorter
-// than the parent one in order to give the process enough time to
-// terminate the underlying process before being killed by parent.
-// The difference between shutdown timeouts for neighbouring levels is
-// GRACE_PERIOD_DELTA. For example, if the innermost timeout
-// (for an executor) is 5s, ExecutorProcess timeout will be
-// (5 + GRACE_PERIOD_DELTA)s and containerizer timeout
-// (5 + 2 * GRACE_PERIOD_DELTA)s.
-extern const Duration EXECUTOR_SHUTDOWN_GRACE_PERIOD;
-extern const Duration GRACE_PERIOD_DELTA;
-
 // Default backoff interval used by the slave to wait before registration.
 extern const Duration REGISTRATION_BACKOFF_FACTOR;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/src/slave/containerizer/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
index 4e0dc3b..c453c33 100644
--- a/src/slave/containerizer/containerizer.cpp
+++ b/src/slave/containerizer/containerizer.cpp
@@ -284,19 +284,6 @@ map<string, string> executorEnvironment(
   env["MESOS_SLAVE_PID"] = stringify(slavePid);
   env["MESOS_CHECKPOINT"] = checkpoint ? "1" : "0";
 
-  // We expect the graceful shutdown timeout to be set either by a
-  // framework or to default value from slave's flags. In case it is
-  // absent for some reason, use the hardcoded default.
-  if (executorInfo.command().has_grace_period_seconds()) {
-    env["MESOS_SHUTDOWN_GRACE_PERIOD"] =
-      stringify(Seconds(executorInfo.command().grace_period_seconds()));
-  } else {
-    LOG(WARNING) << "CommandInfo.grace_period flag is not set, "
-                 << "using default value: " << EXECUTOR_SHUTDOWN_GRACE_PERIOD;
-    env["MESOS_SHUTDOWN_GRACE_PERIOD"] =
-      stringify(EXECUTOR_SHUTDOWN_GRACE_PERIOD);
-  }
-
   if (checkpoint) {
     env["MESOS_RECOVERY_TIMEOUT"] = stringify(recoveryTimeout);
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 6ef64ed..ddb3259 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -137,10 +137,8 @@ public:
 
     add(&Flags::executor_shutdown_grace_period,
         "executor_shutdown_grace_period",
-        "Amount of time to wait for an executor to shut down\n"
-        "(e.g., 60s, 3min, etc). If the flag value is too small\n"
-        "(less than 3s), there may not be enough time for the\n"
-        "executor to react and can result in a hard shutdown.",
+        "Amount of time to wait for an executor\n"
+        "to shut down (e.g., 60secs, 3mins, etc)",
         EXECUTOR_SHUTDOWN_GRACE_PERIOD);
 
     add(&Flags::gc_delay,

http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/src/slave/graceful_shutdown.cpp
----------------------------------------------------------------------
diff --git a/src/slave/graceful_shutdown.cpp b/src/slave/graceful_shutdown.cpp
deleted file mode 100644
index 2ab58f7..0000000
--- a/src/slave/graceful_shutdown.cpp
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "stout/duration.hpp"
-
-#include "logging/logging.hpp"
-
-#include "slave/constants.hpp"
-#include "slave/graceful_shutdown.hpp"
-
-namespace mesos {
-namespace slave {
-
-// Calculates the shutdown grace period (aka shutdown timeout) so it
-// is bigger than the nested one. To adjust the timeout correctly, the
-// caller should provide its level index in the shutdown chain.
-// Timeout adjustment gives the caller process enough time to
-// terminate the underlying process before the caller, in turn, is
-// killed by its parent (see the sequence chart in the
-// graceful_shutdown.hpp). This approach guarantees a nested timeout
-// is always greater than the parent one, but not that it is
-// sufficient for the graceful shutdown to happen.
-Duration calculateGracePeriod(
-    Duration gracePeriod,
-    int callerLevel)
-{
-  if (gracePeriod < Duration::zero()) {
-    LOG(WARNING) << "Shutdown grace period should be nonnegative (got "
-                 << gracePeriod << "), using default value: "
-                 << EXECUTOR_SHUTDOWN_GRACE_PERIOD;
-    gracePeriod = EXECUTOR_SHUTDOWN_GRACE_PERIOD;
-  }
-
-  gracePeriod += GRACE_PERIOD_DELTA * callerLevel;
-
-  return gracePeriod;
-}
-
-
-Duration getContainerizerGracePeriod(const Duration& baseShutdownTimeout)
-{
-  return calculateGracePeriod(baseShutdownTimeout, 2);
-}
-
-
-Duration getExecGracePeriod(const Duration& baseShutdownTimeout)
-{
-  return calculateGracePeriod(baseShutdownTimeout, 1);
-}
-
-
-Duration getExecutorGracePeriod(const Duration& baseShutdownTimeout)
-{
-  return calculateGracePeriod(baseShutdownTimeout, 0);
-}
-
-
-} // namespace slave {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/src/slave/graceful_shutdown.hpp
----------------------------------------------------------------------
diff --git a/src/slave/graceful_shutdown.hpp b/src/slave/graceful_shutdown.hpp
deleted file mode 100644
index 950329d..0000000
--- a/src/slave/graceful_shutdown.hpp
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#ifndef __SLAVE_GRACEFUL_SHUTDOWN_HPP__
-#define __SLAVE_GRACEFUL_SHUTDOWN_HPP__
-
-namespace mesos {
-namespace slave {
-
-// Slave           Exec          Executor
-//  +               +               +
-//  |               |               |
-//  |               |               |
-//  |   shutdown()  |               |
-//  +-^------------->               |
-//  | |             |   shutdown()  |
-//  | |             +-^-------------> shutdown()
-//  | |             | |             | ^
-//  | |             | |             | |
-//  | timeout       | timeout       | | timeout  <-- flags.
-//  | level 2       | level 1       | | level 0        shutdown_grace_period
-//  | |             | |             | v
-//  | |             | |             | escalated()
-//  | |             | v             |
-//  | |             | ShutdownProcess
-//  | |             | ::kill()      |
-//  | v             |               |
-//  | shutdownExecutorTimeout()     |
-//  |               |               |
-//  v               v               v
-//  Containerizer->destroy()
-
-
-// Returns the shutdown grace period for containerizer. We assume it
-// is the 2nd and the last level in the shutdown chain.
-Duration getContainerizerGracePeriod(const Duration& baseShutdownTimeout);
-
-
-// Returns the shutdown grace period for ExecutorProcess. We assume it
-// is the 1st level in the shutdown chain.
-Duration getExecGracePeriod(const Duration& baseShutdownTimeout);
-
-
-// Returns the shutdown grace period for an executor (e.g.
-// CommandExecutorProcess). It is the 0 level in the shutdown chain.
-Duration getExecutorGracePeriod(const Duration& baseShutdownTimeout);
-
-} // namespace slave {
-} // namespace mesos {
-
-#endif // __SLAVE_GRACEFUL_SHUTDOWN_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 1e60454..ec7ec13 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -80,7 +80,6 @@
 
 #include "slave/constants.hpp"
 #include "slave/flags.hpp"
-#include "slave/graceful_shutdown.hpp"
 #include "slave/paths.hpp"
 #include "slave/slave.hpp"
 #include "slave/status_update_manager.hpp"
@@ -1102,30 +1101,6 @@ Future<bool> Slave::unschedule(const string& path)
 }
 
 
-// Returns a TaskInfo with grace shutdown period field added in
-// task's CommandInfo structures.
-TaskInfo updateGracePeriod(TaskInfo task, double gracePeriod)
-{
-  // TODO(alexr): do not overwrite present value for frameworks that
-  // are authorized to set grace periods for their executors.
-
-  // Update CommandInfo in task.
-  if (task.has_command()) {
-    task.mutable_command()->set_grace_period_seconds(gracePeriod);
-  }
-
-  // Update CommandInfo in task's ExecutorInfo.
-  if (task.has_executor() &&
-      task.executor().has_command()) {
-    task.mutable_executor()->mutable_command()->set_grace_period_seconds(
-        gracePeriod);
-  }
-
-  // Return either updated or unchanged TaskInfo.
-  return task;
-}
-
-
 // TODO(vinod): Instead of crashing the slave on checkpoint errors,
 // send TASK_LOST to the framework.
 void Slave::runTask(
@@ -1203,19 +1178,14 @@ void Slave::runTask(
     }
   }
 
-  // Ensure the task has grace shutdown period set.
-  const TaskInfo& task_ = updateGracePeriod(
-      task,
-      Seconds(flags.executor_shutdown_grace_period).value());
-
-  const ExecutorInfo& executorInfo = getExecutorInfo(frameworkId, task_);
+  const ExecutorInfo& executorInfo = getExecutorInfo(frameworkId, task);
   const ExecutorID& executorId = executorInfo.executor_id();
 
   // We add the task to 'pending' to ensure the framework is not
   // removed and the framework and top level executor directories
   // are not scheduled for deletion before '_runTask()' is called.
   CHECK_NOTNULL(framework);
-  framework->pending[executorId][task_.task_id()] = task_;
+  framework->pending[executorId][task.task_id()] = task;
 
   // If we are about to create a new executor, unschedule the top
   // level work and meta directories from getting gc'ed.
@@ -1246,7 +1216,7 @@ void Slave::runTask(
             frameworkInfo,
             frameworkId,
             pid,
-            task_));
+            task));
 }
 
 
@@ -3322,7 +3292,7 @@ void Slave::shutdownExecutor(Framework* framework, Executor* executor)
   send(executor->pid, ShutdownExecutorMessage());
 
   // Prepare for sending a kill if the executor doesn't comply.
-  delay(getContainerizerGracePeriod(flags.executor_shutdown_grace_period),
+  delay(flags.executor_shutdown_grace_period,
         self(),
         &Slave::shutdownExecutorTimeout,
         framework->id,

http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/src/tests/gc_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/gc_tests.cpp b/src/tests/gc_tests.cpp
index 45efce3..c74287a 100644
--- a/src/tests/gc_tests.cpp
+++ b/src/tests/gc_tests.cpp
@@ -49,7 +49,6 @@
 #include "slave/constants.hpp"
 #include "slave/flags.hpp"
 #include "slave/gc.hpp"
-#include "slave/graceful_shutdown.hpp"
 #include "slave/paths.hpp"
 #include "slave/slave.hpp"
 
@@ -440,8 +439,7 @@ TEST_F(GarbageCollectorIntegrationTest, ExitedFramework)
     FUTURE_DISPATCH(_, &GarbageCollectorProcess::schedule);
 
   // Advance clock to kill executor via isolator.
-  Clock::advance(slave::getContainerizerGracePeriod(
-      flags.executor_shutdown_grace_period));
+  Clock::advance(flags.executor_shutdown_grace_period);
 
   Clock::settle();
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/9175d827/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index a02e335..21709e2 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -34,7 +34,6 @@
 #include <process/io.hpp>
 #include <process/owned.hpp>
 #include <process/pid.hpp>
-#include <process/reap.hpp>
 #include <process/subprocess.hpp>
 
 #include <stout/option.hpp>
@@ -47,9 +46,8 @@
 #include "master/master.hpp"
 
 #include "slave/constants.hpp"
-#include "slave/flags.hpp"
 #include "slave/gc.hpp"
-#include "slave/graceful_shutdown.hpp"
+#include "slave/flags.hpp"
 #include "slave/slave.hpp"
 
 #include "slave/containerizer/fetcher.hpp"
@@ -1695,198 +1693,3 @@ TEST_F(SlaveTest, TaskLabels)
 
   Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
-
-
-// This test checks that the mechanism of calculating nested graceful
-// shutdown periods does not break the default behaviour and works as
-// expected.
-TEST_F(SlaveTest, ShutdownGracePeriod)
-{
-  Duration defaultTimeout = slave::EXECUTOR_SHUTDOWN_GRACE_PERIOD;
-  Duration customTimeout = Seconds(10);
-
-  // We used to have a signal escalation timeout constant responsibe
-  // for graceful shutdown period in the CommandExecutor. Make sure
-  // the default behaviour (3s) persists.
-  EXPECT_EQ(Seconds(3), slave::getExecutorGracePeriod(defaultTimeout));
-
-  // The new logic uses a certain delta to calculate nested timeouts.
-  EXPECT_EQ(Duration::zero(), slave::getExecutorGracePeriod(Duration::zero()));
-  EXPECT_EQ(Seconds(2), slave::getContainerizerGracePeriod(Duration::zero()));
-  EXPECT_EQ(customTimeout + Seconds(2),
-            slave::getContainerizerGracePeriod(customTimeout));
-
-  // The grace period in ExecutorProcess should be bigger than the
-  // grace period in an executor.
-  EXPECT_GT(slave::getExecGracePeriod(defaultTimeout),
-            slave::getExecutorGracePeriod(defaultTimeout));
-
-  // Check the graceful shutdown periods that reach the executor in
-  // protobuf messages.
-  // NOTE: We check only the message contents and *not* the value
-  // stored by the executor.
-  Try<PID<Master>> master = StartMaster();
-  ASSERT_SOME(master);
-
-  MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestContainerizer containerizer(&exec);
-
-  slave::Flags flags = CreateSlaveFlags();
-  flags.executor_shutdown_grace_period = slave::EXECUTOR_SHUTDOWN_GRACE_PERIOD;
-
-  Try<PID<Slave>> slave = StartSlave(&containerizer, flags);
-  ASSERT_SOME(slave);
-
-  MockScheduler sched;
-  MesosSchedulerDriver driver(
-      &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
-
-  EXPECT_CALL(sched, registered(&driver, _, _));
-  EXPECT_CALL(exec, registered(_, _, _, _));
-
-  Future<vector<Offer>> offers;
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(FutureArg<1>(&offers))
-    .WillRepeatedly(Return()); // Ignore subsequent offers.
-
-  driver.start();
-
-  AWAIT_READY(offers);
-  EXPECT_FALSE(offers.get().empty());
-  Offer offer = offers.get()[0];
-
-  // Create one task with shutdown grace period set and one without.
-  TaskInfo taskCustom;
-  taskCustom.set_name("Task with custom grace shutdown period");
-  taskCustom.mutable_task_id()->set_value("custom");
-  taskCustom.mutable_slave_id()->MergeFrom(offer.slave_id());
-  taskCustom.mutable_executor()->MergeFrom(DEFAULT_EXECUTOR_INFO);
-  taskCustom.mutable_resources()->CopyFrom(
-      Resources::parse("cpus:0.1;mem:64").get());
-  taskCustom.mutable_executor()->mutable_command()->set_grace_period_seconds(
-      Seconds(customTimeout).value());
-
-  TaskInfo taskDefault;
-  taskDefault.set_name("Task with default grace shutdown period");
-  taskDefault.mutable_task_id()->set_value("default");
-  taskDefault.mutable_slave_id()->MergeFrom(offer.slave_id());
-  taskDefault.mutable_executor()->MergeFrom(DEFAULT_EXECUTOR_INFO);
-  taskDefault.mutable_resources()->CopyFrom(
-      Resources::parse("cpus:0.1;mem:64").get());
-
-  ASSERT_TRUE(Resources(offer.resources()).contains(
-      taskCustom.resources() + taskDefault.resources()));
-
-  vector<TaskInfo> tasks;
-  tasks.push_back(taskCustom);
-  tasks.push_back(taskDefault);
-
-  Future<TaskInfo> task1, task2;
-  EXPECT_CALL(exec, launchTask(_, _))
-    .WillOnce(FutureArg<1>(&task1))
-    .WillOnce(FutureArg<1>(&task2));
-
-  driver.launchTasks(offer.id(), tasks);
-
-  AWAIT_READY(task1);
-  AWAIT_READY(task2);
-
-  // Currently (14 Nov 2014) grace periods customized by frameworks
-  // are ignored.
-  EXPECT_DOUBLE_EQ(
-      Seconds(defaultTimeout).value(),
-      task1.get().executor().command().grace_period_seconds());
-  EXPECT_DOUBLE_EQ
-      (Seconds(defaultTimeout).value(),
-      task2.get().executor().command().grace_period_seconds());
-
-  driver.stop();
-  driver.join();
-
-  Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
-}
-
-
-// This test ensures that the graceful shutdown in the command
-// uses SIGTERM to gracefully shutdown a task.
-TEST_F(SlaveTest, CommandExecutorGracefulShutdown)
-{
-  Try<PID<Master>> master = StartMaster();
-  ASSERT_SOME(master);
-
-  // Explicitly set the grace period for the executor.
-  // NOTE: We ensure that the graceful shutdown is at least
-  // 10 seconds because we've observed the sleep command to take
-  // several seconds to terminate from SIGTERM on some slow CI VMs.
-  slave::Flags flags = CreateSlaveFlags();
-  flags.executor_shutdown_grace_period = std::max(
-      slave::EXECUTOR_SHUTDOWN_GRACE_PERIOD,
-      Duration(Seconds(10)));
-
-  // Ensure that a reap will occur within the grace period.
-  Duration timeout = slave::getExecutorGracePeriod(
-      flags.executor_shutdown_grace_period);
-  EXPECT_GT(timeout, MAX_REAP_INTERVAL());
-
-  Fetcher fetcher;
-  Try<MesosContainerizer*> containerizer = MesosContainerizer::create(
-      flags, true, &fetcher);
-  ASSERT_SOME(containerizer);
-
-  Try<PID<Slave>> slave = StartSlave(containerizer.get(), flags);
-  ASSERT_SOME(slave);
-
-  MockScheduler sched;
-  MesosSchedulerDriver driver(
-      &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
-
-  EXPECT_CALL(sched, registered(&driver, _, _));
-
-  Future<vector<Offer>> offers;
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(FutureArg<1>(&offers))
-    .WillRepeatedly(Return()); // Ignore subsequent offers.
-
-  driver.start();
-
-  AWAIT_READY(offers);
-  EXPECT_FALSE(offers.get().empty());
-  Offer offer = offers.get()[0];
-
-  // Launch a long-running task responsive to SIGTERM.
-  TaskInfo taskResponsive = createTask(offer, "sleep 1000");
-  vector<TaskInfo> tasks;
-  tasks.push_back(taskResponsive);
-
-  Future<TaskStatus> statusRunning, statusKilled;
-  EXPECT_CALL(sched, statusUpdate(&driver, _))
-    .WillOnce(FutureArg<1>(&statusRunning))
-    .WillOnce(FutureArg<1>(&statusKilled));
-
-  driver.launchTasks(offer.id(), tasks);
-
-  AWAIT_READY(statusRunning);
-  EXPECT_EQ(TASK_RUNNING, statusRunning.get().state());
-
-  driver.killTask(taskResponsive.task_id());
-
-  AWAIT_READY(statusKilled);
-  EXPECT_EQ(TASK_KILLED, statusKilled.get().state());
-
-  // CommandExecutor supports graceful shutdown in sending SIGTERM
-  // first. If the task obeys, it will be reaped and we get
-  // appropriate status message.
-  // NOTE: strsignal() behaves differently on Mac OS and Linux.
-  // TODO(alex): By now we have no better way to extract the kill
-  // reason. Change this once we have level 2 enums for task states.
-  EXPECT_TRUE(statusKilled.get().has_message());
-  EXPECT_TRUE(strings::contains(statusKilled.get().message(), "Terminated"))
-    << statusKilled.get().message();
-
-  // Stop the driver while the task is running.
-  driver.stop();
-  driver.join();
-
-  Shutdown();  // Must shutdown before 'containerizer' gets deallocated.
-  delete containerizer.get();
-}