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 2018/12/18 23:24:05 UTC

[mesos] branch master updated (3badf71 -> 7d98ddd)

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

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


    from 3badf71  Simplified verify-reviews.py to be more similar to the Python 2 script.
     new 51a4837  Made master process all authorization results for `LAUNCH_GROUP`.
     new e513376  Made master to authorize `CREATE_DISK` and `DESTROY_DISK` properly.
     new 19ca8c4  Documented `Object.value` authorization field for certain operations.
     new f66fbe7  Set up `Object.value` for `CREATE_DISK` and `DESTROY_DISK` authorization.
     new 7d98ddd  Added MESOS-9474, MESOS-9480 and MESOS-9486 to the 1.7.1 CHANGELOG.

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:
 CHANGELOG                                 |   3 +
 include/mesos/authorizer/authorizer.proto |  15 +++
 src/master/master.cpp                     | 178 +++++++++++++++++++++++-------
 3 files changed, 154 insertions(+), 42 deletions(-)


[mesos] 01/05: Made master process all authorization results for `LAUNCH_GROUP`.

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

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

commit 51a48371aa2566955ca5aead6a95a19b0ecfd0be
Author: Chun-Hung Hsiao <ch...@mesosphere.io>
AuthorDate: Fri Dec 14 11:01:29 2018 -0800

    Made master process all authorization results for `LAUNCH_GROUP`.
    
    This patch fixes a bug where the master does not complete the processing
    of authorization results for `LAUNCH_GROUP`, causing a subsequent
    operation to drop if one of the remaining authorization is denied.
    
    Review: https://reviews.apache.org/r/69571
---
 src/master/master.cpp | 79 +++++++++++++++++++++++++++++----------------------
 1 file changed, 45 insertions(+), 34 deletions(-)

diff --git a/src/master/master.cpp b/src/master/master.cpp
index 6ae53c1..e5fe40e 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -4754,8 +4754,8 @@ void Master::_accept(
   // The order of the conversions is important and preserved.
   vector<ResourceConversion> conversions;
 
-  // The order of `authorizations` must match the order of the operations in
-  // `accept.operations()`, as they are iterated through simultaneously.
+  // The order of `authorizations` must match the order of the operations and/or
+  // tasks in `accept.operations()` as they are iterated through simultaneously.
   CHECK_READY(_authorizations);
   std::deque<Future<bool>> authorizations(
       _authorizations->begin(), _authorizations->end());
@@ -4764,6 +4764,7 @@ void Master::_accept(
     switch (operation.type()) {
       // The RESERVE operation allows a principal to reserve resources.
       case Offer::Operation::RESERVE: {
+        CHECK(!authorizations.empty());
         Future<bool> authorization = authorizations.front();
         authorizations.pop_front();
 
@@ -4839,6 +4840,7 @@ void Master::_accept(
 
       // The UNRESERVE operation allows a principal to unreserve resources.
       case Offer::Operation::UNRESERVE: {
+        CHECK(!authorizations.empty());
         Future<bool> authorization = authorizations.front();
         authorizations.pop_front();
 
@@ -4904,6 +4906,7 @@ void Master::_accept(
       }
 
       case Offer::Operation::CREATE: {
+        CHECK(!authorizations.empty());
         Future<bool> authorization = authorizations.front();
         authorizations.pop_front();
 
@@ -4981,6 +4984,7 @@ void Master::_accept(
       }
 
       case Offer::Operation::DESTROY: {
+        CHECK(!authorizations.empty());
         Future<bool> authorization = authorizations.front();
         authorizations.pop_front();
 
@@ -5074,6 +5078,7 @@ void Master::_accept(
       }
 
       case Offer::Operation::GROW_VOLUME: {
+        CHECK(!authorizations.empty());
         Future<bool> authorization = authorizations.front();
         authorizations.pop_front();
 
@@ -5156,6 +5161,7 @@ void Master::_accept(
       }
 
       case Offer::Operation::SHRINK_VOLUME: {
+        CHECK(!authorizations.empty());
         Future<bool> authorization = authorizations.front();
         authorizations.pop_front();
 
@@ -5239,6 +5245,7 @@ void Master::_accept(
 
       case Offer::Operation::LAUNCH: {
         foreach (const TaskInfo& task, operation.launch().task_infos()) {
+          CHECK(!authorizations.empty());
           Future<bool> authorization = authorizations.front();
           authorizations.pop_front();
 
@@ -5474,51 +5481,50 @@ void Master::_accept(
         // TODO(bmahler): Consider injecting some default (cpus, mem, disk)
         // resources when the framework omits the executor resources.
 
-        // See if there are any validation or authorization errors.
+        // See if there are any authorization or validation errors.
         // Note that we'll only report the first error we encounter
         // for the group.
         //
         // TODO(anindya_sinha): If task group uses shared resources, this
         // validation needs to be enhanced to accommodate multiple copies
         // of shared resources across tasks within the task group.
-        Option<Error> error = validation::task::group::validate(
-            taskGroup, executor, framework, slave, _offeredResources);
+        Option<Error> error;
+        Option<TaskStatus::Reason> reason;
 
-        Option<TaskStatus::Reason> reason = None();
-
-        if (error.isSome()) {
-          reason = TaskStatus::REASON_TASK_GROUP_INVALID;
-        } else {
-          foreach (const TaskInfo& task, taskGroup.tasks()) {
-            Future<bool> authorization = authorizations.front();
-            authorizations.pop_front();
-
-            CHECK(!authorization.isDiscarded());
-
-            if (authorization.isFailed()) {
-              error = Error("Failed to authorize task"
-                            " '" + stringify(task.task_id()) + "'"
-                            ": " + authorization.failure());
+        // NOTE: We check for the authorization errors first and never break the
+        // loop to ensure that all authorization futures for this task group are
+        // iterated through.
+        foreach (const TaskInfo& task, taskGroup.tasks()) {
+          CHECK(!authorizations.empty());
+          Future<bool> authorization = authorizations.front();
+          authorizations.pop_front();
 
-              reason = TaskStatus::REASON_TASK_GROUP_UNAUTHORIZED;
+          CHECK(!authorization.isDiscarded());
 
-              break;
+          if (authorization.isFailed()) {
+            error = Error("Failed to authorize task"
+                          " '" + stringify(task.task_id()) + "'"
+                          ": " + authorization.failure());
+          } else if (!authorization.get()) {
+            string user = framework->info.user(); // Default user.
+            if (task.has_command() && task.command().has_user()) {
+              user = task.command().user();
             }
 
-            if (!authorization.get()) {
-              string user = framework->info.user(); // Default user.
-              if (task.has_command() && task.command().has_user()) {
-                user = task.command().user();
-              }
-
-              error = Error("Task '" + stringify(task.task_id()) + "'"
-                            " is not authorized to launch as"
-                            " user '" + user + "'");
+            error = Error("Task '" + stringify(task.task_id()) + "'"
+                          " is not authorized to launch as"
+                          " user '" + user + "'");
+          }
+        }
 
-              reason = TaskStatus::REASON_TASK_GROUP_UNAUTHORIZED;
+        if (error.isSome()) {
+          reason = TaskStatus::REASON_TASK_GROUP_UNAUTHORIZED;
+        } else {
+          error = validation::task::group::validate(
+              taskGroup, executor, framework, slave, _offeredResources);
 
-              break;
-            }
+          if (error.isSome()) {
+            reason = TaskStatus::REASON_TASK_GROUP_INVALID;
           }
         }
 
@@ -5751,6 +5757,11 @@ void Master::_accept(
     }
   }
 
+  CHECK(authorizations.empty())
+    << "Authorization results not processed: "
+    << stringify(
+           vector<Future<bool>>(authorizations.begin(), authorizations.end()));
+
   // Update the allocator based on the operations.
   if (!conversions.empty()) {
     allocator->updateAllocation(


[mesos] 02/05: Made master to authorize `CREATE_DISK` and `DESTROY_DISK` properly.

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

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

commit e5133763adeabcea8530235c988a8f08b4cf199d
Author: Chun-Hung Hsiao <ch...@mesosphere.io>
AuthorDate: Fri Dec 14 11:10:13 2018 -0800

    Made master to authorize `CREATE_DISK` and `DESTROY_DISK` properly.
    
    This patch fixes a bug where the master does not respect the
    authorization result for `CREATE_DISK` or`DESTROY_DISK`, causing
    subsequent operations to drop if the authorization is denied.
    
    Review: https://reviews.apache.org/r/69563
---
 src/master/master.cpp | 56 +++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 56 insertions(+)

diff --git a/src/master/master.cpp b/src/master/master.cpp
index e5fe40e..6ba2468 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -5673,6 +5673,34 @@ void Master::_accept(
       }
 
       case Offer::Operation::CREATE_DISK: {
+        const Resource::DiskInfo::Source::Type diskType =
+          operation.create_disk().target_type();
+
+        CHECK(!authorizations.empty());
+        Future<bool> authorization = authorizations.front();
+        authorizations.pop_front();
+
+        CHECK(!authorization.isDiscarded());
+
+        if (authorization.isFailed()) {
+          // TODO(greggomann): We may want to retry this failed authorization
+          // request rather than dropping it immediately.
+          drop(framework,
+               operation,
+               "Authorization of principal '" + framework->info.principal() +
+                 "' to create a " + stringify(diskType) + " disk failed: " +
+                 authorization.failure());
+
+          continue;
+        } else if (!authorization.get()) {
+          drop(framework,
+               operation,
+               "Not authorized to create a " + stringify(diskType) +
+                 " disk as '" + framework->info.principal() + "'");
+
+          continue;
+        }
+
         if (!slave->capabilities.resourceProvider) {
           drop(framework,
                operation,
@@ -5712,6 +5740,34 @@ void Master::_accept(
       }
 
       case Offer::Operation::DESTROY_DISK: {
+        const Resource::DiskInfo::Source::Type diskType =
+          operation.destroy_disk().source().disk().source().type();
+
+        CHECK(!authorizations.empty());
+        Future<bool> authorization = authorizations.front();
+        authorizations.pop_front();
+
+        CHECK(!authorization.isDiscarded());
+
+        if (authorization.isFailed()) {
+          // TODO(greggomann): We may want to retry this failed authorization
+          // request rather than dropping it immediately.
+          drop(framework,
+               operation,
+               "Authorization of principal '" + framework->info.principal() +
+                 "' to destroy a " + stringify(diskType) + " disk failed: " +
+                 authorization.failure());
+
+          continue;
+        } else if (!authorization.get()) {
+          drop(framework,
+               operation,
+               "Not authorized to destroy a " + stringify(diskType) +
+                 " disk as '" + framework->info.principal() + "'");
+
+          continue;
+        }
+
         if (!slave->capabilities.resourceProvider) {
           drop(framework,
                operation,


[mesos] 04/05: Set up `Object.value` for `CREATE_DISK` and `DESTROY_DISK` authorization.

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

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

commit f66fbe75a4745790f266516f39c6107833546f97
Author: Chun-Hung Hsiao <ch...@mesosphere.io>
AuthorDate: Mon Dec 17 22:11:52 2018 -0800

    Set up `Object.value` for `CREATE_DISK` and `DESTROY_DISK` authorization.
    
    This patch sets up `Object.value` to the role of the resource for
    authorization actions `CREATE_BLOCK_DISK`, `DESTROY_BLOCK_DISK`,
    `CREATE_MOUNT_DISK` and `DESTROY_MOUNT_DISK` so an old-school authorizer
    can rely on the field to perform authorization.
    
    Review: https://reviews.apache.org/r/69578
---
 src/master/master.cpp | 43 +++++++++++++++++++++++++++++++++++--------
 1 file changed, 35 insertions(+), 8 deletions(-)

diff --git a/src/master/master.cpp b/src/master/master.cpp
index 6ba2468..b4faf2b 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -3833,8 +3833,9 @@ Future<bool> Master::authorizeCreateDisk(
     return true; // Authorization is disabled.
   }
 
-  Option<authorization::Action> action;
+  const Resource& resource = createDisk.source();
 
+  Option<authorization::Action> action;
   switch (createDisk.target_type()) {
     case Resource::DiskInfo::Source::MOUNT: {
       action = authorization::CREATE_MOUNT_DISK;
@@ -3851,8 +3852,7 @@ Future<bool> Master::authorizeCreateDisk(
           "Failed to authorize principal '" +
           (principal.isSome() ? stringify(principal.get()) : "ANY") +
           "' to create a " + stringify(createDisk.target_type()) +
-          " disk from '" + stringify(createDisk.source()) +
-          "': Unsupported disk type");
+          " disk from '" + stringify(resource) + "': Unsupported disk type");
     }
   }
 
@@ -3864,7 +3864,20 @@ Future<bool> Master::authorizeCreateDisk(
     request.mutable_subject()->CopyFrom(subject.get());
   }
 
-  request.mutable_object()->mutable_resource()->CopyFrom(createDisk.source());
+  request.mutable_object()->mutable_resource()->CopyFrom(resource);
+
+  // We set `object.value` in addition to `object.resource` to support legacy
+  // authorizers making only use of this deprecated field.
+  //
+  // NOTE: We rely on the master to ensure that the resource is in the
+  // post-reservation-refinement format and set the value to the most refined
+  // role, or default to '*' for consistency if there is no reservation.
+  CHECK(!resource.has_role()) << resource;
+  CHECK(!resource.has_reservation()) << resource;
+  request.mutable_object()->set_value(
+      resource.reservations().empty()
+        ? "*"
+        : resource.reservations().rbegin()->role());
 
   LOG(INFO) << "Authorizing principal '"
             << (principal.isSome() ? stringify(principal.get()) : "ANY")
@@ -3883,9 +3896,10 @@ Future<bool> Master::authorizeDestroyDisk(
     return true; // Authorization is disabled.
   }
 
-  Option<authorization::Action> action;
+  const Resource& resource = destroyDisk.source();
 
-  switch (destroyDisk.source().disk().source().type()) {
+  Option<authorization::Action> action;
+  switch (resource.disk().source().type()) {
     case Resource::DiskInfo::Source::MOUNT: {
       action = authorization::DESTROY_MOUNT_DISK;
       break;
@@ -3900,7 +3914,7 @@ Future<bool> Master::authorizeDestroyDisk(
       return Failure(
           "Failed to authorize principal '" +
           (principal.isSome() ? stringify(principal.get()) : "ANY") +
-          "' to destroy disk '" + stringify(destroyDisk.source()) +
+          "' to destroy disk '" + stringify(resource) +
           "': Unsupported disk type");
     }
   }
@@ -3913,7 +3927,20 @@ Future<bool> Master::authorizeDestroyDisk(
     request.mutable_subject()->CopyFrom(subject.get());
   }
 
-  request.mutable_object()->mutable_resource()->CopyFrom(destroyDisk.source());
+  request.mutable_object()->mutable_resource()->CopyFrom(resource);
+
+  // We set `object.value` in addition to `object.resource` to support legacy
+  // authorizers making only use of this deprecated field.
+  //
+  // NOTE: We rely on the master to ensure that the resource is in the
+  // post-reservation-refinement format and set the value to the most refined
+  // role, or default to '*' for consistency if there is no reservation.
+  CHECK(!resource.has_role()) << resource;
+  CHECK(!resource.has_reservation()) << resource;
+  request.mutable_object()->set_value(
+      resource.reservations().empty()
+        ? "*"
+        : resource.reservations().rbegin()->role());
 
   LOG(INFO) << "Authorizing principal '"
             << (principal.isSome() ? stringify(principal.get()) : "ANY")


[mesos] 05/05: Added MESOS-9474, MESOS-9480 and MESOS-9486 to the 1.7.1 CHANGELOG.

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

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

commit 7d98ddddb5ada2d5ec6a7b821635479b7d2aadcc
Author: Chun-Hung Hsiao <ch...@mesosphere.io>
AuthorDate: Tue Dec 18 14:37:04 2018 -0800

    Added MESOS-9474, MESOS-9480 and MESOS-9486 to the 1.7.1 CHANGELOG.
---
 CHANGELOG | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/CHANGELOG b/CHANGELOG
index 434b0df..23e36b4 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -32,6 +32,8 @@ Release Notes - Mesos - Version 1.7.1 (WIP)
   * [MESOS-9411] - Validation of JWT tokens using HS256 hashing algorithm is not thread safe.
   * [MESOS-9418] - Add support for the `Discard` blkio operation type.
   * [MESOS-9419] - Executor to framework message crashes master if framework has not re-registered.
+  * [MESOS-9474] - Master does not respect authorization result for `CREATE_DISK` and `DESTROY_DISK`.
+  * [MESOS-9480] - Master may skip processing authorization results for `LAUNCH_GROUP`.
 
 ** Improvement:
   * [MESOS-6765] - Make the Resources wrapper "copy-on-write" to improve performance.
@@ -41,6 +43,7 @@ Release Notes - Mesos - Version 1.7.1 (WIP)
   * [MESOS-9305] - Create cgoup recursively to workaround systemd deleting cgroups_root.
   * [MESOS-9321] - Add an optional `vendor` field in `Resource.DiskInfo.Source`.
   * [MESOS-9340] - Log all socket errors in libprocess.
+  * [MESOS-9486] - Set up `object.value` for `CREATE_DISK` and `DESTROY_DISK` authorizations.
 
 
 Release Notes - Mesos - Version 1.7.0


[mesos] 03/05: Documented `Object.value` authorization field for certain operations.

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

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

commit 19ca8c44362c1b8b42094ac22fb3c01a93a4ddf1
Author: Chun-Hung Hsiao <ch...@mesosphere.io>
AuthorDate: Mon Dec 17 22:10:31 2018 -0800

    Documented `Object.value` authorization field for certain operations.
    
    Review: https://reviews.apache.org/r/69577
---
 include/mesos/authorizer/authorizer.proto | 15 +++++++++++++++
 1 file changed, 15 insertions(+)

diff --git a/include/mesos/authorizer/authorizer.proto b/include/mesos/authorizer/authorizer.proto
index a51d2f2..f906053 100644
--- a/include/mesos/authorizer/authorizer.proto
+++ b/include/mesos/authorizer/authorizer.proto
@@ -264,18 +264,33 @@ enum Action {
   PRUNE_IMAGES = 41;
 
   // `RESIZE_VOLUME` will have an object with `Resource` set.
+  //
+  // NOTE: For consistency, the `value` field will be set with the most refined
+  // role until all `*_WITH_ROLE` aliases are removed.
   RESIZE_VOLUME = 42;
 
   // `CREATE_BLOCK_DISK` will have an object with `Resource` set.
+  //
+  // NOTE: For consistency, the `value` field will be set with the most refined
+  // role until all `*_WITH_ROLE` aliases are removed.
   CREATE_BLOCK_DISK = 43;
 
   // `DESTROY_BLOCK_DISK` will have an object with `Resource` set.
+  //
+  // NOTE: For consistency, the `value` field will be set with the most refined
+  // role until all `*_WITH_ROLE` aliases are removed.
   DESTROY_BLOCK_DISK = 44;
 
   // `CREATE_MOUNT_DISK` will have an object with `Resource` set.
+  //
+  // NOTE: For consistency, the `value` field will be set with the most refined
+  // role until all `*_WITH_ROLE` aliases are removed.
   CREATE_MOUNT_DISK = 45;
 
   // `DESTROY_MOUNT_DISK` will have an object with `Resource` set.
+  //
+  // NOTE: For consistency, the `value` field will be set with the most refined
+  // role until all `*_WITH_ROLE` aliases are removed.
   DESTROY_MOUNT_DISK = 46;
 }