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 2017/02/21 07:00:02 UTC

[1/3] mesos git commit: Added authorization tests of launching container sessions API.

Repository: mesos
Updated Branches:
  refs/heads/master 201eb88be -> eef4340ef


Added authorization tests of launching container sessions API.

Introduced a test which ensures that unauthorized users are unable to
launch nested container sessions.

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


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

Branch: refs/heads/master
Commit: 80644e2de06e0b856bf7a96eb7bfd73eb64bdfc2
Parents: 201eb88
Author: Alexander Rojas <al...@mesosphere.io>
Authored: Mon Feb 20 20:13:28 2017 -0800
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Feb 20 20:13:28 2017 -0800

----------------------------------------------------------------------
 src/tests/api_tests.cpp | 103 +++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 103 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/80644e2d/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 7715da9..2c3ad54 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -4126,6 +4126,109 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentAPITest, LaunchNestedContainerSession)
 }
 
 
+// This tests verifies that unauthorized principals are unable to
+// launch nested container sessions.
+TEST_P(AgentAPITest, LaunchNestedContainerSessionUnauthorized)
+{
+  ContentType contentType = GetParam();
+
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> mesosContainerizer =
+    MesosContainerizer::create(flags, false, &fetcher);
+
+  ASSERT_SOME(mesosContainerizer);
+
+  Owned<slave::Containerizer> containerizer(mesosContainerizer.get());
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  {
+    // Default principal is not allowed to launch nested container sessions.
+    mesos::ACL::LaunchNestedContainerSessionUnderParentWithUser* acl =
+      flags.acls.get()
+        .add_launch_nested_container_sessions_under_parent_with_user();
+    acl->mutable_principals()->add_values(DEFAULT_CREDENTIAL.principal());
+    acl->mutable_users()->set_type(mesos::ACL::Entity::NONE);
+  }
+
+  Try<Owned<cluster::Slave>> slave =
+    StartSlave(detector.get(), containerizer.get(), flags);
+
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  Future<TaskStatus> status;
+  EXPECT_CALL(sched, statusUpdate(_, _))
+    .WillOnce(FutureArg<1>(&status));
+
+  TaskInfo task = createTask(offers.get()[0], "sleep 1000");
+
+  driver.launchTasks(offers.get()[0].id(), {task});
+
+  AWAIT_READY(status);
+  ASSERT_EQ(TASK_RUNNING, status->state());
+
+  // Attempt to launch a nested container which does nothing.
+
+  Future<hashset<ContainerID>> containerIds = containerizer->containers();
+  AWAIT_READY(containerIds);
+  ASSERT_EQ(1u, containerIds->size());
+
+  v1::ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+  containerId.mutable_parent()->set_value(containerIds->begin()->value());
+
+  string command = "sleep 1000";
+
+  v1::agent::Call call;
+  call.set_type(v1::agent::Call::LAUNCH_NESTED_CONTAINER_SESSION);
+
+  call.mutable_launch_nested_container_session()->mutable_container_id()
+    ->CopyFrom(containerId);
+
+  call.mutable_launch_nested_container_session()->mutable_command()->set_value(
+      command);
+
+  http::Headers headers = createBasicAuthHeaders(DEFAULT_CREDENTIAL);
+  headers["Accept"] = stringify(contentType);
+
+  Future<http::Response> response = http::streaming::post(
+    slave.get()->pid,
+    "api/v1",
+    headers,
+    serialize(contentType, call),
+    stringify(contentType));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::Forbidden().status, response);
+
+  containerIds = containerizer->containers();
+  AWAIT_READY(containerIds);
+  EXPECT_EQ(1u, containerIds->size());
+
+  driver.stop();
+  driver.join();
+}
+
+
 // This test verifies that launching a nested container session with `TTYInfo`
 // results in stdout and stderr being streamed to the client as stdout.
 TEST_P_TEMP_DISABLED_ON_WINDOWS(


[2/3] mesos git commit: Added authorization tests when trying to attach to a container input.

Posted by me...@apache.org.
Added authorization tests when trying to attach to a container input.

Introduced a test where a principal is allowed to launch a container
session, but not to attach to its input, therefore its attempts should
fail.

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


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

Branch: refs/heads/master
Commit: 190c9e0f1e4e9e847474dd48a928dbcca125a957
Parents: 80644e2
Author: Alexander Rojas <al...@mesosphere.io>
Authored: Mon Feb 20 20:13:55 2017 -0800
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Feb 20 20:13:55 2017 -0800

----------------------------------------------------------------------
 src/tests/api_tests.cpp | 130 +++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 130 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/190c9e0f/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 2c3ad54..378612d 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -4643,6 +4643,136 @@ TEST_F(AgentAPITest, AttachContainerInputFailure)
 }
 
 
+// Verifies that unauthorized users are not able to attach to a
+// nested container input.
+TEST_P(AgentAPITest, AttachContainerInputAuthorization)
+{
+  ContentType contentType = GetParam();
+
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  {
+    mesos::ACL::AttachContainerInput* acl =
+      flags.acls.get().add_attach_containers_input();
+    acl->mutable_principals()->add_values(DEFAULT_CREDENTIAL.principal());
+    acl->mutable_users()->set_type(mesos::ACL::Entity::NONE);
+  }
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> _containerizer =
+    MesosContainerizer::create(flags, false, &fetcher);
+
+  ASSERT_SOME(_containerizer);
+
+  Owned<slave::Containerizer> containerizer(_containerizer.get());
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave =
+    StartSlave(detector.get(), containerizer.get(), flags);
+
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  Future<TaskStatus> status;
+  EXPECT_CALL(sched, statusUpdate(_, _))
+    .WillOnce(FutureArg<1>(&status));
+
+  TaskInfo task = createTask(offers.get()[0], "sleep 1000");
+
+  driver.launchTasks(offers.get()[0].id(), {task});
+
+  AWAIT_READY(status);
+  ASSERT_EQ(TASK_RUNNING, status->state());
+
+  // Launch a nested container session which runs a shell.
+
+  Future<hashset<ContainerID>> containerIds = containerizer->containers();
+  AWAIT_READY(containerIds);
+  ASSERT_EQ(1u, containerIds->size());
+
+  v1::ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+  containerId.mutable_parent()->set_value(containerIds->begin()->value());
+
+  {
+    string command = "sh";
+
+    v1::agent::Call call;
+    call.set_type(v1::agent::Call::LAUNCH_NESTED_CONTAINER_SESSION);
+
+    call.mutable_launch_nested_container_session()->mutable_container_id()
+        ->CopyFrom(containerId);
+
+    call.mutable_launch_nested_container_session()
+        ->mutable_command()->set_value(command);
+
+    http::Headers headers = createBasicAuthHeaders(DEFAULT_CREDENTIAL);
+    headers["Accept"] = stringify(contentType);
+
+    Future<http::Response> response = http::streaming::post(
+      slave.get()->pid,
+      "api/v1",
+      headers,
+      serialize(contentType, call),
+      stringify(contentType));
+
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::OK().status, response);
+  }
+
+  // Attempt to attach to the container session's input.
+
+  {
+    v1::agent::Call call;
+    call.set_type(v1::agent::Call::ATTACH_CONTAINER_INPUT);
+
+    call.mutable_attach_container_input()
+        ->set_type(v1::agent::Call::AttachContainerInput::CONTAINER_ID);
+
+    call.mutable_attach_container_input()->mutable_container_id()
+        ->CopyFrom(containerId);
+
+    ContentType contentType = ContentType::RECORDIO;
+    ContentType messageContentType = ContentType::PROTOBUF;
+
+    ::recordio::Encoder<v1::agent::Call> encoder(
+        lambda::bind(serialize, messageContentType, lambda::_1));
+
+    http::Headers headers = createBasicAuthHeaders(DEFAULT_CREDENTIAL);
+    headers[MESSAGE_CONTENT_TYPE] = stringify(messageContentType);
+
+    Future<http::Response> response = http::post(
+      slave.get()->pid,
+      "api/v1",
+      headers,
+      encoder.encode(call),
+      stringify(contentType));
+
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::Forbidden().status, response);
+  }
+
+  driver.stop();
+  driver.join();
+}
+
+
 TEST_F(AgentAPITest, AttachContainerInputValidation)
 {
   Clock::pause();


[3/3] mesos git commit: Cleaned up weights handling code.

Posted by me...@apache.org.
Cleaned up weights handling code.

Removed a redundant statement, fixed log message style, cleaned up
comments.

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


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

Branch: refs/heads/master
Commit: eef4340ef26289d7dd6cc6b0cc56139ce5f0059e
Parents: 190c9e0
Author: Neil Conway <ne...@gmail.com>
Authored: Mon Feb 20 20:18:48 2017 -0800
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Feb 20 20:18:48 2017 -0800

----------------------------------------------------------------------
 src/master/master.cpp               |  6 ++--
 src/master/weights_handler.cpp      |  2 +-
 src/tests/dynamic_weights_tests.cpp | 50 ++++++++++++++++----------------
 3 files changed, 28 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/eef4340e/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 0f9fc39..2ef8365 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -1720,8 +1720,8 @@ Future<Nothing> Master::_recover(const Registry& registry)
     // for weights, so the `--weights` flag can be deprecated and this check
     // can eventually be removed.
     if (!weights.empty()) {
-      LOG(WARNING) << "Ignoring the --weights flag '" << flags.weights.get()
-                   << "', and recovering the weights from registry.";
+      LOG(WARNING) << "Ignoring --weights flag '" << flags.weights.get()
+                   << "' and recovering the weights from registry";
 
       // Before recovering weights from the registry, the allocator was already
       // initialized with `--weights`, so here we need to reset (to 1.0)
@@ -1734,8 +1734,6 @@ Future<Nothing> Master::_recover(const Registry& registry)
           weightInfos.push_back(weightInfo);
         }
       }
-      // Clear weights specified by `--weights` flag.
-      weights.clear();
     }
 
     // Recover `weights` with `registry_weights`.

http://git-wip-us.apache.org/repos/asf/mesos/blob/eef4340e/src/master/weights_handler.cpp
----------------------------------------------------------------------
diff --git a/src/master/weights_handler.cpp b/src/master/weights_handler.cpp
index da0b995..4f6a4cd 100644
--- a/src/master/weights_handler.cpp
+++ b/src/master/weights_handler.cpp
@@ -57,7 +57,7 @@ Future<http::Response> Master::WeightsHandler::get(
     const http::Request& request,
     const Option<string>& principal) const
 {
-  VLOG(1) << "Handling get weights request.";
+  VLOG(1) << "Handling get weights request";
 
   // Check that the request type is GET which is guaranteed by the master.
   CHECK_EQ("GET", request.method);

http://git-wip-us.apache.org/repos/asf/mesos/blob/eef4340e/src/tests/dynamic_weights_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/dynamic_weights_tests.cpp b/src/tests/dynamic_weights_tests.cpp
index ce577ce..5571ab7 100644
--- a/src/tests/dynamic_weights_tests.cpp
+++ b/src/tests/dynamic_weights_tests.cpp
@@ -145,7 +145,7 @@ protected:
 
 
 // Update weights requests with invalid JSON structure
-// should return a '400 Bad Request'.
+// should return '400 Bad Request'.
 TEST_F(DynamicWeightsTest, PutInvalidRequest)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
@@ -196,14 +196,14 @@ TEST_F(DynamicWeightsTest, PutInvalidRequest)
 }
 
 
-// A update weights request with zero value
-// should return a '400 Bad Request'.
+// An update weights request with zero value
+// should return '400 Bad Request'.
 TEST_F(DynamicWeightsTest, ZeroWeight)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Send a weight update request to update the weight of 'role1' to 0.
+  // Send a request to update the weight of 'role1' to 0.
   RepeatedPtrField<WeightInfo> infos = createWeightInfos("role1=0");
   Future<Response> response = process::http::request(
       process::http::createRequest(
@@ -221,14 +221,14 @@ TEST_F(DynamicWeightsTest, ZeroWeight)
 }
 
 
-// A update weights request with negative value
-// should return a '400 Bad Request'.
+// An update weights request with negative value
+// should return '400 Bad Request'.
 TEST_F(DynamicWeightsTest, NegativeWeight)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Send a weight update request to update the weight of 'role1' to -2.0.
+  // Send a request to update the weight of 'role1' to -2.0.
   RepeatedPtrField<WeightInfo> infos = createWeightInfos("role1=-2.0");
   Future<Response> response = process::http::request(
       process::http::createRequest(
@@ -246,14 +246,14 @@ TEST_F(DynamicWeightsTest, NegativeWeight)
 }
 
 
-// A update weights request with non-numeric value
-// should return a '400 Bad Request'.
+// An update weights request with non-numeric value
+// should return '400 Bad Request'.
 TEST_F(DynamicWeightsTest, NonNumericWeight)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Send a weight update request to update the weight of 'role1' to 'two'.
+  // Send a request to update the weight of 'role1' to 'two'.
   RepeatedPtrField<WeightInfo> infos = createWeightInfos("role1=two");
   Future<Response> response = process::http::request(
       process::http::createRequest(
@@ -272,7 +272,7 @@ TEST_F(DynamicWeightsTest, NonNumericWeight)
 
 
 // Updates must be explicit about what role they are updating,
-// and a missing role request should return a '400 Bad Request'.
+// and a missing role request should return '400 Bad Request'.
 TEST_F(DynamicWeightsTest, MissingRole)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
@@ -312,7 +312,7 @@ TEST_F(DynamicWeightsTest, MissingRole)
 
 
 // Verifies that an update request for an unknown role (not specified
-// in --roles flag) is rejected when using the explicit roles (--roles flag
+// in --roles flag) is rejected when using explicit roles (--roles flag
 // is specified when master is started).
 TEST_F(DynamicWeightsTest, UnknownRole)
 {
@@ -341,8 +341,8 @@ TEST_F(DynamicWeightsTest, UnknownRole)
 
 
 // Verifies that a weights update request for a whitelisted role
-// (contained in --roles flag) succeeds when using the explicit
-// roles (--roles flag is specified when master is started).
+// (contained in --roles flag) succeeds when using explicit roles
+// (--roles flag is specified when master is started).
 TEST_F(DynamicWeightsTest, UpdateWeightsWithExplictRoles)
 {
   // Specify --roles whitelist when starting master.
@@ -353,7 +353,7 @@ TEST_F(DynamicWeightsTest, UpdateWeightsWithExplictRoles)
 
   checkWithGetRequest(master.get()->pid, DEFAULT_CREDENTIAL);
 
-  // Send a weight update request for the specified roles in UPDATED_WEIGHTS1.
+  // Send a weight update request for the roles in UPDATED_WEIGHTS1.
   RepeatedPtrField<WeightInfo> infos = createWeightInfos(UPDATED_WEIGHTS1);
   Future<Response> response = process::http::request(
       process::http::createRequest(
@@ -384,7 +384,7 @@ TEST_F(DynamicWeightsTest, UnauthenticatedUpdateWeightRequest)
   credential.set_principal("unknown-principal");
   credential.set_secret("test-secret");
 
-  // Send a weight update request for the specified roles in UPDATED_WEIGHTS1.
+  // Send a weight update request for the roles in UPDATED_WEIGHTS1.
   RepeatedPtrField<WeightInfo> infos = createWeightInfos(UPDATED_WEIGHTS1);
   Future<Response> response1 = process::http::request(
       process::http::createRequest(
@@ -473,7 +473,7 @@ TEST_F(DynamicWeightsTest, AuthorizedGetWeightsRequest)
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
 
-  // Send a weight update request for the specified roles in UPDATED_WEIGHTS1.
+  // Send a weight update request for the roles in UPDATED_WEIGHTS1.
   RepeatedPtrField<WeightInfo> infos = createWeightInfos(UPDATED_WEIGHTS1);
   Future<Response> response = process::http::request(
       process::http::createRequest(
@@ -517,7 +517,7 @@ TEST_F(DynamicWeightsTest, AuthorizedWeightUpdateRequest)
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
 
-  // Send a weight update request for the specified roles in UPDATED_WEIGHTS1.
+  // Send a weight update request for the roles in UPDATED_WEIGHTS1.
   RepeatedPtrField<WeightInfo> infos = createWeightInfos(UPDATED_WEIGHTS1);
   Future<Response> response = process::http::request(
       process::http::createRequest(
@@ -562,7 +562,7 @@ TEST_F(DynamicWeightsTest, AuthorizedUpdateWeightRequestWithoutPrincipal)
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
 
-  // Send a weight update request for the specified roles in UPDATED_WEIGHTS1.
+  // Send a weight update request for the roles in UPDATED_WEIGHTS1.
   RepeatedPtrField<WeightInfo> infos = createWeightInfos(UPDATED_WEIGHTS1);
   Future<Response> response = process::http::request(
       process::http::createRequest(
@@ -594,7 +594,7 @@ TEST_F(DynamicWeightsTest, UnauthorizedWeightUpdateRequest)
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
 
-  // Send a weight update request for the specified roles in UPDATED_WEIGHTS1.
+  // Send a weight update request for the roles in UPDATED_WEIGHTS1.
   RepeatedPtrField<WeightInfo> infos = createWeightInfos(UPDATED_WEIGHTS1);
   Future<Response> response = process::http::request(
       process::http::createRequest(
@@ -618,13 +618,13 @@ TEST_F(DynamicWeightsTest, RecoveredWeightsFromRegistry)
   // Start a master with `--weights` flag.
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.registry = "replicated_log";
-
   masterFlags.weights = UPDATED_WEIGHTS1;
+
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
 
-  // Tests whether the weights replicated log is initialized with the
-  // `--weights` flag when bootstrapping the cluster.
+  // Tests whether the weights stored in the registry are initialized
+  // using the `--weights` flag when the cluster is bootstrapped.
   {
     checkWithGetRequest(
         master.get()->pid,
@@ -645,8 +645,8 @@ TEST_F(DynamicWeightsTest, RecoveredWeightsFromRegistry)
         UPDATED_WEIGHTS1);
   }
 
-  // Tests whether the weights replicated log can be updated with
-  // `/weights` endpoint.
+  // Tests whether the weights stored in the registry are updated
+  // successfully using the `/weights` endpoint.
   {
     // Send a weights update request for the specified roles.
     RepeatedPtrField<WeightInfo> infos = createWeightInfos(UPDATED_WEIGHTS2);