You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by al...@apache.org on 2017/03/13 18:16:09 UTC

[2/7] mesos git commit: Implemented the Agent API call `RemoveNestedContainer`.

Implemented the Agent API call `RemoveNestedContainer`.

This new Agent API call makes it possible to destroy the sandbox and
runtime directories of a terminated nested container.

Review: https://reviews.apache.org/r/57387/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/74106898
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/74106898
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/74106898

Branch: refs/heads/master
Commit: 74106898edc6b5022a02a3f9e6acc93de57d896e
Parents: 92c9236
Author: Gast�n Kleiman <ga...@mesosphere.io>
Authored: Mon Mar 13 18:23:35 2017 +0100
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Mon Mar 13 19:15:40 2017 +0100

----------------------------------------------------------------------
 src/authorizer/local/authorizer.cpp  |  14 +++
 src/slave/http.cpp                   |  64 ++++++++++++++
 src/slave/slave.hpp                  |   6 ++
 src/slave/validation.cpp             |  22 +++++
 src/tests/authorization_tests.cpp    | 142 ++++++++++++++++++++++++++++++
 src/tests/slave_validation_tests.cpp |  33 +++++++
 6 files changed, 281 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/74106898/src/authorizer/local/authorizer.cpp
----------------------------------------------------------------------
diff --git a/src/authorizer/local/authorizer.cpp b/src/authorizer/local/authorizer.cpp
index 2227b24..ad0bd6e 100644
--- a/src/authorizer/local/authorizer.cpp
+++ b/src/authorizer/local/authorizer.cpp
@@ -355,6 +355,7 @@ public:
         }
         case authorization::ATTACH_CONTAINER_INPUT:
         case authorization::ATTACH_CONTAINER_OUTPUT:
+        case authorization::REMOVE_NESTED_CONTAINER:
         case authorization::KILL_NESTED_CONTAINER:
         case authorization::WAIT_NESTED_CONTAINER: {
           aclObject.set_type(mesos::ACL::Entity::ANY);
@@ -1001,6 +1002,19 @@ private:
         return acls_;
         break;
       }
+      case authorization::REMOVE_NESTED_CONTAINER: {
+        foreach (const ACL::RemoveNestedContainer& acl,
+            acls.remove_nested_containers()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.users();
+
+          acls_.push_back(acl_);
+        }
+
+        return acls_;
+        break;
+      }
       case authorization::VIEW_FRAMEWORK:
         foreach (const ACL::ViewFramework& acl, acls.view_frameworks()) {
           GenericACL acl_;

http://git-wip-us.apache.org/repos/asf/mesos/blob/74106898/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 7b4b952..1ab6f94 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -599,6 +599,9 @@ Future<Response> Slave::Http::_api(
     case mesos::agent::Call::KILL_NESTED_CONTAINER:
       return killNestedContainer(call, mediaTypes.accept, principal);
 
+    case mesos::agent::Call::REMOVE_NESTED_CONTAINER:
+      return removeNestedContainer(call, mediaTypes.accept, principal);
+
     case mesos::agent::Call::LAUNCH_NESTED_CONTAINER_SESSION:
       return launchNestedContainerSession(call, mediaTypes, principal);
 
@@ -2448,6 +2451,67 @@ Future<Response> Slave::Http::killNestedContainer(
 }
 
 
+Future<Response> Slave::Http::removeNestedContainer(
+    const mesos::agent::Call& call,
+    ContentType acceptType,
+    const Option<Principal>& principal) const
+{
+  CHECK_EQ(mesos::agent::Call::REMOVE_NESTED_CONTAINER, call.type());
+  CHECK(call.has_remove_nested_container());
+
+  Future<Owned<ObjectApprover>> approver;
+
+  if (slave->authorizer.isSome()) {
+    Option<authorization::Subject> subject = createSubject(principal);
+
+    approver = slave->authorizer.get()->getObjectApprover(
+        subject, authorization::REMOVE_NESTED_CONTAINER);
+  } else {
+    approver = Owned<ObjectApprover>(new AcceptingObjectApprover());
+  }
+
+  return approver.then(defer(slave->self(),
+    [this, call](const Owned<ObjectApprover>& removeApprover)
+        -> Future<Response> {
+      const ContainerID& containerId =
+        call.remove_nested_container().container_id();
+
+      Executor* executor = slave->getExecutor(containerId);
+      if (executor == nullptr) {
+        return OK();
+      }
+
+      Framework* framework = slave->getFramework(executor->frameworkId);
+      CHECK_NOTNULL(framework);
+
+      ObjectApprover::Object object;
+      object.executor_info = &(executor->info);
+      object.framework_info = &(framework->info);
+
+      Try<bool> approved = removeApprover.get()->approved(object);
+
+      if (approved.isError()) {
+        return Failure(approved.error());
+      } else if (!approved.get()) {
+        return Forbidden();
+      }
+
+      Future<Nothing> remove = slave->containerizer->remove(containerId);
+
+      return remove.then(
+          [containerId](const Future<Nothing>& result) -> Response {
+            if (result.isFailed()) {
+              LOG(ERROR) << "Failed to remove nested container " << containerId
+                         << ": " << result.failure();
+              return InternalServerError(result.failure());
+            }
+
+            return OK();
+          });
+    }));
+}
+
+
 Future<Response> Slave::Http::_attachContainerInput(
     const mesos::agent::Call& call,
     Owned<Reader<mesos::agent::Call>>&& decoder,

http://git-wip-us.apache.org/repos/asf/mesos/blob/74106898/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 978edd6..3ff9281 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -695,6 +695,12 @@ private:
         const Option<process::http::authentication::Principal>&
             principal) const;
 
+    process::Future<process::http::Response> removeNestedContainer(
+        const mesos::agent::Call& call,
+        ContentType acceptType,
+        const Option<process::http::authentication::Principal>&
+            principal) const;
+
     process::Future<process::http::Response> launchNestedContainerSession(
         const mesos::agent::Call& call,
         const RequestMediaTypes& mediaTypes,

http://git-wip-us.apache.org/repos/asf/mesos/blob/74106898/src/slave/validation.cpp
----------------------------------------------------------------------
diff --git a/src/slave/validation.cpp b/src/slave/validation.cpp
index 3dbd0fa..85c43ca 100644
--- a/src/slave/validation.cpp
+++ b/src/slave/validation.cpp
@@ -229,6 +229,28 @@ Option<Error> validate(
       return None();
     }
 
+    case mesos::agent::Call::REMOVE_NESTED_CONTAINER: {
+      if (!call.has_remove_nested_container()) {
+        return Error("Expecting 'remove_nested_container' to be present");
+      }
+
+      Option<Error> error = validation::container::validateContainerId(
+          call.remove_nested_container().container_id());
+
+      if (error.isSome()) {
+        return Error("'remove_nested_container.container_id' is invalid"
+                     ": " + error->message);
+      }
+
+      // Nested containers always have at least one parent.
+      if (!call.remove_nested_container().container_id().has_parent()) {
+        return Error("Expecting 'remove_nested_container.container_id.parent'"
+                     " to be present");
+      }
+
+      return None();
+    }
+
     case mesos::agent::Call::LAUNCH_NESTED_CONTAINER_SESSION: {
       if (!call.has_launch_nested_container_session()) {
         return Error(

http://git-wip-us.apache.org/repos/asf/mesos/blob/74106898/src/tests/authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/authorization_tests.cpp b/src/tests/authorization_tests.cpp
index 42edecc..cd15add 100644
--- a/src/tests/authorization_tests.cpp
+++ b/src/tests/authorization_tests.cpp
@@ -3822,6 +3822,148 @@ TYPED_TEST(AuthorizationTest, KillNestedContainer)
 }
 
 
+// This tests the authorization of removing a nested container.
+TYPED_TEST(AuthorizationTest, RemoveNestedContainer)
+{
+  // Setup ACLs.
+  ACLs acls;
+
+  {
+    // "foo" principal cannot remove any nested container.
+    mesos::ACL::RemoveNestedContainer* acl =
+        acls.add_remove_nested_containers();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_users()->set_type(mesos::ACL::Entity::NONE);
+  }
+
+  {
+    // "bar" principal can remove nested containers running under user "bar".
+    mesos::ACL::RemoveNestedContainer* acl =
+      acls.add_remove_nested_containers();
+    acl->mutable_principals()->add_values("bar");
+    acl->mutable_users()->add_values("bar");
+  }
+
+  {
+    // "ops" principal can remove all nested containers.
+    mesos::ACL::RemoveNestedContainer* acl =
+      acls.add_remove_nested_containers();
+    acl->mutable_principals()->add_values("ops");
+    acl->mutable_users()->set_type(mesos::ACL::Entity::ANY);
+  }
+
+  {
+    // No one else can remove any nested container.
+    mesos::ACL::RemoveNestedContainer* acl =
+        acls.add_remove_nested_containers();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    acl->mutable_users()->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());
+
+  FrameworkInfo frameworkInfo;
+  frameworkInfo.set_user("user");
+
+  // Create ExecutorInfo with a user not mentioned in the ACLs in
+  // command as object to be authorized.
+  ExecutorInfo executorInfo;
+  {
+    executorInfo.set_name("Task");
+    executorInfo.mutable_executor_id()->set_value("t");
+    executorInfo.mutable_command()->set_value("echo hello");
+    executorInfo.mutable_command()->set_user("user");
+  }
+
+  // Create ExecutorInfo with user "bar" in command as object to be
+  // authorized.
+  ExecutorInfo executorInfoBar;
+  {
+    executorInfoBar.set_name("Executor");
+    executorInfoBar.mutable_executor_id()->set_value("e");
+    executorInfoBar.mutable_command()->set_value("echo hello");
+    executorInfoBar.mutable_command()->set_user("bar");
+  }
+
+  // Create ExecutorInfo with no user in command as object to be
+  // authorized.
+  ExecutorInfo executorInfoNoUser;
+  {
+    executorInfoNoUser.set_name("Executor");
+    executorInfoNoUser.mutable_executor_id()->set_value("e");
+    executorInfoNoUser.mutable_command()->set_value("echo hello");
+  }
+
+  // Principal "foo" cannot remove a nested container with a request
+  // with ExecutorInfo running with user "user".
+  {
+    authorization::Request request;
+    request.set_action(authorization::REMOVE_NESTED_CONTAINER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->mutable_executor_info()->MergeFrom(executorInfo);
+
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
+
+  // Principal "bar" cannot remove a nested container with a request
+  // with ExecutorInfo running with user "user".
+  {
+    authorization::Request request;
+    request.set_action(authorization::REMOVE_NESTED_CONTAINER);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->mutable_executor_info()->MergeFrom(executorInfo);
+    request.mutable_object()->mutable_framework_info()->MergeFrom(
+        frameworkInfo);
+
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
+
+  // Principal "ops" can remove a nested container with a request with
+  // ExecutorInfo running with user "user".
+  {
+    authorization::Request request;
+    request.set_action(authorization::REMOVE_NESTED_CONTAINER);
+    request.mutable_subject()->set_value("ops");
+    request.mutable_object()->mutable_executor_info()->MergeFrom(executorInfo);
+    request.mutable_object()->mutable_framework_info()->MergeFrom(
+        frameworkInfo);
+
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  // Principal "bar" can remove a nested container with a request with
+  // ExecutorInfo running with user "bar".
+  {
+    authorization::Request request;
+    request.set_action(authorization::REMOVE_NESTED_CONTAINER);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->mutable_executor_info()->MergeFrom(
+        executorInfoBar);
+    request.mutable_object()->mutable_framework_info()->MergeFrom(
+        frameworkInfo);
+
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  // Principal "ops" can remove nested container with a request with
+  // an ExecutorInfo without user.
+  {
+    authorization::Request request;
+    request.set_action(authorization::REMOVE_NESTED_CONTAINER);
+    request.mutable_subject()->set_value("ops");
+    request.mutable_object()->mutable_executor_info()->MergeFrom(
+        executorInfoNoUser);
+    request.mutable_object()->mutable_framework_info()->MergeFrom(
+        frameworkInfo);
+
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+}
+
+
 // This tests that a missing request.object is allowed for an ACL whose
 // Object is ANY.
 // NOTE: The only usecase for this behavior is currently teardownFramework.

http://git-wip-us.apache.org/repos/asf/mesos/blob/74106898/src/tests/slave_validation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_validation_tests.cpp b/src/tests/slave_validation_tests.cpp
index 7845284..e81b65b 100644
--- a/src/tests/slave_validation_tests.cpp
+++ b/src/tests/slave_validation_tests.cpp
@@ -397,6 +397,39 @@ TEST(AgentCallValidationTest, KillNestedContainer)
 }
 
 
+TEST(AgentCallValidationTest, RemoveNestedContainer)
+{
+  // Missing `remove_nested_container`.
+  agent::Call call;
+  call.set_type(agent::Call::REMOVE_NESTED_CONTAINER);
+
+  Option<Error> error = validation::agent::call::validate(call);
+  EXPECT_SOME(error);
+
+  // Expecting a `container_id.parent`.
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  agent::Call::RemoveNestedContainer* removeNestedContainer =
+    call.mutable_remove_nested_container();
+
+  removeNestedContainer->mutable_container_id()->CopyFrom(containerId);
+
+  error = validation::agent::call::validate(call);
+  EXPECT_SOME(error);
+
+  // Test the valid case.
+  ContainerID parentContainerId;
+  parentContainerId.set_value(UUID::random().toString());
+
+  removeNestedContainer->mutable_container_id()->mutable_parent()->CopyFrom(
+      containerId);
+
+  error = validation::agent::call::validate(call);
+  EXPECT_NONE(error);
+}
+
+
 TEST(AgentCallValidationTest, LaunchNestedContainerSession)
 {
   // Missing `launch_nested_container_session`.