From ee370d389350b5a6e8afa89a67995fa8b8dbde55 Mon Sep 17 00:00:00 2001 From: Alex Diachenko Date: Thu, 20 Dec 2018 14:24:05 -0800 Subject: [PATCH] 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 --- .../connect/runtime/rest/RestServer.java | 2 + .../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 15386430bc5..c0d83f29103 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; @@ -171,6 +172,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 c66ce36d8b0..a0fb685aff1 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; @@ -158,6 +157,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 @@ -175,12 +201,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();