You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by as...@apache.org on 2020/03/03 07:52:48 UTC

[mesos] 01/08: Introduced `getApprover(...)` authorizer interface.

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

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

commit c6cede0800f11c208078dcf3eaa756997166ac7b
Author: Andrei Sekretenko <as...@mesosphere.com>
AuthorDate: Mon Jan 20 16:05:48 2020 +0100

    Introduced `getApprover(...)` authorizer interface.
    
    This patch introduces a breaking change in the Authorizer interface:
    `getObjectApprover(...)` method that returnes ObjectApprover
    which should not be stored for a long time is replaced with
    `getApprover(...)` method that returns ObjectApprover
    that must be kept valid (by authorizer implementation) throughout its
    whole lifetime.
    
    This unblocks way to synchronous (without dispatch to another actor)
    authorization in cases where principal is known to be long-lived;
    examples are the scheduler API (see MESOS-10056) and v1 operator API
    events (see MESOS-10057).
    
    The local authorizer is modified accordingly.
    
    NOTE: This patch breaks compatibility with custom authorizers which
    do not implement this method!
    
    Review: https://reviews.apache.org/r/72089
---
 docs/authorization.md                   | 10 +++-
 include/mesos/authorizer/authorizer.hpp | 45 +++++++++++++---
 src/authorizer/local/authorizer.cpp     | 94 +++++++++++++++++++--------------
 src/authorizer/local/authorizer.hpp     |  3 +-
 src/common/http.cpp                     | 14 ++---
 src/common/http.hpp                     | 19 ++++---
 src/tests/api_tests.cpp                 | 30 ++++++-----
 src/tests/master_load_tests.cpp         | 25 +++++----
 src/tests/mesos.cpp                     |  5 +-
 src/tests/mesos.hpp                     |  3 +-
 10 files changed, 159 insertions(+), 89 deletions(-)

diff --git a/docs/authorization.md b/docs/authorization.md
index 698e485..96ec973 100644
--- a/docs/authorization.md
+++ b/docs/authorization.md
@@ -911,7 +911,7 @@ backwards compatibility when adding new fields (see
 Third, the `ObjectApprover` interface. In order to support efficient
 authorization of large objects and multiple objects a user can request an
 `ObjectApprover` via
-`Future<Owned<ObjectApprover>> getObjectApprover(const authorization::Subject& subject, const authorization::Action& action)`.
+`Future<shared_ptr<const ObjectApprover>> getApprover(const authorization::Subject& subject, const authorization::Action& action)`.
 The resulting `ObjectApprover` provides
 `Try<bool> approved(const ObjectApprover::Object& object)` to synchronously
 check whether objects are authorized. The `ObjectApprover::Object` follows the
@@ -932,5 +932,13 @@ struct Object
 As the fields take pointer to each entity the `ObjectApprover::Object` does not
 require the entity to be copied.
 
+Authorizer must ensure that `ObjectApprover`s returned by `getApprover(...)` method
+are valid throughout their whole lifetime. This is relied upon by parts of Mesos code
+(Scheduler API, Operator API events and so on) that have a need to frequently authorize
+a limited number of long-lived authorization subjects.
+This code on the Mesos side, on its part, must ensure that it does not store
+`ObjectApprover` for authorization subjects that it no longer uses (i.e. that it
+does not leak `ObjectApprover`s).
+
 NOTE: As the `ObjectApprover` is run synchronously in a different actor process
 `ObjectApprover.approved()` call must not block!
diff --git a/include/mesos/authorizer/authorizer.hpp b/include/mesos/authorizer/authorizer.hpp
index a86a6ee..1c540f2 100644
--- a/include/mesos/authorizer/authorizer.hpp
+++ b/include/mesos/authorizer/authorizer.hpp
@@ -36,6 +36,13 @@ class ACLs;
  * This interface represents a function object returned by the
  * authorizer which can be used locally (and synchronously) to
  * check whether a specific object is authorized.
+ *
+ * Authorizer implementations must ensure that ObjectApprover is valid
+ * throughout its lifetime (by updating the internal state of ObjectApprover
+ * if/when necessary). Components of Mesos side are allowed
+ * to store `ObjectApprover`s for long-lived authorization subjects indefinitely
+ * (as long as they have a potential need to authorize objects for corresponding
+ * subject-action pair) and can rely on ObjectApprover being valid at any time.
  */
 class ObjectApprover
 {
@@ -214,11 +221,29 @@ public:
   };
 
   /**
-   * NOTE: As this function can be used synchronously by actors
-   * it is essential that it does not block!
+   * This method returns whether access to the specified object is authorized
+   * or not, or `Error`. The `Error` is returned in case of:
+   * - transient authorization failures
+   * - authorizer or underlying systems being in invalid state
+   * - the `Object` provided by Mesos is invalid
+   *
+   * Note that this method is not idempotent; the result might change due to
+   * modifications of internal state of `ObjectApprover` performed by the
+   * authorizer to keep `ObjectApprover` valid.
+   *
+   * For example, if the authorizer is backed by an external IAM, from which it
+   * fetches permissions, changing permissions for the authorization Subject in
+   * the IAM might result in the response changing from `false` to `true` for
+   * the same Object. Also, in this example, failure to keep permissions
+   * up-to-date due to malfunctions of the IAM/network will be reported as an
+   * Error being returned by this method until the permissions are updated
+   * successfully.
+   *
+   * NOTE: As this method can be used synchronously by actors,
+   * it is essential that its implementation does not block. Specifically,
+   * calling blocking libprocess functions from this method can cause deadlock!
    */
-  virtual Try<bool> approved(
-      const Option<Object>& object) const noexcept = 0;
+  virtual Try<bool> approved(const Option<Object>& object) const noexcept = 0;
 
   virtual ~ObjectApprover() = default;
 };
@@ -287,9 +312,16 @@ public:
       const authorization::Request& request) = 0;
 
   /**
-   * Creates an `ObjectApprover` which can synchronously check authorization on
+   * Returns an `ObjectApprover` which can synchronously check authorization on
    * an object.
    *
+   * The returned `ObjectApprover` is valid throuhout its whole
+   * lifetime or the lifetime of the authorizer, whichever is smaller.
+   *
+   * Calls to `approved(...)` method can return different values depending
+   * on the internal state maintained by the authorizer (which can change
+   * due to the need to keep `ObjectApprover` up-to-date).
+   *
    * @param subject `authorization::Subject` subject for which the
    *     `ObjectApprover` should be created.
    *
@@ -298,7 +330,8 @@ public:
    *
    * @return An `ObjectApprover` for the given `subject` and `action`.
    */
-  virtual process::Future<process::Owned<ObjectApprover>> getObjectApprover(
+  virtual process::Future<std::shared_ptr<const ObjectApprover>>
+  getApprover(
       const Option<authorization::Subject>& subject,
       const authorization::Action& action) = 0;
 
diff --git a/src/authorizer/local/authorizer.cpp b/src/authorizer/local/authorizer.cpp
index 16c0ffa..8d29fa8 100644
--- a/src/authorizer/local/authorizer.cpp
+++ b/src/authorizer/local/authorizer.cpp
@@ -16,6 +16,7 @@
 
 #include "authorizer/local/authorizer.hpp"
 
+#include <memory>
 #include <string>
 #include <vector>
 
@@ -43,12 +44,12 @@
 #include "common/parse.hpp"
 #include "common/protobuf_utils.hpp"
 
+using std::shared_ptr;
 using std::string;
 using std::vector;
 
 using process::Failure;
 using process::Future;
-using process::Owned;
 
 using process::dispatch;
 
@@ -832,19 +833,24 @@ public:
       subject = request.subject();
     }
 
-    return getObjectApprover(subject, request.action())
-      .then([=](const Owned<ObjectApprover>& objectApprover) -> Future<bool> {
-        Option<ObjectApprover::Object> object = None();
-        if (request.has_object()) {
-          object = ObjectApprover::Object(request.object());
-        }
+    Option<ObjectApprover::Object> object = None();
+    if (request.has_object()) {
+      object = ObjectApprover::Object(request.object());
+    }
 
-        Try<bool> result = objectApprover->approved(object);
-        if (result.isError()) {
-          return Failure(result.error());
-        }
-        return result.get();
-      });
+    const Try<shared_ptr<const ObjectApprover>> approver =
+      getApprover(subject, request.action());
+
+    if (approver.isError()) {
+      return Failure(approver.error());
+    }
+
+    Try<bool> result = (*approver)->approved(object);
+    if (result.isError()) {
+      return Failure(result.error());
+    }
+
+    return result.get();
   }
 
   template <typename SomeACL>
@@ -895,7 +901,7 @@ public:
     return acls;
   }
 
-  Future<Owned<ObjectApprover>> getHierarchicalRoleApprover(
+  shared_ptr<const ObjectApprover> getHierarchicalRoleApprover(
       const Option<authorization::Subject>& subject,
       const authorization::Action& action) const
   {
@@ -1010,12 +1016,11 @@ public:
         UNREACHABLE();
     }
 
-    return Owned<ObjectApprover>(
-        new LocalHierarchicalRoleApprover(
-            hierarchicalRoleACLs, subject, action, acls.permissive()));
+    return std::make_shared<LocalHierarchicalRoleApprover>(
+        hierarchicalRoleACLs, subject, action, acls.permissive());
   }
 
-  Future<Owned<ObjectApprover>> getNestedContainerObjectApprover(
+  shared_ptr<const ObjectApprover> getNestedContainerObjectApprover(
       const Option<authorization::Subject>& subject,
       const authorization::Action& action) const
   {
@@ -1064,15 +1069,15 @@ public:
       }
     }
 
-    return Owned<ObjectApprover>(new LocalNestedContainerObjectApprover(
+    return std::make_shared<LocalNestedContainerObjectApprover>(
         runAsUserAcls,
         parentRunningAsUserAcls,
         subject,
         action,
-        acls.permissive()));
+        acls.permissive());
   }
 
-  Future<Owned<ObjectApprover>> getImplicitExecutorObjectApprover(
+  shared_ptr<const ObjectApprover> getImplicitExecutorObjectApprover(
       const Option<authorization::Subject>& subject,
       const authorization::Action& action)
   {
@@ -1098,14 +1103,14 @@ public:
     if (subjectContainerId.isNone()) {
       // If the subject's claims do not include a ContainerID,
       // we deny all objects.
-      return Owned<ObjectApprover>(new RejectingObjectApprover());
+      return std::make_shared<RejectingObjectApprover>();
     }
 
-    return Owned<ObjectApprover>(new LocalImplicitExecutorObjectApprover(
-        subjectContainerId.get()));
+    return std::make_shared<LocalImplicitExecutorObjectApprover>(
+        subjectContainerId.get());
   }
 
-  Future<Owned<ObjectApprover>> getImplicitResourceProviderObjectApprover(
+  shared_ptr<const ObjectApprover> getImplicitResourceProviderObjectApprover(
       const Option<authorization::Subject>& subject,
       const authorization::Action& action)
   {
@@ -1128,15 +1133,14 @@ public:
     if (subjectPrefix.isNone()) {
       // If the subject's claims do not include a namespace string,
       // we deny all objects.
-      return Owned<ObjectApprover>(new RejectingObjectApprover());
+      return std::make_shared<RejectingObjectApprover>();
     }
 
-    return Owned<ObjectApprover>(
-        new LocalImplicitResourceProviderObjectApprover(
-            subjectPrefix.get()));
+    return std::make_shared<LocalImplicitResourceProviderObjectApprover>(
+        subjectPrefix.get());
   }
 
-  Future<Owned<ObjectApprover>> getObjectApprover(
+  Try<shared_ptr<const ObjectApprover>> getApprover(
       const Option<authorization::Subject>& subject,
       const authorization::Action& action)
   {
@@ -1173,7 +1177,7 @@ public:
     // subjects that do not have the `value` field set. If the previous case was
     // not true and `value` is not set, then we should fail all requests.
     if (subject.isSome() && !subject->has_value()) {
-      return Owned<ObjectApprover>(new RejectingObjectApprover());
+      return std::make_shared<RejectingObjectApprover>();
     }
 
     switch (action) {
@@ -1238,16 +1242,15 @@ public:
         Result<vector<GenericACL>> genericACLs =
           createGenericACLs(action, acls);
         if (genericACLs.isError()) {
-          return Failure(genericACLs.error());
+          return Error(genericACLs.error());
         }
         if (genericACLs.isNone()) {
           // If we could not create acls, we deny all objects.
-          return Owned<ObjectApprover>(new RejectingObjectApprover());
+          return std::make_shared<RejectingObjectApprover>();
         }
 
-        return Owned<ObjectApprover>(
-            new LocalAuthorizerObjectApprover(
-                genericACLs.get(), subject, action, acls.permissive()));
+        return std::make_shared<LocalAuthorizerObjectApprover>(
+            genericACLs.get(), subject, action, acls.permissive());
       }
     }
 
@@ -1944,15 +1947,24 @@ process::Future<bool> LocalAuthorizer::authorized(
 }
 
 
-Future<Owned<ObjectApprover>> LocalAuthorizer::getObjectApprover(
-      const Option<authorization::Subject>& subject,
-      const authorization::Action& action)
+Future<shared_ptr<const ObjectApprover>> LocalAuthorizer::getApprover(
+    const Option<authorization::Subject>& subject,
+    const authorization::Action& action)
 {
   return dispatch(
       process,
-      &LocalAuthorizerProcess::getObjectApprover,
+      &LocalAuthorizerProcess::getApprover,
       subject,
-      action);
+      action)
+    .then(
+        [](const Try<shared_ptr<const ObjectApprover>>& approver)
+          -> Future<shared_ptr<const ObjectApprover>> {
+          if (approver.isError()) {
+            return Failure(approver.error());
+          }
+
+          return *approver;
+        });
 }
 
 } // namespace internal {
diff --git a/src/authorizer/local/authorizer.hpp b/src/authorizer/local/authorizer.hpp
index 2516a37..d2c5560 100644
--- a/src/authorizer/local/authorizer.hpp
+++ b/src/authorizer/local/authorizer.hpp
@@ -65,10 +65,11 @@ public:
   process::Future<bool> authorized(
       const authorization::Request& request) override;
 
-  process::Future<process::Owned<ObjectApprover>> getObjectApprover(
+  process::Future<std::shared_ptr<const ObjectApprover>> getApprover(
       const Option<authorization::Subject>& subject,
       const authorization::Action& action) override;
 
+
 private:
   LocalAuthorizer(const ACLs& acls);
 
diff --git a/src/common/http.cpp b/src/common/http.cpp
index c5b2a91..3dd77dc 100644
--- a/src/common/http.cpp
+++ b/src/common/http.cpp
@@ -15,6 +15,7 @@
 // limitations under the License.
 
 #include <map>
+#include <memory>
 #include <ostream>
 #include <set>
 #include <string>
@@ -63,6 +64,7 @@
 using std::map;
 using std::ostream;
 using std::set;
+using std::shared_ptr;
 using std::string;
 using std::vector;
 
@@ -1182,12 +1184,10 @@ Future<Owned<ObjectApprovers>> ObjectApprovers::create(
     authorization::createSubject(principal);
 
   if (authorizer.isNone()) {
-    hashmap<authorization::Action, Owned<ObjectApprover>> approvers;
+    hashmap<authorization::Action, shared_ptr<const ObjectApprover>> approvers;
 
     foreach (authorization::Action action, _actions) {
-      approvers.put(
-          action,
-          Owned<ObjectApprover>(new AcceptingObjectApprover()));
+      approvers.put(action, std::make_shared<AcceptingObjectApprover>());
     }
 
     return Owned<ObjectApprovers>(
@@ -1195,11 +1195,11 @@ Future<Owned<ObjectApprovers>> ObjectApprovers::create(
   }
 
   return process::collect(lambda::map<vector>(
-      [&](authorization::Action action) -> Future<Owned<ObjectApprover>> {
-        return authorizer.get()->getObjectApprover(subject, action);
+      [&](authorization::Action action) {
+        return authorizer.get()->getApprover(subject, action);
       },
       _actions))
-    .then([=](const vector<Owned<ObjectApprover>>& _approvers) {
+    .then([=](const vector<shared_ptr<const ObjectApprover>>& _approvers) {
       return Owned<ObjectApprovers>(
           new ObjectApprovers(lambda::zip(_actions, _approvers), principal));
     });
diff --git a/src/common/http.hpp b/src/common/http.hpp
index 4a0f4a8..98000db 100644
--- a/src/common/http.hpp
+++ b/src/common/http.hpp
@@ -371,7 +371,7 @@ public:
       std::initializer_list<authorization::Action> actions);
 
   template <authorization::Action action, typename... Args>
-  bool approved(const Args&... args)
+  bool approved(const Args&... args) const
   {
     if (!approvers.contains(action)) {
       LOG(WARNING)
@@ -381,10 +381,16 @@ public:
       return false;
     }
 
-    Try<bool> approved = approvers[action]->approved(
-        ObjectApprover::Object(args...));
+    Try<bool> approved =
+      approvers.at(action)->approved(ObjectApprover::Object(args...));
 
     if (approved.isError()) {
+      // NOTE: Silently dropping errors here creates a potential for
+      // _transient_ authorization errors to make API events subscriber's view
+      // inconsistent (see MESOS-10085). Also, this creates potential for an
+      // object to silently disappear from Operator API endpoint response in
+      // case of an authorization error (see MESOS-10099).
+      //
       // TODO(joerg84): Expose these errors back to the caller.
       LOG(WARNING)
           << "Failed to authorize principal "
@@ -402,18 +408,19 @@ private:
   ObjectApprovers(
       hashmap<
           authorization::Action,
-          process::Owned<ObjectApprover>>&& _approvers,
+          std::shared_ptr<const ObjectApprover>>&& _approvers,
       const Option<process::http::authentication::Principal>& _principal)
     : principal(_principal),
       approvers(std::move(_approvers)) {}
 
-  hashmap<authorization::Action, process::Owned<ObjectApprover>> approvers;
+  hashmap<authorization::Action, std::shared_ptr<const ObjectApprover>>
+    approvers;
 };
 
 
 template <>
 inline bool ObjectApprovers::approved<authorization::VIEW_ROLE>(
-    const Resource& resource)
+    const Resource& resource) const
 {
   // Necessary because recovered agents are presented in old format.
   if (resource.has_role() && resource.role() != "*" &&
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 8755016..21c090a 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -14,6 +14,7 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#include <memory>
 #include <string>
 #include <tuple>
 #include <utility>
@@ -71,6 +72,7 @@
 
 namespace http = process::http;
 
+using std::shared_ptr;
 
 using google::protobuf::RepeatedPtrField;
 
@@ -3358,24 +3360,24 @@ TEST_P(MasterAPITest, EventAuthorizationDelayed)
 
   // When the authorizer is called, return pending futures
   // that we can satisfy later.
-  Promise<Owned<ObjectApprover>> taskAddedApprover;
-  Promise<Owned<ObjectApprover>> updateRunningApprover;
-  Promise<Owned<ObjectApprover>> updateFinishedApprover;
+  Promise<shared_ptr<const ObjectApprover>> taskAddedApprover;
+  Promise<shared_ptr<const ObjectApprover>> updateRunningApprover;
+  Promise<shared_ptr<const ObjectApprover>> updateFinishedApprover;
 
   Sequence approverSequence;
 
   // Each event results in 4 calls into the authorizer.
   // NOTE: This may change when the operator event stream code is refactored
   // to avoid unnecessary authorizer calls. See MESOS-8475.
-  EXPECT_CALL(authorizer, getObjectApprover(_, _))
+  EXPECT_CALL(authorizer, getApprover(_, _))
     .Times(4)
     .InSequence(approverSequence)
     .WillRepeatedly(Return(taskAddedApprover.future()));
-  EXPECT_CALL(authorizer, getObjectApprover(_, _))
+  EXPECT_CALL(authorizer, getApprover(_, _))
     .Times(4)
     .InSequence(approverSequence)
     .WillRepeatedly(Return(updateRunningApprover.future()));
-  EXPECT_CALL(authorizer, getObjectApprover(_, _))
+  EXPECT_CALL(authorizer, getApprover(_, _))
     .Times(4)
     .InSequence(approverSequence)
     .WillRepeatedly(Return(updateFinishedApprover.future()));
@@ -3432,7 +3434,8 @@ TEST_P(MasterAPITest, EventAuthorizationDelayed)
   AWAIT_READY(acknowledgeFinished);
 
   {
-    taskAddedApprover.set(Owned<ObjectApprover>(new AcceptingObjectApprover()));
+    taskAddedApprover.set(shared_ptr<const ObjectApprover>(
+        std::make_shared<AcceptingObjectApprover>()));
 
     AWAIT_READY(event);
 
@@ -3444,8 +3447,8 @@ TEST_P(MasterAPITest, EventAuthorizationDelayed)
   event = decoder.read();
 
   {
-    updateRunningApprover.set(Owned<ObjectApprover>(
-        new AcceptingObjectApprover()));
+    updateRunningApprover.set(shared_ptr<const ObjectApprover>(
+        std::make_shared<AcceptingObjectApprover>()));
 
     AWAIT_READY(event);
 
@@ -3461,8 +3464,8 @@ TEST_P(MasterAPITest, EventAuthorizationDelayed)
   event = decoder.read();
 
   {
-    updateFinishedApprover.set(Owned<ObjectApprover>(
-        new AcceptingObjectApprover()));
+    updateFinishedApprover.set(shared_ptr<const ObjectApprover>(
+        std::make_shared<AcceptingObjectApprover>()));
 
     AWAIT_READY(event);
 
@@ -3477,9 +3480,8 @@ TEST_P(MasterAPITest, EventAuthorizationDelayed)
 
   EXPECT_TRUE(reader.close());
 
-  EXPECT_CALL(authorizer, getObjectApprover(_, _))
-    .WillRepeatedly(Return(Owned<ObjectApprover>(
-        new AcceptingObjectApprover())));
+  EXPECT_CALL(authorizer, getApprover(_, _))
+    .WillRepeatedly(Return(std::make_shared<AcceptingObjectApprover>()));
 
   EXPECT_CALL(*executor, shutdown(_))
     .Times(AtMost(1));
diff --git a/src/tests/master_load_tests.cpp b/src/tests/master_load_tests.cpp
index 6bbc1c0..c7c1b12 100644
--- a/src/tests/master_load_tests.cpp
+++ b/src/tests/master_load_tests.cpp
@@ -14,6 +14,8 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#include <memory>
+
 #include <mesos/mesos.hpp>
 
 #include <process/async.hpp>
@@ -26,6 +28,8 @@
 
 #include "tests/mesos.hpp"
 
+using std::shared_ptr;
+
 using mesos::authorization::VIEW_EXECUTOR;
 using mesos::authorization::VIEW_FLAGS;
 using mesos::authorization::VIEW_FRAMEWORK;
@@ -104,10 +108,13 @@ protected:
 };
 
 
-// This authorizer will not satisfy any futures from `getObjectApprover()`
+// This authorizer will not satisfy any futures from `getApprover()`
 // until it is told to, presumably from the test body.
 //
 // It effectively acts as a giant gate for certain requests.
+//
+// TODO(asekretenko): Find a way to gate requests without relying on
+// authorizer.
 class BlockingAuthorizerProcess
   : public process::Process<BlockingAuthorizerProcess>
 {
@@ -122,12 +129,12 @@ public:
     return underlying_->authorized(request);
   }
 
-  Future<Owned<ObjectApprover>> getObjectApprover(
+  Future<shared_ptr<const ObjectApprover>> getApprover(
       const Option<authorization::Subject>& subject,
       const authorization::Action& action)
   {
-    Future<Owned<ObjectApprover>> future =
-      underlying_->getObjectApprover(subject, action);
+    Future<shared_ptr<const ObjectApprover>> future =
+      underlying_->getApprover(subject, action);
 
     if (!blocked_) {
       return future;
@@ -144,7 +151,7 @@ public:
     return promises_.size();
   }
 
-  // Satisfies all future and prior calls made to `getObjectApprover`.
+  // Satisfies all future and prior calls made to `getApprover`.
   Future<Nothing> unleash()
   {
     CHECK_EQ(promises_.size(), futures_.size());
@@ -163,8 +170,8 @@ public:
 
 private:
   Authorizer* underlying_;
-  std::queue<Future<Owned<ObjectApprover>>> futures_;
-  std::queue<Promise<Owned<ObjectApprover>>> promises_;
+  std::queue<Future<shared_ptr<const ObjectApprover>>> futures_;
+  std::queue<Promise<shared_ptr<const ObjectApprover>>> promises_;
   bool blocked_;
 };
 
@@ -192,13 +199,13 @@ public:
         request);
   }
 
-  Future<Owned<ObjectApprover>> getObjectApprover(
+  Future<shared_ptr<const ObjectApprover>> getApprover(
       const Option<authorization::Subject>& subject,
       const authorization::Action& action) override
   {
     return process::dispatch(
         process_.get(),
-        &BlockingAuthorizerProcess::getObjectApprover,
+        &BlockingAuthorizerProcess::getApprover,
         subject,
         action);
   }
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index 664c302..d6933a6 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -543,9 +543,8 @@ MockAuthorizer::MockAuthorizer()
   EXPECT_CALL(*this, authorized(_))
     .WillRepeatedly(Return(true));
 
-  EXPECT_CALL(*this, getObjectApprover(_, _))
-    .WillRepeatedly(Return(Owned<ObjectApprover>(
-        new AcceptingObjectApprover())));
+  EXPECT_CALL(*this, getApprover(_, _))
+    .WillRepeatedly(Return(std::make_shared<AcceptingObjectApprover>()));
 }
 
 
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 73b1866..17fcaa3 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -3553,7 +3553,8 @@ public:
       authorized, process::Future<bool>(const authorization::Request& request));
 
   MOCK_METHOD2(
-      getObjectApprover, process::Future<process::Owned<ObjectApprover>>(
+      getApprover,
+      process::Future<std::shared_ptr<const ObjectApprover>>(
           const Option<authorization::Subject>& subject,
           const authorization::Action& action));
 };