From commits-return-10911-apmail-kafka-commits-archive=kafka.apache.org@kafka.apache.org Thu Dec 20 22:29:46 2018 Return-Path: X-Original-To: apmail-kafka-commits-archive@www.apache.org Delivered-To: apmail-kafka-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 68B9E181F5 for ; Thu, 20 Dec 2018 22:29:40 +0000 (UTC) Received: (qmail 46315 invoked by uid 500); 20 Dec 2018 22:29:40 -0000 Delivered-To: apmail-kafka-commits-archive@kafka.apache.org Received: (qmail 46274 invoked by uid 500); 20 Dec 2018 22:29:40 -0000 Mailing-List: contact commits-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kafka.apache.org Delivered-To: mailing list commits@kafka.apache.org Received: (qmail 46263 invoked by uid 99); 20 Dec 2018 22:29:40 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 20 Dec 2018 22:29:40 +0000 Received: by gitbox.apache.org (ASF Mail Server at gitbox.apache.org, from userid 33) id 903C78528A; Thu, 20 Dec 2018 22:29:39 +0000 (UTC) Date: Thu, 20 Dec 2018 22:29:38 +0000 To: "commits@kafka.apache.org" Subject: [kafka] branch 2.0 updated: KAFKA-7759; Disable WADL output in the Connect REST API (#6051) MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit Message-ID: <154534497771.17786.15046842271810112421@gitbox.apache.org> From: jgus@apache.org X-Git-Host: gitbox.apache.org X-Git-Repo: kafka X-Git-Refname: refs/heads/2.0 X-Git-Reftype: branch X-Git-Oldrev: ea9cb3ce4cdce31be4431c7db9f83b0335a261a3 X-Git-Newrev: 9736f97d2f6d6a10d31ae7449558a28b08f26dcc X-Git-Rev: 9736f97d2f6d6a10d31ae7449558a28b08f26dcc X-Git-NotificationType: ref_changed_plus_diff X-Git-Multimail-Version: 1.5.dev Auto-Submitted: auto-generated This is an automated email from the ASF dual-hosted git repository. jgus pushed a commit to branch 2.0 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/2.0 by this push: new 9736f97 KAFKA-7759; Disable WADL output in the Connect REST API (#6051) 9736f97 is described below commit 9736f97d2f6d6a10d31ae7449558a28b08f26dcc Author: Alex Diachenko AuthorDate: Thu Dec 20 14:24:05 2018 -0800 KAFKA-7759; Disable WADL output in the Connect REST API (#6051) This patch disables support for WADL output in the Connect REST API since it was never intended to be exposed. Reviewers: Randall Hauch , Jason Gustafson --- .../kafka/connect/runtime/rest/RestServer.java | 2 ++ .../kafka/connect/runtime/rest/RestServerTest.java | 39 +++++++++++++++++----- 2 files changed, 33 insertions(+), 8 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java index 5a589db..73997a5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java @@ -45,6 +45,7 @@ import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.servlets.CrossOriginFilter; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.glassfish.jersey.server.ResourceConfig; +import org.glassfish.jersey.server.ServerProperties; import org.glassfish.jersey.servlet.ServletContainer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -170,6 +171,7 @@ public class RestServer { resourceConfig.register(new ConnectorPluginsResource(herder)); resourceConfig.register(ConnectExceptionMapper.class); + resourceConfig.property(ServerProperties.WADL_FEATURE_DISABLE, true); registerRestExtensions(herder, resourceConfig); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java index 2f8704a..892693a 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java @@ -25,7 +25,6 @@ import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.util.Callback; import org.easymock.Capture; import org.easymock.EasyMock; -import org.easymock.IAnswer; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -41,11 +40,11 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; - import javax.ws.rs.client.Client; import javax.ws.rs.client.ClientBuilder; import javax.ws.rs.client.Invocation; import javax.ws.rs.client.WebTarget; +import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import static org.junit.Assert.assertEquals; @@ -155,6 +154,33 @@ public class RestServerTest { Assert.assertEquals("http://my-hostname:8080/", server.advertisedUrl().toString()); } + @Test + public void testOptionsDoesNotIncludeWadlOutput() { + Map configMap = new HashMap<>(baseWorkerProps()); + DistributedConfig workerConfig = new DistributedConfig(configMap); + + EasyMock.expect(herder.plugins()).andStubReturn(plugins); + EasyMock.expect(plugins.newPlugins(Collections.emptyList(), + workerConfig, + ConnectRestExtension.class)) + .andStubReturn(Collections.emptyList()); + PowerMock.replayAll(); + + server = new RestServer(workerConfig); + server.start(herder); + + Response response = request("/connectors") + .accept(MediaType.WILDCARD) + .options(); + Assert.assertEquals(MediaType.TEXT_PLAIN_TYPE, response.getMediaType()); + Assert.assertArrayEquals( + response.getAllowedMethods().toArray(new String[0]), + response.readEntity(String.class).split(", ") + ); + + PowerMock.verifyAll(); + } + public void checkCORSRequest(String corsDomain, String origin, String expectedHeader, String method) { // To be able to set the Origin, we need to toggle this flag @@ -172,12 +198,9 @@ public class RestServerTest { final Capture>> connectorsCallback = EasyMock.newCapture(); herder.connectors(EasyMock.capture(connectorsCallback)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - connectorsCallback.getValue().onCompletion(null, Arrays.asList("a", "b")); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + connectorsCallback.getValue().onCompletion(null, Arrays.asList("a", "b")); + return null; }); PowerMock.replayAll();