You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bb...@apache.org on 2017/12/08 10:47:40 UTC

[4/4] mesos git commit: Added a V1 API call to list resource providers.

Added a V1 API call to list resource providers.

The 'GET_RESOURCE_PROVIDERS' call will list all subscribed local
resource providers of an agent.

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


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

Branch: refs/heads/master
Commit: d142d38e3183c6b6ac56ab1c1c1629a6acde3ad5
Parents: 12695ec
Author: Jan Schlicht <ja...@mesosphere.io>
Authored: Fri Dec 8 11:26:23 2017 +0100
Committer: Benjamin Bannier <bb...@apache.org>
Committed: Fri Dec 8 11:26:23 2017 +0100

----------------------------------------------------------------------
 include/mesos/agent/agent.proto    |  31 ++++++++--
 include/mesos/v1/agent/agent.proto |  31 ++++++++--
 src/slave/http.cpp                 |  34 +++++++++++
 src/slave/http.hpp                 |   5 ++
 src/slave/validation.cpp           |   3 +
 src/tests/api_tests.cpp            | 102 ++++++++++++++++++++++++++++++++
 6 files changed, 198 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d142d38e/include/mesos/agent/agent.proto
----------------------------------------------------------------------
diff --git a/include/mesos/agent/agent.proto b/include/mesos/agent/agent.proto
index 0f92f73..eb5f1b4 100644
--- a/include/mesos/agent/agent.proto
+++ b/include/mesos/agent/agent.proto
@@ -53,10 +53,21 @@ message Call {
     GET_STATE = 9;
 
     GET_CONTAINERS = 10;
-    GET_FRAMEWORKS = 11;    // Retrieves the information about known frameworks.
-    GET_EXECUTORS = 12;     // Retrieves the information about known executors.
-    GET_TASKS = 13;         // Retrieves the information about known tasks.
-    GET_AGENT = 20;         // Retrieves the agent information.
+
+    // Retrieves the information about known frameworks.
+    GET_FRAMEWORKS = 11;
+
+    // Retrieves the information about known executors.
+    GET_EXECUTORS = 12;
+
+    // Retrieves the information about known tasks.
+    GET_TASKS = 13;
+
+    // Retrieves the agent information.
+    GET_AGENT = 20;
+
+    // Retrieves the information about known resource providers.
+    GET_RESOURCE_PROVIDERS = 26;
 
     // Calls for managing nested containers underneath an executor's container.
     // Some of these calls are deprecated in favor of the calls
@@ -321,6 +332,7 @@ message Response {
     GET_EXECUTORS = 11;            // See 'GetExecutors' below.
     GET_TASKS = 12;                // See 'GetTasks' below.
     GET_AGENT = 14;                // See 'GetAgent' below.
+    GET_RESOURCE_PROVIDERS = 16;   // See 'GetResourceProviders' below.
 
     WAIT_NESTED_CONTAINER = 13 [deprecated = true];
     WAIT_CONTAINER = 15;           // See 'WaitContainer' below.
@@ -437,6 +449,16 @@ message Response {
     optional SlaveInfo slave_info = 1;
   }
 
+  // Lists information about all resource providers known to the agent
+  // at the current time.
+  message GetResourceProviders {
+    message ResourceProvider {
+      required ResourceProviderInfo resource_provider_info = 1;
+    }
+
+    repeated ResourceProvider resource_providers = 1;
+  }
+
   // Returns termination information about the nested container.
   message WaitNestedContainer {
     // Wait status of the lead process in the container. Note that this
@@ -496,6 +518,7 @@ message Response {
   optional GetExecutors get_executors = 12;
   optional GetTasks get_tasks = 13;
   optional GetAgent get_agent = 15;
+  optional GetResourceProviders get_resource_providers = 17;
   optional WaitNestedContainer wait_nested_container = 14;
   optional WaitContainer wait_container = 16;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/d142d38e/include/mesos/v1/agent/agent.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/agent/agent.proto b/include/mesos/v1/agent/agent.proto
index 012ffef..3aedfe8 100644
--- a/include/mesos/v1/agent/agent.proto
+++ b/include/mesos/v1/agent/agent.proto
@@ -53,10 +53,21 @@ message Call {
     GET_STATE = 9;
 
     GET_CONTAINERS = 10;
-    GET_FRAMEWORKS = 11;    // Retrieves the information about known frameworks.
-    GET_EXECUTORS = 12;     // Retrieves the information about known executors.
-    GET_TASKS = 13;         // Retrieves the information about known tasks.
-    GET_AGENT = 20;         // Retrieves the agent information.
+
+    // Retrieves the information about known frameworks.
+    GET_FRAMEWORKS = 11;
+
+    // Retrieves the information about known executors.
+    GET_EXECUTORS = 12;
+
+    // Retrieves the information about known tasks.
+    GET_TASKS = 13;
+
+    // Retrieves the agent information.
+    GET_AGENT = 20;
+
+    // Retrieves the information about known resource providers.
+    GET_RESOURCE_PROVIDERS = 26;
 
     // Calls for managing nested containers underneath an executor's container.
     // Some of these calls are deprecated in favor of the calls
@@ -321,6 +332,7 @@ message Response {
     GET_EXECUTORS = 11;            // See 'GetExecutors' below.
     GET_TASKS = 12;                // See 'GetTasks' below.
     GET_AGENT = 14;                // See 'GetAgent' below.
+    GET_RESOURCE_PROVIDERS = 16;   // See 'GetResourceProviders' below.
 
     WAIT_NESTED_CONTAINER = 13 [deprecated = true];
     WAIT_CONTAINER = 15;           // See 'WaitContainer' below.
@@ -437,6 +449,16 @@ message Response {
     optional AgentInfo agent_info = 1;
   }
 
+  // Lists information about all resource providers known to the agent
+  // at the current time.
+  message GetResourceProviders {
+    message ResourceProvider {
+      required ResourceProviderInfo resource_provider_info = 1;
+    }
+
+    repeated ResourceProvider resource_providers = 1;
+  }
+
   // Returns termination information about the nested container.
   message WaitNestedContainer {
     // Wait status of the lead process in the container. Note that this
@@ -496,6 +518,7 @@ message Response {
   optional GetExecutors get_executors = 12;
   optional GetTasks get_tasks = 13;
   optional GetAgent get_agent = 15;
+  optional GetResourceProviders get_resource_providers = 17;
   optional WaitNestedContainer wait_nested_container = 14;
   optional WaitContainer wait_container = 16;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/d142d38e/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index fd0e809..49278a3 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -577,6 +577,9 @@ Future<Response> Http::_api(
     case mesos::agent::Call::GET_AGENT:
       return getAgent(call, mediaTypes.accept, principal);
 
+    case mesos::agent::Call::GET_RESOURCE_PROVIDERS:
+      return getResourceProviders(call, mediaTypes.accept, principal);
+
     case mesos::agent::Call::LAUNCH_NESTED_CONTAINER:
       return launchNestedContainer(call, mediaTypes.accept, principal);
 
@@ -1853,6 +1856,37 @@ Future<Response> Http::getAgent(
 }
 
 
+Future<Response> Http::getResourceProviders(
+    const mesos::agent::Call& call,
+    ContentType acceptType,
+    const Option<Principal>& principal) const
+{
+  CHECK_EQ(mesos::agent::Call::GET_RESOURCE_PROVIDERS, call.type());
+
+  LOG(INFO) << "Processing GET_RESOURCE_PROVIDERS call";
+
+  // TODO(nfnt): Authorize this call (MESOS-8314).
+
+  agent::Response response;
+  response.set_type(mesos::agent::Response::GET_RESOURCE_PROVIDERS);
+
+  agent::Response::GetResourceProviders* resourceProviders =
+    response.mutable_get_resource_providers();
+
+  foreachvalue (
+      const ResourceProviderInfo& resourceProviderInfo,
+      slave->resourceProviderInfos) {
+    agent::Response::GetResourceProviders::ResourceProvider* resourceProvider =
+      resourceProviders->add_resource_providers();
+
+    resourceProvider->mutable_resource_provider_info()->CopyFrom(
+        resourceProviderInfo);
+  }
+
+  return OK(serialize(acceptType, evolve(response)), stringify(acceptType));
+}
+
+
 Future<Response> Http::getState(
     const mesos::agent::Call& call,
     ContentType acceptType,

http://git-wip-us.apache.org/repos/asf/mesos/blob/d142d38e/src/slave/http.hpp
----------------------------------------------------------------------
diff --git a/src/slave/http.hpp b/src/slave/http.hpp
index a51831c..5eecb2a 100644
--- a/src/slave/http.hpp
+++ b/src/slave/http.hpp
@@ -200,6 +200,11 @@ private:
       ContentType acceptType,
       const Option<process::http::authentication::Principal>& principal) const;
 
+  process::Future<process::http::Response> getResourceProviders(
+      const mesos::agent::Call& call,
+      ContentType acceptType,
+      const Option<process::http::authentication::Principal>& principal) const;
+
   process::Future<process::http::Response> getState(
       const mesos::agent::Call& call,
       ContentType acceptType,

http://git-wip-us.apache.org/repos/asf/mesos/blob/d142d38e/src/slave/validation.cpp
----------------------------------------------------------------------
diff --git a/src/slave/validation.cpp b/src/slave/validation.cpp
index 32781fd..4a3a78a 100644
--- a/src/slave/validation.cpp
+++ b/src/slave/validation.cpp
@@ -156,6 +156,9 @@ Option<Error> validate(
     case mesos::agent::Call::GET_AGENT:
       return None();
 
+    case mesos::agent::Call::GET_RESOURCE_PROVIDERS:
+      return None();
+
     case mesos::agent::Call::LAUNCH_NESTED_CONTAINER: {
       if (!call.has_launch_nested_container()) {
         return Error("Expecting 'launch_nested_container' to be present");

http://git-wip-us.apache.org/repos/asf/mesos/blob/d142d38e/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 53c705e..c8855d0 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -6009,6 +6009,108 @@ TEST_P(AgentAPITest, DefaultAccept)
 }
 
 
+TEST_P(AgentAPITest, GetResourceProviders)
+{
+  Clock::pause();
+
+  const ContentType contentType = GetParam();
+
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  StandaloneMasterDetector detector(master.get()->pid);
+
+  Future<UpdateSlaveMessage> updateSlaveMessage =
+    FUTURE_PROTOBUF(UpdateSlaveMessage(), _, _);
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.authenticate_http_readwrite = false;
+
+  constexpr SlaveInfo::Capability::Type capabilities[] = {
+    SlaveInfo::Capability::MULTI_ROLE,
+    SlaveInfo::Capability::HIERARCHICAL_ROLE,
+    SlaveInfo::Capability::RESERVATION_REFINEMENT,
+    SlaveInfo::Capability::RESOURCE_PROVIDER};
+
+  slaveFlags.agent_features = SlaveCapabilities();
+  foreach (SlaveInfo::Capability::Type type, capabilities) {
+    SlaveInfo::Capability* capability =
+      slaveFlags.agent_features->add_capabilities();
+    capability->set_type(type);
+  }
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(&detector, slaveFlags);
+  ASSERT_SOME(slave);
+
+  Clock::advance(slaveFlags.registration_backoff_factor);
+  Clock::settle();
+  AWAIT_READY(updateSlaveMessage);
+
+  v1::agent::Call v1Call;
+  v1Call.set_type(v1::agent::Call::GET_RESOURCE_PROVIDERS);
+
+  Future<v1::agent::Response> v1Response =
+    post(slave.get()->pid, v1Call, contentType);
+
+  AWAIT_READY(v1Response);
+  ASSERT_TRUE(v1Response->IsInitialized());
+  ASSERT_EQ(v1::agent::Response::GET_RESOURCE_PROVIDERS, v1Response->type());
+
+  EXPECT_TRUE(
+      v1Response->get_resource_providers().resource_providers().empty());
+
+  mesos::v1::ResourceProviderInfo info;
+  info.set_type("org.apache.mesos.rp.test");
+  info.set_name("test");
+
+  v1::MockResourceProvider resourceProvider(
+      info,
+      v1::createDiskResource(
+          "200", "*", None(), None(), v1::createDiskSourceRaw()));
+
+  // Start and register a resource provider.
+  string scheme = "http";
+
+#ifdef USE_SSL_SOCKET
+  if (process::network::openssl::flags().enabled) {
+    scheme = "https";
+  }
+#endif
+
+  http::URL url(
+      scheme,
+      slave.get()->pid.address.ip,
+      slave.get()->pid.address.port,
+      slave.get()->pid.id + "/api/v1/resource_provider");
+
+  Owned<EndpointDetector> endpointDetector(new ConstantEndpointDetector(url));
+
+  updateSlaveMessage = FUTURE_PROTOBUF(UpdateSlaveMessage(), _, _);
+
+  resourceProvider.start(endpointDetector, contentType, v1::DEFAULT_CREDENTIAL);
+
+  // Wait until the agent's resources have been updated to include the
+  // resource provider resources.
+  AWAIT_READY(updateSlaveMessage);
+
+  v1Response = post(slave.get()->pid, v1Call, contentType);
+
+  AWAIT_READY(v1Response);
+  ASSERT_TRUE(v1Response->IsInitialized());
+  ASSERT_EQ(v1::agent::Response::GET_RESOURCE_PROVIDERS, v1Response->type());
+
+  EXPECT_EQ(1, v1Response->get_resource_providers().resource_providers_size());
+
+  const mesos::v1::ResourceProviderInfo& responseInfo =
+    v1Response->get_resource_providers()
+      .resource_providers(0)
+      .resource_provider_info();
+
+  EXPECT_EQ(info.type(), responseInfo.type());
+  EXPECT_EQ(info.name(), responseInfo.name());
+}
+
+
 class AgentAPIStreamingTest
   : public MesosTest,
     public WithParamInterface<ContentType> {};