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 2019/01/11 21:33:27 UTC

[mesos] branch master updated (594ea4c -> 2a770e5)

This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git.


    from 594ea4c  Fixed the CNI_NETNS handling in port mapper CNI plugin.
     new eda0912  Made agent not read the forked pid and libprocess pid after reboot.
     new 43772df  Updated `SlaveRecoveryTest.Reboot` to expect none pids.
     new a555937  Added a test `SlaveRecoveryTest.RebootWithExecutorPidReused`.
     new 63714e5  Added MESOS-9502 to 1.7.1 CHANGELOG.
     new 1986ef4  Added MESOS-9502 to 1.6.2 CHANGELOG.
     new fbb5d01  Added MESOS-9502 to 1.5.2 CHANGELOG.
     new 795e3e7  Added MESOS-9502 to 1.4.3 CHANGELOG.
     new 6bf1580  Added MESOS-9501 to 1.7.1 CHANGELOG.
     new 0217dce  Added MESOS-9501 to 1.6.2 CHANGELOG.
     new 98526e3  Added MESOS-9501 to 1.5.2 CHANGELOG.
     new 2a770e5  Added MESOS-9501 to 1.4.3 CHANGELOG.

The 11 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 CHANGELOG                          |   8 ++
 src/slave/state.cpp                |  57 +++++++++++----
 src/slave/state.hpp                |  12 ++-
 src/tests/slave_recovery_tests.cpp | 145 +++++++++++++++++++++++++++++++++++--
 4 files changed, 200 insertions(+), 22 deletions(-)


[mesos] 09/11: Added MESOS-9501 to 1.6.2 CHANGELOG.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 0217dce09ad549e2ca74a897b2c38bdfe3b5d810
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Fri Jan 11 13:11:14 2019 -0800

    Added MESOS-9501 to 1.6.2 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index 50be081..fad64e1 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -468,6 +468,7 @@ Release Notes - Mesos - Version 1.6.2 (WIP)
 ** Improvement
   * [MESOS-9305] - Create cgoup recursively to workaround systemd deleting cgroups_root.
   * [MESOS-9340] - Log all socket errors in libprocess.
+  * [MESOS-9501] - Mesos executor fails to terminate and gets stuck after agent host reboot.
   * [MESOS-9502] - IOswitchboard cleanup could get stuck due to FD leak from a race.
   * [MESOS-9510] - Disallowed nan, inf and so on in `Value::Scalar`.
   * [MESOS-9516] - Extend `min_allocatable_resources` flag to cover non-scalar resources.


[mesos] 01/11: Made agent not read the forked pid and libprocess pid after reboot.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit eda091215a132a7ea758a95cd4f66945a462387e
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Fri Jan 11 12:21:18 2019 -0800

    Made agent not read the forked pid and libprocess pid after reboot.
    
    After agent host is rebooted, the forked pid and libprocess pid in
    agent's meta directory are obsolete, so we should not read them during
    agent recovery, otherwise containerizer may wait for an irrelevant
    process if the forked pid is reused by another process after reboot.
    
    Review: https://reviews.apache.org/r/69705/
---
 src/slave/state.cpp | 57 +++++++++++++++++++++++++++++++++++++++++------------
 src/slave/state.hpp | 12 +++++++----
 2 files changed, 52 insertions(+), 17 deletions(-)

diff --git a/src/slave/state.cpp b/src/slave/state.cpp
index e7cf849..ae16d6f 100644
--- a/src/slave/state.cpp
+++ b/src/slave/state.cpp
@@ -123,7 +123,9 @@ Try<State> recover(const string& rootDir, bool strict)
   SlaveID slaveId;
   slaveId.set_value(Path(directory.get()).basename());
 
-  Try<SlaveState> slave = SlaveState::recover(rootDir, slaveId, strict);
+  Try<SlaveState> slave =
+    SlaveState::recover(rootDir, slaveId, strict, state.rebooted);
+
   if (slave.isError()) {
     return Error(slave.error());
   }
@@ -137,7 +139,8 @@ Try<State> recover(const string& rootDir, bool strict)
 Try<SlaveState> SlaveState::recover(
     const string& rootDir,
     const SlaveID& slaveId,
-    bool strict)
+    bool strict,
+    bool rebooted)
 {
   SlaveState state;
   state.id = slaveId;
@@ -188,7 +191,7 @@ Try<SlaveState> SlaveState::recover(
     frameworkId.set_value(Path(path).basename());
 
     Try<FrameworkState> framework =
-      FrameworkState::recover(rootDir, slaveId, frameworkId, strict);
+      FrameworkState::recover(rootDir, slaveId, frameworkId, strict, rebooted);
 
     if (framework.isError()) {
       return Error("Failed to recover framework " + frameworkId.value() +
@@ -207,7 +210,8 @@ Try<FrameworkState> FrameworkState::recover(
     const string& rootDir,
     const SlaveID& slaveId,
     const FrameworkID& frameworkId,
-    bool strict)
+    bool strict,
+    bool rebooted)
 {
   FrameworkState state;
   state.id = frameworkId;
@@ -295,8 +299,8 @@ Try<FrameworkState> FrameworkState::recover(
     ExecutorID executorId;
     executorId.set_value(Path(path).basename());
 
-    Try<ExecutorState> executor =
-      ExecutorState::recover(rootDir, slaveId, frameworkId, executorId, strict);
+    Try<ExecutorState> executor = ExecutorState::recover(
+        rootDir, slaveId, frameworkId, executorId, strict, rebooted);
 
     if (executor.isError()) {
       return Error("Failed to recover executor '" + executorId.value() +
@@ -316,7 +320,8 @@ Try<ExecutorState> ExecutorState::recover(
     const SlaveID& slaveId,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    bool strict)
+    bool strict,
+    bool rebooted)
 {
   ExecutorState state;
   state.id = executorId;
@@ -356,7 +361,13 @@ Try<ExecutorState> ExecutorState::recover(
       containerId.set_value(Path(path).basename());
 
       Try<RunState> run = RunState::recover(
-          rootDir, slaveId, frameworkId, executorId, containerId, strict);
+          rootDir,
+          slaveId,
+          frameworkId,
+          executorId,
+          containerId,
+          strict,
+          rebooted);
 
       if (run.isError()) {
         return Error(
@@ -423,7 +434,8 @@ Try<RunState> RunState::recover(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
     const ContainerID& containerId,
-    bool strict)
+    bool strict,
+    bool rebooted)
 {
   RunState state;
   state.id = containerId;
@@ -469,18 +481,37 @@ Try<RunState> RunState::recover(
     state.errors += task->errors;
   }
 
-  // Read the forked pid.
   path = paths::getForkedPidPath(
       rootDir, slaveId, frameworkId, executorId, containerId);
+
+  // If agent host is rebooted, we do not read the forked pid and libprocess pid
+  // since those two pids are obsolete after reboot. And we remove the forked
+  // pid file to make sure we will not read it in the case the agent process is
+  // restarted after we checkpoint the new boot ID in `Slave::__recover` (i.e.,
+  // agent recovery is done after the reboot).
+  if (rebooted) {
+    if (os::exists(path)) {
+      Try<Nothing> rm = os::rm(path);
+      if (rm.isError()) {
+        return Error(
+            "Failed to remove executor forked pid file '" + path + "': " +
+            rm.error());
+      }
+    }
+
+    return state;
+  }
+
   if (!os::exists(path)) {
-    // This could happen if the slave died before the isolator
-    // checkpointed the forked pid.
+    // This could happen if the slave died before the containerizer checkpointed
+    // the forked pid or agent process is restarted after agent host is rebooted
+    // since we remove this file in the above code.
     LOG(WARNING) << "Failed to find executor forked pid file '" << path << "'";
     return state;
   }
 
+  // Read the forked pid.
   Result<string> pid = state::read<string>(path);
-
   if (pid.isError()) {
     message = "Failed to read executor forked pid from '" + path +
               "': " + pid.error();
diff --git a/src/slave/state.hpp b/src/slave/state.hpp
index 4f3d4ce..e2180ae 100644
--- a/src/slave/state.hpp
+++ b/src/slave/state.hpp
@@ -279,7 +279,8 @@ struct RunState
       const FrameworkID& frameworkId,
       const ExecutorID& executorId,
       const ContainerID& containerId,
-      bool strict);
+      bool strict,
+      bool rebooted);
 
   Option<ContainerID> id;
   hashmap<TaskID, TaskState> tasks;
@@ -306,7 +307,8 @@ struct ExecutorState
       const SlaveID& slaveId,
       const FrameworkID& frameworkId,
       const ExecutorID& executorId,
-      bool strict);
+      bool strict,
+      bool rebooted);
 
   ExecutorID id;
   Option<ExecutorInfo> info;
@@ -324,7 +326,8 @@ struct FrameworkState
       const std::string& rootDir,
       const SlaveID& slaveId,
       const FrameworkID& frameworkId,
-      bool strict);
+      bool strict,
+      bool rebooted);
 
   FrameworkID id;
   Option<FrameworkInfo> info;
@@ -359,7 +362,8 @@ struct SlaveState
   static Try<SlaveState> recover(
       const std::string& rootDir,
       const SlaveID& slaveId,
-      bool strict);
+      bool strict,
+      bool rebooted);
 
   SlaveID id;
   Option<SlaveInfo> info;


[mesos] 03/11: Added a test `SlaveRecoveryTest.RebootWithExecutorPidReused`.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit a55593728e04cc0ecc0893b9f3796df9f22f83d4
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Fri Jan 11 12:21:27 2019 -0800

    Added a test `SlaveRecoveryTest.RebootWithExecutorPidReused`.
    
    Review: https://reviews.apache.org/r/69717/
---
 src/tests/slave_recovery_tests.cpp | 138 +++++++++++++++++++++++++++++++++++++
 1 file changed, 138 insertions(+)

diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index 91fa00a..8eb5081 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -2810,6 +2810,144 @@ TYPED_TEST(SlaveRecoveryTest, Reboot)
 }
 
 
+// When the slave is down we modify the BOOT_ID_FILE and the executor's forked
+// pid file to simulate executor's pid is reused after agent host is rebooted,
+// the framework should receive `TASK_FAILED` status update after the reboot.
+// This is a regression test for MESOS-9501.
+TYPED_TEST(SlaveRecoveryTest, RebootWithExecutorPidReused)
+{
+  Try<Owned<cluster::Master>> master = this->StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = this->CreateSlaveFlags();
+
+  Fetcher fetcher(flags);
+
+  Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
+  ASSERT_SOME(_containerizer);
+  Owned<slave::Containerizer> containerizer(_containerizer.get());
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave =
+    this->StartSlave(detector.get(), containerizer.get(), flags);
+  ASSERT_SOME(slave);
+
+  // Enable checkpointing for the framework.
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.set_checkpoint(true);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, frameworkInfo, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(_, _, _));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  ASSERT_FALSE(offers->empty());
+
+  TaskInfo task = createTask(offers.get()[0], SLEEP_COMMAND(1000));
+
+  // Capture the slave and framework ids.
+  SlaveID slaveId1 = offers.get()[0].slave_id();
+  FrameworkID frameworkId = offers.get()[0].framework_id();
+
+  Future<Message> registerExecutorMessage =
+    FUTURE_MESSAGE(Eq(RegisterExecutorMessage().GetTypeName()), _, _);
+
+  Future<TaskStatus> startingStatus;
+  Future<TaskStatus> runningStatus;
+  Future<TaskStatus> failedStatus;
+  EXPECT_CALL(sched, statusUpdate(_, _))
+    .WillOnce(FutureArg<1>(&startingStatus))
+    .WillOnce(FutureArg<1>(&runningStatus))
+    .WillOnce(FutureArg<1>(&failedStatus))
+    .WillRepeatedly(Return()); // Ignore subsequent updates.
+
+  driver.launchTasks(offers.get()[0].id(), {task});
+
+  // Capture the executor ID and PID.
+  AWAIT_READY(registerExecutorMessage);
+
+  RegisterExecutorMessage registerExecutor;
+  registerExecutor.ParseFromString(registerExecutorMessage->body);
+  ExecutorID executorId = registerExecutor.executor_id();
+  UPID executorPid = registerExecutorMessage->from;
+
+  // Wait for TASK_STARTING and TASK_RUNNING updates.
+  AWAIT_READY(startingStatus);
+  EXPECT_EQ(TASK_STARTING, startingStatus->state());
+
+  AWAIT_READY(runningStatus);
+  EXPECT_EQ(TASK_RUNNING, runningStatus->state());
+
+  // Capture the container ID.
+  Future<hashset<ContainerID>> containers = containerizer->containers();
+
+  AWAIT_READY(containers);
+  ASSERT_EQ(1u, containers->size());
+
+  ContainerID containerId = *containers->begin();
+
+  slave.get()->terminate();
+
+  // Get the executor's pid so we can reap it to properly simulate a
+  // reboot.
+  string pidPath = paths::getForkedPidPath(
+      paths::getMetaRootDir(flags.work_dir),
+      slaveId1,
+      frameworkId,
+      executorId,
+      containerId);
+
+  Try<string> read = os::read(pidPath);
+  ASSERT_SOME(read);
+
+  Try<pid_t> pid = numify<pid_t>(read.get());
+  ASSERT_SOME(pid);
+
+  Future<Option<int>> executorStatus = process::reap(pid.get());
+
+  // Shut down the executor manually and wait until it's been reaped.
+  process::post(slave.get()->pid, executorPid, ShutdownExecutorMessage());
+
+  AWAIT_READY(executorStatus);
+
+  // Modify the boot ID to simulate a reboot.
+  ASSERT_SOME(os::write(
+      paths::getBootIdPath(paths::getMetaRootDir(flags.work_dir)),
+      "rebooted! ;)"));
+
+  // Modify the executor's pid to simulate the pid is reused after reboot.
+  ASSERT_SOME(os::write(pidPath, stringify(::getpid())));
+
+  Future<ReregisterSlaveMessage> reregisterSlaveMessage =
+    FUTURE_PROTOBUF(ReregisterSlaveMessage(), _, _);
+
+  // Restart the slave (use same flags) with a new containerizer.
+  _containerizer = TypeParam::create(flags, true, &fetcher);
+  ASSERT_SOME(_containerizer);
+  containerizer.reset(_containerizer.get());
+
+  slave = this->StartSlave(detector.get(), containerizer.get(), flags);
+  ASSERT_SOME(slave);
+
+  AWAIT_READY(reregisterSlaveMessage);
+
+  AWAIT_READY(failedStatus);
+  EXPECT_EQ(TASK_FAILED, failedStatus->state());
+
+  driver.stop();
+  driver.join();
+}
+
+
 // When the agent is down we modify the BOOT_ID_FILE to simulate a
 // reboot and change the resources flag to cause a mismatch
 // between the recovered agent's agent info and the one of the new agent


[mesos] 05/11: Added MESOS-9502 to 1.6.2 CHANGELOG.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 1986ef46e048c82e09b23daef5734895334b4826
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Fri Jan 11 13:07:52 2019 -0800

    Added MESOS-9502 to 1.6.2 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index 63f2966..be50169 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -467,6 +467,7 @@ Release Notes - Mesos - Version 1.6.2 (WIP)
 ** Improvement
   * [MESOS-9305] - Create cgoup recursively to workaround systemd deleting cgroups_root.
   * [MESOS-9340] - Log all socket errors in libprocess.
+  * [MESOS-9502] - IOswitchboard cleanup could get stuck due to FD leak from a race.
   * [MESOS-9510] - Disallowed nan, inf and so on in `Value::Scalar`.
   * [MESOS-9516] - Extend `min_allocatable_resources` flag to cover non-scalar resources.
 


[mesos] 10/11: Added MESOS-9501 to 1.5.2 CHANGELOG.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 98526e33ef2ec197c763f8ca1a1a7188b8d93f19
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Fri Jan 11 13:11:31 2019 -0800

    Added MESOS-9501 to 1.5.2 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index fad64e1..9d36405 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -921,6 +921,7 @@ Release Notes - Mesos - Version 1.5.2
   * [MESOS-9308] - URI disk profile adaptor could deadlock.
   * [MESOS-9334] - Container stuck at ISOLATING state due to libevent poll never returns.
   * [MESOS-9480] - Master may skip processing authorization results for `LAUNCH_GROUP`.
+  * [MESOS-9501] - Mesos executor fails to terminate and gets stuck after agent host reboot.
   * [MESOS-9502] - IOswitchboard cleanup could get stuck due to FD leak from a race.
 
 ** Improvement:


[mesos] 07/11: Added MESOS-9502 to 1.4.3 CHANGELOG.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 795e3e75e45ac1a0632d7f9c20a6635b6eb7b414
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Fri Jan 11 13:09:57 2019 -0800

    Added MESOS-9502 to 1.4.3 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index 21077f9..b33adc9 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1472,6 +1472,7 @@ Release Notes - Mesos - Version 1.4.3 (WIP)
   * [MESOS-9419] - Executor to framework message crashes master if framework has not re-registered.
   * [MESOS-9480] - Master may skip processing authorization results for `LAUNCH_GROUP`.
   * [MESOS-9492] - Persist CNI working directory across reboot.
+  * [MESOS-9502] - IOswitchboard cleanup could get stuck due to FD leak from a race.
 
 ** Improvement:
   * [MESOS-9510] - Disallowed nan, inf and so on in `Value::Scalar`.


[mesos] 04/11: Added MESOS-9502 to 1.7.1 CHANGELOG.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 63714e56a04048874ff99e87f6e092208b148aa9
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Fri Jan 11 13:07:03 2019 -0800

    Added MESOS-9502 to 1.7.1 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index 5b4d3f8..63f2966 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -47,6 +47,7 @@ Release Notes - Mesos - Version 1.7.1
   * [MESOS-9479] - SLRP does not set RP ID in produced OperationStatus.
   * [MESOS-9480] - Master may skip processing authorization results for `LAUNCH_GROUP`.
   * [MESOS-9492] - Persist CNI working directory across reboot.
+  * [MESOS-9502] - IOswitchboard cleanup could get stuck due to FD leak from a race.
   * [MESOS-9505] - `make check` failed with linking errors when c-ares is installed.
   * [MESOS-9508] - Official 1.7.0 tarball can't be built on Ubuntu 16.04 LTS.
 


[mesos] 08/11: Added MESOS-9501 to 1.7.1 CHANGELOG.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 6bf158018d4cc7a69ff3a21430051ac80b7f966c
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Fri Jan 11 13:10:51 2019 -0800

    Added MESOS-9501 to 1.7.1 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index b33adc9..50be081 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -47,6 +47,7 @@ Release Notes - Mesos - Version 1.7.1
   * [MESOS-9479] - SLRP does not set RP ID in produced OperationStatus.
   * [MESOS-9480] - Master may skip processing authorization results for `LAUNCH_GROUP`.
   * [MESOS-9492] - Persist CNI working directory across reboot.
+  * [MESOS-9501] - Mesos executor fails to terminate and gets stuck after agent host reboot.
   * [MESOS-9502] - IOswitchboard cleanup could get stuck due to FD leak from a race.
   * [MESOS-9505] - `make check` failed with linking errors when c-ares is installed.
   * [MESOS-9508] - Official 1.7.0 tarball can't be built on Ubuntu 16.04 LTS.


[mesos] 06/11: Added MESOS-9502 to 1.5.2 CHANGELOG.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit fbb5d01275390783b2712e681c5ccebfc2a77d78
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Fri Jan 11 13:09:22 2019 -0800

    Added MESOS-9502 to 1.5.2 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index be50169..21077f9 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -919,6 +919,7 @@ Release Notes - Mesos - Version 1.5.2
   * [MESOS-9308] - URI disk profile adaptor could deadlock.
   * [MESOS-9334] - Container stuck at ISOLATING state due to libevent poll never returns.
   * [MESOS-9480] - Master may skip processing authorization results for `LAUNCH_GROUP`.
+  * [MESOS-9502] - IOswitchboard cleanup could get stuck due to FD leak from a race.
 
 ** Improvement:
   * [MESOS-9510] - Disallowed nan, inf and so on in `Value::Scalar`.


[mesos] 02/11: Updated `SlaveRecoveryTest.Reboot` to expect none pids.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 43772dfa7111ebdf759b066728736921062cc4fc
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Fri Jan 11 12:21:22 2019 -0800

    Updated `SlaveRecoveryTest.Reboot` to expect none pids.
    
    After agent host is rebooted, the recovered executor's forked pid and
    libprocess pid will be `NONE()` since we do not read them from agent's
    meta directory which are actually obsolete after reboot.
    
    Review: https://reviews.apache.org/r/69716/
---
 src/tests/slave_recovery_tests.cpp | 7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)

diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index 0eb47e2..91fa00a 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -2798,11 +2798,8 @@ TYPED_TEST(SlaveRecoveryTest, Reboot)
 
   EXPECT_TRUE(executorState.runs.contains(containerId2.get()));
 
-  EXPECT_SOME_EQ(
-      executorPid,
-      executorState
-        .runs[containerId2.get()]
-        .libprocessPid);
+  EXPECT_NONE(executorState.runs[containerId2.get()].forkedPid);
+  EXPECT_NONE(executorState.runs[containerId2.get()].libprocessPid);
 
   EXPECT_TRUE(executorState
                 .runs[containerId2.get()]


[mesos] 11/11: Added MESOS-9501 to 1.4.3 CHANGELOG.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 2a770e535b89bf90ac57108efc23574bf4bf5f54
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Fri Jan 11 13:12:23 2019 -0800

    Added MESOS-9501 to 1.4.3 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index 9d36405..7f1e9ab 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1475,6 +1475,7 @@ Release Notes - Mesos - Version 1.4.3 (WIP)
   * [MESOS-9419] - Executor to framework message crashes master if framework has not re-registered.
   * [MESOS-9480] - Master may skip processing authorization results for `LAUNCH_GROUP`.
   * [MESOS-9492] - Persist CNI working directory across reboot.
+  * [MESOS-9501] - Mesos executor fails to terminate and gets stuck after agent host reboot.
   * [MESOS-9502] - IOswitchboard cleanup could get stuck due to FD leak from a race.
 
 ** Improvement: