You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ch...@apache.org on 2019/07/18 00:07:51 UTC

[mesos] branch 1.8.x updated (4ae0644 -> edaf639)

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

chhsiao pushed a change to branch 1.8.x
in repository https://gitbox.apache.org/repos/asf/mesos.git.


    from 4ae0644  Updated CHANGELOG for 1.8.1 release.
     new dece081  Sequentialized all events to master's `/api/v1` subscribers.
     new c889ba6  Notifies master `/api/v1` subscribers about recovered frameworks.
     new edaf639  Added MESOS-9785 to the 1.8.2 CHANGELOG.

The 3 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                     |  9 +++++++++
 src/common/protobuf_utils.cpp |  4 ----
 src/master/master.cpp         | 22 ++++++++++++++++++++--
 src/master/master.hpp         | 11 +++++++++++
 src/tests/api_tests.cpp       | 35 +++++++++++++++++++++++++----------
 5 files changed, 65 insertions(+), 16 deletions(-)


[mesos] 02/03: Notifies master `/api/v1` subscribers about recovered frameworks.

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

chhsiao pushed a commit to branch 1.8.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit c889ba6b7818e8eea8b2a11c442bac81b6a237a2
Author: Chun-Hung Hsiao <ch...@mesosphere.io>
AuthorDate: Wed May 15 21:54:20 2019 -0700

    Notifies master `/api/v1` subscribers about recovered frameworks.
    
    If one subscribes to master's `/api/v1` endpoint after a master failover
    but before an agent reregistration, frameworks recovered through the
    agent registration should be notified to the subscriber, otherwise
    recovered tasks will have framework IDs referring to frameworks unknown
    to the subscriber.
    
    Review: https://reviews.apache.org/r/70651
---
 src/common/protobuf_utils.cpp |  4 ----
 src/master/master.cpp         |  7 +++++++
 src/tests/api_tests.cpp       | 35 +++++++++++++++++++++++++----------
 3 files changed, 32 insertions(+), 14 deletions(-)

diff --git a/src/common/protobuf_utils.cpp b/src/common/protobuf_utils.cpp
index 8b252cb..6a93ac7 100644
--- a/src/common/protobuf_utils.cpp
+++ b/src/common/protobuf_utils.cpp
@@ -1337,10 +1337,6 @@ mesos::master::Event createTaskAdded(const Task& task)
 mesos::master::Event createFrameworkAdded(
     const mesos::internal::master::Framework& _framework)
 {
-  CHECK(_framework.active());
-  CHECK(_framework.connected());
-  CHECK(!_framework.recovered());
-
   mesos::master::Event event;
   event.set_type(mesos::master::Event::FRAMEWORK_ADDED);
 
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 06a89bc..5488b7b 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -10521,6 +10521,13 @@ void Master::recoverFramework(
 
   Framework* framework = new Framework(this, flags, info);
 
+  // Send a `FRAMEWORK_ADDED` event to subscribers before adding recovered tasks
+  // so the framework ID referred by any succeeding `TASK_ADDED` event will be
+  // known to subscribers.
+  if (!subscribers.subscribed.empty()) {
+    subscribers.send(protobuf::master::event::createFrameworkAdded(*framework));
+  }
+
   // Add active operations, tasks, and executors to the framework.
   foreachvalue (Slave* slave, slaves.registered) {
     if (slave->tasks.contains(framework->id())) {
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 4850ba6..539c704 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -2711,8 +2711,9 @@ TEST_P(MasterAPITest, SubscribersReceiveHealthUpdates)
 
 
 // This test verifies that subscribing to the 'api/v1' endpoint between
-// a master failover and an agent re-registration won't cause the master
-// to crash. See MESOS-8601.
+// a master failover and an agent reregistration won't cause the master
+// to crash, and frameworks recovered through agent reregistration will be
+// broadcast to subscribers. See MESOS-8601 and MESOS-9785.
 TEST_P(MasterAPITest, MasterFailover)
 {
   ContentType contentType = GetParam();
@@ -2863,21 +2864,35 @@ TEST_P(MasterAPITest, MasterFailover)
 
   AWAIT_READY(slaveReregisteredMessage);
 
-  // The agent re-registration should result in an `AGENT_ADDED` event
-  // and a `TASK_ADDED` event.
-  set<v1::master::Event::Type> expectedEvents =
-    {v1::master::Event::AGENT_ADDED, v1::master::Event::TASK_ADDED};
-  set<v1::master::Event::Type> observedEvents;
+  // The agent re-registration should result in an `AGENT_ADDED` event,
+  // a `FRAMEWORK_ADDED` event and a `TASK_ADDED` event in order.
+  event = decoder.read();
+  AWAIT_READY(event);
+
+  EXPECT_EQ(v1::master::Event::AGENT_ADDED, event.get()->type());
+  const v1::master::Event::AgentAdded& agentAdded = event.get()->agent_added();
+
+  EXPECT_EQ(agentId, agentAdded.agent().agent_info().id());
 
   event = decoder.read();
   AWAIT_READY(event);
-  observedEvents.insert(event->get().type());
+
+  EXPECT_EQ(v1::master::Event::FRAMEWORK_ADDED, event.get()->type());
+  const v1::master::Event::FrameworkAdded& frameworkAdded =
+    event.get()->framework_added();
+
+  EXPECT_EQ(frameworkId, frameworkAdded.framework().framework_info().id());
+  EXPECT_FALSE(frameworkAdded.framework().active());
+  EXPECT_FALSE(frameworkAdded.framework().connected());
+  EXPECT_TRUE(frameworkAdded.framework().recovered());
 
   event = decoder.read();
   AWAIT_READY(event);
-  observedEvents.insert(event->get().type());
 
-  EXPECT_EQ(expectedEvents, observedEvents);
+  EXPECT_EQ(v1::master::Event::TASK_ADDED, event.get()->type());
+  const v1::master::Event::TaskAdded& taskAdded = event.get()->task_added();
+
+  EXPECT_EQ(task.task_id(), taskAdded.task().task_id());
 }
 
 


[mesos] 03/03: Added MESOS-9785 to the 1.8.2 CHANGELOG.

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

chhsiao pushed a commit to branch 1.8.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit edaf639d5e5d9af368bb986b331186a4cae4edc5
Author: Chun-Hung Hsiao <ch...@mesosphere.io>
AuthorDate: Wed May 22 14:44:28 2019 -0700

    Added MESOS-9785 to the 1.8.2 CHANGELOG.
---
 CHANGELOG | 9 +++++++++
 1 file changed, 9 insertions(+)

diff --git a/CHANGELOG b/CHANGELOG
index d03be88..248e382 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,3 +1,11 @@
+Release Notes - Mesos - Version 1.8.2 (WIP)
+-------------------------------------------
+* This is a bug fix release.
+
+** Bug
+  * [MESOS-9785] - Frameworks recovered from reregistered agents are not reported to master `/api/v1` subscribers.
+
+
 Release Notes - Mesos - Version 1.8.1
 -------------------------------------
 * This is a bug fix release.
@@ -22,6 +30,7 @@ Release Notes - Mesos - Version 1.8.1
   * [MESOS-9759] - Log required quota headroom and available quota headroom in the allocator.
   * [MESOS-9787] - Log slow SSL (TLS) peer reverse DNS lookup.
 
+
 Release Notes - Mesos - Version 1.8.0
 -------------------------------------
 This release contains the following highlights:


[mesos] 01/03: Sequentialized all events to master's `/api/v1` subscribers.

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

chhsiao pushed a commit to branch 1.8.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit dece081d33c8a6d4e1b56589ec5d4e16013d6643
Author: Chun-Hung Hsiao <ch...@mesosphere.io>
AuthorDate: Wed May 22 21:08:09 2019 -0700

    Sequentialized all events to master's `/api/v1` subscribers.
    
    The master needs to create object approvers before sending an event to
    its `/api/v1` subscribers. The creation calls `process::collect`, which
    does not have any ordering guarantee. As a result, events might be
    reordered, which could be unexpected by subscribers.
    
    This patch imposes an order between events by sequentializing the
    creation of object approvers. The actual creations can still go in
    parallel, but the returned futures will be completed in the creation
    order.
    
    Review: https://reviews.apache.org/r/70702
---
 src/master/master.cpp | 15 +++++++++++++--
 src/master/master.hpp | 11 +++++++++++
 2 files changed, 24 insertions(+), 2 deletions(-)

diff --git a/src/master/master.cpp b/src/master/master.cpp
index f1ab034..06a89bc 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -12792,9 +12792,8 @@ void Master::Subscribers::send(
   Shared<Task> sharedTask(task.isSome() ? new Task(task.get()) : nullptr);
 
   foreachvalue (const Owned<Subscriber>& subscriber, subscribed) {
-    ObjectApprovers::create(
+    subscriber->getApprovers(
         master->authorizer,
-        subscriber->principal,
         {VIEW_ROLE, VIEW_FRAMEWORK, VIEW_TASK, VIEW_EXECUTOR})
       .then(defer(
           master->self(),
@@ -12811,6 +12810,18 @@ void Master::Subscribers::send(
 }
 
 
+Future<Owned<ObjectApprovers>> Master::Subscribers::Subscriber::getApprovers(
+    const Option<Authorizer*>& authorizer,
+    std::initializer_list<authorization::Action> actions)
+{
+  Future<Owned<ObjectApprovers>> approvers =
+    ObjectApprovers::create(authorizer, principal, actions);
+
+  return approversSequence.add<Owned<ObjectApprovers>>(
+      [approvers] { return approvers; });
+}
+
+
 void Master::Subscribers::Subscriber::send(
     const Shared<mesos::master::Event>& event,
     const Owned<ObjectApprovers>& approvers,
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 94891af..ed83167 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -49,6 +49,7 @@
 #include <process/owned.hpp>
 #include <process/process.hpp>
 #include <process/protobuf.hpp>
+#include <process/sequence.hpp>
 #include <process/timer.hpp>
 
 #include <process/metrics/counter.hpp>
@@ -2140,6 +2141,12 @@ private:
       Subscriber(const Subscriber&) = delete;
       Subscriber& operator=(const Subscriber&) = delete;
 
+      // Creates object approvers. The futures returned by this method will be
+      // completed in the calling order.
+      process::Future<process::Owned<ObjectApprovers>> getApprovers(
+          const Option<Authorizer*>& authorizer,
+          std::initializer_list<authorization::Action> actions);
+
       // TODO(greggomann): Refactor this function into multiple event-specific
       // overloads. See MESOS-8475.
       void send(
@@ -2160,6 +2167,10 @@ private:
       StreamingHttpConnection<v1::master::Event> http;
       ResponseHeartbeater<mesos::master::Event, v1::master::Event> heartbeater;
       const Option<process::http::authentication::Principal> principal;
+
+      // We maintain a sequence to coordinate the creation of object approvers
+      // in order to sequentialize all events to the subscriber.
+      process::Sequence approversSequence;
     };
 
     // Sends the event to all subscribers connected to the 'api/vX' endpoint.