You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bm...@apache.org on 2018/08/15 22:49:14 UTC

[mesos] 07/07: Added a test for master's handling of stale authentication requests.

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

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

commit d557637c8a854bd40fff4c95a88c8c6c8120e90e
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Fri Aug 10 17:16:07 2018 -0700

    Added a test for master's handling of stale authentication requests.
    
    This test ensures that when the master sees a new authentication
    request for a particular agent or scheduler (we just test the
    scheduler case is tested here since the master does not distinguish),
    the master will discard the old one and proceed with the new one.
    
    Review: https://reviews.apache.org/r/68307
---
 src/internal/devolve.hpp           |  4 +--
 src/internal/evolve.hpp            |  6 ++--
 src/master/master.hpp              | 52 ++++++++++++++---------------
 src/master/metrics.hpp             | 10 +++---
 src/tests/authentication_tests.cpp | 68 ++++++++++++++++++++++++++++++++++++++
 5 files changed, 104 insertions(+), 36 deletions(-)

diff --git a/src/internal/devolve.hpp b/src/internal/devolve.hpp
index 8007f42..1bc2a32 100644
--- a/src/internal/devolve.hpp
+++ b/src/internal/devolve.hpp
@@ -75,8 +75,8 @@ mesos::resource_provider::Call devolve(const v1::resource_provider::Call& call);
 mesos::resource_provider::Event devolve(
     const v1::resource_provider::Event& event);
 
-scheduler::Call devolve(const v1::scheduler::Call& call);
-scheduler::Event devolve(const v1::scheduler::Event& event);
+mesos::scheduler::Call devolve(const v1::scheduler::Call& call);
+mesos::scheduler::Event devolve(const v1::scheduler::Event& event);
 
 executor::Call devolve(const v1::executor::Call& call);
 executor::Event devolve(const v1::executor::Event& event);
diff --git a/src/internal/evolve.hpp b/src/internal/evolve.hpp
index e792ff5..ca462ea 100644
--- a/src/internal/evolve.hpp
+++ b/src/internal/evolve.hpp
@@ -114,9 +114,9 @@ google::protobuf::RepeatedPtrField<T1> evolve(
 }
 
 
-v1::scheduler::Call evolve(const scheduler::Call& call);
-v1::scheduler::Event evolve(const scheduler::Event& event);
-v1::scheduler::Response evolve(const scheduler::Response& response);
+v1::scheduler::Call evolve(const mesos::scheduler::Call& call);
+v1::scheduler::Event evolve(const mesos::scheduler::Event& event);
+v1::scheduler::Response evolve(const mesos::scheduler::Response& response);
 
 
 // Helper functions that evolve old style internal messages to a
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 85ef14c..36994a1 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -1044,7 +1044,7 @@ private:
 
   void drop(
       const process::UPID& from,
-      const scheduler::Call& call,
+      const mesos::scheduler::Call& call,
       const std::string& message);
 
   void drop(
@@ -1054,27 +1054,27 @@ private:
 
   void drop(
       Framework* framework,
-      const scheduler::Call& call,
+      const mesos::scheduler::Call& call,
       const std::string& message);
 
   void drop(
       Framework* framework,
-      const scheduler::Call::Suppress& suppress,
+      const mesos::scheduler::Call::Suppress& suppress,
       const std::string& message);
 
   void drop(
       Framework* framework,
-      const scheduler::Call::Revive& revive,
+      const mesos::scheduler::Call::Revive& revive,
       const std::string& message);
 
   // Call handlers.
   void receive(
       const process::UPID& from,
-      scheduler::Call&& call);
+      mesos::scheduler::Call&& call);
 
   void subscribe(
       HttpConnection http,
-      const scheduler::Call::Subscribe& subscribe);
+      const mesos::scheduler::Call::Subscribe& subscribe);
 
   void _subscribe(
       HttpConnection http,
@@ -1085,7 +1085,7 @@ private:
 
   void subscribe(
       const process::UPID& from,
-      const scheduler::Call::Subscribe& subscribe);
+      const mesos::scheduler::Call::Subscribe& subscribe);
 
   void _subscribe(
       const process::UPID& from,
@@ -1103,67 +1103,67 @@ private:
 
   void accept(
       Framework* framework,
-      scheduler::Call::Accept&& accept);
+      mesos::scheduler::Call::Accept&& accept);
 
   void _accept(
       const FrameworkID& frameworkId,
       const SlaveID& slaveId,
       const Resources& offeredResources,
-      scheduler::Call::Accept&& accept,
+      mesos::scheduler::Call::Accept&& accept,
       const process::Future<
           std::vector<process::Future<bool>>>& authorizations);
 
   void acceptInverseOffers(
       Framework* framework,
-      const scheduler::Call::AcceptInverseOffers& accept);
+      const mesos::scheduler::Call::AcceptInverseOffers& accept);
 
   void decline(
       Framework* framework,
-      scheduler::Call::Decline&& decline);
+      mesos::scheduler::Call::Decline&& decline);
 
   void declineInverseOffers(
       Framework* framework,
-      const scheduler::Call::DeclineInverseOffers& decline);
+      const mesos::scheduler::Call::DeclineInverseOffers& decline);
 
   void revive(
       Framework* framework,
-      const scheduler::Call::Revive& revive);
+      const mesos::scheduler::Call::Revive& revive);
 
   void kill(
       Framework* framework,
-      const scheduler::Call::Kill& kill);
+      const mesos::scheduler::Call::Kill& kill);
 
   void shutdown(
       Framework* framework,
-      const scheduler::Call::Shutdown& shutdown);
+      const mesos::scheduler::Call::Shutdown& shutdown);
 
   void acknowledge(
       Framework* framework,
-      scheduler::Call::Acknowledge&& acknowledge);
+      mesos::scheduler::Call::Acknowledge&& acknowledge);
 
   void acknowledgeOperationStatus(
       Framework* framework,
-      scheduler::Call::AcknowledgeOperationStatus&& acknowledge);
+      mesos::scheduler::Call::AcknowledgeOperationStatus&& acknowledge);
 
   void reconcile(
       Framework* framework,
-      scheduler::Call::Reconcile&& reconcile);
+      mesos::scheduler::Call::Reconcile&& reconcile);
 
-  scheduler::Response::ReconcileOperations reconcileOperations(
+  mesos::scheduler::Response::ReconcileOperations reconcileOperations(
       Framework* framework,
-      const scheduler::Call::ReconcileOperations& reconcile);
+      const mesos::scheduler::Call::ReconcileOperations& reconcile);
 
   void message(
       Framework* framework,
-      scheduler::Call::Message&& message);
+      mesos::scheduler::Call::Message&& message);
 
   void request(
       Framework* framework,
-      const scheduler::Call::Request& request);
+      const mesos::scheduler::Call::Request& request);
 
   void suppress(
       Framework* framework,
-      const scheduler::Call::Suppress& suppress);
+      const mesos::scheduler::Call::Suppress& suppress);
 
   bool elected() const
   {
@@ -1832,7 +1832,7 @@ private:
 
     process::Future<process::http::Response> reconcileOperations(
         Framework* framework,
-        const scheduler::Call::ReconcileOperations& call,
+        const mesos::scheduler::Call::ReconcileOperations& call,
         ContentType contentType) const;
 
     Master* master;
@@ -2500,8 +2500,8 @@ struct Framework
   hashmap<SlaveID, Resources> offeredResources;
 
   // This is only set for HTTP frameworks.
-  Option<process::Owned<Heartbeater<scheduler::Event, v1::scheduler::Event>>>
-    heartbeater;
+  Option<process::Owned<
+      Heartbeater<mesos::scheduler::Event, v1::scheduler::Event>>> heartbeater;
 
   // This is used for per-framwork metrics.
   FrameworkMetrics metrics;
diff --git a/src/master/metrics.hpp b/src/master/metrics.hpp
index df28a48..14f894a 100644
--- a/src/master/metrics.hpp
+++ b/src/master/metrics.hpp
@@ -209,7 +209,7 @@ struct Metrics
   std::vector<process::metrics::PullGauge> resources_revocable_used;
   std::vector<process::metrics::PullGauge> resources_revocable_percent;
 
-  void incrementInvalidSchedulerCalls(const scheduler::Call& call);
+  void incrementInvalidSchedulerCalls(const mesos::scheduler::Call& call);
 
   void incrementTasksStates(
       const TaskState& state,
@@ -224,9 +224,9 @@ struct FrameworkMetrics
 
   ~FrameworkMetrics();
 
-  void incrementCall(const scheduler::Call::Type& callType);
+  void incrementCall(const mesos::scheduler::Call::Type& callType);
 
-  void incrementEvent(const scheduler::Event& event);
+  void incrementEvent(const mesos::scheduler::Event& event);
 
   void incrementTaskState(const TaskState& state);
   void decrementActiveTaskState(const TaskState& state);
@@ -238,10 +238,10 @@ struct FrameworkMetrics
   process::metrics::PushGauge subscribed;
 
   process::metrics::Counter calls;
-  hashmap<scheduler::Call::Type, process::metrics::Counter> call_types;
+  hashmap<mesos::scheduler::Call::Type, process::metrics::Counter> call_types;
 
   process::metrics::Counter events;
-  hashmap<scheduler::Event::Type, process::metrics::Counter> event_types;
+  hashmap<mesos::scheduler::Event::Type, process::metrics::Counter> event_types;
 
   process::metrics::Counter offers_sent;
   process::metrics::Counter offers_accepted;
diff --git a/src/tests/authentication_tests.cpp b/src/tests/authentication_tests.cpp
index c9a8f85..0e8a758 100644
--- a/src/tests/authentication_tests.cpp
+++ b/src/tests/authentication_tests.cpp
@@ -30,6 +30,8 @@
 
 #include "master/detector/standalone.hpp"
 
+#include "sched/constants.hpp"
+
 #include "tests/mesos.hpp"
 #include "tests/utils.hpp"
 
@@ -393,6 +395,72 @@ TEST_F(AuthenticationTest, RetrySlaveAuthentication)
 }
 
 
+// This test ensures that when the master sees a new authentication
+// request for a particular agent or scheduler (we just test the
+// scheduler case here since the master does not distinguish),
+// the master will discard the old one and proceed with the new one.
+//
+// TODO(bmahler): Use a mock authenticator for this test instead
+// of using the default one and dropping the exited message.
+TEST_F(AuthenticationTest, MasterRetriedAuthenticationHandling)
+{
+  Clock::pause();
+
+  // Set the master authentication timeout to a very large value
+  // so that we can exercise the case of a new authentication
+  // request arriving before the previous one times out.
+  master::Flags flags = CreateMasterFlags();
+  flags.authentication_v0_timeout = Days(30);
+
+  Try<Owned<cluster::Master>> master = StartMaster(flags);
+  ASSERT_SOME(master);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  // Drop the first step message from the authenticator.
+  Future<Message> authenticationStepMessage =
+    DROP_MESSAGE(Eq(AuthenticationStepMessage().GetTypeName()), _, _);
+
+  driver.start();
+
+  AWAIT_READY(authenticationStepMessage);
+
+  Future<Nothing> registered;
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .WillOnce(FutureSatisfy(&registered));
+
+  // Now the master will have a pending authentication and we have
+  // the scheduler driver retry. Since the master's authentication
+  // timeout is larger than the scheduler's, the scheduler will
+  // retry and the master should discard the stale one and let
+  // the new one proceed. First, we drop the exited event that
+  // the master's authenticator will listen for to ensure the
+  // authentication remains outstanding from the master's
+  // perspective.
+  const UPID& authenticatee = authenticationStepMessage->to;
+  const UPID& authenticator = authenticationStepMessage->from;
+
+  Future<Nothing> exited = DROP_EXITED(authenticatee, authenticator);
+
+  Clock::advance(
+      mesos::internal::scheduler::DEFAULT_AUTHENTICATION_TIMEOUT);
+  Clock::settle();
+  Clock::advance(
+      mesos::internal::scheduler::DEFAULT_AUTHENTICATION_BACKOFF_FACTOR * 2);
+
+  // Make sure the exited was dropped.
+  AWAIT_READY(exited);
+
+  // Scheduler should be able to get registered.
+  AWAIT_READY(registered);
+
+  driver.stop();
+  driver.join();
+}
+
+
 // This test verifies that the framework properly retries
 // authentication when an intermediate message in SASL protocol
 // is lost.