You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by vi...@apache.org on 2017/04/21 21:24:54 UTC

[1/2] mesos git commit: Added test for failed HTTP executor authentication.

Repository: mesos
Updated Branches:
  refs/heads/master 3f5b2c90c -> 02d8426a3


Added test for failed HTTP executor authentication.

This patch adds a new test, `SlaveTest.HTTPExecutorBadAuthentication`,
which verifies that an HTTP executor cannot subscribe or launch nested
containers when it provides an authentication token that has not been
properly signed.

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


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

Branch: refs/heads/master
Commit: 3cd927c981f44cb71b4d1783f73d1e8d46d98bf0
Parents: 3f5b2c9
Author: Greg Mann <gr...@mesosphere.io>
Authored: Fri Apr 21 14:24:05 2017 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Apr 21 14:24:05 2017 -0700

----------------------------------------------------------------------
 src/tests/slave_tests.cpp | 208 +++++++++++++++++++++++++++++++++++++++++
 1 file changed, 208 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3cd927c9/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index ba7c6c3..8c97dc6 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -31,9 +31,11 @@
 
 #include <mesos/authentication/http/basic_authenticator_factory.hpp>
 
+
 #include <process/clock.hpp>
 #include <process/future.hpp>
 #include <process/gmock.hpp>
+#include <process/http.hpp>
 #include <process/owned.hpp>
 #include <process/pid.hpp>
 #include <process/reap.hpp>
@@ -49,6 +51,10 @@
 #include <stout/strings.hpp>
 #include <stout/try.hpp>
 
+#ifdef USE_SSL_SOCKET
+#include "authentication/executor/jwt_secret_generator.hpp"
+#endif // USE_SSL_SOCKET
+
 #include "common/build.hpp"
 #include "common/http.hpp"
 #include "common/protobuf_utils.hpp"
@@ -78,6 +84,10 @@
 
 using namespace mesos::internal::slave;
 
+#ifdef USE_SSL_SOCKET
+using mesos::authentication::executor::JWTSecretGenerator;
+#endif // USE_SSL_SOCKET
+
 using mesos::internal::master::Master;
 
 using mesos::internal::protobuf::createLabel;
@@ -107,6 +117,8 @@ using process::http::Response;
 using process::http::ServiceUnavailable;
 using process::http::Unauthorized;
 
+using process::http::authentication::Principal;
+
 using std::map;
 using std::shared_ptr;
 using std::string;
@@ -1852,6 +1864,202 @@ TEST_F(SlaveTest, ReadonlyHTTPEndpointsNoAuthentication)
 }
 
 
+// Since executor authentication currently has SSL as a dependency, we cannot
+// test executor authentication when Mesos has not been built with SSL.
+#ifdef USE_SSL_SOCKET
+// This test verifies that HTTP executor SUBSCRIBE and LAUNCH_NESTED_CONTAINER
+// calls fail if the executor provides an incorrectly-signed authentication
+// token with valid claims.
+TEST_F(SlaveTest, HTTPExecutorBadAuthentication)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  auto executor = std::make_shared<v1::MockHTTPExecutor>();
+
+  v1::Resources resources =
+    v1::Resources::parse("cpus:0.1;mem:32;disk:32").get();
+
+  v1::ExecutorInfo executorInfo;
+  executorInfo.set_type(v1::ExecutorInfo::DEFAULT);
+  executorInfo.mutable_executor_id()->CopyFrom(v1::DEFAULT_EXECUTOR_ID);
+  executorInfo.mutable_resources()->CopyFrom(resources);
+
+  Owned<TestContainerizer> containerizer(
+      new TestContainerizer(devolve(executorInfo.executor_id()), executor));
+
+  // This pointer is passed to the agent, which will perform the cleanup.
+  MockSecretGenerator* mockSecretGenerator = new MockSecretGenerator();
+
+  MockSlave slave(
+      CreateSlaveFlags(),
+      detector.get(),
+      containerizer.get(),
+      None(),
+      None(),
+      mockSecretGenerator);
+  process::PID<Slave> slavePid = spawn(slave);
+
+  auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
+
+  Future<Nothing> connected;
+  EXPECT_CALL(*scheduler, connected(_))
+    .WillOnce(FutureSatisfy(&connected));
+
+  v1::scheduler::TestMesos mesos(
+      master.get()->pid,
+      ContentType::PROTOBUF,
+      scheduler);
+
+  AWAIT_READY(connected);
+
+  Future<v1::scheduler::Event::Subscribed> subscribed;
+  EXPECT_CALL(*scheduler, subscribed(_, _))
+    .WillOnce(FutureArg<1>(&subscribed));
+
+  Future<v1::scheduler::Event::Offers> offers;
+  EXPECT_CALL(*scheduler, offers(_, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  EXPECT_CALL(*scheduler, heartbeat(_))
+    .WillRepeatedly(Return()); // Ignore heartbeats.
+
+  {
+    v1::scheduler::Call call;
+    call.set_type(v1::scheduler::Call::SUBSCRIBE);
+    v1::scheduler::Call::Subscribe* subscribe = call.mutable_subscribe();
+    subscribe->mutable_framework_info()->CopyFrom(v1::DEFAULT_FRAMEWORK_INFO);
+
+    mesos.send(call);
+  }
+
+  AWAIT_READY(subscribed);
+  v1::FrameworkID frameworkId(subscribed->framework_id());
+
+  executorInfo.mutable_framework_id()->CopyFrom(frameworkId);
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0, offers->offers().size());
+
+  Future<v1::executor::Mesos*> executorLib;
+  EXPECT_CALL(*executor, connected(_))
+    .WillOnce(FutureArg<0>(&executorLib));
+
+  Promise<Secret> secret;
+  Future<Principal> principal;
+  EXPECT_CALL(*mockSecretGenerator, generate(_))
+    .WillOnce(DoAll(FutureArg<0>(&principal),
+                    Return(secret.future())));
+
+  const v1::Offer& offer = offers->offers(0);
+  const v1::AgentID& agentId = offer.agent_id();
+
+  {
+    v1::TaskInfo taskInfo =
+      v1::createTask(agentId, resources, SLEEP_COMMAND(1000));
+
+    v1::TaskGroupInfo taskGroup;
+    taskGroup.add_tasks()->CopyFrom(taskInfo);
+
+    v1::scheduler::Call call;
+    call.mutable_framework_id()->CopyFrom(frameworkId);
+    call.set_type(v1::scheduler::Call::ACCEPT);
+
+    v1::scheduler::Call::Accept* accept = call.mutable_accept();
+    accept->add_offer_ids()->CopyFrom(offer.id());
+
+    v1::Offer::Operation* operation = accept->add_operations();
+    operation->set_type(v1::Offer::Operation::LAUNCH_GROUP);
+
+    v1::Offer::Operation::LaunchGroup* launchGroup =
+      operation->mutable_launch_group();
+
+    launchGroup->mutable_executor()->CopyFrom(executorInfo);
+    launchGroup->mutable_task_group()->CopyFrom(taskGroup);
+
+    mesos.send(call);
+  }
+
+  AWAIT_READY(principal);
+
+  // Create a secret generator initialized with an incorrect key.
+  Owned<JWTSecretGenerator> jwtSecretGenerator(
+      new JWTSecretGenerator("incorrect_key"));
+
+  Future<Secret> authenticationToken =
+    jwtSecretGenerator->generate(principal.get());
+
+  AWAIT_READY(authenticationToken);
+
+  secret.set(authenticationToken.get());
+
+  {
+    AWAIT_READY(executorLib);
+
+    v1::executor::Call call;
+    call.mutable_framework_id()->CopyFrom(frameworkId);
+    call.mutable_executor_id()->CopyFrom(v1::DEFAULT_EXECUTOR_ID);
+
+    call.set_type(v1::executor::Call::SUBSCRIBE);
+
+    call.mutable_subscribe();
+
+    executorLib.get()->send(call);
+
+    Future<v1::executor::Event::Error> error;
+    EXPECT_CALL(*executor, error(_, _))
+      .WillOnce(FutureArg<1>(&error));
+
+    AWAIT_READY(error);
+    EXPECT_EQ(
+        error->message(),
+        "Received unexpected '401 Unauthorized' () for SUBSCRIBE");
+  }
+
+  {
+    ASSERT_TRUE(principal->claims.contains("cid"));
+
+    v1::ContainerID parentContainerId;
+    parentContainerId.set_value(principal->claims.at("cid"));
+
+    v1::ContainerID containerId;
+    containerId.set_value(UUID::random().toString());
+    containerId.mutable_parent()->CopyFrom(parentContainerId);
+
+    v1::agent::Call call;
+    call.set_type(v1::agent::Call::LAUNCH_NESTED_CONTAINER);
+
+    call.mutable_launch_nested_container()->mutable_container_id()
+      ->CopyFrom(containerId);
+
+    process::http::Headers headers;
+    headers["Authorization"] =
+      "Bearer " + authenticationToken.get().value().data();
+
+    Future<Response> response = process::http::post(
+      slavePid,
+      "api/v1",
+      headers,
+      serialize(ContentType::PROTOBUF, call),
+      stringify(ContentType::PROTOBUF));
+
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(Unauthorized({}).status, response);
+
+    ASSERT_TRUE(response->headers.contains("WWW-Authenticate"));
+    ASSERT_TRUE(strings::contains(
+        response->headers.at("WWW-Authenticate"),
+        "Invalid JWT: Token signature does not match"));
+  }
+
+  terminate(slave);
+  wait(slave);
+}
+#endif // USE_SSL_SOCKET
+
+
 // This test verifies correct handling of statistics endpoint when
 // there is no exeuctor running.
 TEST_F(SlaveTest, StatisticsEndpointNoExecutor)


[2/2] mesos git commit: Added documentation for executor authorization.

Posted by vi...@apache.org.
Added documentation for executor authorization.

This patch adds documentation regarding the implicit authorization
of HTTP executors.

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


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

Branch: refs/heads/master
Commit: 02d8426a3b1dc6b3269a014ada541a6008a6ab05
Parents: 3cd927c
Author: Greg Mann <gr...@mesosphere.io>
Authored: Fri Apr 21 14:24:35 2017 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Apr 21 14:24:35 2017 -0700

----------------------------------------------------------------------
 docs/authorization.md | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/02d8426a/docs/authorization.md
----------------------------------------------------------------------
diff --git a/docs/authorization.md b/docs/authorization.md
index 913765f..42337e7 100644
--- a/docs/authorization.md
+++ b/docs/authorization.md
@@ -25,6 +25,26 @@ briefly discusses how to implement a custom authorizer; this section is not
 directed at operators but at engineers who wish to build their own authorizer
 back end.
 
+## HTTP Executor Authorization
+
+When the agent's `--authenticate_http_executors` flag is set, HTTP executors are
+required to authenticate with the HTTP executor API. When they do so, a simple
+implicit authorization rule is applied. In plain language, the rule states that
+executors can only perform actions on themselves. More specifically, an
+executor's authenticated principal must contain claims with keys `fid`, `eid`,
+and `cid`, with values equal to the currently-running executor's framework ID,
+executor ID, and container ID, respectively. By default, an authentication token
+containing these claims is injected into the executor's environment (see the
+[authentication documentation](authentication.md) for more information).
+
+Similarly, when the agent's `--authenticate_http_readwrite` flag is set, HTTP
+executor's are required to authenticate with the HTTP operator API when making
+calls such as `LAUNCH_NESTED_CONTAINER`. In this case, executor authorization is
+performed via the loaded authorizer module, if present. The default Mesos local
+authorizer applies a simple implicit authorization rule, requiring that the
+executor's principal contain a claim with key `cid` and a value equal to the
+currently-running executor's container ID.
+
 ## Local Authorizer
 
 ### Role vs. Principal