You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2017/11/09 09:51:13 UTC

[1/4] flink git commit: [FLINK-8000] Sort Rest handler URLS in RestServerEndpoint

Repository: flink
Updated Branches:
  refs/heads/master 551653340 -> 541fe4366


[FLINK-8000] Sort Rest handler URLS in RestServerEndpoint

Introduce special RestHandlerUrlComparator to sort REST URLs such that
URLs with path parameters are sorted after those without or fewer.

E.g. the following order would be established

/jobs
/jobs/overview
/jobs/:jobid
/jobs/:jobid/config
/:*

This closes #4958.


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

Branch: refs/heads/master
Commit: 34fdf569d7defb4393849fb9ecb2763b14532cc6
Parents: 5516533
Author: Till Rohrmann <tr...@apache.org>
Authored: Sat Nov 4 14:56:11 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Nov 9 10:48:38 2017 +0100

----------------------------------------------------------------------
 .../program/rest/RestClusterClientTest.java     | 12 +--
 .../dispatcher/DispatcherRestEndpoint.java      |  6 +-
 .../flink/runtime/rest/RestServerEndpoint.java  | 90 +++++++++++++++++++-
 .../flink/runtime/rest/RestEndpointITCase.java  |  5 +-
 .../runtime/rest/RestServerEndpointTest.java    | 63 ++++++++++++++
 5 files changed, 162 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/34fdf569/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java
index 0542d50..1bc04eb 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java
@@ -111,9 +111,9 @@ public class RestClusterClientTest extends TestLogger {
 
 		RestServerEndpoint rse = new RestServerEndpoint(rsec) {
 			@Override
-			protected Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
+			protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
 
-				Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>();
+				List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>();
 				handlers.add(Tuple2.of(portHandler.getMessageHeaders(), portHandler));
 				handlers.add(Tuple2.of(submitHandler.getMessageHeaders(), submitHandler));
 				handlers.add(Tuple2.of(terminationHandler.getMessageHeaders(), terminationHandler));
@@ -212,9 +212,9 @@ public class RestClusterClientTest extends TestLogger {
 
 		RestServerEndpoint rse = new RestServerEndpoint(rsec) {
 			@Override
-			protected Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
+			protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
 
-				Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>();
+				List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>();
 				handlers.add(Tuple2.of(triggerHandler.getMessageHeaders(), triggerHandler));
 				return handlers;
 			}
@@ -280,9 +280,9 @@ public class RestClusterClientTest extends TestLogger {
 
 		RestServerEndpoint rse = new RestServerEndpoint(rsec) {
 			@Override
-			protected Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
+			protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
 
-				Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>();
+				List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>();
 				handlers.add(Tuple2.of(listJobsHandler.getMessageHeaders(), listJobsHandler));
 				return handlers;
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/34fdf569/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
index 6766784..a479749 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
@@ -85,7 +85,7 @@ import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -137,7 +137,7 @@ public class DispatcherRestEndpoint extends RestServerEndpoint {
 	}
 
 	@Override
-	protected Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
+	protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
 		ArrayList<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = new ArrayList<>(3);
 
 		final Time timeout = restConfiguration.getTimeout();
@@ -352,8 +352,8 @@ public class DispatcherRestEndpoint extends RestServerEndpoint {
 		handlers.add(Tuple2.of(jobSubmitHandler.getMessageHeaders(), jobSubmitHandler));
 		handlers.add(Tuple2.of(TaskManagersHeaders.getInstance(), taskManagersHandler));
 		handlers.add(Tuple2.of(TaskManagerDetailsHeaders.getInstance(), taskManagerDetailsHandler));
+		handlers.add(Tuple2.of(SubtasksTimesHeaders.getInstance(), subtasksTimesHandler));
 
-		// This handler MUST be added last, as it otherwise masks all subsequent GET handlers
 		optWebContent.ifPresent(
 			webContent -> handlers.add(Tuple2.of(WebContentHandlerSpecification.getInstance(), webContent)));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34fdf569/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 18766c0..ea18e15 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
@@ -45,8 +45,11 @@ import org.slf4j.LoggerFactory;
 
 import javax.net.ssl.SSLEngine;
 
+import java.io.Serializable;
 import java.net.InetSocketAddress;
-import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 
@@ -88,7 +91,7 @@ public abstract class RestServerEndpoint {
 	 * @param restAddressFuture future rest address of the RestServerEndpoint
 	 * @return Collection of AbstractRestHandler which are added to the server endpoint
 	 */
-	protected abstract Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture);
+	protected abstract List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture);
 
 	/**
 	 * Starts this REST server endpoint.
@@ -107,7 +110,23 @@ public abstract class RestServerEndpoint {
 			final Router router = new Router();
 			final CompletableFuture<String> restAddressFuture = new CompletableFuture<>();
 
-			initializeHandlers(restAddressFuture).forEach(handler -> registerHandler(router, handler));
+			List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> handlers = initializeHandlers(restAddressFuture);
+
+			/* sort the handlers such that they are ordered the following:
+			 * /jobs
+			 * /jobs/overview
+			 * /jobs/:jobid
+			 * /jobs/:jobid/config
+			 * /:*
+			 */
+			Collections.sort(
+				handlers,
+				RestHandlerUrlComparator.INSTANCE);
+
+			handlers.forEach(handler -> {
+				log.debug("Register handler {} under {}@{}.", handler.f1, handler.f0.getHttpMethod(), handler.f0.getTargetRestEndpointURL());
+				registerHandler(router, handler);
+			});
 
 			ChannelInitializer<SocketChannel> initializer = new ChannelInitializer<SocketChannel>() {
 
@@ -268,4 +287,69 @@ public abstract class RestServerEndpoint {
 				throw new RuntimeException("Unsupported http method: " + specificationHandler.f0.getHttpMethod() + '.');
 		}
 	}
+
+	/**
+	 * Comparator for Rest URLs.
+	 *
+	 * <p>The comparator orders the Rest URLs such that URLs with path parameters are ordered behind
+	 * those without parameters. E.g.:
+	 * /jobs
+	 * /jobs/overview
+	 * /jobs/:jobid
+	 * /jobs/:jobid/config
+	 * /:*
+	 *
+	 * <p>IMPORTANT: This comparator is highly specific to how Netty path parameter are encoded. Namely
+	 * via a preceding ':' character.
+	 */
+	static final class RestHandlerUrlComparator implements Comparator<Tuple2<RestHandlerSpecification, ChannelInboundHandler>>, Serializable {
+
+		private static final long serialVersionUID = 2388466767835547926L;
+
+		private static final Comparator<String> CASE_INSENSITIVE_ORDER = new CaseInsensitiveOrderComparator();
+
+		static final RestHandlerUrlComparator INSTANCE = new RestHandlerUrlComparator();
+
+		@Override
+		public int compare(
+				Tuple2<RestHandlerSpecification, ChannelInboundHandler> o1,
+				Tuple2<RestHandlerSpecification, ChannelInboundHandler> o2) {
+			return CASE_INSENSITIVE_ORDER.compare(o1.f0.getTargetRestEndpointURL(), o2.f0.getTargetRestEndpointURL());
+		}
+
+		static final class CaseInsensitiveOrderComparator implements Comparator<String>, Serializable {
+			private static final long serialVersionUID = 8550835445193437027L;
+
+			@Override
+			public int compare(String s1, String s2) {
+				int n1 = s1.length();
+				int n2 = s2.length();
+				int min = Math.min(n1, n2);
+				for (int i = 0; i < min; i++) {
+					char c1 = s1.charAt(i);
+					char c2 = s2.charAt(i);
+					if (c1 != c2) {
+						c1 = Character.toUpperCase(c1);
+						c2 = Character.toUpperCase(c2);
+						if (c1 != c2) {
+							c1 = Character.toLowerCase(c1);
+							c2 = Character.toLowerCase(c2);
+							if (c1 != c2) {
+								if (c1 == ':') {
+									// c2 is less than c1 because it is also different
+									return 1;
+								} else if (c2 == ':') {
+									// c1 is less than c2
+									return -1;
+								} else {
+									return c1 - c2;
+								}
+							}
+						}
+					}
+				}
+				return n1 - n2;
+			}
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34fdf569/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 4d3c6b5..ee59da7 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
@@ -57,6 +57,7 @@ import javax.annotation.Nonnull;
 import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
@@ -205,8 +206,8 @@ public class RestEndpointITCase extends TestLogger {
 		}
 
 		@Override
-		protected Collection<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
-			return Collections.singleton(Tuple2.of(new TestHeaders(), testHandler));
+		protected List<Tuple2<RestHandlerSpecification, ChannelInboundHandler>> initializeHandlers(CompletableFuture<String> restAddressFuture) {
+			return Collections.singletonList(Tuple2.of(new TestHeaders(), testHandler));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34fdf569/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java
new file mode 100644
index 0000000..3cb4a55
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.runtime.rest;
+
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test cases for the {@link RestServerEndpoint}.
+ */
+public class RestServerEndpointTest extends TestLogger {
+
+	/**
+	 * Tests that the REST handler URLs are properly sorted.
+	 */
+	@Test
+	public void testRestHandlerUrlSorting() {
+		final int numberHandlers = 5;
+
+		final List<String> handlerUrls = new ArrayList<>(numberHandlers);
+
+		handlerUrls.add("/jobs/overview");
+		handlerUrls.add("/jobs/:jobid");
+		handlerUrls.add("/jobs");
+		handlerUrls.add("/:*");
+		handlerUrls.add("/jobs/:jobid/config");
+
+		final List<String> expected = new ArrayList<>(numberHandlers);
+
+		expected.add("/jobs");
+		expected.add("/jobs/overview");
+		expected.add("/jobs/:jobid");
+		expected.add("/jobs/:jobid/config");
+		expected.add("/:*");
+
+		Collections.sort(handlerUrls, new RestServerEndpoint.RestHandlerUrlComparator.CaseInsensitiveOrderComparator());
+
+		assertEquals(expected, handlerUrls);
+	}
+}


[2/4] flink git commit: [FLINK-8024] Let ClusterOverviewHandler directly extend from AbstractRestHandler

Posted by tr...@apache.org.
[FLINK-8024] Let ClusterOverviewHandler directly extend from AbstractRestHandler

This closes #4982.


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

Branch: refs/heads/master
Commit: f03393e807f51e6496f5bd54771987a64287b154
Parents: 34fdf56
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Nov 6 18:48:53 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Nov 9 10:48:39 2017 +0100

----------------------------------------------------------------------
 .../dispatcher/DispatcherRestEndpoint.java      | 10 +--
 .../handler/cluster/ClusterOverviewHandler.java | 65 ++++++++++++++++++++
 .../handler/legacy/ClusterOverviewHandler.java  | 15 +----
 3 files changed, 69 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f03393e8/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
index a479749..1a0c584 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
@@ -27,6 +27,7 @@ import org.apache.flink.runtime.rest.RestServerEndpointConfiguration;
 import org.apache.flink.runtime.rest.handler.LegacyRestHandlerAdapter;
 import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration;
 import org.apache.flink.runtime.rest.handler.RestHandlerSpecification;
+import org.apache.flink.runtime.rest.handler.cluster.ClusterOverviewHandler;
 import org.apache.flink.runtime.rest.handler.job.BlobServerPortHandler;
 import org.apache.flink.runtime.rest.handler.job.JobAccumulatorsHandler;
 import org.apache.flink.runtime.rest.handler.job.JobConfigHandler;
@@ -44,12 +45,10 @@ import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointStatsCach
 import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler;
 import org.apache.flink.runtime.rest.handler.job.checkpoints.TaskCheckpointStatisticDetailsHandler;
 import org.apache.flink.runtime.rest.handler.legacy.ClusterConfigHandler;
-import org.apache.flink.runtime.rest.handler.legacy.ClusterOverviewHandler;
 import org.apache.flink.runtime.rest.handler.legacy.DashboardConfigHandler;
 import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
 import org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandler;
 import org.apache.flink.runtime.rest.handler.legacy.files.WebContentHandlerSpecification;
-import org.apache.flink.runtime.rest.handler.legacy.messages.ClusterOverviewWithVersion;
 import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher;
 import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerDetailsHandler;
 import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagersHandler;
@@ -143,15 +142,12 @@ public class DispatcherRestEndpoint extends RestServerEndpoint {
 		final Time timeout = restConfiguration.getTimeout();
 		final Map<String, String> responseHeaders = restConfiguration.getResponseHeaders();
 
-		LegacyRestHandlerAdapter<DispatcherGateway, ClusterOverviewWithVersion, EmptyMessageParameters> clusterOverviewHandler = new LegacyRestHandlerAdapter<>(
+		ClusterOverviewHandler<DispatcherGateway> clusterOverviewHandler = new ClusterOverviewHandler<>(
 			restAddressFuture,
 			leaderRetriever,
 			timeout,
 			responseHeaders,
-			ClusterOverviewHeaders.getInstance(),
-			new ClusterOverviewHandler(
-				executor,
-				timeout));
+			ClusterOverviewHeaders.getInstance());
 
 		LegacyRestHandlerAdapter<DispatcherGateway, DashboardConfiguration, EmptyMessageParameters> dashboardConfigurationHandler = new LegacyRestHandlerAdapter<>(
 			restAddressFuture,

http://git-wip-us.apache.org/repos/asf/flink/blob/f03393e8/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/ClusterOverviewHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/ClusterOverviewHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/ClusterOverviewHandler.java
new file mode 100644
index 0000000..19b94c5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/ClusterOverviewHandler.java
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.runtime.rest.handler.cluster;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.messages.webmonitor.ClusterOverview;
+import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.legacy.messages.ClusterOverviewWithVersion;
+import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+
+import javax.annotation.Nonnull;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Handler which returns the cluster overview information with version.
+ *
+ * @param <T> type of the leader gateway
+ */
+public class ClusterOverviewHandler<T extends RestfulGateway> extends AbstractRestHandler<T, EmptyRequestBody, ClusterOverviewWithVersion, EmptyMessageParameters> {
+
+	private static final String version = EnvironmentInformation.getVersion();
+
+	private static final String commitID = EnvironmentInformation.getRevisionInformation().commitId;
+
+	public ClusterOverviewHandler(
+			CompletableFuture<String> localRestAddress,
+			GatewayRetriever<T> leaderRetriever,
+			Time timeout,
+			Map<String, String> responseHeaders,
+			MessageHeaders<EmptyRequestBody, ClusterOverviewWithVersion, EmptyMessageParameters> messageHeaders) {
+		super(localRestAddress, leaderRetriever, timeout, responseHeaders, messageHeaders);
+	}
+
+	@Override
+	public CompletableFuture<ClusterOverviewWithVersion> handleRequest(@Nonnull HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request, @Nonnull T gateway) {
+		CompletableFuture<ClusterOverview> overviewFuture = gateway.requestClusterOverview(timeout);
+
+		return overviewFuture.thenApply(
+			statusOverview -> ClusterOverviewWithVersion.fromStatusOverview(statusOverview, version, commitID));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f03393e8/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandler.java
index 30e7fec..0a26084 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterOverviewHandler.java
@@ -20,16 +20,11 @@ package org.apache.flink.runtime.rest.handler.legacy;
 
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.concurrent.FutureUtils;
-import org.apache.flink.runtime.dispatcher.DispatcherGateway;
 import org.apache.flink.runtime.jobmaster.JobManagerGateway;
 import org.apache.flink.runtime.messages.webmonitor.ClusterOverview;
 import org.apache.flink.runtime.messages.webmonitor.JobsOverview;
-import org.apache.flink.runtime.rest.handler.HandlerRequest;
-import org.apache.flink.runtime.rest.handler.LegacyRestHandler;
 import org.apache.flink.runtime.rest.handler.legacy.messages.ClusterOverviewWithVersion;
 import org.apache.flink.runtime.rest.messages.ClusterOverviewHeaders;
-import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
-import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.util.FlinkException;
 
@@ -48,7 +43,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * Responder that returns the status of the Flink cluster, such as how many
  * TaskManagers are currently connected, and how many jobs are running.
  */
-public class ClusterOverviewHandler extends AbstractJsonRequestHandler implements LegacyRestHandler<DispatcherGateway, ClusterOverviewWithVersion, EmptyMessageParameters> {
+public class ClusterOverviewHandler extends AbstractJsonRequestHandler {
 
 	private static final String version = EnvironmentInformation.getVersion();
 
@@ -108,12 +103,4 @@ public class ClusterOverviewHandler extends AbstractJsonRequestHandler implement
 			return FutureUtils.completedExceptionally(new FlinkException("Failed to fetch list of all running jobs: ", e));
 		}
 	}
-
-	@Override
-	public CompletableFuture<ClusterOverviewWithVersion> handleRequest(HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request, DispatcherGateway gateway) {
-		CompletableFuture<ClusterOverview> overviewFuture = gateway.requestClusterOverview(timeout);
-
-		return overviewFuture.thenApply(
-			statusOverview -> ClusterOverviewWithVersion.fromStatusOverview(statusOverview, version, commitID));
-	}
 }


[3/4] flink git commit: [FLINK-8026] Let ClusterConfigHandler directly extend AbstractRestHandler

Posted by tr...@apache.org.
[FLINK-8026] Let ClusterConfigHandler directly extend AbstractRestHandler

This closes #4984.


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

Branch: refs/heads/master
Commit: 541fe43663d2a24b1ae66bc2b5228c49dfd43e7b
Parents: fa967df
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Nov 7 10:56:33 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Nov 9 10:48:39 2017 +0100

----------------------------------------------------------------------
 .../dispatcher/DispatcherRestEndpoint.java      | 11 +---
 .../handler/cluster/ClusterConfigHandler.java   | 65 ++++++++++++++++++++
 .../handler/legacy/ClusterConfigHandler.java    | 19 +-----
 3 files changed, 69 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/541fe436/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
index 96af2eb..3b262c7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
@@ -24,9 +24,9 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.runtime.rest.RestServerEndpoint;
 import org.apache.flink.runtime.rest.RestServerEndpointConfiguration;
-import org.apache.flink.runtime.rest.handler.LegacyRestHandlerAdapter;
 import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration;
 import org.apache.flink.runtime.rest.handler.RestHandlerSpecification;
+import org.apache.flink.runtime.rest.handler.cluster.ClusterConfigHandler;
 import org.apache.flink.runtime.rest.handler.cluster.ClusterOverviewHandler;
 import org.apache.flink.runtime.rest.handler.cluster.DashboardConfigHandler;
 import org.apache.flink.runtime.rest.handler.job.BlobServerPortHandler;
@@ -45,18 +45,15 @@ import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointStatistic
 import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointStatsCache;
 import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler;
 import org.apache.flink.runtime.rest.handler.job.checkpoints.TaskCheckpointStatisticDetailsHandler;
-import org.apache.flink.runtime.rest.handler.legacy.ClusterConfigHandler;
 import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
 import org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandler;
 import org.apache.flink.runtime.rest.handler.legacy.files.WebContentHandlerSpecification;
 import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher;
 import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerDetailsHandler;
 import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagersHandler;
-import org.apache.flink.runtime.rest.messages.ClusterConfigurationInfo;
 import org.apache.flink.runtime.rest.messages.ClusterConfigurationInfoHeaders;
 import org.apache.flink.runtime.rest.messages.ClusterOverviewHeaders;
 import org.apache.flink.runtime.rest.messages.DashboardConfigurationHeaders;
-import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
 import org.apache.flink.runtime.rest.messages.JobAccumulatorsHeaders;
 import org.apache.flink.runtime.rest.messages.JobConfigHeaders;
 import org.apache.flink.runtime.rest.messages.JobExceptionsHeaders;
@@ -163,15 +160,13 @@ public class DispatcherRestEndpoint extends RestServerEndpoint {
 			responseHeaders,
 			JobsOverviewHeaders.getInstance());
 
-		LegacyRestHandlerAdapter<DispatcherGateway, ClusterConfigurationInfo, EmptyMessageParameters> clusterConfigurationHandler = new LegacyRestHandlerAdapter<>(
+		ClusterConfigHandler<DispatcherGateway> clusterConfigurationHandler = new ClusterConfigHandler<>(
 			restAddressFuture,
 			leaderRetriever,
 			timeout,
 			responseHeaders,
 			ClusterConfigurationInfoHeaders.getInstance(),
-			new ClusterConfigHandler(
-				executor,
-				clusterConfiguration));
+			clusterConfiguration);
 
 		JobTerminationHandler jobTerminationHandler = new JobTerminationHandler(
 			restAddressFuture,

http://git-wip-us.apache.org/repos/asf/flink/blob/541fe436/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/ClusterConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/ClusterConfigHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/ClusterConfigHandler.java
new file mode 100644
index 0000000..ef95c61
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/ClusterConfigHandler.java
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.runtime.rest.handler.cluster;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.messages.ClusterConfigurationInfo;
+import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nonnull;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Handler which serves the cluster's configuration.
+ *
+ * @param <T> type of the leader gateway
+ */
+public class ClusterConfigHandler<T extends RestfulGateway> extends AbstractRestHandler<T, EmptyRequestBody, ClusterConfigurationInfo, EmptyMessageParameters> {
+
+	private final ClusterConfigurationInfo clusterConfig;
+
+	public ClusterConfigHandler(
+			CompletableFuture<String> localRestAddress,
+			GatewayRetriever<? extends T> leaderRetriever,
+			Time timeout,
+			Map<String, String> responseHeaders,
+			MessageHeaders<EmptyRequestBody, ClusterConfigurationInfo, EmptyMessageParameters> messageHeaders,
+			Configuration configuration) {
+		super(localRestAddress, leaderRetriever, timeout, responseHeaders, messageHeaders);
+
+		Preconditions.checkNotNull(configuration);
+		this.clusterConfig = ClusterConfigurationInfo.from(configuration);
+	}
+
+	@Override
+	protected CompletableFuture<ClusterConfigurationInfo> handleRequest(@Nonnull HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request, @Nonnull T gateway) throws RestHandlerException {
+		return CompletableFuture.completedFuture(clusterConfig);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/541fe436/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterConfigHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterConfigHandler.java
index efdc9c9..76221b5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterConfigHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterConfigHandler.java
@@ -19,15 +19,9 @@
 package org.apache.flink.runtime.rest.handler.legacy;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.dispatcher.DispatcherGateway;
 import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.rest.handler.HandlerRequest;
-import org.apache.flink.runtime.rest.handler.LegacyRestHandler;
-import org.apache.flink.runtime.rest.messages.ClusterConfigurationInfo;
 import org.apache.flink.runtime.rest.messages.ClusterConfigurationInfoEntry;
 import org.apache.flink.runtime.rest.messages.ClusterConfigurationInfoHeaders;
-import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
-import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
 import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.Preconditions;
 
@@ -43,17 +37,14 @@ import java.util.concurrent.Executor;
 /**
  * Returns the Job Manager's configuration.
  */
-public class ClusterConfigHandler extends AbstractJsonRequestHandler
-		implements LegacyRestHandler<DispatcherGateway, ClusterConfigurationInfo, EmptyMessageParameters> {
+public class ClusterConfigHandler extends AbstractJsonRequestHandler {
 
-	private final ClusterConfigurationInfo clusterConfig;
 	private final String clusterConfigJson;
 
 	public ClusterConfigHandler(Executor executor, Configuration config) {
 		super(executor);
 
 		Preconditions.checkNotNull(config);
-		this.clusterConfig = ClusterConfigurationInfo.from(config);
 		this.clusterConfigJson = createConfigJson(config);
 	}
 
@@ -63,14 +54,6 @@ public class ClusterConfigHandler extends AbstractJsonRequestHandler
 	}
 
 	@Override
-	public CompletableFuture<ClusterConfigurationInfo> handleRequest(
-			HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request,
-			DispatcherGateway gateway) {
-
-		return CompletableFuture.completedFuture(clusterConfig);
-	}
-
-	@Override
 	public CompletableFuture<String> handleJsonRequest(
 			Map<String, String> pathParams,
 			Map<String, String> queryParams,


[4/4] flink git commit: [FLINK-8025] Let DashboardConfigHandler directly extend AbstractRestHandler

Posted by tr...@apache.org.
[FLINK-8025] Let DashboardConfigHandler directly extend AbstractRestHandler

This closes #4983.


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

Branch: refs/heads/master
Commit: fa967dfc0cddec0a5d0c7d76e564a8307ec3fb35
Parents: f03393e
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Nov 6 18:56:01 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Nov 9 10:48:39 2017 +0100

----------------------------------------------------------------------
 .../dispatcher/DispatcherRestEndpoint.java      | 11 ++--
 .../handler/cluster/DashboardConfigHandler.java | 62 ++++++++++++++++++++
 .../handler/legacy/DashboardConfigHandler.java  | 12 +---
 3 files changed, 67 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fa967dfc/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
index 1a0c584..96af2eb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
@@ -28,6 +28,7 @@ import org.apache.flink.runtime.rest.handler.LegacyRestHandlerAdapter;
 import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration;
 import org.apache.flink.runtime.rest.handler.RestHandlerSpecification;
 import org.apache.flink.runtime.rest.handler.cluster.ClusterOverviewHandler;
+import org.apache.flink.runtime.rest.handler.cluster.DashboardConfigHandler;
 import org.apache.flink.runtime.rest.handler.job.BlobServerPortHandler;
 import org.apache.flink.runtime.rest.handler.job.JobAccumulatorsHandler;
 import org.apache.flink.runtime.rest.handler.job.JobConfigHandler;
@@ -45,7 +46,6 @@ import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointStatsCach
 import org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler;
 import org.apache.flink.runtime.rest.handler.job.checkpoints.TaskCheckpointStatisticDetailsHandler;
 import org.apache.flink.runtime.rest.handler.legacy.ClusterConfigHandler;
-import org.apache.flink.runtime.rest.handler.legacy.DashboardConfigHandler;
 import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
 import org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandler;
 import org.apache.flink.runtime.rest.handler.legacy.files.WebContentHandlerSpecification;
@@ -55,7 +55,6 @@ import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagersHandler;
 import org.apache.flink.runtime.rest.messages.ClusterConfigurationInfo;
 import org.apache.flink.runtime.rest.messages.ClusterConfigurationInfoHeaders;
 import org.apache.flink.runtime.rest.messages.ClusterOverviewHeaders;
-import org.apache.flink.runtime.rest.messages.DashboardConfiguration;
 import org.apache.flink.runtime.rest.messages.DashboardConfigurationHeaders;
 import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
 import org.apache.flink.runtime.rest.messages.JobAccumulatorsHeaders;
@@ -149,15 +148,13 @@ public class DispatcherRestEndpoint extends RestServerEndpoint {
 			responseHeaders,
 			ClusterOverviewHeaders.getInstance());
 
-		LegacyRestHandlerAdapter<DispatcherGateway, DashboardConfiguration, EmptyMessageParameters> dashboardConfigurationHandler = new LegacyRestHandlerAdapter<>(
+		DashboardConfigHandler<DispatcherGateway> dashboardConfigHandler = new DashboardConfigHandler<>(
 			restAddressFuture,
 			leaderRetriever,
 			timeout,
 			responseHeaders,
 			DashboardConfigurationHeaders.getInstance(),
-			new DashboardConfigHandler(
-				executor,
-				restConfiguration.getRefreshInterval()));
+			restConfiguration.getRefreshInterval());
 
 		JobsOverviewHandler<DispatcherGateway> jobsOverviewHandler = new JobsOverviewHandler<>(
 			restAddressFuture,
@@ -331,7 +328,7 @@ public class DispatcherRestEndpoint extends RestServerEndpoint {
 
 		handlers.add(Tuple2.of(ClusterOverviewHeaders.getInstance(), clusterOverviewHandler));
 		handlers.add(Tuple2.of(ClusterConfigurationInfoHeaders.getInstance(), clusterConfigurationHandler));
-		handlers.add(Tuple2.of(DashboardConfigurationHeaders.getInstance(), dashboardConfigurationHandler));
+		handlers.add(Tuple2.of(DashboardConfigurationHeaders.getInstance(), dashboardConfigHandler));
 		handlers.add(Tuple2.of(JobsOverviewHeaders.getInstance(), jobsOverviewHandler));
 		handlers.add(Tuple2.of(JobTerminationHeaders.getInstance(), jobTerminationHandler));
 		handlers.add(Tuple2.of(JobConfigHeaders.getInstance(), jobConfigHandler));

http://git-wip-us.apache.org/repos/asf/flink/blob/fa967dfc/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/DashboardConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/DashboardConfigHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/DashboardConfigHandler.java
new file mode 100644
index 0000000..785cd72
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/DashboardConfigHandler.java
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.runtime.rest.handler.cluster;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.messages.DashboardConfiguration;
+import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+
+import javax.annotation.Nonnull;
+
+import java.time.ZonedDateTime;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Handler which returns the dashboard configuration.
+ *
+ * @param <T> type of the leader gateway
+ */
+public class DashboardConfigHandler<T extends RestfulGateway> extends AbstractRestHandler<T, EmptyRequestBody, DashboardConfiguration, EmptyMessageParameters> {
+
+	private final DashboardConfiguration dashboardConfiguration;
+
+	public DashboardConfigHandler(
+			CompletableFuture<String> localRestAddress,
+			GatewayRetriever<? extends T> leaderRetriever,
+			Time timeout,
+			Map<String, String> responseHeaders,
+			MessageHeaders<EmptyRequestBody, DashboardConfiguration, EmptyMessageParameters> messageHeaders,
+			long refreshInterval) {
+		super(localRestAddress, leaderRetriever, timeout, responseHeaders, messageHeaders);
+
+		dashboardConfiguration = DashboardConfiguration.from(refreshInterval, ZonedDateTime.now());
+	}
+
+	@Override
+	public CompletableFuture<DashboardConfiguration> handleRequest(@Nonnull HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request, @Nonnull T gateway) {
+		return CompletableFuture.completedFuture(dashboardConfiguration);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fa967dfc/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandler.java
index 89f5fca..4f055f3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/DashboardConfigHandler.java
@@ -18,13 +18,8 @@
 
 package org.apache.flink.runtime.rest.handler.legacy;
 
-import org.apache.flink.runtime.dispatcher.DispatcherGateway;
 import org.apache.flink.runtime.jobmaster.JobManagerGateway;
-import org.apache.flink.runtime.rest.handler.HandlerRequest;
-import org.apache.flink.runtime.rest.handler.LegacyRestHandler;
 import org.apache.flink.runtime.rest.messages.DashboardConfiguration;
-import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
-import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
 
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator;
 
@@ -40,7 +35,7 @@ import java.util.concurrent.Executor;
  * against this web server should behave. It defines for example the refresh interval,
  * and time zone of the server timestamps.
  */
-public class DashboardConfigHandler extends AbstractJsonRequestHandler implements LegacyRestHandler<DispatcherGateway, DashboardConfiguration, EmptyMessageParameters> {
+public class DashboardConfigHandler extends AbstractJsonRequestHandler {
 
 	public static final String DASHBOARD_CONFIG_REST_PATH = "/config";
 
@@ -68,11 +63,6 @@ public class DashboardConfigHandler extends AbstractJsonRequestHandler implement
 	}
 
 	@Override
-	public CompletableFuture<DashboardConfiguration> handleRequest(HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request, DispatcherGateway gateway) {
-		return CompletableFuture.completedFuture(dashboardConfiguration);
-	}
-
-	@Override
 	public CompletableFuture<String> handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, JobManagerGateway jobManagerGateway) {
 		return CompletableFuture.completedFuture(configString);
 	}