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/07/18 16:19:08 UTC

[mesos] branch master updated (2ecbe56 -> f3b827f)

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 2ecbe56  Updated 1.8.1 CHANGELOG to its final form.
     new bf01659  Added `PendingFutureTracker` class for tracking pending futures.
     new 5112ab8  Added `IsolatorTracker` for tracking calls of isolator methods.
     new c3964a8  Wrapped isolators in `IsolatorTracker`.
     new 5e01feb  Added `LauncherTracker` for tracking calls of launcher methods.
     new 59c2c75  Wrapped launcher in `LauncherTracker`.
     new 9604977  Added `/containerizer/debug` endpoint.
     new f3b827f  Implemented `FutureTrackTest` tests.

The 7 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:
 docs/authorization.md                              |   1 +
 src/CMakeLists.txt                                 |   2 +
 src/Makefile.am                                    |   6 +
 src/common/authorization.cpp                       |   1 +
 src/common/future_tracker.hpp                      | 159 +++++++++++++++++++++
 src/local/local.cpp                                |   1 +
 src/slave/constants.hpp                            |   3 +
 src/slave/containerizer/containerizer.cpp          |  13 +-
 src/slave/containerizer/containerizer.hpp          |   5 +-
 src/slave/containerizer/mesos/containerizer.cpp    |  49 +++++--
 src/slave/containerizer/mesos/containerizer.hpp    |   3 +-
 src/slave/containerizer/mesos/isolator_tracker.cpp | 152 ++++++++++++++++++++
 .../filesystem/posix.hpp => isolator_tracker.hpp}  |  65 ++++-----
 src/slave/containerizer/mesos/launcher_tracker.cpp | 109 ++++++++++++++
 .../{linux_launcher.hpp => launcher_tracker.hpp}   |  35 ++---
 src/slave/http.cpp                                 |  96 +++++++++++++
 src/slave/http.hpp                                 |   9 ++
 src/slave/main.cpp                                 |  13 +-
 src/slave/slave.cpp                                |  10 ++
 src/slave/slave.hpp                                |   3 +
 src/tests/CMakeLists.txt                           |   1 +
 src/tests/cluster.cpp                              |  12 +-
 src/tests/common/future_tracker_tests.cpp          | 148 +++++++++++++++++++
 src/tests/mock_slave.cpp                           |   2 +
 src/tests/mock_slave.hpp                           |   1 +
 25 files changed, 819 insertions(+), 80 deletions(-)
 create mode 100644 src/common/future_tracker.hpp
 create mode 100644 src/slave/containerizer/mesos/isolator_tracker.cpp
 copy src/slave/containerizer/mesos/{isolators/filesystem/posix.hpp => isolator_tracker.hpp} (59%)
 create mode 100644 src/slave/containerizer/mesos/launcher_tracker.cpp
 copy src/slave/containerizer/mesos/{linux_launcher.hpp => launcher_tracker.hpp} (68%)
 create mode 100644 src/tests/common/future_tracker_tests.cpp


[mesos] 01/07: Added `PendingFutureTracker` class for tracking pending futures.

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 bf01659f5b064842d9a8d088f3fa925c20a660ed
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Thu Jul 18 09:10:39 2019 -0700

    Added `PendingFutureTracker` class for tracking pending futures.
    
    This patch introduces a mechanism for tracking pending futures.
    This feature allows detection of hanging operations, which get
    stuck on a blocking operation or asynchronously. However, this
    feature does not provide any mechanism for tracking pending
    promises, because `Promise` objects might not be accessible in
    various cases. Thereby, we introduce a new class that can be
    used to track pending futures, so it might facilitate debugging
    of stuck issues.
    
    Review: https://reviews.apache.org/r/70887/
---
 src/Makefile.am               |   1 +
 src/common/future_tracker.hpp | 159 ++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 160 insertions(+)

diff --git a/src/Makefile.am b/src/Makefile.am
index 761dde1..ecdced4 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1067,6 +1067,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   common/build.hpp							\
   common/command_utils.cpp						\
   common/command_utils.hpp						\
+  common/future_tracker.hpp						\
   common/heartbeater.hpp						\
   common/http.cpp							\
   common/http.hpp							\
diff --git a/src/common/future_tracker.hpp b/src/common/future_tracker.hpp
new file mode 100644
index 0000000..a3f191a
--- /dev/null
+++ b/src/common/future_tracker.hpp
@@ -0,0 +1,159 @@
+// 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 __FUTURE_TRACKER_HPP__
+#define __FUTURE_TRACKER_HPP__
+
+#include <list>
+#include <map>
+#include <string>
+#include <vector>
+
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+#include <process/future.hpp>
+#include <process/id.hpp>
+#include <process/owned.hpp>
+#include <process/process.hpp>
+
+namespace mesos {
+namespace internal {
+
+struct FutureMetadata
+{
+  std::string operation;
+  std::string component;
+  std::map<std::string, std::string> args;
+
+  inline bool operator==(const FutureMetadata& that) const
+  {
+    return operation == that.operation &&
+           component == that.component &&
+           args == that.args;
+  }
+};
+
+
+class PendingFutureTrackerProcess
+  : public process::Process<PendingFutureTrackerProcess>
+{
+public:
+  PendingFutureTrackerProcess()
+    : ProcessBase(process::ID::generate("pending-future-tracker")) {}
+
+  template <typename T>
+  void addFuture(const process::Future<T>& future, FutureMetadata&& metadata)
+  {
+    auto it = pending.emplace(pending.end(), std::move(metadata));
+
+    future
+      .onAny(process::defer(
+          self(), &PendingFutureTrackerProcess::eraseFuture, it))
+      .onAbandoned(process::defer(
+          self(), &PendingFutureTrackerProcess::eraseFuture, it))
+      .onDiscard(process::defer(
+          self(), &PendingFutureTrackerProcess::eraseFuture, it));
+  }
+
+  void eraseFuture(typename std::list<FutureMetadata>::iterator it)
+  {
+    pending.erase(it);
+  }
+
+  process::Future<std::vector<FutureMetadata>> pendingFutures()
+  {
+    return std::vector<FutureMetadata>(pending.begin(), pending.end());
+  }
+
+private:
+  std::list<FutureMetadata> pending;
+};
+
+
+class PendingFutureTracker
+{
+public:
+  static Try<PendingFutureTracker*> create()
+  {
+    return new PendingFutureTracker(process::Owned<PendingFutureTrackerProcess>(
+        new PendingFutureTrackerProcess));
+  }
+
+  ~PendingFutureTracker()
+  {
+    terminate(process.get());
+    process::wait(process.get());
+  }
+
+  /**
+   * This method subscribes on state transitions of the `future` to keep track
+   * of pending operations/promises associated with this future.
+   *
+   * @param operation Operation's name identifies the place in the code related
+   * to this future. E.g., "some/isolator::prepare".
+   *
+   * @param component Component is used to distinguish pending futures
+   * related to different components so that they can be exposed by
+   * different API endpoints.
+   *
+   * @param args A list of pairs <argument name, argument value> representing
+   * arguments passed to the function that returned the given future.
+   *
+   * @return The same `future` which is passed as the first argument.
+   */
+  template <typename T>
+  process::Future<T> track(
+      const process::Future<T>& future,
+      const std::string& operation,
+      const std::string& component,
+      const std::map<std::string, std::string>& args = {})
+  {
+    process::dispatch(
+        process.get(),
+        &PendingFutureTrackerProcess::addFuture<T>,
+        future,
+        FutureMetadata{operation, component, args});
+
+    return future;
+  }
+
+  /**
+   * This method returns a list of pending futures represented as objects of
+   * `FutureMetadata` class, whose variables are initialized by the arguments
+   * passed to the `track` method.
+   */
+  process::Future<std::vector<FutureMetadata>> pendingFutures()
+  {
+    return process::dispatch(
+        process.get(),
+        &PendingFutureTrackerProcess::pendingFutures);
+  }
+
+private:
+  explicit PendingFutureTracker(
+      const process::Owned<PendingFutureTrackerProcess>& _process)
+    : process(_process)
+  {
+    spawn(process.get());
+  }
+
+  process::Owned<PendingFutureTrackerProcess> process;
+};
+
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __FUTURE_TRACKER_HPP__


[mesos] 07/07: Implemented `FutureTrackTest` tests.

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 f3b827fa0206715ea1242ab839c3f7c0d7f685f4
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Thu Jul 18 09:10:54 2019 -0700

    Implemented `FutureTrackTest` tests.
    
    These tests verify functionality provided by the
    `PendingFutureTracker` class.
    
    Review: https://reviews.apache.org/r/71065/
---
 src/Makefile.am                           |   1 +
 src/tests/CMakeLists.txt                  |   1 +
 src/tests/common/future_tracker_tests.cpp | 148 ++++++++++++++++++++++++++++++
 3 files changed, 150 insertions(+)

diff --git a/src/Makefile.am b/src/Makefile.am
index 80ae6c0..3c600bd 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -2675,6 +2675,7 @@ mesos_tests_SOURCES =						\
   tests/zookeeper_test_server.hpp				\
   tests/zookeeper_url_tests.cpp					\
   tests/common/command_utils_tests.cpp				\
+  tests/common/future_tracker_tests.cpp				\
   tests/common/http_tests.cpp					\
   tests/common/recordio_tests.cpp				\
   tests/common/type_utils_tests.cpp				\
diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt
index 5eb9c65..57344a1 100644
--- a/src/tests/CMakeLists.txt
+++ b/src/tests/CMakeLists.txt
@@ -152,6 +152,7 @@ if (ENABLE_SSL)
 endif ()
 
 list(APPEND MESOS_TESTS_SRC
+  common/future_tracker_tests.cpp
   common/http_tests.cpp
   common/recordio_tests.cpp
   common/type_utils_tests.cpp)
diff --git a/src/tests/common/future_tracker_tests.cpp b/src/tests/common/future_tracker_tests.cpp
new file mode 100644
index 0000000..29d43cc
--- /dev/null
+++ b/src/tests/common/future_tracker_tests.cpp
@@ -0,0 +1,148 @@
+// 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 <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include <process/gmock.hpp>
+#include <process/gtest.hpp>
+#include <process/owned.hpp>
+
+#include <stout/lambda.hpp>
+
+#include "common/future_tracker.hpp"
+
+using std::string;
+using std::vector;
+
+using process::Future;
+using process::Owned;
+using process::Promise;
+
+using testing::_;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+class FutureTrackerTest : public ::testing::Test
+{
+protected:
+  void SetUp() override
+  {
+    Try<PendingFutureTracker*> _tracker = PendingFutureTracker::create();
+    ASSERT_FALSE(_tracker.isError());
+    tracker.reset(_tracker.get());
+  }
+
+  Owned<PendingFutureTracker> tracker;
+};
+
+
+TEST_F(FutureTrackerTest, ListPending)
+{
+  Promise<bool> promise1;
+  Promise<bool> promise2;
+
+  Future<bool> future1 = promise1.future();
+  Future<bool> future2 = promise2.future();
+
+  const FutureMetadata data1{"f1", "test1", {}};
+  const FutureMetadata data2{
+    "f2", "test2", {{"arg1", "val1"}, {"arg2", "val2"}}};
+
+  auto track1 =
+    tracker->track(future1, data1.operation, data1.component, data1.args);
+
+  auto track2 =
+    tracker->track(future2, data2.operation, data2.component, data2.args);
+
+  EXPECT_EQ(track1, future1);
+  EXPECT_EQ(track2, future2);
+
+  auto pending = tracker->pendingFutures();
+
+  AWAIT_READY(pending);
+
+  EXPECT_EQ(pending->size(), 2u);
+  EXPECT_EQ(pending->at(0), data1);
+  EXPECT_EQ(pending->at(1), data2);
+}
+
+
+TEST_F(FutureTrackerTest, ListReady)
+{
+  Promise<bool> promise1;
+  Promise<bool> promise2;
+
+  const FutureMetadata data1{"f1", "test1", {}};
+  const FutureMetadata data2{"f2", "test2", {}};
+
+  tracker->track(promise1.future(), data1.operation, data1.component);
+  tracker->track(promise2.future(), data2.operation, data2.component);
+
+  Future<Nothing> eraseFuture =
+    FUTURE_DISPATCH(_, &PendingFutureTrackerProcess::eraseFuture);
+
+  promise2.set(true);
+
+  AWAIT_READY(eraseFuture);
+
+  auto pending = tracker->pendingFutures();
+
+  AWAIT_READY(pending);
+
+  EXPECT_EQ(pending->size(), 1u);
+  EXPECT_EQ(pending->front(), data1);
+}
+
+
+TEST_F(FutureTrackerTest, ListAfterCancellation)
+{
+  vector<lambda::function<void(Owned<Promise<bool>>&)>> cancellations = {
+    [](Owned<Promise<bool>>& promise) {
+      promise->discard();
+    },
+    [](Owned<Promise<bool>>& promise) {
+      promise.reset();
+    }
+  };
+
+  for (auto& cancel : cancellations) {
+    Owned<Promise<bool>> promise(new Promise<bool>());
+
+    tracker->track(promise->future(), "f", "test");
+
+    Future<Nothing> eraseFuture =
+      FUTURE_DISPATCH(_, &PendingFutureTrackerProcess::eraseFuture);
+
+    cancel(promise);
+
+    AWAIT_READY(eraseFuture);
+
+    auto pending = tracker->pendingFutures();
+
+    AWAIT_READY(pending);
+
+    EXPECT_TRUE(pending->empty());
+  }
+}
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {


[mesos] 04/07: Added `LauncherTracker` for tracking calls of launcher methods.

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 5e01feb96874c0f7327ada82c39165900610c1b4
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Thu Jul 18 09:10:44 2019 -0700

    Added `LauncherTracker` for tracking calls of launcher methods.
    
    This patch adds a new `LauncherTracker` class that proxies
    calls to the real `Launcher` and keeps track of returned futures
    by employing `PendingFutureTracker` class.
    
    Review: https://reviews.apache.org/r/70890/
---
 src/CMakeLists.txt                                 |   1 +
 src/Makefile.am                                    |   2 +
 src/slave/containerizer/mesos/launcher_tracker.cpp | 109 +++++++++++++++++++++
 src/slave/containerizer/mesos/launcher_tracker.hpp |  63 ++++++++++++
 4 files changed, 175 insertions(+)

diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index c4d32df..99402cb 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -173,6 +173,7 @@ set(AGENT_SRC
   slave/containerizer/mesos/isolator_tracker.cpp
   slave/containerizer/mesos/launch.cpp
   slave/containerizer/mesos/launcher.cpp
+  slave/containerizer/mesos/launcher_tracker.cpp
   slave/containerizer/mesos/mount.cpp
   slave/containerizer/mesos/paths.cpp
   slave/containerizer/mesos/io/switchboard.cpp
diff --git a/src/Makefile.am b/src/Makefile.am
index 395b9f7..80ae6c0 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1254,6 +1254,8 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   slave/containerizer/mesos/launch.hpp					\
   slave/containerizer/mesos/launcher.cpp				\
   slave/containerizer/mesos/launcher.hpp				\
+  slave/containerizer/mesos/launcher_tracker.cpp			\
+  slave/containerizer/mesos/launcher_tracker.hpp			\
   slave/containerizer/mesos/mount.cpp					\
   slave/containerizer/mesos/mount.hpp					\
   slave/containerizer/mesos/paths.cpp					\
diff --git a/src/slave/containerizer/mesos/launcher_tracker.cpp b/src/slave/containerizer/mesos/launcher_tracker.cpp
new file mode 100644
index 0000000..4434e57
--- /dev/null
+++ b/src/slave/containerizer/mesos/launcher_tracker.cpp
@@ -0,0 +1,109 @@
+// 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 <process/async.hpp>
+
+#include "common/future_tracker.hpp"
+
+#include "slave/constants.hpp"
+
+#include "slave/containerizer/mesos/launcher_tracker.hpp"
+
+using std::map;
+using std::string;
+using std::vector;
+
+using process::Future;
+using process::Promise;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+LauncherTracker::LauncherTracker(
+    const process::Owned<Launcher>& _launcher, PendingFutureTracker* _tracker)
+  : launcher(_launcher), tracker(_tracker)
+{}
+
+
+Future<hashset<ContainerID>> LauncherTracker::recover(
+    const vector<mesos::slave::ContainerState>& states)
+{
+  return tracker->track(
+      launcher->recover(states),
+      "launcher::recover",
+      COMPONENT_NAME_CONTAINERIZER);
+}
+
+
+Try<pid_t> LauncherTracker::fork(
+    const ContainerID& containerId,
+    const string& path,
+    const vector<string>& argv,
+    const mesos::slave::ContainerIO& containerIO,
+    const flags::FlagsBase* flags,
+    const Option<map<string, string>>& environment,
+    const Option<int>& enterNamespaces,
+    const Option<int>& cloneNamespaces,
+    const vector<int_fd>& whitelistFds)
+{
+  Promise<Try<pid_t>> promise;
+
+  tracker->track(
+      promise.future(),
+      "launcher::fork",
+      COMPONENT_NAME_CONTAINERIZER,
+      {{"containerId", stringify(containerId)},
+       {"path", path}});
+
+  Try<pid_t> forked = launcher->fork(
+      containerId,
+      path,
+      argv,
+      containerIO,
+      flags,
+      environment,
+      enterNamespaces,
+      cloneNamespaces,
+      whitelistFds);
+
+  promise.set(forked);
+  return forked;
+}
+
+
+Future<Nothing> LauncherTracker::destroy(const ContainerID& containerId)
+{
+  return tracker->track(
+      launcher->destroy(containerId),
+      "launcher::destroy",
+      COMPONENT_NAME_CONTAINERIZER,
+      {{"containerId", stringify(containerId)}});
+}
+
+
+Future<ContainerStatus> LauncherTracker::status(const ContainerID& containerId)
+{
+  return tracker->track(
+      launcher->status(containerId),
+      "launcher::status",
+      COMPONENT_NAME_CONTAINERIZER,
+      {{"containerId", stringify(containerId)}});
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
diff --git a/src/slave/containerizer/mesos/launcher_tracker.hpp b/src/slave/containerizer/mesos/launcher_tracker.hpp
new file mode 100644
index 0000000..999f6be
--- /dev/null
+++ b/src/slave/containerizer/mesos/launcher_tracker.hpp
@@ -0,0 +1,63 @@
+// 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 __LAUNCHER_TRACKER_HPP__
+#define __LAUNCHER_TRACKER_HPP__
+
+#include <process/owned.hpp>
+
+#include "slave/containerizer/mesos/launcher.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class LauncherTracker : public Launcher
+{
+public:
+  LauncherTracker(
+      const process::Owned<Launcher>& _launcher,
+      PendingFutureTracker* _tracker);
+
+  process::Future<hashset<ContainerID>> recover(
+      const std::vector<mesos::slave::ContainerState>& states) override;
+
+  Try<pid_t> fork(
+      const ContainerID& containerId,
+      const std::string& path,
+      const std::vector<std::string>& argv,
+      const mesos::slave::ContainerIO& containerIO,
+      const flags::FlagsBase* flags,
+      const Option<std::map<std::string, std::string>>& environment,
+      const Option<int>& enterNamespaces,
+      const Option<int>& cloneNamespaces,
+      const std::vector<int_fd>& whitelistFds) override;
+
+  process::Future<Nothing> destroy(const ContainerID& containerId) override;
+
+  process::Future<ContainerStatus> status(
+      const ContainerID& containerId) override;
+
+private:
+  process::Owned<Launcher> launcher;
+  PendingFutureTracker* tracker;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __LAUNCHER_TRACKER_HPP__


[mesos] 02/07: Added `IsolatorTracker` for tracking calls of isolator methods.

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 5112ab806ca0fdcdbf8a8fa55e87dbf7437e0f9d
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Thu Jul 18 09:10:40 2019 -0700

    Added `IsolatorTracker` for tracking calls of isolator methods.
    
    This patch adds a new `IsolatorTracker` class that proxies
    calls to the real isolator and keeps track of returned futures
    by employing `PendingFutureTracker` class.
    
    Review: https://reviews.apache.org/r/70888/
---
 src/CMakeLists.txt                                 |   1 +
 src/Makefile.am                                    |   2 +
 src/slave/constants.hpp                            |   3 +
 src/slave/containerizer/mesos/isolator_tracker.cpp | 152 +++++++++++++++++++++
 src/slave/containerizer/mesos/isolator_tracker.hpp |  79 +++++++++++
 5 files changed, 237 insertions(+)

diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 6d14090..c4d32df 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -170,6 +170,7 @@ set(AGENT_SRC
   slave/containerizer/fetcher.cpp
   slave/containerizer/mesos/containerizer.cpp
   slave/containerizer/mesos/isolator.cpp
+  slave/containerizer/mesos/isolator_tracker.cpp
   slave/containerizer/mesos/launch.cpp
   slave/containerizer/mesos/launcher.cpp
   slave/containerizer/mesos/mount.cpp
diff --git a/src/Makefile.am b/src/Makefile.am
index ecdced4..395b9f7 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1225,6 +1225,8 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   slave/containerizer/mesos/io/switchboard.hpp				\
   slave/containerizer/mesos/isolator.cpp				\
   slave/containerizer/mesos/isolator.hpp				\
+  slave/containerizer/mesos/isolator_tracker.cpp			\
+  slave/containerizer/mesos/isolator_tracker.hpp			\
   slave/containerizer/mesos/isolators/docker/volume/driver.cpp		\
   slave/containerizer/mesos/isolators/docker/volume/driver.hpp		\
   slave/containerizer/mesos/isolators/docker/volume/paths.cpp		\
diff --git a/src/slave/constants.hpp b/src/slave/constants.hpp
index cc81e51..721afe1 100644
--- a/src/slave/constants.hpp
+++ b/src/slave/constants.hpp
@@ -202,6 +202,9 @@ constexpr char MESOS_DEFAULT_EXECUTOR[] = "mesos-default-executor";
 constexpr char MESOS_EXECUTOR[] = "mesos-executor";
 #endif // __WINDOWS__
 
+// Name of the component used for describing pending futures.
+constexpr char COMPONENT_NAME_CONTAINERIZER[] = "containerizer";
+
 
 // Virtual path on which agent logs are mounted in `/files/` endpoint.
 constexpr char AGENT_LOG_VIRTUAL_PATH[] = "/slave/log";
diff --git a/src/slave/containerizer/mesos/isolator_tracker.cpp b/src/slave/containerizer/mesos/isolator_tracker.cpp
new file mode 100644
index 0000000..c78e883
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolator_tracker.cpp
@@ -0,0 +1,152 @@
+// 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 "common/future_tracker.hpp"
+
+#include "slave/constants.hpp"
+
+#include "slave/containerizer/mesos/isolator_tracker.hpp"
+
+using std::string;
+using std::vector;
+
+using process::Future;
+using process::Owned;
+
+using mesos::slave::ContainerConfig;
+using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::ContainerLimitation;
+using mesos::slave::ContainerState;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+IsolatorTracker::IsolatorTracker(
+    const Owned<mesos::slave::Isolator>& _isolator,
+    const string& _isolatorName,
+    PendingFutureTracker* _tracker)
+  : isolator(_isolator),
+    isolatorName(_isolatorName),
+    tracker(_tracker)
+{}
+
+
+bool IsolatorTracker::supportsNesting()
+{
+  return isolator->supportsNesting();
+}
+
+
+bool IsolatorTracker::supportsStandalone()
+{
+  return isolator->supportsStandalone();
+}
+
+
+Future<Nothing> IsolatorTracker::recover(
+    const vector<ContainerState>& state,
+    const hashset<ContainerID>& orphans)
+{
+  return tracker->track(
+      isolator->recover(state, orphans),
+      strings::format("%s::recover", isolatorName).get(),
+      COMPONENT_NAME_CONTAINERIZER);
+}
+
+
+Future<Option<ContainerLaunchInfo>> IsolatorTracker::prepare(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig)
+{
+  return tracker->track(
+      isolator->prepare(containerId, containerConfig),
+      strings::format("%s::prepare", isolatorName).get(),
+      COMPONENT_NAME_CONTAINERIZER,
+      {{"containerId", stringify(containerId)}});
+}
+
+
+Future<Nothing> IsolatorTracker::isolate(
+    const ContainerID& containerId,
+    pid_t pid)
+{
+  return tracker->track(
+      isolator->isolate(containerId, pid),
+      strings::format("%s::isolate", isolatorName).get(),
+      COMPONENT_NAME_CONTAINERIZER,
+      {{"containerId", stringify(containerId)},
+       {"pid", stringify(pid)}});
+}
+
+
+Future<ContainerLimitation> IsolatorTracker::watch(
+    const ContainerID& containerId)
+{
+  // Do not track `watch` method, since it is supposed
+  // to be pending as long as the container is running.
+  return isolator->watch(containerId);
+}
+
+
+Future<Nothing> IsolatorTracker::update(
+    const ContainerID& containerId,
+    const Resources& resources)
+{
+  return tracker->track(
+      isolator->update(containerId, resources),
+      strings::format("%s::update", isolatorName).get(),
+      COMPONENT_NAME_CONTAINERIZER,
+      {{"containerId", stringify(containerId)},
+       {"resources", stringify(resources)}});
+}
+
+
+Future<ResourceStatistics> IsolatorTracker::usage(
+    const ContainerID& containerId)
+{
+  return tracker->track(
+      isolator->usage(containerId),
+      strings::format("%s::usage", isolatorName).get(),
+      COMPONENT_NAME_CONTAINERIZER,
+      {{"containerId", stringify(containerId)}});
+}
+
+
+Future<ContainerStatus> IsolatorTracker::status(
+    const ContainerID& containerId)
+{
+  return tracker->track(
+      isolator->status(containerId),
+      strings::format("%s::status", isolatorName).get(),
+      COMPONENT_NAME_CONTAINERIZER,
+      {{"containerId", stringify(containerId)}});
+}
+
+
+Future<Nothing> IsolatorTracker::cleanup(
+    const ContainerID& containerId)
+{
+  return tracker->track(
+      isolator->cleanup(containerId),
+      strings::format("%s::cleanup", isolatorName).get(),
+      COMPONENT_NAME_CONTAINERIZER,
+      {{"containerId", stringify(containerId)}});
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
diff --git a/src/slave/containerizer/mesos/isolator_tracker.hpp b/src/slave/containerizer/mesos/isolator_tracker.hpp
new file mode 100644
index 0000000..563db10
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolator_tracker.hpp
@@ -0,0 +1,79 @@
+// 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 __ISOLATOR_TRACKER_HPP__
+#define __ISOLATOR_TRACKER_HPP__
+
+#include <mesos/slave/isolator.hpp>
+
+#include <process/owned.hpp>
+#include <process/process.hpp>
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class IsolatorTracker : public mesos::slave::Isolator
+{
+public:
+  IsolatorTracker(
+      const process::Owned<mesos::slave::Isolator>& _isolator,
+      const std::string& _isolatorName,
+      PendingFutureTracker* _tracker);
+
+  bool supportsNesting() override;
+  bool supportsStandalone() override;
+
+  process::Future<Nothing> recover(
+      const std::vector<mesos::slave::ContainerState>& states,
+      const hashset<ContainerID>& orphans) override;
+
+  process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig) override;
+
+  process::Future<Nothing> isolate(
+      const ContainerID& containerId,
+      pid_t pid) override;
+
+  process::Future<mesos::slave::ContainerLimitation> watch(
+      const ContainerID& containerId) override;
+
+  process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources) override;
+
+  process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId) override;
+
+  process::Future<ContainerStatus> status(
+      const ContainerID& containerId) override;
+
+  process::Future<Nothing> cleanup(
+      const ContainerID& containerId) override;
+
+private:
+  process::Owned<mesos::slave::Isolator> isolator;
+  std::string isolatorName;
+  PendingFutureTracker* tracker;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+
+#endif // __ISOLATOR_TRACKER_HPP__


[mesos] 05/07: Wrapped launcher in `LauncherTracker`.

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 59c2c75d0982385271c3ba86e3cbbf6c21fa7bae
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Thu Jul 18 09:10:45 2019 -0700

    Wrapped launcher in `LauncherTracker`.
    
    This patch wraps a container launcher in instance of `LauncherTracker`
    class. If the launcher gets stuck in some operation, `pendingFutures`
    will return the method name along with its arguments such as
    `containerId`, `pid`, etc.
    
    Review: https://reviews.apache.org/r/70891/
---
 src/slave/containerizer/mesos/containerizer.cpp | 15 +++++++++++----
 1 file changed, 11 insertions(+), 4 deletions(-)

diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index b4d10a7..6f76527 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -73,6 +73,7 @@
 #include "slave/containerizer/mesos/isolator_tracker.hpp"
 #include "slave/containerizer/mesos/launch.hpp"
 #include "slave/containerizer/mesos/launcher.hpp"
+#include "slave/containerizer/mesos/launcher_tracker.hpp"
 #include "slave/containerizer/mesos/paths.hpp"
 #include "slave/containerizer/mesos/utils.hpp"
 
@@ -316,7 +317,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
   LOG(INFO) << "Using isolation " << stringify(isolations.get());
 
   // Create the launcher for the MesosContainerizer.
-  Try<Launcher*> launcher = [&flags]() -> Try<Launcher*> {
+  Try<Launcher*> _launcher = [&flags]() -> Try<Launcher*> {
 #ifdef __linux__
     if (flags.launcher == "linux") {
       return LinuxLauncher::create(flags);
@@ -340,8 +341,14 @@ Try<MesosContainerizer*> MesosContainerizer::create(
 #endif // __linux__
   }();
 
-  if (launcher.isError()) {
-    return Error("Failed to create launcher: " + launcher.error());
+  if (_launcher.isError()) {
+    return Error("Failed to create launcher: " + _launcher.error());
+  }
+
+  Owned<Launcher> launcher = Owned<Launcher>(_launcher.get());
+
+  if (futureTracker != nullptr) {
+    launcher = Owned<Launcher>(new LauncherTracker(launcher, futureTracker));
   }
 
   Try<Owned<Provisioner>> _provisioner =
@@ -597,7 +604,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
       local,
       fetcher,
       gc,
-      Owned<Launcher>(launcher.get()),
+      launcher,
       provisioner,
       isolators,
       volumeGidManager);


[mesos] 06/07: Added `/containerizer/debug` endpoint.

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 960497728be1d06724b077f73a978d1c798705b7
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Thu Jul 18 09:10:47 2019 -0700

    Added `/containerizer/debug` endpoint.
    
    This patch introduces an agent's `/containerizer/debug` endpoint,
    which exposes the debug info related to Mesos containerizer.
    Currently, this endpoint returns a list of pending futures related to
    isolators or containerizer launcher. This endpoint is experimental,
    and the format of its output may change over time.
    
    Review: https://reviews.apache.org/r/70892/
---
 docs/authorization.md        |  1 +
 src/common/authorization.cpp |  1 +
 src/local/local.cpp          |  1 +
 src/slave/http.cpp           | 96 ++++++++++++++++++++++++++++++++++++++++++++
 src/slave/http.hpp           |  9 +++++
 src/slave/main.cpp           |  1 +
 src/slave/slave.cpp          | 10 +++++
 src/slave/slave.hpp          |  3 ++
 src/tests/cluster.cpp        |  2 +
 src/tests/mock_slave.cpp     |  2 +
 src/tests/mock_slave.hpp     |  1 +
 11 files changed, 127 insertions(+)

diff --git a/docs/authorization.md b/docs/authorization.md
index 91dc03b..698e485 100644
--- a/docs/authorization.md
+++ b/docs/authorization.md
@@ -366,6 +366,7 @@ The `get_endpoints` action covers:
 * `/logging/toggle`
 * `/metrics/snapshot`
 * `/slave(id)/containers`
+* `/slave(id)/containerizer/debug`
 * `/slave(id)/monitor/statistics`
 
 ### Examples
diff --git a/src/common/authorization.cpp b/src/common/authorization.cpp
index 8948721..fa71b0e 100644
--- a/src/common/authorization.cpp
+++ b/src/common/authorization.cpp
@@ -46,6 +46,7 @@ namespace authorization {
 // action `GET_ENDPOINTS_WITH_PATH`.
 hashset<string> AUTHORIZABLE_ENDPOINTS{
     "/containers",
+    "/containerizer/debug",
     "/files/debug",
     "/logging/toggle",
     "/metrics/snapshot",
diff --git a/src/local/local.cpp b/src/local/local.cpp
index 6ac6b02..68dc9fb 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -534,6 +534,7 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
         qosControllers->back(),
         secretGenerators->back(),
         nullptr,
+        nullptr,
         authorizer_); // Same authorizer as master.
 
     slaves[containerizer.get()] = slave;
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 321dca7..d9f113d 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -59,6 +59,7 @@
 
 #include "common/authorization.hpp"
 #include "common/build.hpp"
+#include "common/future_tracker.hpp"
 #include "common/http.hpp"
 #include "common/recordio.hpp"
 #include "common/resources_utils.hpp"
@@ -71,6 +72,7 @@
 
 #include "resource_provider/local.hpp"
 
+#include "slave/constants.hpp"
 #include "slave/http.hpp"
 #include "slave/slave.hpp"
 #include "slave/validation.hpp"
@@ -2350,6 +2352,100 @@ Future<JSON::Array> Http::__containers(
 }
 
 
+string Http::CONTAINERIZER_DEBUG_HELP()
+{
+  return HELP(
+      TLDR(
+          "Retrieve debug information for the Mesos containerizer."),
+      DESCRIPTION(
+          "Returns a list of pending operations related to Mesos",
+          "containerizer. This endpoint can help investigating",
+          "container stuck issues.",
+          "",
+          "**Note**: There is no fixed schema for a pending operation.",
+          "Thereby, the output of this endpoint should not be used by",
+          "automated tools.",
+          "",
+          "Example (**Note**: this is not exhaustive):",
+          "",
+          "```",
+          "{",
+          "    \"pending\":[",
+          "        {",
+          "            \"operation\":\"network/cni::attach\",",
+          "            \"args\":{",
+          "                \"containerId\":\"container\"",
+          "            }",
+          "        }",
+          "    ]",
+          "}",
+          "```"),
+      AUTHENTICATION(true));
+}
+
+
+Future<Response> Http::containerizerDebug(
+    const Request& request,
+    const Option<Principal>& principal) const
+{
+  // TODO(a10gupta): Remove check for enabled
+  // authorization as part of MESOS-5346.
+  if (request.method != "GET" && slave->authorizer.isSome()) {
+    return MethodNotAllowed({"GET"}, request.method);
+  }
+
+  Try<string> endpoint = extractEndpoint(request.url);
+  if (endpoint.isError()) {
+    return Failure("Failed to extract endpoint: " + endpoint.error());
+  }
+
+  return authorizeEndpoint(
+      endpoint.get(),
+      request.method,
+      slave->authorizer,
+      principal)
+    .then(defer(
+        slave->self(),
+        [this, request](bool authorized) -> Future<Response> {
+          if (!authorized) {
+            return Forbidden();
+          }
+
+          return _containerizerDebug()
+           .then([request](const JSON::Object& result) -> Response {
+              return process::http::OK(result, request.url.query.get("jsonp"));
+           });
+        }));
+}
+
+
+Future<JSON::Object> Http::_containerizerDebug() const
+{
+  return slave->futureTracker->pendingFutures().then(
+      defer(slave->self(), [](const vector<FutureMetadata>& pending) {
+        JSON::Object result;
+
+        JSON::Array futures;
+        foreach (const FutureMetadata& metadata, pending) {
+          if (metadata.component != COMPONENT_NAME_CONTAINERIZER) {
+            continue;
+          }
+
+          JSON::Object args;
+          foreachpair (const string& key, const string& value, metadata.args) {
+            args.values[key] = JSON::String(value);
+          }
+
+          futures.values.emplace_back(JSON::Object{
+              {"operation", JSON::String(metadata.operation)}, {"args", args}});
+        }
+        result.values["pending"] = std::move(futures);
+
+        return result;
+      }));
+}
+
+
 Future<Response> Http::pruneImages(
     const agent::Call& call,
     ContentType acceptType,
diff --git a/src/slave/http.hpp b/src/slave/http.hpp
index b8c83f1..0afdad9 100644
--- a/src/slave/http.hpp
+++ b/src/slave/http.hpp
@@ -80,6 +80,11 @@ public:
       const process::http::Request& request,
       const Option<process::http::authentication::Principal>& principal) const;
 
+  // /slave/containerizer/debug
+  process::Future<process::http::Response> containerizerDebug(
+      const process::http::Request& request,
+      const Option<process::http::authentication::Principal>& principal) const;
+
   static std::string API_HELP();
   static std::string EXECUTOR_HELP();
   static std::string RESOURCE_PROVIDER_HELP();
@@ -88,6 +93,7 @@ public:
   static std::string STATE_HELP();
   static std::string STATISTICS_HELP();
   static std::string CONTAINERS_HELP();
+  static std::string CONTAINERIZER_DEBUG_HELP();
 
 private:
   JSON::Object _flags() const;
@@ -120,6 +126,9 @@ private:
       bool showNestedContainers,
       bool showStandaloneContainers) const;
 
+  // Continuation for `/containerizer/debug` endpoint
+  process::Future<JSON::Object> _containerizerDebug() const;
+
   // Helper routines for endpoint authorization.
   Try<std::string> extractEndpoint(const process::http::URL& url) const;
 
diff --git a/src/slave/main.cpp b/src/slave/main.cpp
index c974ba0..fd58637 100644
--- a/src/slave/main.cpp
+++ b/src/slave/main.cpp
@@ -619,6 +619,7 @@ int main(int argc, char** argv)
       qosController.get(),
       secretGenerator,
       volumeGidManager,
+      futureTracker.get(),
       authorizer_);
 
   process::spawn(slave);
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 0e7e4d4..9c14784 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -198,6 +198,7 @@ Slave::Slave(const string& id,
              QoSController* _qosController,
              SecretGenerator* _secretGenerator,
              VolumeGidManager* _volumeGidManager,
+             PendingFutureTracker* _futureTracker,
              const Option<Authorizer*>& _authorizer)
   : ProcessBase(id),
     state(RECOVERING),
@@ -228,6 +229,7 @@ Slave::Slave(const string& id,
     qosController(_qosController),
     secretGenerator(_secretGenerator),
     volumeGidManager(_volumeGidManager),
+    futureTracker(_futureTracker),
     authorizer(_authorizer),
     resourceVersion(protobuf::createUUID()) {}
 
@@ -836,6 +838,14 @@ void Slave::initialize()
               logResponse(request, response);
             });
         });
+  route("/containerizer/debug",
+        READONLY_HTTP_AUTHENTICATION_REALM,
+        Http::CONTAINERIZER_DEBUG_HELP(),
+        [this](const http::Request& request,
+               const Option<Principal>& principal) {
+          logRequest(request);
+          return http.containerizerDebug(request, principal);
+        });
 
   // TODO(tillt): Use generalized lambda capture once we adopt C++14.
   Option<Authorizer*> _authorizer = authorizer;
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 58a5608..556d8ea 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -129,6 +129,7 @@ public:
         mesos::slave::QoSController* qosController,
         mesos::SecretGenerator* secretGenerator,
         VolumeGidManager* volumeGidManager,
+        PendingFutureTracker* futureTracker,
         const Option<Authorizer*>& authorizer);
 
   ~Slave() override;
@@ -868,6 +869,8 @@ private:
 
   VolumeGidManager* volumeGidManager;
 
+  PendingFutureTracker* futureTracker;
+
   const Option<Authorizer*> authorizer;
 
   // The most recent estimate of the total amount of oversubscribed
diff --git a/src/tests/cluster.cpp b/src/tests/cluster.cpp
index 9f180cc..1646516 100644
--- a/src/tests/cluster.cpp
+++ b/src/tests/cluster.cpp
@@ -621,6 +621,7 @@ Try<process::Owned<Slave>> Slave::create(
         qosController.getOrElse(slave->qosController.get()),
         secretGenerator.getOrElse(slave->secretGenerator.get()),
         volumeGidManager,
+        futureTracker.get(),
         authorizer));
   } else {
     slave->slave.reset(new slave::Slave(
@@ -635,6 +636,7 @@ Try<process::Owned<Slave>> Slave::create(
         qosController.getOrElse(slave->qosController.get()),
         secretGenerator.getOrElse(slave->secretGenerator.get()),
         volumeGidManager,
+        futureTracker.get(),
         authorizer));
   }
 
diff --git a/src/tests/mock_slave.cpp b/src/tests/mock_slave.cpp
index dd458e3..71be957 100644
--- a/src/tests/mock_slave.cpp
+++ b/src/tests/mock_slave.cpp
@@ -103,6 +103,7 @@ MockSlave::MockSlave(
     QoSController* qosController,
     SecretGenerator* secretGenerator,
     VolumeGidManager* volumeGidManager,
+    PendingFutureTracker* futureTracker,
     const Option<Authorizer*>& authorizer)
   // It is necessary to explicitly call `ProcessBase` constructor here even
   // though the direct parent `Slave` already does this. This is because
@@ -122,6 +123,7 @@ MockSlave::MockSlave(
         qosController,
         secretGenerator,
         volumeGidManager,
+        futureTracker,
         authorizer)
 {
   // Set up default behaviors, calling the original methods.
diff --git a/src/tests/mock_slave.hpp b/src/tests/mock_slave.hpp
index c057b40..eb31a0f 100644
--- a/src/tests/mock_slave.hpp
+++ b/src/tests/mock_slave.hpp
@@ -100,6 +100,7 @@ public:
       mesos::slave::QoSController* qosController,
       SecretGenerator* secretGenerator,
       slave::VolumeGidManager* volumeGidManager,
+      PendingFutureTracker* futureTracker,
       const Option<Authorizer*>& authorizer);
 
   MOCK_METHOD6(___run, void(


[mesos] 03/07: Wrapped isolators in `IsolatorTracker`.

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 c3964a83a773a43321a7dfa6455482002ed96e64
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Thu Jul 18 09:10:41 2019 -0700

    Wrapped isolators in `IsolatorTracker`.
    
    This patch wraps every isolator in instance of `IsolatorTracker` class.
    If an isolator gets stuck in some operation, `pendingFutures` will
    return the isolator name, method name along with its arguments such as
    `containerId`, `pid`, etc.
    
    Review: https://reviews.apache.org/r/70889/
---
 src/slave/containerizer/containerizer.cpp       | 13 ++++++++--
 src/slave/containerizer/containerizer.hpp       |  5 +++-
 src/slave/containerizer/mesos/containerizer.cpp | 34 ++++++++++++++++++-------
 src/slave/containerizer/mesos/containerizer.hpp |  3 ++-
 src/slave/main.cpp                              | 12 ++++++++-
 src/tests/cluster.cpp                           | 10 +++++++-
 6 files changed, 62 insertions(+), 15 deletions(-)

diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
index 5ce0d9c..9e44e5e 100644
--- a/src/slave/containerizer/containerizer.cpp
+++ b/src/slave/containerizer/containerizer.cpp
@@ -219,7 +219,8 @@ Try<Containerizer*> Containerizer::create(
     Fetcher* fetcher,
     GarbageCollector* gc,
     SecretResolver* secretResolver,
-    VolumeGidManager* volumeGidManager)
+    VolumeGidManager* volumeGidManager,
+    PendingFutureTracker* futureTracker)
 {
   // Get the set of containerizer types.
   const vector<string> _types = strings::split(flags.containerizers, ",");
@@ -289,7 +290,15 @@ Try<Containerizer*> Containerizer::create(
   foreach (const string& type, containerizerTypes) {
     if (type == "mesos") {
       Try<MesosContainerizer*> containerizer = MesosContainerizer::create(
-          flags, local, fetcher, gc, secretResolver, nvidia, volumeGidManager);
+          flags,
+          local,
+          fetcher,
+          gc,
+          secretResolver,
+          nvidia,
+          volumeGidManager,
+          futureTracker);
+
       if (containerizer.isError()) {
         return Error("Could not create MesosContainerizer: " +
                      containerizer.error());
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index d33c65c..6a9ebed 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -36,6 +36,8 @@
 #include <stout/option.hpp>
 #include <stout/try.hpp>
 
+#include "common/future_tracker.hpp"
+
 #include "slave/gc.hpp"
 
 #include "slave/volume_gid_manager/volume_gid_manager.hpp"
@@ -75,7 +77,8 @@ public:
       Fetcher* fetcher,
       GarbageCollector* gc,
       SecretResolver* secretResolver = nullptr,
-      VolumeGidManager* volumeGidManager = nullptr);
+      VolumeGidManager* volumeGidManager = nullptr,
+      PendingFutureTracker* futureTracker = nullptr);
 
   // Determine slave resources from flags, probing the system or
   // querying a delegate.
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index c9a369b..b4d10a7 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -70,6 +70,7 @@
 
 #include "slave/containerizer/mesos/constants.hpp"
 #include "slave/containerizer/mesos/containerizer.hpp"
+#include "slave/containerizer/mesos/isolator_tracker.hpp"
 #include "slave/containerizer/mesos/launch.hpp"
 #include "slave/containerizer/mesos/launcher.hpp"
 #include "slave/containerizer/mesos/paths.hpp"
@@ -177,7 +178,8 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     GarbageCollector* gc,
     SecretResolver* secretResolver,
     const Option<NvidiaComponents>& nvidia,
-    VolumeGidManager* volumeGidManager)
+    VolumeGidManager* volumeGidManager,
+    PendingFutureTracker* futureTracker)
 {
   Try<hashset<string>> isolations = [&flags]() -> Try<hashset<string>> {
     const vector<string> tokens(strings::tokenize(flags.isolation, ","));
@@ -537,26 +539,40 @@ Try<MesosContainerizer*> MesosContainerizer::create(
       cgroupsIsolatorCreated = true;
     }
 
-    Try<Isolator*> isolator = creator.second(flags);
-    if (isolator.isError()) {
+    Try<Isolator*> _isolator = creator.second(flags);
+    if (_isolator.isError()) {
       return Error("Failed to create isolator '" + creator.first + "': " +
-                   isolator.error());
+                   _isolator.error());
+    }
+
+    Owned<Isolator> isolator(_isolator.get());
+
+    if (futureTracker != nullptr) {
+      isolator = Owned<Isolator>(
+          new IsolatorTracker(isolator, creator.first, futureTracker));
     }
 
-    isolators.push_back(Owned<Isolator>(isolator.get()));
+    isolators.push_back(isolator);
   }
 
   // Next, apply any custom isolators in the order given by the flags.
   foreach (const string& name, strings::tokenize(flags.isolation, ",")) {
     if (ModuleManager::contains<Isolator>(name)) {
-      Try<Isolator*> isolator = ModuleManager::create<Isolator>(name);
+      Try<Isolator*> _isolator = ModuleManager::create<Isolator>(name);
 
-      if (isolator.isError()) {
+      if (_isolator.isError()) {
         return Error("Failed to create isolator '" + name + "': " +
-                    isolator.error());
+                    _isolator.error());
+      }
+
+      Owned<Isolator> isolator(_isolator.get());
+
+      if (futureTracker != nullptr) {
+        isolator = Owned<Isolator>(
+            new IsolatorTracker(isolator, name, futureTracker));
       }
 
-      isolators.push_back(Owned<Isolator>(isolator.get()));
+      isolators.push_back(isolator);
       continue;
     }
 
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 558e412..271d632 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -72,7 +72,8 @@ public:
       GarbageCollector* gc = nullptr,
       SecretResolver* secretResolver = nullptr,
       const Option<NvidiaComponents>& nvidia = None(),
-      VolumeGidManager* volumeGidManager = nullptr);
+      VolumeGidManager* volumeGidManager = nullptr,
+      PendingFutureTracker* futureTracker = nullptr);
 
   static Try<MesosContainerizer*> create(
       const Flags& flags,
diff --git a/src/slave/main.cpp b/src/slave/main.cpp
index ef5ea02..c974ba0 100644
--- a/src/slave/main.cpp
+++ b/src/slave/main.cpp
@@ -491,13 +491,21 @@ int main(int argc, char** argv)
   }
 #endif // __WINDOWS__
 
+  // Initialize PendingFutureTracker.
+  Try<PendingFutureTracker*> futureTracker = PendingFutureTracker::create();
+  if (futureTracker.isError()) {
+    EXIT(EXIT_FAILURE) << "Failed to initialize pending future tracker: "
+                       << futureTracker.error();
+  }
+
   Try<Containerizer*> containerizer = Containerizer::create(
       flags,
       false,
       fetcher,
       gc,
       secretResolver.get(),
-      volumeGidManager);
+      volumeGidManager,
+      futureTracker.get());
 
   if (containerizer.isError()) {
     EXIT(EXIT_FAILURE)
@@ -636,6 +644,8 @@ int main(int argc, char** argv)
 
   delete containerizer.get();
 
+  delete futureTracker.get();
+
 #ifndef __WINDOWS__
   delete volumeGidManager;
 #endif // __WINDOWS__
diff --git a/src/tests/cluster.cpp b/src/tests/cluster.cpp
index b43f806..9f180cc 100644
--- a/src/tests/cluster.cpp
+++ b/src/tests/cluster.cpp
@@ -71,6 +71,7 @@
 #include "authorizer/local/authorizer.hpp"
 
 #include "common/authorization.hpp"
+#include "common/future_tracker.hpp"
 #include "common/http.hpp"
 
 #include "files/files.hpp"
@@ -446,6 +447,12 @@ Try<process::Owned<Slave>> Slave::create(
   }
 #endif // __WINDOWS__
 
+  Try<PendingFutureTracker*> futureTracker = PendingFutureTracker::create();
+  if (futureTracker.isError()) {
+    return Error(
+        "Failed to create pending future tracker: " + futureTracker.error());
+  }
+
   // If the containerizer is not provided, create a default one.
   if (containerizer.isSome()) {
     slave->containerizer = containerizer.get();
@@ -460,7 +467,8 @@ Try<process::Owned<Slave>> Slave::create(
           slave->fetcher.get(),
           gc.getOrElse(slave->gc.get()),
           nullptr,
-          volumeGidManager);
+          volumeGidManager,
+          futureTracker.get());
 
     if (_containerizer.isError()) {
       return Error("Failed to create containerizer: " + _containerizer.error());