You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by me...@apache.org on 2016/12/07 11:04:20 UTC

mesos git commit: Added authorization support when attaching to container input/output.

Repository: mesos
Updated Branches:
  refs/heads/master 9da19f5cc -> 77b16d8ee


Added authorization support when attaching to container input/output.

Introduces authorization support for the newly introduced debug API
calls: `AttachContainerInput` and `AttachContainerOutput`.

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


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

Branch: refs/heads/master
Commit: 77b16d8eee7e4130371cebd6d1076410fc4de675
Parents: 9da19f5
Author: Alexander Rojas <al...@mesosphere.io>
Authored: Wed Dec 7 01:53:22 2016 -0800
Committer: Adam B <ad...@mesosphere.io>
Committed: Wed Dec 7 01:53:22 2016 -0800

----------------------------------------------------------------------
 src/slave/http.cpp      | 163 ++++++++++++++++++++++++++++++++++++++-----
 src/slave/slave.cpp     |  19 +++++
 src/slave/slave.hpp     |  13 ++++
 src/tests/api_tests.cpp |  22 +++++-
 4 files changed, 196 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/77b16d8e/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 91eebe1..0c0e769 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -2312,23 +2312,12 @@ Future<Response> Slave::Http::killNestedContainer(
 }
 
 
-Future<Response> Slave::Http::attachContainerInput(
+Future<Response> Slave::Http::_attachContainerInput(
     const mesos::agent::Call& call,
     Owned<Reader<mesos::agent::Call>>&& decoder,
     ContentType contentType,
-    ContentType acceptType,
-    const Option<string>& principal) const
+    ContentType acceptType) const
 {
-  CHECK_EQ(mesos::agent::Call::ATTACH_CONTAINER_INPUT, call.type());
-  CHECK(call.has_attach_container_input());
-
-  if (call.attach_container_input().type() !=
-      mesos::agent::Call::AttachContainerInput::CONTAINER_ID) {
-    return BadRequest(
-        "Expecting 'attach_container_input.type' to be CONTAINER_ID");
-  }
-
-  CHECK(call.attach_container_input().has_container_id());
   const ContainerID& containerId = call.attach_container_input().container_id();
 
   Pipe pipe;
@@ -2390,6 +2379,82 @@ Future<Response> Slave::Http::attachContainerInput(
 }
 
 
+Future<Response> Slave::Http::attachContainerInput(
+    const mesos::agent::Call& call,
+    Owned<Reader<mesos::agent::Call>>&& decoder,
+    ContentType contentType,
+    ContentType acceptType,
+    const Option<string>& principal) const
+{
+  CHECK_EQ(mesos::agent::Call::ATTACH_CONTAINER_INPUT, call.type());
+  CHECK(call.has_attach_container_input());
+
+  if (call.attach_container_input().type() !=
+      mesos::agent::Call::AttachContainerInput::CONTAINER_ID) {
+    return BadRequest(
+        "Expecting 'attach_container_input.type' to be CONTAINER_ID");
+  }
+
+  CHECK(call.attach_container_input().has_container_id());
+
+  Option<Framework*> framework;
+  Option<Executor*> executor;
+  Future<Owned<ObjectApprover>> approver;
+
+  if (slave->authorizer.isSome()) {
+    executor =
+        slave->locateExecutor(call.attach_container_input().container_id());
+    if (executor.get() != nullptr) {
+      framework = slave->frameworks[executor.get()->frameworkId];
+    } else {
+      framework = nullptr;
+    }
+
+    authorization::Subject subject;
+    if (principal.isSome()) {
+      subject.set_value(principal.get());
+    }
+
+    approver = slave->authorizer.get()->getObjectApprover(
+        subject, authorization::ATTACH_CONTAINER_INPUT);
+  } else {
+    approver = Owned<ObjectApprover>(new AcceptingObjectApprover());
+  }
+
+  return approver.then(defer(slave->self(),
+    [this, call, decoder, contentType, acceptType, executor, framework](
+        const Owned<ObjectApprover>& attachInputApprover) -> Future<Response> {
+      if (executor.isSome() && executor.get() == nullptr){
+        return NotFound(
+            "Container " +
+            stringify(call.attach_container_input().container_id()) +
+            " cannot be found");
+      }
+
+      ObjectApprover::Object object;
+      if (executor.isSome()) {
+        object.executor_info = &(executor.get()->info);
+      }
+      if (framework.isSome()) {
+        object.framework_info = &(framework.get()->info);
+      }
+
+      Try<bool> approved = attachInputApprover.get()->approved(object);
+
+      if (approved.isError()) {
+        return Failure(approved.error());
+      } else if (!approved.get()) {
+        return Forbidden();
+      }
+
+      Owned<Reader<mesos::agent::Call>> decoder_ = decoder;
+
+      return _attachContainerInput(
+          call, std::move(decoder_), contentType, acceptType);
+  }));
+}
+
+
 // Helper that reads data from `writer` and writes to `reader`.
 // Returns a failed future if there are any errors reading or writing.
 // The future is satisfied when we get a EOF.
@@ -2550,15 +2615,11 @@ Future<Response> Slave::Http::launchNestedContainerSession(
 }
 
 
-Future<Response> Slave::Http::attachContainerOutput(
+Future<Response> Slave::Http::_attachContainerOutput(
     const mesos::agent::Call& call,
     ContentType contentType,
-    ContentType acceptType,
-    const Option<string>& principal) const
+    ContentType acceptType) const
 {
-  CHECK_EQ(mesos::agent::Call::ATTACH_CONTAINER_OUTPUT, call.type());
-  CHECK(call.has_attach_container_output());
-
   const ContainerID& containerId =
     call.attach_container_output().container_id();
 
@@ -2638,6 +2699,70 @@ Future<Response> Slave::Http::attachContainerOutput(
     });
 }
 
+
+Future<Response> Slave::Http::attachContainerOutput(
+    const mesos::agent::Call& call,
+    ContentType contentType,
+    ContentType acceptType,
+    const Option<string>& principal) const
+{
+  CHECK_EQ(mesos::agent::Call::ATTACH_CONTAINER_OUTPUT, call.type());
+  CHECK(call.has_attach_container_output());
+
+  Future<Owned<ObjectApprover>> approver;
+  Option<Framework*> framework;
+  Option<Executor*> executor;
+
+  if (slave->authorizer.isSome()) {
+    executor =
+        slave->locateExecutor(call.attach_container_output().container_id());
+    if (executor.get() != nullptr) {
+      framework = slave->frameworks[executor.get()->frameworkId];
+    } else {
+      framework = nullptr;
+    }
+
+    authorization::Subject subject;
+    if (principal.isSome()) {
+      subject.set_value(principal.get());
+    }
+
+    approver = slave->authorizer.get()->getObjectApprover(
+        subject, authorization::ATTACH_CONTAINER_OUTPUT);
+  } else {
+    approver = Owned<ObjectApprover>(new AcceptingObjectApprover());
+  }
+
+  return approver.then(defer(slave->self(),
+    [this, call, contentType, acceptType, executor, framework](
+        const Owned<ObjectApprover>& attachOutputApprover) -> Future<Response> {
+      if (executor.isSome() && executor.get() == nullptr) {
+        return NotFound(
+            "Container " +
+            stringify(call.attach_container_output().container_id()) +
+            " cannot be found");
+      }
+
+      ObjectApprover::Object object;
+      if (executor.isSome()) {
+        object.executor_info = &(executor.get()->info);
+      }
+      if (framework.isSome()) {
+        object.framework_info = &(framework.get()->info);
+      }
+
+      Try<bool> approved = attachOutputApprover.get()->approved(object);
+
+      if (approved.isError()) {
+        return Failure(approved.error());
+      } else if (!approved.get()) {
+        return Forbidden();
+      }
+
+      return _attachContainerOutput(call, contentType, acceptType);
+  }));
+}
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/77b16d8e/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 7eb4503..c54d335 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -6215,6 +6215,25 @@ double Slave::_resources_revocable_percent(const string& name)
 }
 
 
+Executor* Slave::locateExecutor(const ContainerID& containerId) const
+{
+  // Locate the executor (for now we just loop since we don't
+  // index based on container id and this likely won't have a
+  // significant performance impact due to the low number of
+  // executors per-agent).
+  // TODO(adam-mesos): Support more levels of nesting.
+  foreachvalue (Framework* framework, frameworks) {
+    foreachvalue (Executor* executor, framework->executors) {
+      if (executor->containerId == containerId ||
+          executor->containerId == containerId.parent()) {
+        return executor;
+      }
+    }
+  }
+  return nullptr;
+}
+
+
 Framework::Framework(
     Slave* _slave,
     const Flags& slaveFlags,

http://git-wip-us.apache.org/repos/asf/mesos/blob/77b16d8e/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 4b94dff..ecfca05 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -468,6 +468,8 @@ private:
       const FrameworkID& frameworkId,
       const ExecutorID& executorId);
 
+  Executor* locateExecutor(const ContainerID& containerId) const;
+
   // Inner class used to namespace HTTP route handlers (see
   // slave/http.cpp for implementations).
   class Http
@@ -672,12 +674,23 @@ private:
         ContentType acceptType,
         const Option<std::string>& principal) const;
 
+    process::Future<process::http::Response> _attachContainerInput(
+        const mesos::agent::Call& call,
+        process::Owned<recordio::Reader<agent::Call>>&& decoder,
+        ContentType contentType,
+        ContentType acceptType) const;
+
     process::Future<process::http::Response> attachContainerOutput(
         const mesos::agent::Call& call,
         ContentType contentType,
         ContentType acceptType,
         const Option<std::string>& principal) const;
 
+    process::Future<process::http::Response> _attachContainerOutput(
+        const mesos::agent::Call& call,
+        ContentType contentType,
+        ContentType acceptType) const;
+
     Slave* slave;
 
     // Used to rate limit the statistics endpoint.

http://git-wip-us.apache.org/repos/asf/mesos/blob/77b16d8e/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 7c70f28..6cd1f83 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -3711,7 +3711,16 @@ TEST_P(AgentAPITest, AttachContainerOutputFailure)
   EXPECT_CALL(mockContainerizer, recover(_))
     .WillOnce(Return(Future<Nothing>(Nothing())));
 
-  Try<Owned<cluster::Slave>> slave = StartSlave(&detector, &mockContainerizer);
+  slave::Flags flags = CreateSlaveFlags();
+
+  // Disable authorization in the agent.
+  flags.acls = None();
+
+  Try<Owned<cluster::Slave>> slave =
+      StartSlave(
+          &detector,
+          &mockContainerizer,
+          flags);
 
   ASSERT_SOME(slave);
 
@@ -3757,7 +3766,16 @@ TEST_F(AgentAPITest, AttachContainerInputFailure)
   EXPECT_CALL(mockContainerizer, recover(_))
     .WillOnce(Return(Future<Nothing>(Nothing())));
 
-  Try<Owned<cluster::Slave>> slave = StartSlave(&detector, &mockContainerizer);
+  slave::Flags flags = CreateSlaveFlags();
+
+  // Disable authorization in the agent.
+  flags.acls = None();
+
+  Try<Owned<cluster::Slave>> slave =
+      StartSlave(
+          &detector,
+          &mockContainerizer,
+          flags);
 
   ASSERT_SOME(slave);