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 2015/09/22 00:05:16 UTC

[1/3] mesos git commit: Added devolve operation for V1 executor protobuf.

Repository: mesos
Updated Branches:
  refs/heads/master 57361f10c -> 870801a9b


Added devolve operation for V1 executor protobuf.

This change adds a trivial devolve call for V1 executor protobuf ->
unversioned protobuf that can be used internally.

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


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

Branch: refs/heads/master
Commit: 067df4b9265550d3623af3932980ed9b3bc5cb67
Parents: 57361f1
Author: Anand Mazumdar <ma...@gmail.com>
Authored: Mon Sep 21 14:57:57 2015 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Mon Sep 21 15:05:05 2015 -0700

----------------------------------------------------------------------
 src/internal/devolve.cpp | 6 ++++++
 src/internal/devolve.hpp | 5 +++++
 2 files changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/067df4b9/src/internal/devolve.cpp
----------------------------------------------------------------------
diff --git a/src/internal/devolve.cpp b/src/internal/devolve.cpp
index 6cace66..e2bb22b 100644
--- a/src/internal/devolve.cpp
+++ b/src/internal/devolve.cpp
@@ -111,6 +111,12 @@ Credential devolve(const v1::Credential& credential)
 }
 
 
+executor::Call devolve(const v1::executor::Call& call)
+{
+  return devolve<executor::Call>(call);
+}
+
+
 scheduler::Call devolve(const v1::scheduler::Call& call)
 {
   return devolve<scheduler::Call>(call);

http://git-wip-us.apache.org/repos/asf/mesos/blob/067df4b9/src/internal/devolve.hpp
----------------------------------------------------------------------
diff --git a/src/internal/devolve.hpp b/src/internal/devolve.hpp
index f03cac5..f275d3d 100644
--- a/src/internal/devolve.hpp
+++ b/src/internal/devolve.hpp
@@ -23,10 +23,14 @@
 
 #include <mesos/mesos.hpp>
 
+#include <mesos/executor/executor.hpp>
+
 #include <mesos/scheduler/scheduler.hpp>
 
 #include <mesos/v1/mesos.hpp>
 
+#include <mesos/v1/executor/executor.hpp>
+
 #include <mesos/v1/scheduler/scheduler.hpp>
 
 #include <stout/foreach.hpp>
@@ -47,6 +51,7 @@ Credential devolve(const v1::Credential& credential);
 scheduler::Call devolve(const v1::scheduler::Call& call);
 scheduler::Event devolve(const v1::scheduler::Event& event);
 
+executor::Call devolve(const v1::executor::Call& call);
 
 // Helper for repeated field devolving to 'T1' from 'T2'.
 template <typename T1, typename T2>


[3/3] mesos git commit: Added tests for executor endpoint on agent.

Posted by vi...@apache.org.
Added tests for executor endpoint on agent.

This change introduces basic validation tests for the executor HTTP
endpoint on agent.

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


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

Branch: refs/heads/master
Commit: 870801a9b4b857744e1e9c040d7ac7522dd98f1c
Parents: 5080508
Author: Anand Mazumdar <ma...@gmail.com>
Authored: Mon Sep 21 15:02:41 2015 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Mon Sep 21 15:05:06 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                       |   1 +
 src/tests/executor_http_api_tests.cpp | 401 +++++++++++++++++++++++++++++
 2 files changed, 402 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/870801a9/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 834f69a..e224060 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1674,6 +1674,7 @@ mesos_tests_SOURCES =						\
   tests/environment.cpp						\
   tests/examples_tests.cpp					\
   tests/exception_tests.cpp					\
+  tests/executor_http_api_tests.cpp				\
   tests/health_check_tests.cpp					\
   tests/fault_tolerance_tests.cpp				\
   tests/fetcher_cache_tests.cpp					\

http://git-wip-us.apache.org/repos/asf/mesos/blob/870801a9/src/tests/executor_http_api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/executor_http_api_tests.cpp b/src/tests/executor_http_api_tests.cpp
new file mode 100644
index 0000000..9dbc519
--- /dev/null
+++ b/src/tests/executor_http_api_tests.cpp
@@ -0,0 +1,401 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <string>
+
+#include <mesos/v1/executor/executor.hpp>
+
+#include <mesos/http.hpp>
+
+#include <process/future.hpp>
+#include <process/gtest.hpp>
+#include <process/http.hpp>
+#include <process/pid.hpp>
+
+#include "common/http.hpp"
+
+#include "master/master.hpp"
+
+#include "tests/mesos.hpp"
+
+using mesos::internal::master::Master;
+
+using mesos::internal::slave::Slave;
+
+using mesos::v1::executor::Call;
+
+using process::Future;
+using process::PID;
+
+using process::http::BadRequest;
+using process::http::MethodNotAllowed;
+using process::http::NotAcceptable;
+using process::http::OK;
+using process::http::Response;
+using process::http::UnsupportedMediaType;
+
+using std::string;
+
+using testing::WithParamInterface;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+
+class ExecutorHttpApiTest
+  : public MesosTest,
+    public WithParamInterface<ContentType> {};
+
+
+// The tests are parameterized by the content type of the request.
+INSTANTIATE_TEST_CASE_P(
+    ContentType,
+    ExecutorHttpApiTest,
+    ::testing::Values(ContentType::PROTOBUF, ContentType::JSON));
+
+
+// TODO(anand): Add more validation tests for:
+// - If the slave is still recovering, the call should return
+//   ServiceUnavailable.
+// - If Executor is not found, the call should return
+//   BadRequest.
+// - If Executor has not registered and sends a Call message other
+//   than Subscribe, the call should return Forbidden.
+
+
+// This test expects a BadRequest when 'Content-Type' is omitted.
+TEST_F(ExecutorHttpApiTest, NoContentType)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Try<PID<Slave>> slave = StartSlave();
+  ASSERT_SOME(slave);
+
+  // Wait for recovery to be complete.
+  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  AWAIT_READY(__recover);
+
+  Call call;
+  call.set_type(Call::MESSAGE);
+
+  call.mutable_message()->set_data("hello world");
+  call.mutable_framework_id()->set_value("dummy_framework_id");
+  call.mutable_executor_id()->set_value("dummy_executor_id");
+
+  Future<Response> response = process::http::post(
+      slave.get(),
+      "api/v1/executor",
+      None(),
+      serialize(ContentType::JSON, call),
+      None());
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(BadRequest().status, response);
+
+  Shutdown();
+}
+
+
+// This test sends a valid JSON blob that cannot be deserialized
+// into a valid protobuf resulting in a BadRequest.
+TEST_F(ExecutorHttpApiTest, ValidJsonButInvalidProtobuf)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Try<PID<Slave>> slave = StartSlave();
+  ASSERT_SOME(slave);
+
+  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  AWAIT_READY(__recover);
+
+  JSON::Object object;
+  object.values["string"] = "valid_json";
+
+  hashmap<string, string> headers;
+  headers["Accept"] = APPLICATION_JSON;
+
+  Future<Response> response = process::http::post(
+      slave.get(),
+      "api/v1/executor",
+      headers,
+      stringify(object),
+      APPLICATION_JSON);
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(BadRequest().status, response);
+
+  Shutdown();
+}
+
+
+// This test sends a malformed body that cannot be deserialized
+// into a valid protobuf resulting in a BadRequest.
+TEST_P(ExecutorHttpApiTest, MalformedContent)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Try<PID<Slave>> slave = StartSlave();
+  ASSERT_SOME(slave);
+
+  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  AWAIT_READY(__recover);
+
+  const string body = "MALFORMED_CONTENT";
+
+  const ContentType contentType = GetParam();
+  hashmap<string, string> headers;
+  headers["Accept"] = stringify(contentType);
+
+  Future<Response> response = process::http::post(
+      slave.get(),
+      "api/v1/executor",
+      headers,
+      body,
+      stringify(contentType));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(BadRequest().status, response);
+
+  Shutdown();
+}
+
+
+// This test sets an unsupported media type as Content-Type. This
+// should result in a 415 (UnsupportedMediaType) response.
+TEST_P(ExecutorHttpApiTest, UnsupportedContentMediaType)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Try<PID<Slave>> slave = StartSlave();
+  ASSERT_SOME(slave);
+
+  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  AWAIT_READY(__recover);
+
+  ContentType contentType = GetParam();
+  hashmap<string, string> headers;
+  headers["Accept"] = stringify(contentType);
+
+  Call call;
+  call.set_type(Call::SUBSCRIBE);
+
+  call.mutable_framework_id()->set_value("dummy_framework_id");
+  call.mutable_executor_id()->set_value("dummy_executor_id");
+
+  const string unknownMediaType = "application/unknown-media-type";
+
+  Future<Response> response = process::http::post(
+      slave.get(),
+      "api/v1/executor",
+      headers,
+      serialize(contentType, call),
+      unknownMediaType);
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(UnsupportedMediaType().status, response);
+
+  Shutdown();
+}
+
+
+// This test sends a Call from an unknown FrameworkID. The call
+// should return a BadRequest.
+TEST_P(ExecutorHttpApiTest, MessageFromUnknownFramework)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Try<PID<Slave>> slave = StartSlave();
+  ASSERT_SOME(slave);
+
+  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  AWAIT_READY(__recover);
+
+  ContentType contentType = GetParam();
+  hashmap<string, string> headers;
+  headers["Accept"] = stringify(contentType);
+
+  Call call;
+  call.set_type(Call::MESSAGE);
+
+  call.mutable_message()->set_data("hello world");
+  call.mutable_framework_id()->set_value("dummy_framework_id");
+  call.mutable_executor_id()->set_value("dummy_executor_id");
+
+  Future<Response> response = process::http::post(
+      slave.get(),
+      "api/v1/executor",
+      headers,
+      serialize(contentType, call),
+      stringify(contentType));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(BadRequest().status, response);
+
+  Shutdown();
+}
+
+
+// This test sends a GET request to the executor HTTP endpoint instead
+// of a POST. The call should return a MethodNotAllowed response.
+TEST_F(ExecutorHttpApiTest, GetRequest)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Try<PID<Slave>> slave = StartSlave();
+  ASSERT_SOME(slave);
+
+  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  AWAIT_READY(__recover);
+
+  Future<Response> response = process::http::get(
+      slave.get(),
+      "api/v1/executor");
+
+  AWAIT_READY(response);
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(MethodNotAllowed().status, response);
+
+  Shutdown();
+}
+
+
+// This test sends in a Accept:*/* header meaning it would Accept any
+// media type as response. We return the default "application/json"
+// media type as response.
+TEST_P(ExecutorHttpApiTest, DefaultAccept)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Try<PID<Slave>> slave = StartSlave();
+  ASSERT_SOME(slave);
+
+  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  AWAIT_READY(__recover);
+
+  hashmap<string, string> headers;
+  headers["Accept"] = "*/*";
+
+  // Only subscribe needs to 'Accept' JSON or protobuf.
+  Call call;
+  call.set_type(Call::SUBSCRIBE);
+
+  call.mutable_framework_id()->set_value("dummy_framework_id");
+  call.mutable_executor_id()->set_value("dummy_executor_id");
+
+  // Retrieve the parameter passed as content type to this test.
+  const ContentType contentType = GetParam();
+
+  Future<Response> response = process::http::streaming::post(
+      slave.get(),
+      "api/v1/executor",
+      headers,
+      serialize(contentType, call),
+      stringify(contentType));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+  EXPECT_SOME_EQ(APPLICATION_JSON, response.get().headers.get("Content-Type"));
+
+  Shutdown();
+}
+
+
+// This test does not set any Accept header for the subscribe call.
+// The default response media type should be "application/json" in
+// this case.
+TEST_P(ExecutorHttpApiTest, NoAcceptHeader)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Try<PID<Slave>> slave = StartSlave();
+  ASSERT_SOME(slave);
+
+  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  AWAIT_READY(__recover);
+
+  // Retrieve the parameter passed as content type to this test.
+  const ContentType contentType = GetParam();
+
+  // No 'Accept' header leads to all media types considered
+  // acceptable. JSON will be chosen by default.
+  hashmap<string, string> headers;
+
+  // Only subscribe needs to 'Accept' JSON or protobuf.
+  Call call;
+  call.set_type(Call::SUBSCRIBE);
+
+  call.mutable_framework_id()->set_value("dummy_framework_id");
+  call.mutable_executor_id()->set_value("dummy_executor_id");
+
+  Future<Response> response = process::http::streaming::post(
+      slave.get(),
+      "api/v1/executor",
+      headers,
+      serialize(contentType, call),
+      stringify(contentType));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+  EXPECT_SOME_EQ(APPLICATION_JSON, response.get().headers.get("Content-Type"));
+
+  Shutdown();
+}
+
+
+// This test sends a unsupported Accept media type for the Accept
+// header. The response should be NotAcceptable in this case.
+TEST_P(ExecutorHttpApiTest, NotAcceptable)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Try<PID<Slave>> slave = StartSlave();
+  ASSERT_SOME(slave);
+
+  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  AWAIT_READY(__recover);
+
+  // Retrieve the parameter passed as content type to this test.
+  const ContentType contentType = GetParam();
+
+  hashmap<string, string> headers;
+  headers["Accept"] = "foo";
+
+  // Only subscribe needs to 'Accept' JSON or protobuf.
+  Call call;
+  call.set_type(Call::SUBSCRIBE);
+
+  call.mutable_framework_id()->set_value("dummy_framework_id");
+  call.mutable_executor_id()->set_value("dummy_executor_id");
+
+  Future<Response> response = process::http::streaming::post(
+      slave.get(),
+      "api/v1/executor",
+      headers,
+      serialize(contentType, call),
+      stringify(contentType));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(NotAcceptable().status, response);
+
+  Shutdown();
+}
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {


[2/3] mesos git commit: Introduced executor HTTP endpoint on agent.

Posted by vi...@apache.org.
Introduced executor HTTP endpoint on agent.

This change introduces a stub endpoint on agent. As of now, it doesn't
do much except validating the `Content-Type`, `Accept` headers among
other trivial validations. Most of the functionality already existed in
`src/master/http.cpp`.

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


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

Branch: refs/heads/master
Commit: 50805083d4045126fc44ff2d027428a04fc2e636
Parents: 067df4b
Author: Anand Mazumdar <ma...@gmail.com>
Authored: Mon Sep 21 15:01:50 2015 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Mon Sep 21 15:05:06 2015 -0700

----------------------------------------------------------------------
 src/slave/http.cpp  | 137 +++++++++++++++++++++++++++++++++++++++++++++++
 src/slave/slave.cpp |   7 +++
 src/slave/slave.hpp |   5 ++
 3 files changed, 149 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/50805083/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 101aa06..12a4d39 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -22,6 +22,10 @@
 #include <string>
 #include <vector>
 
+#include <mesos/executor/executor.hpp>
+
+#include <mesos/v1/executor/executor.hpp>
+
 #include <mesos/type_utils.hpp>
 
 #include <process/help.hpp>
@@ -41,6 +45,8 @@
 #include "common/build.hpp"
 #include "common/http.hpp"
 
+#include "internal/devolve.hpp"
+
 #include "mesos/mesos.hpp"
 #include "mesos/resources.hpp"
 
@@ -55,8 +61,17 @@ using process::Owned;
 using process::TLDR;
 using process::USAGE;
 
+using process::http::Accepted;
+using process::http::BadRequest;
+using process::http::Forbidden;
 using process::http::InternalServerError;
+using process::http::MethodNotAllowed;
+using process::http::NotAcceptable;
+using process::http::NotImplemented;
 using process::http::OK;
+using process::http::Pipe;
+using process::http::ServiceUnavailable;
+using process::http::UnsupportedMediaType;
 
 using process::metrics::internal::MetricsProcess;
 
@@ -182,6 +197,128 @@ void Slave::Http::log(const Request& request)
 }
 
 
+const string Slave::Http::EXECUTOR_HELP = HELP(
+    TLDR(
+        "Endpoint for the Executor HTTP API."),
+    DESCRIPTION(
+        "This endpoint is used by the executors to interact with the ",
+        "agent via Call/Event messages."
+        "Returns 200 OK iff the initial SUBSCRIBE Call is successful."
+        "This would result in a streaming response via chunked "
+        "transfer encoding. The executors can process the response "
+        "incrementally."
+        "Returns 202 Accepted for all other Call messages iff the "
+        "request is accepted."));
+
+
+Future<Response> Slave::Http::executor(const Request& request) const
+{
+  // TODO(anand): Add metrics for rejected requests.
+
+  if (slave->state == Slave::RECOVERING) {
+    return ServiceUnavailable("Agent has not finished recovery");
+  }
+
+  if (request.method != "POST") {
+    return MethodNotAllowed(
+        "Expecting a 'POST' request, received '" + request.method + "'");
+  }
+
+  v1::executor::Call v1Call;
+
+  Option<string> contentType = request.headers.get("Content-Type");
+  if (contentType.isNone()) {
+    return BadRequest("Expecting 'Content-Type' to be present");
+  }
+
+  if (contentType.get() == APPLICATION_PROTOBUF) {
+    if (!v1Call.ParseFromString(request.body)) {
+      return BadRequest("Failed to parse body into Call protobuf");
+    }
+  } else if (contentType.get() == APPLICATION_JSON) {
+    Try<JSON::Value> value = JSON::parse(request.body);
+    if (value.isError()) {
+      return BadRequest("Failed to parse body into JSON: " + value.error());
+    }
+
+    Try<v1::executor::Call> parse =
+      ::protobuf::parse<v1::executor::Call>(value.get());
+
+    if (parse.isError()) {
+      return BadRequest("Failed to convert JSON into Call protobuf: " +
+                        parse.error());
+    }
+
+    v1Call = parse.get();
+  } else {
+    return UnsupportedMediaType(
+        string("Expecting 'Content-Type' of ") +
+        APPLICATION_JSON + " or " + APPLICATION_PROTOBUF);
+  }
+
+  const executor::Call call = devolve(v1Call);
+
+  // TODO(anand): Validate the protobuf (MESOS-2906) before proceeding
+  // further.
+
+  if (call.type() == executor::Call::SUBSCRIBE) {
+    // We default to JSON since an empty 'Accept' header
+    // results in all media types considered acceptable.
+    ContentType responseContentType;
+
+    if (request.acceptsMediaType(APPLICATION_JSON)) {
+      responseContentType = ContentType::JSON;
+    } else if (request.acceptsMediaType(APPLICATION_PROTOBUF)) {
+      responseContentType = ContentType::PROTOBUF;
+    } else {
+      return NotAcceptable(
+          string("Expecting 'Accept' to allow ") +
+          "'" + APPLICATION_PROTOBUF + "' or '" + APPLICATION_JSON + "'");
+    }
+
+    Pipe pipe;
+    OK ok;
+    ok.headers["Content-Type"] = stringify(responseContentType);
+
+    ok.type = Response::PIPE;
+    ok.reader = pipe.reader();
+
+    return ok;
+  }
+
+
+  // We consolidate the framework/executor lookup logic here because
+  // it is common for all the call handlers.
+  Framework* framework = slave->getFramework(call.framework_id());
+  if (framework == NULL) {
+    return BadRequest("Framework cannot be found");
+  }
+
+  Executor* executor = framework->getExecutor(call.executor_id());
+  if (executor == NULL) {
+    return BadRequest("Executor cannot be found");
+  }
+
+  if (executor->state == Executor::REGISTERING) {
+    return Forbidden("Executor is not subscribed");
+  }
+
+  switch (call.type()) {
+    case executor::Call::UPDATE:
+      return Accepted();
+
+    case executor::Call::MESSAGE:
+      return Accepted();
+
+    default:
+      // Should be caught during call validation above.
+      LOG(FATAL) << "Unexpected " << call.type() << " call";
+  }
+
+  return NotImplemented();
+}
+
+
 const string Slave::Http::HEALTH_HELP = HELP(
     TLDR(
         "Health check of the Slave."),

http://git-wip-us.apache.org/repos/asf/mesos/blob/50805083/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index ad710d7..29865ec 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -494,6 +494,13 @@ void Slave::initialize()
   // Setup HTTP routes.
   Http http = Http(this);
 
+  route("/api/v1/executor",
+        Http::EXECUTOR_HELP,
+        [http](const process::http::Request& request) {
+          Http::log(request);
+          return http.executor(request);
+        });
+
   // TODO(ijimenez): Remove this endpoint at the end of the
   // deprecation cycle on 0.26.
   route("/state.json",

http://git-wip-us.apache.org/repos/asf/mesos/blob/50805083/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 32e1830..7a54fad 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -399,6 +399,10 @@ private:
     // desired request handler to get consistent request logging.
     static void log(const process::http::Request& request);
 
+    // /slave/api/v1/executor
+    process::Future<process::http::Response> executor(
+        const process::http::Request& request) const;
+
     // /slave/health
     process::Future<process::http::Response> health(
         const process::http::Request& request) const;
@@ -407,6 +411,7 @@ private:
     process::Future<process::http::Response> state(
         const process::http::Request& request) const;
 
+    static const std::string EXECUTOR_HELP;
     static const std::string HEALTH_HELP;
     static const std::string STATE_HELP;