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 2020/01/06 18:27:37 UTC

[mesos] branch master updated: Moved creating authorization Object out of `Master::authorizeTask`.

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


The following commit(s) were added to refs/heads/master by this push:
     new 765a631  Moved creating authorization Object out of `Master::authorizeTask`.
765a631 is described below

commit 765a6317ccab6653e967bebac42dd55ad559f8cf
Author: Andrei Sekretenko <as...@mesosphere.io>
AuthorDate: Mon Jan 6 12:56:47 2020 -0500

    Moved creating authorization Object out of `Master::authorizeTask`.
    
    This is the first patch in the chain that extract Master code
    generating Action-Object pairs into a dedicated ActionObject class,
    thus seperating authz Object creation from feeding them into authorizer.
    
    This is a prerequisite to using ObjectApprover interface for
    synchronous authorization of Scheduler API calls.
    
    Review: https://reviews.apache.org/r/71859/
---
 src/CMakeLists.txt           |   1 +
 src/Makefile.am              |   2 +
 src/master/authorization.cpp |  64 ++++++++++++++++++++++
 src/master/authorization.hpp |  69 ++++++++++++++++++++++++
 src/master/master.cpp        | 124 +++++++++++++++++++++++++------------------
 src/master/master.hpp        |  15 ++++--
 6 files changed, 219 insertions(+), 56 deletions(-)

diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index ef9382d..0c97e3e 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -408,6 +408,7 @@ set(LOGGING_SRC
   logging/logging.cpp)
 
 set(MASTER_SRC
+  master/authorization.cpp
   master/constants.cpp
   master/flags.cpp
   master/framework.cpp
diff --git a/src/Makefile.am b/src/Makefile.am
index 111c156..47ad1bd 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1133,6 +1133,8 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   master/allocator/mesos/sorter/random/sorter.hpp				\
   master/allocator/mesos/sorter/random/utils.hpp				\
   master/allocator/mesos/sorter/sorter.hpp					\
+  master/authorization.cpp						\
+  master/authorization.hpp						\
   master/constants.cpp							\
   master/constants.hpp							\
   master/contender/contender.cpp					\
diff --git a/src/master/authorization.cpp b/src/master/authorization.cpp
new file mode 100644
index 0000000..eabeb41
--- /dev/null
+++ b/src/master/authorization.cpp
@@ -0,0 +1,64 @@
+// 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 <stout/protobuf.hpp>
+#include "common/protobuf_utils.hpp"
+
+#include "master/authorization.hpp"
+
+using std::ostream;
+
+
+namespace mesos {
+namespace authorization {
+
+
+ActionObject ActionObject::taskLaunch(
+    const TaskInfo& task,
+    const FrameworkInfo& framework)
+{
+  Object object;
+  *object.mutable_task_info() = task;
+  *object.mutable_framework_info() = framework;
+
+  return ActionObject(authorization::RUN_TASK, std::move(object));
+}
+
+
+ostream& operator<<(ostream& stream, const ActionObject& actionObject)
+{
+  const Option<Object>& object = actionObject.object();
+
+  if (object.isNone()) {
+    return stream << "perform action " << Action_Name(actionObject.action())
+                  << " on ANY object";
+  }
+
+  switch (actionObject.action()) {
+    case authorization::RUN_TASK:
+      return stream << "launch task " << object->task_info().task_id()
+                    << " of framework " << object->framework_info().id();
+    default:
+      break;
+  }
+
+  return stream << "perform action " << Action_Name(actionObject.action())
+                << " on object " << jsonify(JSON::Protobuf(*object));
+}
+
+
+} // namespace authorization {
+} // namespace mesos {
diff --git a/src/master/authorization.hpp b/src/master/authorization.hpp
new file mode 100644
index 0000000..c0f083c
--- /dev/null
+++ b/src/master/authorization.hpp
@@ -0,0 +1,69 @@
+// 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 __MASTER_AUTHORIZATION_HPP__
+#define __MASTER_AUTHORIZATION_HPP__
+
+#include <ostream>
+
+#include <stout/option.hpp>
+
+#include <mesos/mesos.pb.h>
+#include <mesos/authorizer/authorizer.pb.h>
+
+namespace mesos {
+namespace authorization {
+
+
+class ActionObject
+{
+public:
+  Action action() const { return action_; }
+
+  const Option<Object>& object() const& { return object_; }
+  Option<Object>&& object() && { return std::move(object_); }
+
+  // Returns action-object pair for authorizing a task launch.
+  static ActionObject taskLaunch(
+      const TaskInfo& task,
+      const FrameworkInfo& framework);
+
+private:
+  Action action_;
+  Option<Object> object_;
+
+  ActionObject(Action action, Option<Object>&& object)
+    : action_(action), object_(object){};
+};
+
+// Outputs human-readable description of an ActionObject.
+//
+// NOTE: For more convenient use in authorization-related messages
+// ("Authorizing principal 'baz' to launch task", "Forbidden to create disk...",
+// and so on), the description starts with a verb.
+//
+// Output examples:
+//  - "launch task 123 of framework de-adbe-ef",
+//  - "perform action FOO_BAR on object '{task_info: ..., machine_id: ...}"
+std::ostream& operator<<(
+    std::ostream& stream,
+    const ActionObject& actionObject);
+
+
+} // namespace authorization {
+} // namespace mesos {
+
+#endif // __MASTER_AUTHORIZATION_HPP__
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 14b90a5..a81ee79 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -90,6 +90,7 @@
 #include "logging/flags.hpp"
 #include "logging/logging.hpp"
 
+#include "master/authorization.hpp"
 #include "master/flags.hpp"
 #include "master/master.hpp"
 #include "master/registry_operations.hpp"
@@ -146,6 +147,8 @@ using mesos::authorization::VIEW_FRAMEWORK;
 using mesos::authorization::VIEW_TASK;
 using mesos::authorization::VIEW_EXECUTOR;
 
+using mesos::authorization::ActionObject;
+
 using mesos::master::contender::MasterContender;
 
 using mesos::master::detector::MasterDetector;
@@ -3716,35 +3719,36 @@ void Master::launchTasks(
 }
 
 
-Future<bool> Master::authorizeTask(
-    const TaskInfo& task,
-    Framework* framework)
+Future<bool> Master::authorize(
+    const Option<Principal>& principal,
+    ActionObject&& actionObject)
 {
-  CHECK_NOTNULL(framework);
 
   if (authorizer.isNone()) {
-    return true; // Authorization is disabled.
+    return true;
   }
 
-  // Authorize the task.
+  const Option<authorization::Subject> subject = createSubject(principal);
+
   authorization::Request request;
 
-  if (framework->info.has_principal()) {
-    request.mutable_subject()->set_value(framework->info.principal());
+  if (subject.isSome()) {
+    *request.mutable_subject() = *subject;
   }
 
-  request.set_action(authorization::RUN_TASK);
-
-  authorization::Object* object = request.mutable_object();
-
-  object->mutable_task_info()->CopyFrom(task);
-  object->mutable_framework_info()->CopyFrom(framework->info);
+  LOG(INFO) << "Authorizing"
+            << (principal.isSome()
+                  ? " principal '" + stringify(*principal) + "'"
+                  : " ANY principal")
+            << " to " << actionObject;
 
-  LOG(INFO)
-    << "Authorizing framework principal '"
-    << (framework->info.has_principal() ? framework->info.principal() : "ANY")
-    << "' to launch task " << task.task_id();
+  request.set_action(actionObject.action());
+  if (actionObject.object().isSome()) {
+    *request.mutable_object() = *(std::move(actionObject).object());
+  }
 
+  // TODO(asekretenko): Use a background-refreshed ObjectApprover
+  // when they become available (see MESOS-10056).
   return authorizer.get()->authorized(request);
 }
 
@@ -4343,6 +4347,7 @@ void Master::addTask(
 }
 
 
+
 void Master::accept(
     Framework* framework,
     scheduler::Call::Accept&& accept)
@@ -4789,43 +4794,60 @@ void Master::accept(
   LOG(INFO) << "Processing ACCEPT call for offers: " << accept.offer_ids()
             << " on agent " << *slave << " for framework " << *framework;
 
+  auto getOperationTasks =
+    [](const Offer::Operation& operation) -> const RepeatedPtrField<TaskInfo>& {
+    if (operation.type() == Offer::Operation::LAUNCH) {
+      return operation.launch().task_infos();
+    }
+
+    if (operation.type() == Offer::Operation::LAUNCH_GROUP) {
+      return operation.launch_group().task_group().tasks();
+    }
+
+    UNREACHABLE();
+  };
+
+  // Add tasks to be launched to the framework's list of pending tasks
+  // before authorizing.
+  //
+  // NOTE: If two tasks have the same ID, the second one will
+  // not be put into 'framework->pendingTasks', therefore
+  // will not be launched (and TASK_ERROR will be sent).
+  // Unfortunately, we can't tell the difference between a
+  // duplicate TaskID and getting killed while pending
+  // (removed from the map). So it's possible that we send
+  // a TASK_ERROR after a TASK_KILLED (see _accept())!
+  for (const Offer::Operation& operation : accept.operations()) {
+    if (operation.type() == Offer::Operation::LAUNCH ||
+        operation.type() == Offer::Operation::LAUNCH_GROUP) {
+      for (const TaskInfo& task : getOperationTasks(operation)) {
+        if (!framework->pendingTasks.contains(task.task_id())) {
+          framework->pendingTasks[task.task_id()] = task;
+        }
+
+        // Add to the slave's list of pending tasks.
+        if (!slave->pendingTasks.contains(framework->id()) ||
+            !slave->pendingTasks[framework->id()].contains(task.task_id())) {
+          slave->pendingTasks[framework->id()][task.task_id()] = task;
+        }
+      }
+    }
+  }
+
+  const Option<Principal> principal = framework->info.has_principal()
+    ? Principal(framework->info.principal())
+    : Option<Principal>::none();
+
+  // TODO (asekretenko): use background-refreshed ObjectApprovers
+  // instead of asynchronous authorization.
   vector<Future<bool>> futures;
-  foreach (const Offer::Operation& operation, accept.operations()) {
+  for (const Offer::Operation& operation : accept.operations()) {
     switch (operation.type()) {
       case Offer::Operation::LAUNCH:
       case Offer::Operation::LAUNCH_GROUP: {
-        const RepeatedPtrField<TaskInfo>& tasks = [&]() {
-          if (operation.type() == Offer::Operation::LAUNCH) {
-            return operation.launch().task_infos();
-          } else if (operation.type() == Offer::Operation::LAUNCH_GROUP) {
-            return operation.launch_group().task_group().tasks();
-          }
-          UNREACHABLE();
-        }();
-
-        // Authorize the tasks. A task is in 'framework->pendingTasks'
-        // and 'slave->pendingTasks' before it is authorized.
-        foreach (const TaskInfo& task, tasks) {
-          futures.push_back(authorizeTask(task, framework));
-
-          // Add to the framework's list of pending tasks.
-          //
-          // NOTE: If two tasks have the same ID, the second one will
-          // not be put into 'framework->pendingTasks', therefore
-          // will not be launched (and TASK_ERROR will be sent).
-          // Unfortunately, we can't tell the difference between a
-          // duplicate TaskID and getting killed while pending
-          // (removed from the map). So it's possible that we send
-          // a TASK_ERROR after a TASK_KILLED (see _accept())!
-          if (!framework->pendingTasks.contains(task.task_id())) {
-            framework->pendingTasks[task.task_id()] = task;
-          }
-
-          // Add to the slave's list of pending tasks.
-          if (!slave->pendingTasks.contains(framework->id()) ||
-              !slave->pendingTasks[framework->id()].contains(task.task_id())) {
-            slave->pendingTasks[framework->id()][task.task_id()] = task;
-          }
+        for (const TaskInfo& task : getOperationTasks(operation)) {
+          futures.emplace_back(authorize(
+              principal, ActionObject::taskLaunch(task, framework->info)));
         }
         break;
       }
diff --git a/src/master/master.hpp b/src/master/master.hpp
index f97b085..0fbe3fb 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -79,6 +79,7 @@
 #include "internal/devolve.hpp"
 #include "internal/evolve.hpp"
 
+#include "master/authorization.hpp"
 #include "master/constants.hpp"
 #include "master/flags.hpp"
 #include "master/machine.hpp"
@@ -742,6 +743,15 @@ protected:
       const FrameworkInfo& frameworkInfo,
       const process::UPID& from);
 
+  // Returns whether the principal is authorized for the specified
+  // action-object pair.
+  // Returns failure for transient authorization failures.
+  process::Future<bool> authorize(
+      const Option<process::http::authentication::Principal>& principal,
+      authorization::ActionObject&& actionObject);
+
+  // TODO(asekretenko): get rid of action-specific authorizeSomething() methods.
+
   // Returns whether the framework is authorized.
   // Returns failure for transient authorization failures.
   process::Future<bool> authorizeFramework(
@@ -753,11 +763,6 @@ protected:
       const SlaveInfo& slaveInfo,
       const Option<process::http::authentication::Principal>& principal);
 
-  // Returns whether the task is authorized.
-  // Returns failure for transient authorization failures.
-  process::Future<bool> authorizeTask(
-      const TaskInfo& task,
-      Framework* framework);
 
   /**
    * Authorizes a `RESERVE` operation.