You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ti...@apache.org on 2018/11/20 15:47:06 UTC

[mesos] branch master updated (60a8577 -> 45bd70f)

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

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


    from 60a8577  Added --min_allocatable_resources to the multi-scheduler documentation.
     new 3ed4cd5  Added test reproducing crash on authorization failure.
     new 3501ac5  Added collectAuthorizations helper to master.hpp.
     new 176f5c2  Introduced common/authorization and refactored collectAuthorizations.
     new 00e0707  Refactored createSubject and authorizeLogAccess to common/authorization.
     new 45bd70f  Added test for ACCESS_MESOS_LOG authorization.

The 5 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:
 src/CMakeLists.txt                |   1 +
 src/Makefile.am                   |   2 +
 src/common/authorization.cpp      |  90 ++++++++++++++++++++++++++++
 src/common/authorization.hpp      |  53 ++++++++++++++++
 src/common/http.cpp               |  27 +--------
 src/common/http.hpp               |  10 ----
 src/master/http.cpp               |   1 +
 src/master/master.cpp             |  80 ++++---------------------
 src/master/master.hpp             |   4 +-
 src/master/quota_handler.cpp      |   2 +
 src/master/weights_handler.cpp    |  14 +----
 src/slave/http.cpp                |   1 +
 src/slave/slave.cpp               |  29 ++-------
 src/slave/slave.hpp               |   3 -
 src/tests/authorization_tests.cpp |  54 +++++++++++++++++
 src/tests/master_tests.cpp        | 123 ++++++++++++++++++++++++++++++++++++++
 16 files changed, 347 insertions(+), 147 deletions(-)
 create mode 100644 src/common/authorization.cpp
 create mode 100644 src/common/authorization.hpp


[mesos] 01/05: Added test reproducing crash on authorization failure.

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

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

commit 3ed4cd51f0eea1f86b0a8a25c95f6005565e3010
Author: Till Toenshoff <to...@me.com>
AuthorDate: Tue Nov 20 14:45:38 2018 +0100

    Added test reproducing crash on authorization failure.
    
    This test reproduces the scenario as described in MESOS-9317. The test
    attempts to create a persistent volume by a web request to the
    authorized V1 operator endpoint. The test assures that the underlying
    authorization request fails as it can in production due to failures in
    the authorization backend.
    
    Without fixing MESOS-9317, this test crashes the master process as the
    code-path involved will attempt to access the contents of the awaited
    future even though the future had failed.
    
    Review: https://reviews.apache.org/r/69368/
---
 src/tests/master_tests.cpp | 65 ++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 65 insertions(+)

diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index ac6bf37..8ed1e89 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -106,6 +106,7 @@ using process::PID;
 using process::Promise;
 
 using process::http::Accepted;
+using process::http::InternalServerError;
 using process::http::OK;
 using process::http::Response;
 using process::http::Unauthorized;
@@ -10087,6 +10088,70 @@ TEST_P(MasterTestPrePostReservationRefinement, CreateAndDestroyVolumesV1)
   AWAIT_EXPECT_RESPONSE_STATUS_EQ(Accepted().status, v1DestroyVolumesResponse);
 }
 
+
+// This test validates that an authorization error when requesting
+// volume creation does result in an internal server error.
+// See MESOS-9317.
+TEST_F(MasterTest, CreateVolumesV1AuthorizationFailure)
+{
+  MockAuthorizer authorizer;
+  Try<Owned<cluster::Master>> master = StartMaster(&authorizer);
+  ASSERT_SOME(master);
+
+  // For capturing the `SlaveID` so we can use it in the create/destroy
+  // volumes API call.
+  Future<SlaveRegisteredMessage> slaveRegisteredMessage =
+    FUTURE_PROTOBUF(SlaveRegisteredMessage(), _, _);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  // Do static reservation so we can create persistent volumes from it.
+  slaveFlags.resources = "disk(role1):1024";
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), slaveFlags);
+
+  ASSERT_SOME(slave);
+
+  AWAIT_READY(slaveRegisteredMessage);
+  SlaveID slaveId = slaveRegisteredMessage->slave_id();
+
+  // Create the persistent volume.
+  v1::master::Call v1CreateVolumesCall;
+  v1CreateVolumesCall.set_type(v1::master::Call::CREATE_VOLUMES);
+  v1::master::Call_CreateVolumes* createVolumes =
+    v1CreateVolumesCall.mutable_create_volumes();
+
+  Resources volume = createPersistentVolume(
+      Megabytes(64),
+      "role1",
+      "id1",
+      "path1",
+      None(),
+      None(),
+      DEFAULT_CREDENTIAL.principal());
+
+  createVolumes->mutable_agent_id()->CopyFrom(evolve(slaveId));
+  createVolumes->mutable_volumes()->CopyFrom(v1::Resources(evolve(volume)));
+
+  ContentType contentType = ContentType::PROTOBUF;
+
+  Promise<bool> promise;
+  EXPECT_CALL(authorizer, authorized(_))
+    .WillOnce(Return(promise.future()));
+
+  Future<Response> response = process::http::post(
+      master.get()->pid,
+      "api/v1",
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+      serialize(contentType, v1CreateVolumesCall),
+      stringify(contentType));
+
+  promise.fail("Authorizer failure");
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(InternalServerError().status, response);
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[mesos] 05/05: Added test for ACCESS_MESOS_LOG authorization.

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

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

commit 45bd70f02d9d3c9c5c9067169f020dafc4437f8b
Author: Till Toenshoff <to...@me.com>
AuthorDate: Tue Nov 20 14:46:25 2018 +0100

    Added test for ACCESS_MESOS_LOG authorization.
    
    Review: https://reviews.apache.org/r/69386/
---
 src/tests/authorization_tests.cpp | 54 +++++++++++++++++++++++++++++++++++++++
 1 file changed, 54 insertions(+)

diff --git a/src/tests/authorization_tests.cpp b/src/tests/authorization_tests.cpp
index ac52181..e85cdb6 100644
--- a/src/tests/authorization_tests.cpp
+++ b/src/tests/authorization_tests.cpp
@@ -6997,6 +6997,60 @@ TYPED_TEST(AuthorizationTest, DestroyMountDisk)
   }
 }
 
+
+// This tests the authorization to access Mesos logs.
+TYPED_TEST(AuthorizationTest, LogAccess)
+{
+  // Setup ACLs.
+  ACLs acls;
+
+  {
+    // "foo" principal can access the logs.
+    mesos::ACL::AccessMesosLog* acl = acls.add_access_mesos_logs();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_logs()->set_type(mesos::ACL::Entity::ANY);
+  }
+
+  {
+    // Nobody else can access the logs.
+    mesos::ACL::AccessMesosLog* acl = acls.add_access_mesos_logs();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    acl->mutable_logs()->set_type(mesos::ACL::Entity::NONE);
+  }
+
+  // Create an `Authorizer` with the ACLs.
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
+  ASSERT_SOME(create);
+  Owned<Authorizer> authorizer(create.get());
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::ACCESS_MESOS_LOG);
+    request.mutable_subject()->set_value("foo");
+
+    AWAIT_EXPECT_TRUE(authorizer->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::ACCESS_MESOS_LOG);
+    request.mutable_subject()->set_value("bar");
+    AWAIT_EXPECT_FALSE(authorizer->authorized(request));
+  }
+
+  // Test that no authorizer is created with invalid flags.
+  {
+    ACLs invalid;
+
+    mesos::ACL::AccessMesosLog* acl = invalid.add_access_mesos_logs();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_logs()->add_values("yoda");
+
+    Try<Authorizer*> create = TypeParam::create(parameterize(invalid));
+    EXPECT_ERROR(create);
+  }
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[mesos] 04/05: Refactored createSubject and authorizeLogAccess to common/authorization.

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

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

commit 00e0707e32735886f61ab4f9c03d4b06a295d1b5
Author: Till Toenshoff <to...@me.com>
AuthorDate: Tue Nov 20 14:46:11 2018 +0100

    Refactored createSubject and authorizeLogAccess to common/authorization.
    
    Moves 'createSubject' out of common/http into common/authorization.
    
    Removes duplicate 'authorizeLogAccess' out of master.cpp and slave.cpp.
    Introduces 'authorizeLogAccess' within common/authorization.
    
    Review: https://reviews.apache.org/r/69385/
---
 src/common/authorization.cpp | 49 ++++++++++++++++++++++++++++++++++++++++++++
 src/common/authorization.hpp | 18 ++++++++++++++++
 src/common/http.cpp          | 27 +-----------------------
 src/common/http.hpp          | 10 ---------
 src/master/http.cpp          |  1 +
 src/master/master.cpp        | 25 ++++------------------
 src/master/master.hpp        |  3 ---
 src/master/quota_handler.cpp |  2 ++
 src/slave/http.cpp           |  1 +
 src/slave/slave.cpp          | 29 +++++---------------------
 src/slave/slave.hpp          |  3 ---
 11 files changed, 81 insertions(+), 87 deletions(-)

diff --git a/src/common/authorization.cpp b/src/common/authorization.cpp
index 5064ad2..fa0c0e8 100644
--- a/src/common/authorization.cpp
+++ b/src/common/authorization.cpp
@@ -17,15 +17,22 @@
 #include "common/authorization.hpp"
 
 #include <algorithm>
+#include <string>
+
+#include <mesos/mesos.hpp>
 
 #include <process/collect.hpp>
 
 #include <stout/foreach.hpp>
+#include <stout/none.hpp>
 
+using std::string;
 using std::vector;
 
 using process::Future;
 
+using process::http::authentication::Principal;
+
 namespace mesos {
 namespace authorization {
 
@@ -37,5 +44,47 @@ Future<bool> collectAuthorizations(const vector<Future<bool>>& authorizations)
     });
 }
 
+
+const Option<Subject> createSubject(const Option<Principal>& principal)
+{
+  if (principal.isSome()) {
+    Subject subject;
+
+    if (principal->value.isSome()) {
+      subject.set_value(principal->value.get());
+    }
+
+    foreachpair (const string& key, const string& value, principal->claims) {
+      Label* claim = subject.mutable_claims()->mutable_labels()->Add();
+      claim->set_key(key);
+      claim->set_value(value);
+    }
+
+    return subject;
+  }
+
+  return None();
+}
+
+
+Future<bool> authorizeLogAccess(
+    const Option<Authorizer*>& authorizer,
+    const Option<Principal>& principal)
+{
+  if (authorizer.isNone()) {
+    return true;
+  }
+
+  Request request;
+  request.set_action(ACCESS_MESOS_LOG);
+
+  Option<Subject> subject = createSubject(principal);
+  if (subject.isSome()) {
+    request.mutable_subject()->CopyFrom(subject.get());
+  }
+
+  return authorizer.get()->authorized(request);
+}
+
 } // namespace authorization {
 } // namespace mesos {
diff --git a/src/common/authorization.hpp b/src/common/authorization.hpp
index 439996a..fb73f30 100644
--- a/src/common/authorization.hpp
+++ b/src/common/authorization.hpp
@@ -19,7 +19,14 @@
 
 #include <vector>
 
+#include <mesos/authentication/authenticator.hpp>
+
+#include <mesos/authorizer/authorizer.hpp>
+
 #include <process/future.hpp>
+#include <process/http.hpp>
+
+#include <stout/option.hpp>
 
 namespace mesos {
 namespace authorization {
@@ -29,6 +36,17 @@ namespace authorization {
 process::Future<bool> collectAuthorizations(
     const std::vector<process::Future<bool>>& authorizations);
 
+// Creates a `Subject` for authorization purposes when given an
+// authenticated `Principal`. This function accepts and returns an
+// `Option` to make call sites cleaner, since it is possible that
+// `principal` will be `NONE`.
+const Option<Subject> createSubject(
+    const Option<process::http::authentication::Principal>& principal);
+
+process::Future<bool> authorizeLogAccess(
+    const Option<Authorizer*>& authorizer,
+    const Option<process::http::authentication::Principal>& principal);
+
 } // namespace authorization {
 } // namespace mesos {
 
diff --git a/src/common/http.cpp b/src/common/http.cpp
index eeeed97..2be94b2 100644
--- a/src/common/http.cpp
+++ b/src/common/http.cpp
@@ -48,6 +48,7 @@
 
 #include <stout/os/permissions.hpp>
 
+#include "common/authorization.hpp"
 #include "common/http.hpp"
 
 #include "messages/messages.hpp"
@@ -876,32 +877,6 @@ static void json(JSON::StringWriter* writer, const Value::Text& text)
   writer->set(text.value());
 }
 
-namespace authorization {
-
-const Option<authorization::Subject> createSubject(
-    const Option<Principal>& principal)
-{
-  if (principal.isSome()) {
-    authorization::Subject subject;
-
-    if (principal->value.isSome()) {
-      subject.set_value(principal->value.get());
-    }
-
-    foreachpair (const string& key, const string& value, principal->claims) {
-      Label* claim = subject.mutable_claims()->mutable_labels()->Add();
-      claim->set_key(key);
-      claim->set_value(value);
-    }
-
-    return subject;
-  }
-
-  return None();
-}
-
-} // namespace authorization {
-
 const AuthorizationCallbacks createAuthorizationCallbacks(
     Authorizer* authorizer)
 {
diff --git a/src/common/http.hpp b/src/common/http.hpp
index 6ca54a6..ac9ed5e 100644
--- a/src/common/http.hpp
+++ b/src/common/http.hpp
@@ -173,16 +173,6 @@ void json(
 void json(JSON::ObjectWriter* writer, const Task& task);
 void json(JSON::ObjectWriter* writer, const TaskStatus& status);
 
-namespace authorization {
-
-// Creates a subject for authorization purposes when given an authenticated
-// principal. This function accepts and returns an `Option` to make call sites
-// cleaner, since it is possible that `principal` will be `NONE`.
-const Option<authorization::Subject> createSubject(
-    const Option<process::http::authentication::Principal>& principal);
-
-} // namespace authorization {
-
 const process::http::authorization::AuthorizationCallbacks
   createAuthorizationCallbacks(Authorizer* authorizer);
 
diff --git a/src/master/http.cpp b/src/master/http.cpp
index 75ab6ea..68ee2a6 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -68,6 +68,7 @@
 #include <stout/utils.hpp>
 #include <stout/uuid.hpp>
 
+#include "common/authorization.hpp"
 #include "common/http.hpp"
 #include "common/protobuf_utils.hpp"
 #include "common/resources_utils.hpp"
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 0b43dc7..b4b02d8 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -1074,10 +1074,11 @@ void Master::initialize()
   provide("app", path::join(flags.webui_dir, "app"));
   provide("assets", path::join(flags.webui_dir, "assets"));
 
-  const PID<Master> masterPid = self();
+  // TODO(tillt): Use generalized lambda capture once we adopt C++14.
+  Option<Authorizer*> _authorizer = authorizer;
 
-  auto authorize = [masterPid](const Option<Principal>& principal) {
-    return dispatch(masterPid, &Master::authorizeLogAccess, principal);
+  auto authorize = [_authorizer](const Option<Principal>& principal) {
+    return authorization::authorizeLogAccess(_authorizer, principal);
   };
 
   // Expose the log file for the webui. Fall back to 'log_dir' if
@@ -1413,24 +1414,6 @@ void Master::_exited(Framework* framework)
 }
 
 
-Future<bool> Master::authorizeLogAccess(const Option<Principal>& principal)
-{
-  if (authorizer.isNone()) {
-    return true;
-  }
-
-  authorization::Request request;
-  request.set_action(authorization::ACCESS_MESOS_LOG);
-
-  Option<authorization::Subject> subject = createSubject(principal);
-  if (subject.isSome()) {
-    request.mutable_subject()->CopyFrom(subject.get());
-  }
-
-  return authorizer.get()->authorized(request);
-}
-
-
 void Master::consume(MessageEvent&& event)
 {
   // There are three cases about the message's UPID with respect to
diff --git a/src/master/master.hpp b/src/master/master.hpp
index baa6668..3b3c1a4 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -1179,9 +1179,6 @@ private:
       const hashset<SlaveID>& toRemoveGone,
       const process::Future<bool>& registrarResult);
 
-  process::Future<bool> authorizeLogAccess(
-      const Option<process::http::authentication::Principal>& principal);
-
   std::vector<std::string> filterRoles(
       const process::Owned<ObjectApprovers>& approvers) const;
 
diff --git a/src/master/quota_handler.cpp b/src/master/quota_handler.cpp
index a4975fd..8d417a9 100644
--- a/src/master/quota_handler.cpp
+++ b/src/master/quota_handler.cpp
@@ -37,6 +37,8 @@
 #include <stout/strings.hpp>
 #include <stout/utils.hpp>
 
+#include "common/authorization.hpp"
+
 #include "logging/logging.hpp"
 
 #include "master/quota.hpp"
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 816aed1..ba38926 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -57,6 +57,7 @@
 #include <stout/strings.hpp>
 #include <stout/unreachable.hpp>
 
+#include "common/authorization.hpp"
 #include "common/build.hpp"
 #include "common/http.hpp"
 #include "common/recordio.hpp"
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 74f6fb9..858b786 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -82,6 +82,7 @@
 
 #include "authentication/cram_md5/authenticatee.hpp"
 
+#include "common/authorization.hpp"
 #include "common/build.hpp"
 #include "common/protobuf_utils.hpp"
 #include "common/resources_utils.hpp"
@@ -836,13 +837,11 @@ void Slave::initialize()
             });
         });
 
-  const PID<Slave> slavePid = self();
+  // TODO(tillt): Use generalized lambda capture once we adopt C++14.
+  Option<Authorizer*> _authorizer = authorizer;
 
-  auto authorize = [slavePid](const Option<Principal>& principal) {
-    return dispatch(
-        slavePid,
-        &Slave::authorizeLogAccess,
-        principal);
+  auto authorize = [_authorizer](const Option<Principal>& principal) {
+    return authorization::authorizeLogAccess(_authorizer, principal);
   };
 
   // Expose the log file for the webui. Fall back to 'log_dir' if
@@ -8478,24 +8477,6 @@ Future<bool> Slave::authorizeTask(
 }
 
 
-Future<bool> Slave::authorizeLogAccess(const Option<Principal>& principal)
-{
-  if (authorizer.isNone()) {
-    return true;
-  }
-
-  authorization::Request request;
-  request.set_action(authorization::ACCESS_MESOS_LOG);
-
-  Option<authorization::Subject> subject = createSubject(principal);
-  if (subject.isSome()) {
-    request.mutable_subject()->CopyFrom(subject.get());
-  }
-
-  return authorizer.get()->authorized(request);
-}
-
-
 Future<bool> Slave::authorizeSandboxAccess(
     const Option<Principal>& principal,
     const FrameworkID& frameworkId,
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 0bd3401..edf7269 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -608,9 +608,6 @@ private:
       const TaskInfo& task,
       const FrameworkInfo& frameworkInfo);
 
-  process::Future<bool> authorizeLogAccess(
-      const Option<process::http::authentication::Principal>& principal);
-
   process::Future<bool> authorizeSandboxAccess(
       const Option<process::http::authentication::Principal>& principal,
       const FrameworkID& frameworkId,


[mesos] 03/05: Introduced common/authorization and refactored collectAuthorizations.

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

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

commit 176f5c2fb998b2327059893e180501b32f9450e0
Author: Till Toenshoff <to...@me.com>
AuthorDate: Tue Nov 20 14:46:00 2018 +0100

    Introduced common/authorization and refactored collectAuthorizations.
    
    Adds a new collection of authorization specific helper/s to reduce code
    duplication and increase efficient test coverage.
    
    Moves the newly introduced 'collectAuthorizations' helper into this new
    authorization source unit.
    
    Review: https://reviews.apache.org/r/69384/
---
 src/CMakeLists.txt             |  1 +
 src/Makefile.am                |  2 ++
 src/common/authorization.cpp   | 41 +++++++++++++++++++++++++++++++++++++++++
 src/common/authorization.hpp   | 35 +++++++++++++++++++++++++++++++++++
 src/master/master.cpp          | 11 ++++++-----
 src/master/master.hpp          | 13 -------------
 src/master/weights_handler.cpp |  4 +++-
 src/tests/master_tests.cpp     | 17 +++++++++--------
 8 files changed, 97 insertions(+), 27 deletions(-)

diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 7323682..bde0704 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -221,6 +221,7 @@ set(AUTHORIZER_SRC
   authorizer/local/authorizer.cpp)
 
 set(COMMON_SRC
+  common/authorization.cpp
   common/attributes.cpp
   common/build.cpp
   common/command_utils.cpp
diff --git a/src/Makefile.am b/src/Makefile.am
index 2d9c81b..8da1a05 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -996,6 +996,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   checks/checker_process.cpp						\
   checks/health_checker.cpp						\
   common/attributes.cpp							\
+  common/authorization.cpp						\
   common/command_utils.cpp						\
   common/http.cpp							\
   common/protobuf_utils.cpp						\
@@ -1145,6 +1146,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   checks/checks_runtime.hpp						\
   checks/checks_types.hpp						\
   checks/health_checker.hpp						\
+  common/authorization.hpp						\
   common/build.hpp							\
   common/command_utils.hpp						\
   common/http.hpp							\
diff --git a/src/common/authorization.cpp b/src/common/authorization.cpp
new file mode 100644
index 0000000..5064ad2
--- /dev/null
+++ b/src/common/authorization.cpp
@@ -0,0 +1,41 @@
+// 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/authorization.hpp"
+
+#include <algorithm>
+
+#include <process/collect.hpp>
+
+#include <stout/foreach.hpp>
+
+using std::vector;
+
+using process::Future;
+
+namespace mesos {
+namespace authorization {
+
+Future<bool> collectAuthorizations(const vector<Future<bool>>& authorizations)
+{
+  return process::collect(authorizations)
+    .then([](const vector<bool>& results) -> Future<bool> {
+      return std::find(results.begin(), results.end(), false) == results.end();
+    });
+}
+
+} // namespace authorization {
+} // namespace mesos {
diff --git a/src/common/authorization.hpp b/src/common/authorization.hpp
new file mode 100644
index 0000000..439996a
--- /dev/null
+++ b/src/common/authorization.hpp
@@ -0,0 +1,35 @@
+// 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 __COMMON_AUTHORIZATION_HPP__
+#define __COMMON_AUTHORIZATION_HPP__
+
+#include <vector>
+
+#include <process/future.hpp>
+
+namespace mesos {
+namespace authorization {
+
+// Collects authorization results. Any discarded or failed future
+// results in a failure; any false future results in 'false'.
+process::Future<bool> collectAuthorizations(
+    const std::vector<process::Future<bool>>& authorizations);
+
+} // namespace authorization {
+} // namespace mesos {
+
+#endif // __COMMON_AUTHORIZATION_HPP__
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 6e34cc4..0b43dc7 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -75,6 +75,7 @@
 
 #include "authentication/cram_md5/authenticator.hpp"
 
+#include "common/authorization.hpp"
 #include "common/build.hpp"
 #include "common/http.hpp"
 #include "common/protobuf_utils.hpp"
@@ -3641,7 +3642,7 @@ Future<bool> Master::authorizeReserveResources(
     return authorizer.get()->authorized(request);
   }
 
-  return collectAuthorizations(authorizations);
+  return authorization::collectAuthorizations(authorizations);
 }
 
 
@@ -3694,7 +3695,7 @@ Future<bool> Master::authorizeUnreserveResources(
     return authorizer.get()->authorized(request);
   }
 
-  return collectAuthorizations(authorizations);
+  return authorization::collectAuthorizations(authorizations);
 }
 
 
@@ -3751,7 +3752,7 @@ Future<bool> Master::authorizeCreateVolume(
     return authorizer.get()->authorized(request);
   }
 
-  return collectAuthorizations(authorizations);
+  return authorization::collectAuthorizations(authorizations);
 }
 
 
@@ -3793,7 +3794,7 @@ Future<bool> Master::authorizeDestroyVolume(
     return authorizer.get()->authorized(request);
   }
 
-  return collectAuthorizations(authorizations);
+  return authorization::collectAuthorizations(authorizations);
 }
 
 
@@ -3974,7 +3975,7 @@ Future<bool> Master::authorizeSlave(
         authorizeReserveResources(slaveInfo.resources(), principal));
   }
 
-  return collectAuthorizations(authorizations);
+  return authorization::collectAuthorizations(authorizations);
 }
 
 
diff --git a/src/master/master.hpp b/src/master/master.hpp
index afd829e..baa6668 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -19,7 +19,6 @@
 
 #include <stdint.h>
 
-#include <algorithm>
 #include <list>
 #include <memory>
 #include <set>
@@ -2353,18 +2352,6 @@ private:
 };
 
 
-// Collects authorization results. Any discarded or failed future
-// results in a failure; any false future results in false.
-inline process::Future<bool> collectAuthorizations(
-    const std::vector<process::Future<bool>>& authorizations)
-{
-  return process::collect(authorizations)
-    .then([](const std::vector<bool>& results) -> process::Future<bool> {
-      return std::find(results.begin(), results.end(), false) == results.end();
-    });
-}
-
-
 inline std::ostream& operator<<(
     std::ostream& stream,
     const Framework& framework);
diff --git a/src/master/weights_handler.cpp b/src/master/weights_handler.cpp
index 1d34356..dfb6f06 100644
--- a/src/master/weights_handler.cpp
+++ b/src/master/weights_handler.cpp
@@ -28,6 +28,8 @@
 #include <stout/strings.hpp>
 #include <stout/utils.hpp>
 
+#include "common/authorization.hpp"
+
 #include "master/weights.hpp"
 
 namespace http = process::http;
@@ -345,7 +347,7 @@ Future<bool> Master::WeightsHandler::authorizeUpdateWeights(
     return master->authorizer.get()->authorized(request);
   }
 
-  return collectAuthorizations(authorizations);
+  return authorization::collectAuthorizations(authorizations);
 }
 
 
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index 658051f..651bb9b 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -46,6 +46,7 @@
 #include <stout/strings.hpp>
 #include <stout/try.hpp>
 
+#include "common/authorization.hpp"
 #include "common/build.hpp"
 #include "common/protobuf_utils.hpp"
 
@@ -10160,8 +10161,8 @@ TEST_F(MasterTest, CollectAuthorizations)
     Promise<bool> promise1;
     Promise<bool> promise2;
 
-    Future<bool> result =
-      master::collectAuthorizations({promise1.future(), promise2.future()});
+    Future<bool> result = authorization::collectAuthorizations(
+        {promise1.future(), promise2.future()});
 
     promise1.set(true);
     promise2.set(false);
@@ -10173,8 +10174,8 @@ TEST_F(MasterTest, CollectAuthorizations)
     Promise<bool> promise1;
     Promise<bool> promise2;
 
-    Future<bool> result =
-      master::collectAuthorizations({promise1.future(), promise2.future()});
+    Future<bool> result = authorization::collectAuthorizations(
+        {promise1.future(), promise2.future()});
 
     promise1.set(true);
     promise2.fail("Authorization failure");
@@ -10186,8 +10187,8 @@ TEST_F(MasterTest, CollectAuthorizations)
     Promise<bool> promise1;
     Promise<bool> promise2;
 
-    Future<bool> result =
-      master::collectAuthorizations({promise1.future(), promise2.future()});
+    Future<bool> result = authorization::collectAuthorizations(
+        {promise1.future(), promise2.future()});
 
     promise1.set(true);
     promise2.discard();
@@ -10199,8 +10200,8 @@ TEST_F(MasterTest, CollectAuthorizations)
     Promise<bool> promise1;
     Promise<bool> promise2;
 
-    Future<bool> result =
-      master::collectAuthorizations({promise1.future(), promise2.future()});
+    Future<bool> result = authorization::collectAuthorizations(
+        {promise1.future(), promise2.future()});
 
     promise1.set(true);
     promise2.set(true);


[mesos] 02/05: Added collectAuthorizations helper to master.hpp.

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

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

commit 3501ac59fbe407e4d6372c69af283ea1cb7d9ae0
Author: Till Toenshoff <to...@me.com>
AuthorDate: Tue Nov 20 14:45:50 2018 +0100

    Added collectAuthorizations helper to master.hpp.
    
    Adds the helper function 'collectAuthorizations' to master.hpp. This
    function allows for a simple way to collect authorization futures and
    only if all supplied futures result in an approved authorization will
    the returned future return true.
    
    All identified areas that were formally triggering MESOS-9317 are
    being updated to make use of this new helper.
    
    A helper function has been chosen and preferred over copying this
    pattern into the areas that needed a fix to allow for an efficient and
    complete test coverage.
    
    Additionally we are adding a test validating that new helper.
    
    Review: https://reviews.apache.org/r/69369/
---
 src/master/master.cpp          | 54 ++++-----------------------------------
 src/master/master.hpp          | 14 +++++++++++
 src/master/weights_handler.cpp | 12 +--------
 src/tests/master_tests.cpp     | 57 ++++++++++++++++++++++++++++++++++++++++++
 4 files changed, 77 insertions(+), 60 deletions(-)

diff --git a/src/master/master.cpp b/src/master/master.cpp
index 9458ff1..6e34cc4 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -3641,17 +3641,7 @@ Future<bool> Master::authorizeReserveResources(
     return authorizer.get()->authorized(request);
   }
 
-  return await(authorizations)
-      .then([](const vector<Future<bool>>& authorizations)
-            -> Future<bool> {
-        // Compute a disjunction.
-        foreach (const Future<bool>& authorization, authorizations) {
-          if (!authorization.get()) {
-            return false;
-          }
-        }
-        return true;
-      });
+  return collectAuthorizations(authorizations);
 }
 
 
@@ -3704,17 +3694,7 @@ Future<bool> Master::authorizeUnreserveResources(
     return authorizer.get()->authorized(request);
   }
 
-  return await(authorizations)
-      .then([](const vector<Future<bool>>& authorizations)
-            -> Future<bool> {
-        // Compute a disjunction.
-        foreach (const Future<bool>& authorization, authorizations) {
-          if (!authorization.get()) {
-            return false;
-          }
-        }
-        return true;
-      });
+  return collectAuthorizations(authorizations);
 }
 
 
@@ -3771,17 +3751,7 @@ Future<bool> Master::authorizeCreateVolume(
     return authorizer.get()->authorized(request);
   }
 
-  return await(authorizations)
-      .then([](const vector<Future<bool>>& authorizations)
-            -> Future<bool> {
-        // Compute a disjunction.
-        foreach (const Future<bool>& authorization, authorizations) {
-          if (!authorization.get()) {
-            return false;
-          }
-        }
-        return true;
-      });
+  return collectAuthorizations(authorizations);
 }
 
 
@@ -3823,17 +3793,7 @@ Future<bool> Master::authorizeDestroyVolume(
     return authorizer.get()->authorized(request);
   }
 
-  return await(authorizations)
-      .then([](const vector<Future<bool>>& authorizations)
-            -> Future<bool> {
-        // Compute a disjunction.
-        foreach (const Future<bool>& authorization, authorizations) {
-          if (!authorization.get()) {
-            return false;
-          }
-        }
-        return true;
-      });
+  return collectAuthorizations(authorizations);
 }
 
 
@@ -4014,11 +3974,7 @@ Future<bool> Master::authorizeSlave(
         authorizeReserveResources(slaveInfo.resources(), principal));
   }
 
-  return collect(authorizations)
-    .then([](const vector<bool>& results)
-          -> Future<bool> {
-      return std::find(results.begin(), results.end(), false) == results.end();
-    });
+  return collectAuthorizations(authorizations);
 }
 
 
diff --git a/src/master/master.hpp b/src/master/master.hpp
index e77babf..afd829e 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -19,6 +19,7 @@
 
 #include <stdint.h>
 
+#include <algorithm>
 #include <list>
 #include <memory>
 #include <set>
@@ -42,6 +43,7 @@
 
 #include <mesos/scheduler/scheduler.hpp>
 
+#include <process/collect.hpp>
 #include <process/future.hpp>
 #include <process/limiter.hpp>
 #include <process/http.hpp>
@@ -2351,6 +2353,18 @@ private:
 };
 
 
+// Collects authorization results. Any discarded or failed future
+// results in a failure; any false future results in false.
+inline process::Future<bool> collectAuthorizations(
+    const std::vector<process::Future<bool>>& authorizations)
+{
+  return process::collect(authorizations)
+    .then([](const std::vector<bool>& results) -> process::Future<bool> {
+      return std::find(results.begin(), results.end(), false) == results.end();
+    });
+}
+
+
 inline std::ostream& operator<<(
     std::ostream& stream,
     const Framework& framework);
diff --git a/src/master/weights_handler.cpp b/src/master/weights_handler.cpp
index 222ec75..1d34356 100644
--- a/src/master/weights_handler.cpp
+++ b/src/master/weights_handler.cpp
@@ -345,17 +345,7 @@ Future<bool> Master::WeightsHandler::authorizeUpdateWeights(
     return master->authorizer.get()->authorized(request);
   }
 
-  return await(authorizations)
-      .then([](const vector<Future<bool>>& authorizations)
-            -> Future<bool> {
-        // Compute a disjunction.
-        foreach (const Future<bool>& authorization, authorizations) {
-          if (!authorization.get()) {
-            return false;
-          }
-        }
-        return true;
-      });
+  return collectAuthorizations(authorizations);
 }
 
 
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index 8ed1e89..658051f 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -10152,6 +10152,63 @@ TEST_F(MasterTest, CreateVolumesV1AuthorizationFailure)
   AWAIT_EXPECT_RESPONSE_STATUS_EQ(InternalServerError().status, response);
 }
 
+
+// Test for the authorization collect helper.
+TEST_F(MasterTest, CollectAuthorizations)
+{
+  {
+    Promise<bool> promise1;
+    Promise<bool> promise2;
+
+    Future<bool> result =
+      master::collectAuthorizations({promise1.future(), promise2.future()});
+
+    promise1.set(true);
+    promise2.set(false);
+
+    AWAIT_EXPECT_FALSE(result);
+  }
+
+  {
+    Promise<bool> promise1;
+    Promise<bool> promise2;
+
+    Future<bool> result =
+      master::collectAuthorizations({promise1.future(), promise2.future()});
+
+    promise1.set(true);
+    promise2.fail("Authorization failure");
+
+    AWAIT_EXPECT_FAILED(result);
+  }
+
+  {
+    Promise<bool> promise1;
+    Promise<bool> promise2;
+
+    Future<bool> result =
+      master::collectAuthorizations({promise1.future(), promise2.future()});
+
+    promise1.set(true);
+    promise2.discard();
+
+    AWAIT_EXPECT_FAILED(result);
+  }
+
+  {
+    Promise<bool> promise1;
+    Promise<bool> promise2;
+
+    Future<bool> result =
+      master::collectAuthorizations({promise1.future(), promise2.future()});
+
+    promise1.set(true);
+    promise2.set(true);
+
+    AWAIT_EXPECT_TRUE(result);
+  }
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {