You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by zm...@apache.org on 2016/08/03 22:36:26 UTC

aurora git commit: Support TBinaryProtocol over HTTP

Repository: aurora
Updated Branches:
  refs/heads/master a071af345 -> 0105a151b


Support TBinaryProtocol over HTTP

This replaces the `TServlet` servlet from thrift with our own servlet which
dispatches thrift responses based on the content type of the request. This
enables a client to use either the thrift json protocol or the binary protocol
when communicating with the scheduler.

Without this patch the current behaviour is:
- Regardless of content type of the request, assume the request is json and
  return json with a `application/x-thrift` content type.

With this patch the behaviour becomes:
- A request with no content type header, or a type of `application/x-thrift` or
  `application/json` or `application/vnd.apache.thrift.json` is assumed to be
  JSON.
- A request with a content type header of `application/vnd.apache.thrift.binary`
  is assumed to be binary.
- A request with an `Accept` header of `application/vnd.apache.thrift.binary`
  will have a binary response.
- A request with any other `Accept` header will have a JSON response.

Bugs closed: AURORA-1743

Reviewed at https://reviews.apache.org/r/50685/


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

Branch: refs/heads/master
Commit: 0105a151b0431aec413b0f46021b21b993101424
Parents: a071af3
Author: Zameer Manji <zm...@apache.org>
Authored: Wed Aug 3 15:31:18 2016 -0700
Committer: Zameer Manji <zm...@apache.org>
Committed: Wed Aug 3 15:31:18 2016 -0700

----------------------------------------------------------------------
 RELEASE-NOTES.md                                |   6 +
 .../aurora/scheduler/http/api/ApiModule.java    |  62 ++++++-
 .../http/api/TContentAwareServlet.java          | 166 +++++++++++++++++++
 .../scheduler/http/AbstractJettyTest.java       |   6 +
 .../apache/aurora/scheduler/http/api/ApiIT.java |  70 +++++++-
 5 files changed, 304 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/0105a151/RELEASE-NOTES.md
----------------------------------------------------------------------
diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md
index 19d7f7e..d14c3d7 100644
--- a/RELEASE-NOTES.md
+++ b/RELEASE-NOTES.md
@@ -17,6 +17,12 @@
   from Mesos. This has affected rendering of some of the existing attributes. Furthermore, it now
   dumps additional offer attributes including [reservations](http://mesos.apache.org/documentation/latest/reservation/)
   and [persistent volumes](http://mesos.apache.org/documentation/latest/persistent-volume/).
+* The scheduler API now accepts both thrift JSON and binary thrift. If a request is sent with a
+  `Content-Type` header, or a `Content-Type` header of `application/x-thrift` or `application/json`
+  or `application/vnd.apache.thrift.json` the request is treated as thrift JSON. If a request is
+  sent with a `Content-Type` header of `application/vnd.apache.thrift.binary` the request is treated
+  as binary thrift. If the `Accept` header of the request is `application/vnd.apache.thrift.binary`
+  then the response will be binary thrift. Any other value for `Accept` will result in thrift JSON.
 
 ### Deprecations and removals:
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/0105a151/src/main/java/org/apache/aurora/scheduler/http/api/ApiModule.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/api/ApiModule.java b/src/main/java/org/apache/aurora/scheduler/http/api/ApiModule.java
index cd5adf9..e468209 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/api/ApiModule.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/api/ApiModule.java
@@ -14,6 +14,7 @@
 package org.apache.aurora.scheduler.http.api;
 
 import javax.inject.Singleton;
+import javax.ws.rs.core.MediaType;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.inject.Provides;
@@ -26,14 +27,24 @@ import org.apache.aurora.gen.AuroraAdmin;
 import org.apache.aurora.scheduler.http.CorsFilter;
 import org.apache.aurora.scheduler.http.JettyServerModule;
 import org.apache.aurora.scheduler.http.LeaderRedirectFilter;
+import org.apache.aurora.scheduler.http.api.TContentAwareServlet.ContentFactoryPair;
+import org.apache.aurora.scheduler.http.api.TContentAwareServlet.InputConfig;
+import org.apache.aurora.scheduler.http.api.TContentAwareServlet.OutputConfig;
 import org.apache.aurora.scheduler.thrift.aop.AnnotatedAuroraAdmin;
+import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TJSONProtocol;
-import org.apache.thrift.server.TServlet;
 import org.eclipse.jetty.servlet.DefaultServlet;
 import org.eclipse.jetty.util.resource.Resource;
 
+import static javax.ws.rs.core.MediaType.APPLICATION_JSON_TYPE;
+
 public class ApiModule extends ServletModule {
   public static final String API_PATH = "/api";
+  private static final MediaType GENERIC_THRIFT = new MediaType("application", "x-thrift");
+  private static final MediaType THRIFT_JSON =
+      new MediaType("application", "vnd.apache.thrift.json");
+  private static final MediaType THRIFT_BINARY =
+      new MediaType("application", "vnd.apache.thrift.binary");
 
   /**
    * Set the {@code Access-Control-Allow-Origin} header for API requests. See
@@ -52,7 +63,7 @@ public class ApiModule extends ServletModule {
     if (ENABLE_CORS_FOR.get() != null) {
       filter(API_PATH).through(new CorsFilter(ENABLE_CORS_FOR.get()));
     }
-    serve(API_PATH).with(TServlet.class);
+    serve(API_PATH).with(TContentAwareServlet.class);
 
     filter(ApiBeta.PATH, ApiBeta.PATH + "/*").through(LeaderRedirectFilter.class);
     filter(ApiBeta.PATH, ApiBeta.PATH + "/*")
@@ -69,8 +80,49 @@ public class ApiModule extends ServletModule {
 
   @Provides
   @Singleton
-  TServlet provideApiThriftServlet(AnnotatedAuroraAdmin schedulerThriftInterface) {
-    return new TServlet(
-        new AuroraAdmin.Processor<>(schedulerThriftInterface), new TJSONProtocol.Factory());
+  TContentAwareServlet provideApiThriftServlet(AnnotatedAuroraAdmin schedulerThriftInterface) {
+    /*
+     * For backwards compatibility the servlet is configured to assume `application/x-thrift` and
+     * `application/json` have TJSON bodies.
+     *
+     * Requests that have the registered MIME type for apache thrift are mapped to their respective
+     * protocols. See
+     * http://www.iana.org/assignments/media-types/application/vnd.apache.thrift.binary and
+     * http://www.iana.org/assignments/media-types/application/vnd.apache.thrift.json for details.
+     *
+     * Responses have the registered MIME type so the client can decode appropriately.
+     *
+     * The Accept header is used to determine the response type. By default JSON is sent for any
+     * value except for the binary thrift header.
+     */
+
+    ContentFactoryPair jsonFactory = new ContentFactoryPair(
+        new TJSONProtocol.Factory(),
+        THRIFT_JSON);
+    ContentFactoryPair binFactory = new ContentFactoryPair(
+        new TBinaryProtocol.Factory(),
+        THRIFT_BINARY);
+
+    // Which factory to use based on the Content-Type header of the request for reading the request.
+    InputConfig inputConfig = new InputConfig(GENERIC_THRIFT, ImmutableMap.of(
+        GENERIC_THRIFT, jsonFactory,
+        THRIFT_JSON, jsonFactory,
+        APPLICATION_JSON_TYPE, jsonFactory,
+        THRIFT_BINARY, binFactory
+    ));
+
+    // Which factory to use based on the Accept header of the request for the response.
+    OutputConfig outputConfig = new OutputConfig(APPLICATION_JSON_TYPE, ImmutableMap.of(
+        APPLICATION_JSON_TYPE, jsonFactory,
+        GENERIC_THRIFT, jsonFactory,
+        THRIFT_JSON, jsonFactory,
+        THRIFT_BINARY, binFactory
+        ));
+
+    // A request without a Content-Type (like from curl) should be treated as GENERIC_THRIFT
+    return new TContentAwareServlet(
+        new AuroraAdmin.Processor<>(schedulerThriftInterface),
+        inputConfig,
+        outputConfig);
   }
 }

http://git-wip-us.apache.org/repos/asf/aurora/blob/0105a151/src/main/java/org/apache/aurora/scheduler/http/api/TContentAwareServlet.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/api/TContentAwareServlet.java b/src/main/java/org/apache/aurora/scheduler/http/api/TContentAwareServlet.java
new file mode 100644
index 0000000..1634cb8
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/http/api/TContentAwareServlet.java
@@ -0,0 +1,166 @@
+/**
+ * Licensed 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.
+ */
+package org.apache.aurora.scheduler.http.api;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.Optional;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TIOStreamTransport;
+import org.apache.thrift.transport.TTransport;
+
+import static java.util.Objects.requireNonNull;
+
+import static javax.ws.rs.core.HttpHeaders.ACCEPT;
+
+/**
+ * An implementation of {@link org.apache.thrift.server.TServlet} that can handle multiple thrift
+ * protocols. The protocols are dispatched on HTTP headers.
+ */
+public class TContentAwareServlet extends HttpServlet {
+  private final TProcessor processor;
+  private final InputConfig inputConfig;
+  private final OutputConfig outputConfig;
+
+  /**
+   * Class which contains the mapping of the factory and the content type of the output.
+   */
+  static class ContentFactoryPair implements TProtocolFactory {
+    private final TProtocolFactory factory;
+
+    private final MediaType outputType;
+
+    ContentFactoryPair(TProtocolFactory factory, MediaType outputType) {
+      this.factory = requireNonNull(factory);
+      this.outputType = requireNonNull(outputType);
+    }
+
+    MediaType getOutputType() {
+      return outputType;
+    }
+
+    @Override
+    public TProtocol getProtocol(TTransport tTransport) {
+      return factory.getProtocol(tTransport);
+    }
+  }
+
+  /**
+   * Configures how to interpret the Content-Type of the request.
+   */
+  static class InputConfig {
+    // Type to use when there is no Content-Type
+    private final MediaType defaultType;
+    // Mapping of values in Content-Type to protocol to use to deserialize
+    private final Map<MediaType, ContentFactoryPair> inputMapping;
+
+    InputConfig(MediaType defaultType, Map<MediaType, ContentFactoryPair> inputMapping) {
+      this.defaultType = requireNonNull(defaultType);
+      this.inputMapping = requireNonNull(inputMapping);
+    }
+
+    Optional<ContentFactoryPair> getFactory(Optional<MediaType> mediaType) {
+      return Optional.ofNullable(inputMapping.get(mediaType.orElse(defaultType)));
+    }
+  }
+
+  /**
+   * Configures how to interpret the Accept header of the request. The defaultType's factory is
+   * returned for almost all values to maintain backwards compatibility.
+   */
+  static class OutputConfig {
+    // Type to use when there is no Accept header
+    private final MediaType defaultType;
+    // Mapping of MediaTypes in the Accept header to protocol used to serialize the response
+    private final Map<MediaType, ContentFactoryPair> outputMapping;
+    private final ContentFactoryPair defaultFactory;
+
+    OutputConfig(MediaType defaultType, Map<MediaType, ContentFactoryPair> outputMapping) {
+      this.defaultType = requireNonNull(defaultType);
+      this.outputMapping = requireNonNull(outputMapping);
+      this.defaultFactory = requireNonNull(outputMapping.get(defaultType));
+    }
+
+    ContentFactoryPair getFactory(Optional<MediaType> type) {
+      return Optional.ofNullable(outputMapping.get(type.orElse(defaultType)))
+          .orElse(defaultFactory);
+    }
+  }
+
+  TContentAwareServlet(TProcessor processor, InputConfig inputConfig, OutputConfig outputConfig) {
+    this.processor = requireNonNull(processor);
+    this.inputConfig = requireNonNull(inputConfig);
+    this.outputConfig = requireNonNull(outputConfig);
+  }
+
+  @Override
+  protected void doPost(HttpServletRequest request, HttpServletResponse response)
+      throws ServletException, IOException {
+
+    Optional<ContentFactoryPair> factoryOptional =
+        inputConfig.getFactory(Optional.of(request.getContentType()).map(MediaType::valueOf));
+
+    if (!factoryOptional.isPresent()) {
+      response.setStatus(HttpServletResponse.SC_UNSUPPORTED_MEDIA_TYPE);
+      String msg = "Unsupported Content-Type: " + request.getContentType();
+      response.getOutputStream().write(msg.getBytes(StandardCharsets.UTF_8));
+      return;
+    }
+
+    TTransport transport =
+        new TIOStreamTransport(request.getInputStream(), response.getOutputStream());
+
+    TProtocol inputProtocol = factoryOptional.get().getProtocol(transport);
+
+    Optional<String> acceptHeader = Optional.ofNullable(request.getHeader(ACCEPT));
+    Optional<MediaType> acceptType = Optional.empty();
+    if (acceptHeader.isPresent()) {
+      try {
+        acceptType = acceptHeader.map(MediaType::valueOf);
+      } catch (IllegalArgumentException e) {
+        // Thrown if the Accept header contains more than one type or something else we can't
+        // parse, we just treat is as no header (which will pick up the default value).
+        acceptType = Optional.empty();
+      }
+    }
+
+    ContentFactoryPair outputProtocolFactory = outputConfig.getFactory(acceptType);
+
+    response.setContentType(outputProtocolFactory.getOutputType().toString());
+    TProtocol outputProtocol = outputProtocolFactory.getProtocol(transport);
+    try {
+      processor.process(inputProtocol, outputProtocol);
+      response.getOutputStream().flush();
+    } catch (TException e) {
+      throw new ServletException(e);
+    }
+  }
+
+  @Override
+  protected void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws ServletException, IOException {
+    doPost(request, response);
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/0105a151/src/test/java/org/apache/aurora/scheduler/http/AbstractJettyTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/AbstractJettyTest.java b/src/test/java/org/apache/aurora/scheduler/http/AbstractJettyTest.java
index 7dbe48b..c2ceb4e 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/AbstractJettyTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/AbstractJettyTest.java
@@ -175,6 +175,12 @@ public abstract class AbstractJettyTest extends EasyMockTest {
     return String.format("http://%s:%s%s", httpServer.getHostText(), httpServer.getPort(), path);
   }
 
+  protected WebResource.Builder getPlainRequestBuilder(String path) {
+    assertNotNull("HTTP server must be started first", httpServer);
+    Client client = Client.create(new DefaultClientConfig());
+    return client.resource(makeUrl(path)).getRequestBuilder();
+  }
+
   protected WebResource.Builder getRequestBuilder(String path) {
     assertNotNull("HTTP server must be started first", httpServer);
     ClientConfig config = new DefaultClientConfig();

http://git-wip-us.apache.org/repos/asf/aurora/blob/0105a151/src/test/java/org/apache/aurora/scheduler/http/api/ApiIT.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/ApiIT.java b/src/test/java/org/apache/aurora/scheduler/http/api/ApiIT.java
index 31f5cb3..0a3ff05 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/ApiIT.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/ApiIT.java
@@ -13,8 +13,14 @@
  */
 package org.apache.aurora.scheduler.http.api;
 
+import java.util.Arrays;
+import java.util.List;
+
 import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.primitives.Bytes;
 import com.google.inject.AbstractModule;
 import com.google.inject.Module;
 import com.google.inject.util.Modules;
@@ -26,10 +32,15 @@ import org.apache.aurora.scheduler.thrift.aop.AnnotatedAuroraAdmin;
 import org.junit.Before;
 import org.junit.Test;
 
+import static javax.servlet.http.HttpServletResponse.SC_OK;
+import static javax.servlet.http.HttpServletResponse.SC_UNSUPPORTED_MEDIA_TYPE;
+import static javax.ws.rs.core.HttpHeaders.CONTENT_TYPE;
+
 import static org.easymock.EasyMock.expect;
 import static org.junit.Assert.assertEquals;
 
 public class ApiIT extends AbstractJettyTest {
+  private static final String JSON_FIXTURE = "[1,\"getRoleSummary\",1,0,{}]";
   private AnnotatedAuroraAdmin thrift;
 
   @Before
@@ -58,8 +69,65 @@ public class ApiIT extends AbstractJettyTest {
     ClientResponse response = getRequestBuilder(ApiModule.API_PATH)
         .header(HttpHeaders.ACCEPT_ENCODING, "gzip")
         .type("application/x-thrift")
-        .post(ClientResponse.class, "[1,\"getRoleSummary\",1,0,{}]");
+        .post(ClientResponse.class, JSON_FIXTURE);
 
+    assertEquals(SC_OK, response.getStatus());
     assertEquals("gzip", response.getHeaders().getFirst(HttpHeaders.CONTENT_ENCODING));
   }
+
+  @Test
+  public void testThriftJsonAccepted() throws Exception {
+    expect(thrift.getRoleSummary()).andReturn(new Response());
+
+    replayAndStart();
+
+    ClientResponse response = getPlainRequestBuilder(ApiModule.API_PATH)
+        .type("application/vnd.apache.thrift.json")
+        .accept("application/vnd.apache.thrift.json")
+        .post(ClientResponse.class, JSON_FIXTURE);
+
+    assertEquals(SC_OK, response.getStatus());
+    assertEquals(
+        "application/vnd.apache.thrift.json",
+        response.getHeaders().getFirst(CONTENT_TYPE));
+  }
+
+  @Test
+  public void testUnknownContentTypeRejected() throws Exception {
+    replayAndStart();
+
+    ClientResponse response = getRequestBuilder(ApiModule.API_PATH)
+        .type(MediaType.TEXT_HTML_TYPE)
+        .post(ClientResponse.class, JSON_FIXTURE);
+
+    assertEquals(SC_UNSUPPORTED_MEDIA_TYPE, response.getStatus());
+  }
+
+  @Test
+  public void testBinaryContentTypeAccepted() throws Exception {
+    expect(thrift.getRoleSummary()).andReturn(new Response());
+
+    replayAndStart();
+
+    // This fixture represents a 'getRoleSummary' call encoded as binary thrift.
+    List<Integer> fixture = ImmutableList.<Integer>builder()
+        .addAll(ImmutableList.of(-128, 1, 0, 1, 0, 0, 0, 14, 103))
+        .addAll(ImmutableList.of(101, 116, 82, 111, 108, 101, 83, 117, 109))
+        .addAll(ImmutableList.of(109, 97, 114, 121, 0, 0, 0, 1, 0))
+        .addAll(ImmutableList.of(0, 0, 0, 0, 0))
+        .build();
+
+    // Note the array has to be exactly 27 bytes long.
+    byte[] rawBytes = Arrays.copyOf(Bytes.toArray(fixture), 27);
+
+    ClientResponse response = getPlainRequestBuilder(ApiModule.API_PATH)
+        .type("application/vnd.apache.thrift.binary")
+        .accept("application/vnd.apache.thrift.binary")
+        .post(ClientResponse.class, rawBytes);
+
+    assertEquals(SC_OK, response.getStatus());
+    assertEquals(
+        "application/vnd.apache.thrift.binary",
+        response.getHeaders().getFirst(CONTENT_TYPE));
+  }
 }