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/01/13 01:04:34 UTC

[1/4] mesos git commit: Logged some additional information on a master CHECK.

Repository: mesos
Updated Branches:
  refs/heads/1.5.x 35ac2f047 -> 336b067be


Logged some additional information on a master CHECK.

If the master attempts to remove an unreachable task on a registered
agent, log some additional debugging information in the CHECK to help
track down what happened.

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


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

Branch: refs/heads/1.5.x
Commit: 336b067be8e864f751c292152edf4cb1461e7ce1
Parents: 0f05e05
Author: James Peach <jp...@apache.org>
Authored: Fri Jan 12 17:01:48 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Fri Jan 12 17:04:19 2018 -0800

----------------------------------------------------------------------
 src/master/master.cpp | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/336b067b/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index a6be5a9..e169a60 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -9539,7 +9539,10 @@ void Master::removeFramework(Framework* framework)
 
     // We don't need to remove the task from the slave, because the
     // task was removed when the agent was marked unreachable.
-    CHECK(!slaves.registered.contains(task->slave_id()));
+    CHECK(!slaves.registered.contains(task->slave_id()))
+      << "Unreachable task " << task->task_id()
+      << " of framework " << task->framework_id()
+      << " was found on registered agent " << task->slave_id();
 
     // Move task from unreachable map to completed map.
     framework->addCompletedTask(std::move(*task));


[3/4] mesos git commit: Fixed the default executor flaky testes in tests/cluster.cpp.

Posted by gi...@apache.org.
Fixed the default executor flaky testes in tests/cluster.cpp.

This patch fixes some flaky tests listed below:
1. DefaultExecutorTest.KillTask/0
2. DefaultExecutorTest.TaskWithFileURI/0
3. DefaultExecutorTest.ResourceLimitation/0
4. DefaultExecutorTest.KillMultipleTasks/0

The root cause is that either docker containerizer or mesos
containerizer have wait() and destroy() rely on the same
future `ContainerTermination` which means these two methods
become ready simultaneously, but this is not true for the
composing containerizer because wait() may finish before
destroy in which case the `containers_` hasshmap is not
cleaned up yet in destroy()'s `.onAny` callback.

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


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

Branch: refs/heads/1.5.x
Commit: 0f05e052ca3b8c573437c643f274b0cf5752c22e
Parents: dc6beb7
Author: Gilbert Song <so...@gmail.com>
Authored: Thu Jan 11 18:05:54 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Fri Jan 12 17:04:19 2018 -0800

----------------------------------------------------------------------
 src/tests/cluster.cpp | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0f05e052/src/tests/cluster.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.cpp b/src/tests/cluster.cpp
index f964bf0..de4ddf4 100644
--- a/src/tests/cluster.cpp
+++ b/src/tests/cluster.cpp
@@ -631,8 +631,9 @@ Slave::~Slave()
       process::Future<Option<ContainerTermination>> wait =
         containerizer->wait(containerId);
 
-      containerizer->destroy(containerId);
+      process::Future<bool> destroy = containerizer->destroy(containerId);
 
+      AWAIT(destroy);
       AWAIT(wait);
     }
 


[4/4] mesos git commit: Reverted "Fixed `wait()` and `destroy()` in composing containerizer.".

Posted by gi...@apache.org.
Reverted "Fixed `wait()` and `destroy()` in composing containerizer.".

This reverts commit 95decd404438abd422794524e01d72a889821566.

There are two reasons to revert this commit:
  1. After the agent recovers, if the nested containers that are
     launched beforehand are killed, they will no longer be updated
     with new status, because the `WAIT_NESTED_CONTAINER` call from
     the default executor will end up with a future forever. Please
     see MESOS-8391 for details.
  2. The original commit makes the composing containerizer wait()
     and destroy() rely on the same future of a ContainerTermination
     promise. This would get into the bug that composing containerizer
     destroy() may fail due to the wait() future got discarded.
     Need to protect it by using `undiscardable()`. Please see
     MESOS-7926 for details.

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


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

Branch: refs/heads/1.5.x
Commit: c0559445993a3a1d57603a9f82609f18e0af8966
Parents: 35ac2f0
Author: Gilbert Song <so...@gmail.com>
Authored: Thu Jan 11 17:59:37 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Fri Jan 12 17:04:19 2018 -0800

----------------------------------------------------------------------
 src/slave/containerizer/composing.cpp | 89 +++++++++++++++---------------
 1 file changed, 44 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c0559445/src/slave/containerizer/composing.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.cpp b/src/slave/containerizer/composing.cpp
index 9ace70d..cd840a5 100644
--- a/src/slave/containerizer/composing.cpp
+++ b/src/slave/containerizer/composing.cpp
@@ -118,12 +118,6 @@ private:
       const ContainerID& containerId,
       Containerizer::LaunchResult launchResult);
 
-  // Last step in the container destruction chain; when it finishes, the
-  // container must be removed from the internal collection.
-  void _destroy(
-      const ContainerID& containerId,
-      const Future<Option<ContainerTermination>>& future);
-
   vector<Containerizer*> containerizers_;
 
   // The states that the composing containerizer cares about for the
@@ -141,7 +135,7 @@ private:
   {
     State state;
     Containerizer* containerizer;
-    Promise<Option<ContainerTermination>> termination;
+    Promise<bool> destroyed;
   };
 
   hashmap<ContainerID, Container*> containers_;
@@ -360,7 +354,7 @@ Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
       // This is needed for eventually removing the given container from
       // the list of active containers.
       container->containerizer->wait(containerId)
-        .onAny(defer(self(), &Self::_destroy, containerId, lambda::_1));
+        .onAny(defer(self(), &Self::destroy, containerId));
     }
 
     // Note that the return value is not impacted
@@ -376,10 +370,10 @@ Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
   if (containerizer == containerizers_.end()) {
     // If we are here none of the containerizers support the launch.
 
-    // We set this to `None` because the container has no chance of
+    // We set this to `false` because the container has no chance of
     // getting launched by any containerizer. This is similar to what
     // would happen if the destroy "started" after launch returned false.
-    container->termination.set(Option<ContainerTermination>::none());
+    container->destroyed.set(false);
 
     // We destroy the container irrespective whether
     // a destroy is already in progress, for simplicity.
@@ -395,11 +389,9 @@ Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
     // potentially launch this container. But since a destroy is in progress
     // we do not try any other containerizers.
 
-    // If the destroy-in-progress stopped an launch-in-progress (using the next
-    // containerizer), then we need to set a value to the `termination` promise,
-    // because we consider `wait()` and `destroy()` operations as successful.
-    container->termination.set(
-        Option<ContainerTermination>(ContainerTermination()));
+    // We set this to `true` because the destroy-in-progress stopped an
+    // launch-in-progress (using the next containerizer).
+    container->destroyed.set(true);
 
     containers_.erase(containerId);
     delete container;
@@ -509,7 +501,7 @@ Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
       // This is needed for eventually removing the given container from
       // the list of active containers.
       container->containerizer->wait(containerId)
-        .onAny(defer(self(), &Self::_destroy, containerId, lambda::_1));
+        .onAny(defer(self(), &Self::destroy, containerId));
     }
 
     // Note that the return value is not impacted
@@ -519,10 +511,10 @@ Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
 
   // If we are here, the launch is not supported by the containerizer.
 
-  // We set this to `None` because the container has no chance of
+  // We set this to `false` because the container has no chance of
   // getting launched. This is similar to what would happen if the
   // destroy "started" after launch returned false.
-  container->termination.set(Option<ContainerTermination>::none());
+  container->destroyed.set(false);
 
   // We destroy the container irrespective whether
   // a destroy is already in progress, for simplicity.
@@ -582,10 +574,6 @@ Future<ContainerStatus> ComposingContainerizerProcess::status(
 Future<Option<ContainerTermination>> ComposingContainerizerProcess::wait(
     const ContainerID& containerId)
 {
-  if (containers_.contains(containerId)) {
-    return containers_[containerId]->termination.future();
-  }
-
   // A nested container might have already been terminated, therefore
   // `containers_` might not contain it, but its exit status might have
   // been checkpointed.
@@ -621,16 +609,8 @@ Future<bool> ComposingContainerizerProcess::destroy(
       break; // No-op.
 
     case LAUNCHING:
-    case LAUNCHED:
       container->state = DESTROYING;
 
-      // If the container is destroyed while `launch()` is in progress,
-      // `wait()` will not be called in `_launch()`, so we should call
-      // `wait()` to cleanup state of the container in `_destroy()`.
-      // Note that it is safe to call `_destroy()` multiple times.
-      container->containerizer->wait(containerId)
-        .onAny(defer(self(), &Self::_destroy, containerId, lambda::_1));
-
       // Forward the destroy request to the containerizer. Note that
       // a containerizer is expected to handle a destroy while
       // `launch()` is in progress. If the containerizer could not
@@ -638,27 +618,46 @@ Future<bool> ComposingContainerizerProcess::destroy(
       // then the containerizer may no longer know about this
       // container. If the launch returns false, we will stop trying
       // to launch the container on other containerizers.
-      container->containerizer->destroy(containerId);
+      container->containerizer->destroy(containerId)
+        .onAny(defer(self(), [=](const Future<bool>& destroy) {
+          // We defer the association of the promise in order to
+          // surface a successful destroy (by setting
+          // `Container.destroyed` to true in `_launch()`) when
+          // the containerizer cannot handle this type of container
+          // (`launch()` returns false). If we do not defer here and
+          // instead associate the future right away, the setting of
+          // `Container.destroy` in `_launch()` will be a no-op;
+          // this might result in users waiting on the future
+          // incorrectly thinking that the destroy failed when in
+          // fact the destroy is implicitly successful because the
+          // launch failed.
+          if (containers_.contains(containerId)) {
+            containers_.at(containerId)->destroyed.associate(destroy);
+            delete containers_.at(containerId);
+            containers_.erase(containerId);
+          }
+        }));
 
       break;
-  }
 
-  return container->termination.future()
-    .then([](const Option<ContainerTermination>& termination) {
-      return termination.isSome();
-    });
-}
+    case LAUNCHED:
+      container->state = DESTROYING;
 
+      container->destroyed.associate(
+          container->containerizer->destroy(containerId));
 
-void ComposingContainerizerProcess::_destroy(
-    const ContainerID& containerId,
-    const Future<Option<ContainerTermination>>& future)
-{
-  if (containers_.contains(containerId)) {
-    containers_.at(containerId)->termination.associate(future);
-    delete containers_.at(containerId);
-    containers_.erase(containerId);
+      container->destroyed.future()
+        .onAny(defer(self(), [=](const Future<bool>& destroy) {
+          if (containers_.contains(containerId)) {
+            delete containers_.at(containerId);
+            containers_.erase(containerId);
+          }
+        }));
+
+      break;
   }
+
+  return container->destroyed.future();
 }
 
 


[2/4] mesos git commit: Reverted "Updated composing containerizer tests.".

Posted by gi...@apache.org.
Reverted "Updated composing containerizer tests.".

This reverts commit 84365a140c3730e2d6579ad500118d6749d2f87f.

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


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

Branch: refs/heads/1.5.x
Commit: dc6beb74acf4fe2a1607ec7a2153b09958e8c256
Parents: c055944
Author: Gilbert Song <so...@gmail.com>
Authored: Thu Jan 11 17:59:19 2018 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Fri Jan 12 17:04:19 2018 -0800

----------------------------------------------------------------------
 .../composing_containerizer_tests.cpp           | 34 ++------------------
 1 file changed, 2 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/dc6beb74/src/tests/containerizer/composing_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/composing_containerizer_tests.cpp b/src/tests/containerizer/composing_containerizer_tests.cpp
index 7c22f16..09f7ea4 100644
--- a/src/tests/containerizer/composing_containerizer_tests.cpp
+++ b/src/tests/containerizer/composing_containerizer_tests.cpp
@@ -88,12 +88,6 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
   EXPECT_CALL(*mockContainerizer1, launch(_, _, _, _))
     .WillOnce(Return(launchPromise.future()));
 
-  Future<Nothing> wait;
-  Promise<Option<ContainerTermination>> waitPromise;
-  EXPECT_CALL(*mockContainerizer1, wait(_))
-    .WillOnce(DoAll(FutureSatisfy(&wait),
-                    Return(waitPromise.future())));
-
   Future<Nothing> destroy;
   Promise<bool> destroyPromise;
   EXPECT_CALL(*mockContainerizer1, destroy(_))
@@ -115,9 +109,6 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
   EXPECT_CALL(*mockContainerizer2, launch(_, _, _, _))
     .Times(0);
 
-  // We make sure the wait is being called on the first containerizer.
-  AWAIT_READY(wait);
-
   // We make sure the destroy is being called on the first containerizer.
   // The second containerizer shouldn't be called as well since the
   // container is already destroyed.
@@ -125,7 +116,6 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
 
   launchPromise.set(Containerizer::LaunchResult::NOT_SUPPORTED);
   destroyPromise.set(false);
-  waitPromise.set(Option<ContainerTermination>::none());
 
   // `launched` should be a failure and `destroyed` should be true
   // because the launch was stopped from being tried on the 2nd
@@ -140,7 +130,7 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
 // underlying containerizer's destroy (because it's not sure
 // if the containerizer can handle the type of container being
 // launched). If the launch is successful the destroy future
-// value depends on the containerizer's termination future.
+// value depends on the containerizer's destroy.
 TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
 {
   vector<Containerizer*> containerizers;
@@ -164,12 +154,6 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
   EXPECT_CALL(*mockContainerizer1, launch(_, _, _, _))
     .WillOnce(Return(launchPromise.future()));
 
-  Future<Nothing> wait;
-  Promise<Option<ContainerTermination>> waitPromise;
-  EXPECT_CALL(*mockContainerizer1, wait(_))
-    .WillOnce(DoAll(FutureSatisfy(&wait),
-                    Return(waitPromise.future())));
-
   Future<Nothing> destroy;
   Promise<bool> destroyPromise;
   EXPECT_CALL(*mockContainerizer1, destroy(_))
@@ -191,9 +175,6 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
   EXPECT_CALL(*mockContainerizer2, launch(_, _, _, _))
     .Times(0);
 
-  // We make sure the wait is being called on the first containerizer.
-  AWAIT_READY(wait);
-
   // We make sure the destroy is being called on the first containerizer.
   // The second containerizer shouldn't be called as well since the
   // container is already destroyed.
@@ -201,10 +182,9 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
 
   launchPromise.set(Containerizer::LaunchResult::SUCCESS);
   destroyPromise.set(false);
-  waitPromise.set(Option<ContainerTermination>::none());
 
   // `launched` should return true and `destroyed` should return false
-  // because the launch succeeded and `waitPromise` was set to `None`.
+  // because the launch succeeded and `destroyPromise` was set to false.
   AWAIT_EXPECT_EQ(Containerizer::LaunchResult::SUCCESS, launched);
   AWAIT_EXPECT_EQ(false, destroyed);
 }
@@ -236,12 +216,6 @@ TEST_F(ComposingContainerizerTest, DestroyAfterLaunchLoop)
   EXPECT_CALL(*mockContainerizer1, launch(_, _, _, _))
     .WillOnce(Return(launchPromise.future()));
 
-  Future<Nothing> wait;
-  Promise<Option<ContainerTermination>> waitPromise;
-  EXPECT_CALL(*mockContainerizer1, wait(_))
-    .WillOnce(DoAll(FutureSatisfy(&wait),
-                    Return(waitPromise.future())));
-
   Future<Nothing> destroy;
   Promise<bool> destroyPromise;
   EXPECT_CALL(*mockContainerizer1, destroy(_))
@@ -260,15 +234,11 @@ TEST_F(ComposingContainerizerTest, DestroyAfterLaunchLoop)
 
   Future<bool> destroyed = containerizer.destroy(containerId);
 
-  // We make sure the wait is being called on the containerizer.
-  AWAIT_READY(wait);
-
   // We make sure the destroy is being called on the containerizer.
   AWAIT_READY(destroy);
 
   launchPromise.set(Containerizer::LaunchResult::NOT_SUPPORTED);
   destroyPromise.set(false);
-  waitPromise.set(Option<ContainerTermination>::none());
 
   // `launch` should return false and `destroyed` should return false
   // because none of the containerizers support the launch.