flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From trohrm...@apache.org
Subject flink git commit: [FLINK-7529] Retrieve complete REST address from gateway
Date Tue, 19 Sep 2017 22:38:45 GMT
Repository: flink
Updated Branches:
  refs/heads/master 6a62f1455 -> 327701032


[FLINK-7529] Retrieve complete REST address from gateway

With this change, the complete REST address (protocol://hostname:port) is retrieved
from the RestfulGateway. That way we decouple the RestHandlers from the underlying
RestServerEndpoint/WebRuntimeMonitor because they no longer have to know whether
HTTPs is enabled or not.

This closes #4599.


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

Branch: refs/heads/master
Commit: 32770103253e01cd61c8634378cfa1b26707e19a
Parents: 6a62f14
Author: Till Rohrmann <trohrmann@apache.org>
Authored: Thu Aug 17 16:55:03 2017 +0200
Committer: Till Rohrmann <trohrmann@apache.org>
Committed: Wed Sep 20 00:38:11 2017 +0200

----------------------------------------------------------------------
 .../runtime/webmonitor/RuntimeMonitorHandler.java     | 10 +++-------
 .../flink/runtime/webmonitor/WebRuntimeMonitor.java   | 14 ++++----------
 .../webmonitor/files/StaticFileServerHandler.java     |  5 ++---
 .../webmonitor/handlers/TaskManagerLogHandler.java    |  3 +--
 .../webmonitor/utils/WebFrontendBootstrap.java        |  4 +++-
 .../flink/runtime/webmonitor/RedirectHandlerTest.java | 12 +++++-------
 .../handlers/TaskManagerLogHandlerTest.java           |  3 ---
 .../apache/flink/runtime/rest/RestServerEndpoint.java | 10 ++++++++--
 .../runtime/rest/handler/AbstractRestHandler.java     |  3 +--
 .../flink/runtime/rest/handler/RedirectHandler.java   | 12 +++---------
 .../rest/handler/util/HandlerRedirectUtils.java       |  5 ++---
 .../apache/flink/runtime/jobmanager/JobManager.scala  |  1 +
 .../apache/flink/runtime/rest/RestEndpointITCase.java | 13 +++++--------
 13 files changed, 38 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
index cdcd0fe..b393021 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
@@ -39,7 +39,6 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Routed;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.net.InetSocketAddress;
 import java.net.URLDecoder;
 import java.util.HashMap;
 import java.util.Map;
@@ -71,10 +70,9 @@ public class RuntimeMonitorHandler extends RedirectHandler<JobManagerGateway>
im
 			RequestHandler handler,
 			GatewayRetriever<JobManagerGateway> retriever,
 			CompletableFuture<String> localJobManagerAddressFuture,
-			Time timeout,
-			boolean httpsEnabled) {
+			Time timeout) {
 
-		super(localJobManagerAddressFuture, retriever, timeout, httpsEnabled);
+		super(localJobManagerAddressFuture, retriever, timeout);
 		this.handler = checkNotNull(handler);
 		this.allowOrigin = cfg.getAllowOrigin();
 	}
@@ -99,9 +97,7 @@ public class RuntimeMonitorHandler extends RedirectHandler<JobManagerGateway>
im
 				pathParams.put(key, URLDecoder.decode(routed.pathParams().get(key), ENCODING.toString()));
 			}
 
-			InetSocketAddress address = (InetSocketAddress) ctx.channel().localAddress();
-			queryParams.put(WEB_MONITOR_ADDRESS_KEY,
-				(httpsEnabled ? "https://" : "http://") + address.getHostName() + ":" + address.getPort());
+			queryParams.put(WEB_MONITOR_ADDRESS_KEY, localAddressFuture.get());
 
 			responseFuture = handler.handleRequest(pathParams, queryParams, jobManagerGateway);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
index a15c312..71e1593 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
@@ -277,7 +277,6 @@ public class WebRuntimeMonitor implements WebMonitor {
 				timeout,
 				TaskManagerLogHandler.FileMode.LOG,
 				config,
-				enableSSL,
 				blobView));
 		get(router,
 			new TaskManagerLogHandler(
@@ -287,7 +286,6 @@ public class WebRuntimeMonitor implements WebMonitor {
 				timeout,
 				TaskManagerLogHandler.FileMode.STDOUT,
 				config,
-				enableSSL,
 				blobView));
 		get(router, new TaskManagerMetricsHandler(executor, metricFetcher));
 
@@ -298,12 +296,10 @@ public class WebRuntimeMonitor implements WebMonitor {
 					retriever,
 					localRestAddress,
 					timeout,
-					logFiles.logFile,
-					enableSSL))
+					logFiles.logFile))
 
 			.GET("/jobmanager/stdout", logFiles.stdOutFile == null ? new ConstantTextHandler("(stdout
file unavailable)") :
-				new StaticFileServerHandler<>(retriever, localRestAddress, timeout, logFiles.stdOutFile,
-					enableSSL));
+				new StaticFileServerHandler<>(retriever, localRestAddress, timeout, logFiles.stdOutFile));
 
 		get(router, new JobManagerMetricsHandler(executor, metricFetcher));
 
@@ -357,8 +353,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 			retriever,
 			localRestAddress,
 			timeout,
-			webRootDir,
-			enableSSL));
+			webRootDir));
 
 		// add shutdown hook for deleting the directories and remaining temp files on shutdown
 		try {
@@ -530,8 +525,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 	// ------------------------------------------------------------------------
 
 	private RuntimeMonitorHandler handler(RequestHandler handler) {
-		return new RuntimeMonitorHandler(cfg, handler, retriever, localRestAddress, timeout,
-			serverSSLContext !=  null);
+		return new RuntimeMonitorHandler(cfg, handler, retriever, localRestAddress, timeout);
 	}
 
 	File getBaseDir(Configuration configuration) {

http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
index e6e632e..2445d3f 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
@@ -112,10 +112,9 @@ public class StaticFileServerHandler<T extends RestfulGateway>
extends RedirectH
 			GatewayRetriever<T> retriever,
 			CompletableFuture<String> localJobManagerAddressFuture,
 			Time timeout,
-			File rootPath,
-			boolean httpsEnabled) throws IOException {
+			File rootPath) throws IOException {
 
-		super(localJobManagerAddressFuture, retriever, timeout, httpsEnabled);
+		super(localJobManagerAddressFuture, retriever, timeout);
 
 		this.rootPath = checkNotNull(rootPath).getCanonicalFile();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
index bb9b7f5..d53d2b1 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java
@@ -127,9 +127,8 @@ public class TaskManagerLogHandler extends RedirectHandler<JobManagerGateway>
im
 		Time timeout,
 		FileMode fileMode,
 		Configuration config,
-		boolean httpsEnabled,
 		BlobView blobView) {
-		super(localJobManagerAddressPromise, retriever, timeout, httpsEnabled);
+		super(localJobManagerAddressPromise, retriever, timeout);
 
 		this.executor = checkNotNull(executor);
 		this.config = config;

http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/WebFrontendBootstrap.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/WebFrontendBootstrap.java
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/WebFrontendBootstrap.java
index c3f85d9..740beae 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/WebFrontendBootstrap.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/WebFrontendBootstrap.java
@@ -128,7 +128,9 @@ public class WebFrontendBootstrap {
 
 		this.log.info("Web frontend listening at {}" + ':' + "{}", address, port);
 
-		this.restAddress = address + ':' + port;
+		final String protocol = serverSSLContext != null ? "https://" : "http://";
+
+		this.restAddress = protocol + address + ':' + port;
 	}
 
 	public ServerBootstrap getBootstrap() {

http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/RedirectHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/RedirectHandlerTest.java
b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/RedirectHandlerTest.java
index 23cad19..8935413 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/RedirectHandlerTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/RedirectHandlerTest.java
@@ -67,8 +67,8 @@ public class RedirectHandlerTest extends TestLogger {
 	public void testRedirectHandler() throws Exception {
 		final String restPath = "/testing";
 		final String correctAddress = "foobar:21345";
-		final String redirectionAddress = "foobar:12345";
-		final String expectedRedirection = "http://" + redirectionAddress + restPath;
+		final String redirectionAddress = "http://foobar:12345";
+		final String expectedRedirection = redirectionAddress + restPath;
 
 		final Configuration configuration = new Configuration();
 		final Router router = new Router();
@@ -87,8 +87,7 @@ public class RedirectHandlerTest extends TestLogger {
 		final TestingHandler testingHandler = new TestingHandler(
 			localAddressFuture,
 			gatewayRetriever,
-			timeout,
-			false);
+			timeout);
 
 		router.GET(restPath, testingHandler);
 		WebFrontendBootstrap bootstrap = new WebFrontendBootstrap(
@@ -143,9 +142,8 @@ public class RedirectHandlerTest extends TestLogger {
 		protected TestingHandler(
 				@Nonnull CompletableFuture<String> localAddressFuture,
 				@Nonnull GatewayRetriever<RestfulGateway> leaderRetriever,
-				@Nonnull Time timeout,
-				boolean httpsEnabled) {
-			super(localAddressFuture, leaderRetriever, timeout, httpsEnabled);
+				@Nonnull Time timeout) {
+			super(localAddressFuture, leaderRetriever, timeout);
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
index c11fe6a..faeff13 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
@@ -69,7 +69,6 @@ public class TaskManagerLogHandlerTest {
 			TestingUtils.TIMEOUT(),
 			TaskManagerLogHandler.FileMode.LOG,
 			new Configuration(),
-			false,
 			new VoidBlobStore());
 		String[] pathsLog = handlerLog.getPaths();
 		Assert.assertEquals(1, pathsLog.length);
@@ -82,7 +81,6 @@ public class TaskManagerLogHandlerTest {
 			TestingUtils.TIMEOUT(),
 			TaskManagerLogHandler.FileMode.STDOUT,
 			new Configuration(),
-			false,
 			new VoidBlobStore());
 		String[] pathsOut = handlerOut.getPaths();
 		Assert.assertEquals(1, pathsOut.length);
@@ -124,7 +122,6 @@ public class TaskManagerLogHandlerTest {
 			TestingUtils.TIMEOUT(),
 			TaskManagerLogHandler.FileMode.LOG,
 			new Configuration(),
-			false,
 			new VoidBlobStore());
 
 		final AtomicReference<String> exception = new AtomicReference<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java
index bd7f105..fc37381 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java
@@ -143,8 +143,14 @@ public abstract class RestServerEndpoint {
 
 			log.info("Rest endpoint listening at {}" + ':' + "{}", address, port);
 
-			// TODO: Make it include the protocol (http/https)
-			restAddress = address + ':' + port;
+			final String protocol;
+
+			if (sslEngine != null) {
+				protocol = "https://";
+			} else {
+				protocol = "http://";
+			}
+			restAddress = protocol + address + ':' + port;
 
 			started = true;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
index 95483c3..596c947 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
@@ -69,9 +69,8 @@ public abstract class AbstractRestHandler<T extends RestfulGateway, R
extends Re
 			CompletableFuture<String> localAddressFuture,
 			GatewayRetriever<T> leaderRetriever,
 			Time timeout,
-			boolean httpsEnabled,
 			MessageHeaders<R, P, M> messageHeaders) {
-		super(localAddressFuture, leaderRetriever, timeout, httpsEnabled);
+		super(localAddressFuture, leaderRetriever, timeout);
 		this.messageHeaders = messageHeaders;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RedirectHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RedirectHandler.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RedirectHandler.java
index fb40035..83550cd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RedirectHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RedirectHandler.java
@@ -55,26 +55,21 @@ public abstract class RedirectHandler<T extends RestfulGateway>
extends SimpleCh
 
 	protected final Logger logger = LoggerFactory.getLogger(getClass());
 
-	private final CompletableFuture<String> localAddressFuture;
+	protected final CompletableFuture<String> localAddressFuture;
 
 	protected final GatewayRetriever<T> leaderRetriever;
 
 	protected final Time timeout;
 
-	/** Whether the web service has https enabled. */
-	protected final boolean httpsEnabled;
-
 	private String localAddress;
 
 	protected RedirectHandler(
 			@Nonnull CompletableFuture<String> localAddressFuture,
 			@Nonnull GatewayRetriever<T> leaderRetriever,
-			@Nonnull Time timeout,
-			boolean httpsEnabled) {
+			@Nonnull Time timeout) {
 		this.localAddressFuture = Preconditions.checkNotNull(localAddressFuture);
 		this.leaderRetriever = Preconditions.checkNotNull(leaderRetriever);
 		this.timeout = Preconditions.checkNotNull(timeout);
-		this.httpsEnabled = httpsEnabled;
 		localAddress = null;
 	}
 
@@ -122,8 +117,7 @@ public abstract class RedirectHandler<T extends RestfulGateway>
extends SimpleCh
 								} else if (optRedirectAddress.isPresent()) {
 									response = HandlerRedirectUtils.getRedirectResponse(
 										optRedirectAddress.get(),
-										routed.path(),
-										httpsEnabled);
+										routed.path());
 
 									KeepAliveWrite.flush(channelHandlerContext, routed.request(), response);
 								} else {

http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerRedirectUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerRedirectUtils.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerRedirectUtils.java
index cd32d46..900e1d2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerRedirectUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerRedirectUtils.java
@@ -67,12 +67,11 @@ public class HandlerRedirectUtils {
 			});
 	}
 
-	public static HttpResponse getRedirectResponse(String redirectAddress, String path, boolean
httpsEnabled) {
+	public static HttpResponse getRedirectResponse(String redirectAddress, String path) {
 		checkNotNull(redirectAddress, "Redirect address");
 		checkNotNull(path, "Path");
 
-		String protocol = httpsEnabled ? "https" : "http";
-		String newLocation = String.format("%s://%s%s", protocol, redirectAddress, path);
+		String newLocation = String.format("%s%s", redirectAddress, path);
 
 		HttpResponse redirectResponse = new DefaultFullHttpResponse(
 				HttpVersion.HTTP_1_1, HttpResponseStatus.TEMPORARY_REDIRECT);

http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
index 3989fe4..276e0ff 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
@@ -73,6 +73,7 @@ import org.apache.flink.runtime.messages.webmonitor.{InfoMessage, _}
 import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup
 import org.apache.flink.runtime.metrics.{MetricRegistryConfiguration, MetricRegistry =>
FlinkMetricRegistry}
 import org.apache.flink.runtime.metrics.util.MetricUtils
+import org.apache.flink.runtime.net.SSLUtils
 import org.apache.flink.runtime.process.ProcessReaper
 import org.apache.flink.runtime.query.KvStateMessage.{LookupKvStateLocation, NotifyKvStateRegistered,
NotifyKvStateUnregistered}
 import org.apache.flink.runtime.query.{KvStateMessage, UnknownKvStateLocation}

http://git-wip-us.apache.org/repos/asf/flink/blob/32770103/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestEndpointITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestEndpointITCase.java
b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestEndpointITCase.java
index ff1c1f7..4e06d1f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestEndpointITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestEndpointITCase.java
@@ -72,17 +72,16 @@ public class RestEndpointITCase extends TestLogger {
 		RestServerEndpointConfiguration serverConfig = RestServerEndpointConfiguration.fromConfiguration(config);
 		RestClientConfiguration clientConfig = RestClientConfiguration.fromConfiguration(config);
 
-		final String address = "localhost";
+		final String restAddress = "http://localhost:1234";
 		RestfulGateway mockRestfulGateway = mock(RestfulGateway.class);
-		when(mockRestfulGateway.requestRestAddress(any(Time.class))).thenReturn(CompletableFuture.completedFuture(address));
+		when(mockRestfulGateway.requestRestAddress(any(Time.class))).thenReturn(CompletableFuture.completedFuture(restAddress));
 		GatewayRetriever<RestfulGateway> mockGatewayRetriever = mock(GatewayRetriever.class);
 		when(mockGatewayRetriever.getNow()).thenReturn(Optional.of(mockRestfulGateway));
 
 		TestHandler testHandler = new TestHandler(
-			CompletableFuture.completedFuture(address),
+			CompletableFuture.completedFuture(restAddress),
 			mockGatewayRetriever,
-			RpcUtils.INF_TIMEOUT,
-			true);
+			RpcUtils.INF_TIMEOUT);
 
 		RestServerEndpoint serverEndpoint = new TestRestServerEndpoint(serverConfig, testHandler);
 		RestClient clientEndpoint = new TestRestClient(clientConfig);
@@ -150,13 +149,11 @@ public class RestEndpointITCase extends TestLogger {
 		TestHandler(
 			CompletableFuture<String> localAddressFuture,
 			GatewayRetriever<RestfulGateway> leaderRetriever,
-			Time timeout,
-			boolean httpsEnabled) {
+			Time timeout) {
 			super(
 				localAddressFuture,
 				leaderRetriever,
 				timeout,
-				httpsEnabled,
 				new TestHeaders());
 		}
 


Mime
View raw message