You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by mp...@apache.org on 2015/09/10 00:59:22 UTC

[2/3] mesos git commit: Added /unreserve HTTP endpoint to the master.

Added /unreserve HTTP endpoint to the master.

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


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

Branch: refs/heads/master
Commit: cc9c682e0108e0f5aa127e3afe43df517ab6bad7
Parents: 57a7e7d
Author: Michael Park <mp...@apache.org>
Authored: Wed Aug 5 02:00:15 2015 -0700
Committer: Michael Park <mp...@apache.org>
Committed: Wed Sep 9 15:28:29 2015 -0700

----------------------------------------------------------------------
 src/master/http.cpp                   | 181 +++++++++++++++++++++--------
 src/master/master.cpp                 |   5 +
 src/master/master.hpp                 |  27 +++++
 src/master/validation.cpp             |   2 +-
 src/tests/master_validation_tests.cpp |   1 -
 5 files changed, 164 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/cc9c682e/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index bcf7f93..a052e55 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -745,56 +745,7 @@ Future<Response> Master::Http::reserve(const Request& request) const
 
   // TODO(mpark): Add a reserve ACL for authorization.
 
-  // The resources recovered by rescinding outstanding offers.
-  Resources recovered;
-
-  // The unreserved resources needed to satisfy the RESERVE operation.
-  // This is used in an optimization where we try to only rescind
-  // offers that would contribute to satisfying the Reserve operation.
-  Resources remaining = resources.flatten();
-
-  // We pessimistically assume that what seems like "available"
-  // resources in the allocator will be gone. This can happen due to
-  // the race between the allocator scheduling an 'allocate' call to
-  // itself vs master's request to schedule 'updateAvailable'.
-  // We greedily rescind one offer at time until we've rescinded
-  // enough offers to cover for 'resources'.
-  foreach (Offer* offer, utils::copy(slave->offers)) {
-    // If rescinding the offer would not contribute to satisfying
-    // the remaining resources, skip it.
-    if (remaining == remaining - offer->resources()) {
-      continue;
-    }
-
-    recovered += offer->resources();
-    remaining -= offer->resources();
-
-    // We explicitly pass 'Filters()' which has a default 'refuse_sec'
-    // of 5 seconds rather than 'None()' here, so that we can
-    // virtually always win the race against 'allocate'.
-    master->allocator->recoverResources(
-        offer->framework_id(),
-        offer->slave_id(),
-        offer->resources(),
-        Filters());
-
-    master->removeOffer(offer, true); // Rescind!
-
-    // If we've rescinded enough offers to cover for 'resources',
-    // we're done.
-    Try<Resources> updatedRecovered = recovered.apply(operation);
-    if (updatedRecovered.isSome()) {
-      break;
-    }
-  }
-
-  // Propogate the 'Future<Nothing>' as 'Future<Response>' where
-  // 'Nothing' -> 'OK' and Failed -> 'Conflict'.
-  return master->apply(slave, operation)
-    .then([]() -> Response { return OK(); })
-    .repair([](const Future<Response>& result) {
-       return Conflict(result.failure());
-    });
+  return _operation(slaveId, resources.flatten(), operation);
 }
 
 
@@ -1818,6 +1769,79 @@ Future<Response> Master::Http::maintenanceStatus(const Request& request) const
 }
 
 
+Future<Response> Master::Http::unreserve(const Request& request) const
+{
+  if (request.method != "POST") {
+    return BadRequest("Expecting POST");
+  }
+
+  // Parse the query string in the request body.
+  Try<hashmap<string, string>> decode =
+    process::http::query::decode(request.body);
+
+  if (decode.isError()) {
+    return BadRequest("Unable to decode query string: " + decode.error());
+  }
+
+  const hashmap<string, string>& values = decode.get();
+
+  if (values.get("slaveId").isNone()) {
+    return BadRequest("Missing 'slaveId' query parameter");
+  }
+
+  SlaveID slaveId;
+  slaveId.set_value(values.get("slaveId").get());
+
+  Slave* slave = master->slaves.registered.get(slaveId);
+  if (slave == NULL) {
+    return BadRequest("No slave found with specified ID");
+  }
+
+  if (values.get("resources").isNone()) {
+    return BadRequest("Missing 'resources' query parameter");
+  }
+
+  Try<JSON::Array> parse =
+    JSON::parse<JSON::Array>(values.get("resources").get());
+
+  if (parse.isError()) {
+    return BadRequest(
+        "Error in parsing 'resources' query parameter: " + parse.error());
+  }
+
+  Resources resources;
+  foreach (const JSON::Value& value, parse.get().values) {
+    Try<Resource> resource = ::protobuf::parse<Resource>(value);
+    if (resource.isError()) {
+      return BadRequest(
+          "Error in parsing 'resources' query parameter: " + resource.error());
+    }
+    resources += resource.get();
+  }
+
+  Result<Credential> credential = authenticate(request);
+  if (credential.isError()) {
+    return Unauthorized("Mesos master", credential.error());
+  }
+
+  // Create an offer operation.
+  Offer::Operation operation;
+  operation.set_type(Offer::Operation::UNRESERVE);
+  operation.mutable_unreserve()->mutable_resources()->CopyFrom(resources);
+
+  Option<Error> validate =
+    validation::operation::validate(operation.unreserve(), credential.isSome());
+
+  if (validate.isSome()) {
+    return BadRequest("Invalid UNRESERVE operation: " + validate.get().message);
+  }
+
+  // TODO(mpark): Add a unreserve ACL for authorization.
+
+  return _operation(slaveId, resources, operation);
+}
+
+
 Result<Credential> Master::Http::authenticate(const Request& request) const
 {
   // By default, assume everyone is authenticated if no credentials
@@ -1860,6 +1884,63 @@ Result<Credential> Master::Http::authenticate(const Request& request) const
 }
 
 
+Future<Response> Master::Http::_operation(
+    const SlaveID& slaveId,
+    Resources remaining,
+    const Offer::Operation& operation) const
+{
+  Slave* slave = master->slaves.registered.get(slaveId);
+  if (slave == NULL) {
+    return BadRequest("No slave found with specified ID");
+  }
+
+  // The resources recovered by rescinding outstanding offers.
+  Resources recovered;
+
+  // We pessimistically assume that what seems like "available"
+  // resources in the allocator will be gone. This can happen due to
+  // the race between the allocator scheduling an 'allocate' call to
+  // itself vs master's request to schedule 'updateAvailable'.
+  // We greedily rescind one offer at time until we've rescinded
+  // enough offers to cover for 'resources'.
+  foreach (Offer* offer, utils::copy(slave->offers)) {
+    // If rescinding the offer would not contribute to satisfying
+    // the remaining resources, skip it.
+    if (remaining == remaining - offer->resources()) {
+      continue;
+    }
+
+    recovered += offer->resources();
+    remaining -= offer->resources();
+
+    // We explicitly pass 'Filters()' which has a default 'refuse_sec'
+    // of 5 seconds rather than 'None()' here, so that we can
+    // virtually always win the race against 'allocate'.
+    master->allocator->recoverResources(
+        offer->framework_id(),
+        offer->slave_id(),
+        offer->resources(),
+        Filters());
+
+    master->removeOffer(offer, true); // Rescind!
+
+    // If we've rescinded enough offers to cover for 'resources',
+    // we're done.
+    Try<Resources> updatedRecovered = recovered.apply(operation);
+    if (updatedRecovered.isSome()) {
+      break;
+    }
+  }
+
+  // Propogate the 'Future<Nothing>' as 'Future<Response>' where
+  // 'Nothing' -> 'OK' and Failed -> 'Conflict'.
+  return master->apply(slave, operation)
+    .then([]() -> Response { return OK(); })
+    .repair([](const Future<Response>& result) {
+       return Conflict(result.failure());
+    });
+}
+
 } // namespace master {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/cc9c682e/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index ea7d613..4b60e63 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -845,6 +845,11 @@ void Master::initialize()
           Http::log(request);
           return http.machineUp(request);
         });
+  route("/unreserve",
+        None(),  // TODO(mpark): Add an Http::UNRESERVE_HELP,
+        [http](const process::http::Request& request) {
+          return http.unreserve(request);
+        });
 
   // Provide HTTP assets from a "webui" directory. This is either
   // specified via flags (which is necessary for running out of the

http://git-wip-us.apache.org/repos/asf/mesos/blob/cc9c682e/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 7849d68..1dfc947 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -866,6 +866,10 @@ private:
     process::Future<process::http::Response> machineUp(
         const process::http::Request& request) const;
 
+    // /master/unreserve
+    process::Future<process::http::Response> unreserve(
+        const process::http::Request& request) const;
+
     const static std::string SCHEDULER_HELP;
     const static std::string HEALTH_HELP;
     const static std::string OBSERVE_HELP;
@@ -893,6 +897,29 @@ private:
         const FrameworkID& id,
         bool authorized = true) const;
 
+    /**
+     * Continuation for operations: /reserve, /unreserve, /create and
+     * /destroy. First tries to recover 'remaining' amount of
+     * resources by rescinding outstanding offers, then tries to apply
+     * the operation by calling 'master->apply' and propagates the
+     * 'Future<Nothing>' as 'Future<Response>' where 'Nothing' -> 'OK'
+     * and Failed -> 'Conflict'.
+     *
+     * @param slaveId The ID of the slave that the operation is
+     *     updating.
+     * @param remaining The resources needed to satisfy the operation.
+     *     This is used for an optimization where we try to only
+     *     rescind offers that would contribute to satisfying the
+     *     operation.
+     * @param operation The operation to be performed.
+     *
+     * @return Returns 'OK' if successful, 'Conflict' otherwise.
+     */
+    process::Future<process::http::Response> _operation(
+        const SlaveID& slaveId,
+        Resources remaining,
+        const Offer::Operation& operation) const;
+
     Master* master;
   };
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/cc9c682e/src/master/validation.cpp
----------------------------------------------------------------------
diff --git a/src/master/validation.cpp b/src/master/validation.cpp
index 0361d1f..f97eba6 100644
--- a/src/master/validation.cpp
+++ b/src/master/validation.cpp
@@ -724,7 +724,7 @@ Option<Error> validate(
   }
 
   if (!hasPrincipal) {
-    return Error("A framework without a principal cannot unreserve resources.");
+    return Error("Resources cannot be unreserved without a principal.");
   }
 
   // NOTE: We don't check that 'FrameworkInfo.principal' matches

http://git-wip-us.apache.org/repos/asf/mesos/blob/cc9c682e/src/tests/master_validation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_validation_tests.cpp b/src/tests/master_validation_tests.cpp
index 3513bca..1dff6a3 100644
--- a/src/tests/master_validation_tests.cpp
+++ b/src/tests/master_validation_tests.cpp
@@ -355,7 +355,6 @@ class UnreserveOperationValidationTest : public MesosTest {};
 
 // This test verifies that any resources can be unreserved by any
 // framework with a principal.
-// TODO(mpark): Introduce the "unreserve" ACL to prevent this.
 TEST_F(UnreserveOperationValidationTest, WithoutACL)
 {
   Resource resource = Resources::parse("cpus", "8", "role").get();