flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tzuli...@apache.org
Subject [7/9] flink git commit: [FLINK-7647] [flip6] Rename JobManagerConfigHandler to ClusterConfigHandler
Date Tue, 26 Sep 2017 16:04:49 GMT
[FLINK-7647] [flip6] Rename JobManagerConfigHandler to ClusterConfigHandler

The original naming wouldn't make sense for the FLIP-6 redesign, since
we would have multiple per-job JobManagers for each cluster, which
shares the same configuration.

The REST path is still left untouched and not part of this commit, as
that would involve more changes in flink-runtime-web.


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

Branch: refs/heads/master
Commit: 2b0008c50d27640e5a8a8be491528f4c248fdf32
Parents: 97ff043
Author: Tzu-Li (Gordon) Tai <tzulitai@apache.org>
Authored: Wed Sep 20 18:51:27 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tzulitai@apache.org>
Committed: Tue Sep 26 18:04:07 2017 +0200

----------------------------------------------------------------------
 .../runtime/webmonitor/WebRuntimeMonitor.java   |   4 +-
 .../dispatcher/DispatcherRestEndpoint.java      |   4 +-
 .../handler/legacy/ClusterConfigHandler.java    | 109 +++++++++++++++++++
 .../handler/legacy/JobManagerConfigHandler.java | 109 -------------------
 .../messages/ClusterConfigurationInfo.java      |   4 +-
 .../ClusterConfigurationInfoHeaders.java        |   6 +-
 .../legacy/ClusterConfigHandlerTest.java        |  38 +++++++
 .../legacy/JobManagerConfigHandlerTest.java     |  38 -------
 8 files changed, 157 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2b0008c5/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 cd128de..1af6ab6 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
@@ -28,6 +28,7 @@ import org.apache.flink.runtime.jobmaster.JobManagerGateway;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.net.SSLUtils;
 import org.apache.flink.runtime.rest.handler.WebHandler;
+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.ConstantTextHandler;
 import org.apache.flink.runtime.rest.handler.legacy.CurrentJobIdsHandler;
@@ -40,7 +41,6 @@ import org.apache.flink.runtime.rest.handler.legacy.JobCancellationWithSavepoint
 import org.apache.flink.runtime.rest.handler.legacy.JobConfigHandler;
 import org.apache.flink.runtime.rest.handler.legacy.JobDetailsHandler;
 import org.apache.flink.runtime.rest.handler.legacy.JobExceptionsHandler;
-import org.apache.flink.runtime.rest.handler.legacy.JobManagerConfigHandler;
 import org.apache.flink.runtime.rest.handler.legacy.JobPlanHandler;
 import org.apache.flink.runtime.rest.handler.legacy.JobStoppingHandler;
 import org.apache.flink.runtime.rest.handler.legacy.JobVertexAccumulatorsHandler;
@@ -244,7 +244,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 		get(router, new ClusterOverviewHandler(executor, DEFAULT_REQUEST_TIMEOUT));
 
 		// job manager configuration
-		get(router, new JobManagerConfigHandler(executor, config));
+		get(router, new ClusterConfigHandler(executor, config));
 
 		// overview over jobs
 		get(router, new CurrentJobsOverviewHandler(executor, DEFAULT_REQUEST_TIMEOUT, true, true));

http://git-wip-us.apache.org/repos/asf/flink/blob/2b0008c5/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 3e8a4cb..17f5616 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,10 +27,10 @@ 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.legacy.ClusterConfigHandler;
 import org.apache.flink.runtime.rest.handler.legacy.ClusterOverviewHandler;
 import org.apache.flink.runtime.rest.handler.legacy.CurrentJobsOverviewHandler;
 import org.apache.flink.runtime.rest.handler.legacy.DashboardConfigHandler;
-import org.apache.flink.runtime.rest.handler.legacy.JobManagerConfigHandler;
 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.ClusterConfigurationInfo;
@@ -119,7 +119,7 @@ public class DispatcherRestEndpoint extends RestServerEndpoint {
 			leaderRetriever,
 			timeout,
 			ClusterConfigurationInfoHeaders.getInstance(),
-			new JobManagerConfigHandler(
+			new ClusterConfigHandler(
 				executor,
 				clusterConfiguration));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/2b0008c5/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
new file mode 100644
index 0000000..d3ea160
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ClusterConfigHandler.java
@@ -0,0 +1,109 @@
+/*
+ * 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.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.handler.legacy.messages.ClusterConfigurationInfo;
+import org.apache.flink.runtime.rest.handler.legacy.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;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.Executor;
+
+/**
+ * Returns the Job Manager's configuration.
+ */
+public class ClusterConfigHandler extends AbstractJsonRequestHandler
+		implements LegacyRestHandler<DispatcherGateway, ClusterConfigurationInfo, EmptyMessageParameters>
{
+
+	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);
+	}
+
+	@Override
+	public String[] getPaths() {
+		return new String[]{ClusterConfigurationInfoHeaders.CLUSTER_CONFIG_REST_PATH};
+	}
+
+	@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,
+			JobManagerGateway jobManagerGateway) {
+
+		return CompletableFuture.completedFuture(clusterConfigJson);
+	}
+
+	private static String createConfigJson(Configuration config) {
+		try {
+			StringWriter writer = new StringWriter();
+			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
+
+			gen.writeStartArray();
+			for (String key : config.keySet()) {
+				gen.writeStartObject();
+				gen.writeStringField(ClusterConfigurationInfoEntry.FIELD_NAME_CONFIG_KEY, key);
+
+				String value = config.getString(key, null);
+				// Mask key values which contain sensitive information
+				if (value != null && key.toLowerCase().contains("password")) {
+					value = "******";
+				}
+				gen.writeStringField(ClusterConfigurationInfoEntry.FIELD_NAME_CONFIG_VALUE, value);
+
+				gen.writeEndObject();
+			}
+			gen.writeEndArray();
+
+			gen.close();
+			return writer.toString();
+		} catch (IOException e) {
+			throw new CompletionException(new FlinkException("Could not write configuration.", e));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2b0008c5/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandler.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandler.java
deleted file mode 100644
index 6c0c158..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandler.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * 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.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.handler.legacy.messages.ClusterConfigurationInfo;
-import org.apache.flink.runtime.rest.handler.legacy.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;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CompletionException;
-import java.util.concurrent.Executor;
-
-/**
- * Returns the Job Manager's configuration.
- */
-public class JobManagerConfigHandler extends AbstractJsonRequestHandler
-		implements LegacyRestHandler<DispatcherGateway, ClusterConfigurationInfo, EmptyMessageParameters>
{
-
-	private final ClusterConfigurationInfo clusterConfig;
-	private final String clusterConfigJson;
-
-	public JobManagerConfigHandler(Executor executor, Configuration config) {
-		super(executor);
-
-		Preconditions.checkNotNull(config);
-		this.clusterConfig = ClusterConfigurationInfo.from(config);
-		this.clusterConfigJson = createConfigJson(config);
-	}
-
-	@Override
-	public String[] getPaths() {
-		return new String[]{ClusterConfigurationInfoHeaders.CLUSTER_CONFIG_REST_PATH};
-	}
-
-	@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,
-			JobManagerGateway jobManagerGateway) {
-
-		return CompletableFuture.completedFuture(clusterConfigJson);
-	}
-
-	private static String createConfigJson(Configuration config) {
-		try {
-			StringWriter writer = new StringWriter();
-			JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
-
-			gen.writeStartArray();
-			for (String key : config.keySet()) {
-				gen.writeStartObject();
-				gen.writeStringField(ClusterConfigurationInfoEntry.FIELD_NAME_CONFIG_KEY, key);
-
-				String value = config.getString(key, null);
-				// Mask key values which contain sensitive information
-				if (value != null && key.toLowerCase().contains("password")) {
-					value = "******";
-				}
-				gen.writeStringField(ClusterConfigurationInfoEntry.FIELD_NAME_CONFIG_VALUE, value);
-
-				gen.writeEndObject();
-			}
-			gen.writeEndArray();
-
-			gen.close();
-			return writer.toString();
-		} catch (IOException e) {
-			throw new CompletionException(new FlinkException("Could not write configuration.", e));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2b0008c5/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/messages/ClusterConfigurationInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/messages/ClusterConfigurationInfo.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/messages/ClusterConfigurationInfo.java
index 8b87e63..bf527d9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/messages/ClusterConfigurationInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/messages/ClusterConfigurationInfo.java
@@ -19,13 +19,13 @@
 package org.apache.flink.runtime.rest.handler.legacy.messages;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.rest.handler.legacy.JobManagerConfigHandler;
+import org.apache.flink.runtime.rest.handler.legacy.ClusterConfigHandler;
 import org.apache.flink.runtime.rest.messages.ResponseBody;
 
 import java.util.ArrayList;
 
 /**
- * Response of the {@link JobManagerConfigHandler}, respresented as a list
+ * Response of the {@link ClusterConfigHandler}, respresented as a list
  * of key-value pairs of the cluster {@link Configuration}.
  */
 public class ClusterConfigurationInfo extends ArrayList<ClusterConfigurationInfoEntry>
implements ResponseBody {

http://git-wip-us.apache.org/repos/asf/flink/blob/2b0008c5/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterConfigurationInfoHeaders.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterConfigurationInfoHeaders.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterConfigurationInfoHeaders.java
index 764e497..0ad56b1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterConfigurationInfoHeaders.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterConfigurationInfoHeaders.java
@@ -19,18 +19,20 @@
 package org.apache.flink.runtime.rest.messages;
 
 import org.apache.flink.runtime.rest.HttpMethodWrapper;
-import org.apache.flink.runtime.rest.handler.legacy.JobManagerConfigHandler;
+import org.apache.flink.runtime.rest.handler.legacy.ClusterConfigHandler;
 import org.apache.flink.runtime.rest.handler.legacy.messages.ClusterConfigurationInfo;
 
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
 
 /**
- * Message headers for the {@link JobManagerConfigHandler}.
+ * Message headers for the {@link ClusterConfigHandler}.
  */
 public final class ClusterConfigurationInfoHeaders implements MessageHeaders<EmptyRequestBody,
ClusterConfigurationInfo, EmptyMessageParameters> {
 
 	private static final ClusterConfigurationInfoHeaders INSTANCE = new ClusterConfigurationInfoHeaders();
 
+	// TODO this REST path is inappropriately set due to legacy design reasons, and ideally
should be '/config';
+	// TODO changing it would require corresponding path changes in flink-runtime-web
 	public static final String CLUSTER_CONFIG_REST_PATH = "/jobmanager/config";
 
 	private ClusterConfigurationInfoHeaders() {}

http://git-wip-us.apache.org/repos/asf/flink/blob/2b0008c5/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ClusterConfigHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ClusterConfigHandlerTest.java
b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ClusterConfigHandlerTest.java
new file mode 100644
index 0000000..5094f01
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ClusterConfigHandlerTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.legacy;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.concurrent.Executors;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for the ClusterConfigHandler.
+ */
+public class ClusterConfigHandlerTest {
+	@Test
+	public void testGetPaths() {
+		ClusterConfigHandler handler = new ClusterConfigHandler(Executors.directExecutor(), new
Configuration());
+		String[] paths = handler.getPaths();
+		Assert.assertEquals(1, paths.length);
+		Assert.assertEquals("/jobmanager/config", paths[0]);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2b0008c5/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandlerTest.java
b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandlerTest.java
deleted file mode 100644
index f21b1ed..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/JobManagerConfigHandlerTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.legacy;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.concurrent.Executors;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Tests for the JobManagerConfigHandler.
- */
-public class JobManagerConfigHandlerTest {
-	@Test
-	public void testGetPaths() {
-		JobManagerConfigHandler handler = new JobManagerConfigHandler(Executors.directExecutor(),
new Configuration());
-		String[] paths = handler.getPaths();
-		Assert.assertEquals(1, paths.length);
-		Assert.assertEquals("/jobmanager/config", paths[0]);
-	}
-}


Mime
View raw message