You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2018/10/18 17:07:31 UTC

[GitHub] tillrohrmann closed pull request #6877: [FLINK-10602] Use metric's ActorSystem in MetricFetcher

tillrohrmann closed pull request #6877: [FLINK-10602] Use metric's ActorSystem in MetricFetcher
URL: https://github.com/apache/flink/pull/6877
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java
index 35fad322ef9..150f17fa04c 100755
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java
@@ -54,6 +54,7 @@
 import org.apache.flink.runtime.security.SecurityContext;
 import org.apache.flink.runtime.security.SecurityUtils;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever;
 import org.apache.flink.util.AutoCloseableAsync;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FileUtils;
@@ -219,6 +220,9 @@ private void runCluster(Configuration configuration) throws Exception {
 				heartbeatServices,
 				metricRegistry,
 				archivedExecutionGraphStore,
+				new AkkaQueryServiceRetriever(
+					metricQueryServiceActorSystem,
+					Time.milliseconds(configuration.getLong(WebOptions.TIMEOUT))),
 				this);
 
 			clusterComponent.getShutDownFuture().whenComplete(
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java
index 6d557d0815f..354245d29ba 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java
@@ -22,7 +22,6 @@
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.ConfigurationUtils;
 import org.apache.flink.configuration.RestOptions;
-import org.apache.flink.configuration.WebOptions;
 import org.apache.flink.runtime.blob.BlobServer;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.FutureUtils;
@@ -46,16 +45,14 @@
 import org.apache.flink.runtime.rest.RestEndpointFactory;
 import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
 import org.apache.flink.runtime.webmonitor.RestfulGateway;
 import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint;
 import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever;
-import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
 import org.apache.flink.runtime.webmonitor.retriever.impl.RpcGatewayRetriever;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FlinkException;
 
-import akka.actor.ActorSystem;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -102,6 +99,7 @@ public AbstractDispatcherResourceManagerComponentFactory(
 			HeartbeatServices heartbeatServices,
 			MetricRegistry metricRegistry,
 			ArchivedExecutionGraphStore archivedExecutionGraphStore,
+			MetricQueryServiceRetriever metricQueryServiceRetriever,
 			FatalErrorHandler fatalErrorHandler) throws Exception {
 
 		LeaderRetrievalService dispatcherLeaderRetrievalService = null;
@@ -130,10 +128,6 @@ public AbstractDispatcherResourceManagerComponentFactory(
 				10,
 				Time.milliseconds(50L));
 
-			// TODO: Remove once we have ported the MetricFetcher to the RpcEndpoint
-			final ActorSystem actorSystem = ((AkkaRpcService) rpcService).getActorSystem();
-			final Time timeout = Time.milliseconds(configuration.getLong(WebOptions.TIMEOUT));
-
 			webMonitorEndpoint = restEndpointFactory.createRestEndpoint(
 				configuration,
 				dispatcherGatewayRetriever,
@@ -143,7 +137,7 @@ public AbstractDispatcherResourceManagerComponentFactory(
 					configuration.getInteger(RestOptions.SERVER_NUM_THREADS),
 					configuration.getInteger(RestOptions.SERVER_THREAD_PRIORITY),
 					"DispatcherRestEndpoint"),
-				new AkkaQueryServiceRetriever(actorSystem, timeout),
+				metricQueryServiceRetriever,
 				highAvailabilityServices.getWebMonitorLeaderElectionService(),
 				fatalErrorHandler);
 
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponentFactory.java
index df22a59d955..51ddb8f6df9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponentFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponentFactory.java
@@ -27,6 +27,7 @@
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
 
 /**
  * Factory for the {@link DispatcherResourceManagerComponent}.
@@ -41,5 +42,6 @@
 		HeartbeatServices heartbeatServices,
 		MetricRegistry metricRegistry,
 		ArchivedExecutionGraphStore archivedExecutionGraphStore,
+		MetricQueryServiceRetriever metricQueryServiceRetriever,
 		FatalErrorHandler fatalErrorHandler) throws Exception;
 }
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/VoidMetricQueryServiceRetriever.java b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/VoidMetricQueryServiceRetriever.java
new file mode 100644
index 00000000000..542d68e3fb1
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/VoidMetricQueryServiceRetriever.java
@@ -0,0 +1,39 @@
+/*
+ * 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.webmonitor.retriever.impl;
+
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceGateway;
+import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
+import org.apache.flink.util.FlinkException;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * {@link MetricQueryServiceRetriever} implementation which always fails the retrieval of
+ * the metric query service.
+ */
+public enum VoidMetricQueryServiceRetriever implements MetricQueryServiceRetriever {
+	INSTANCE;
+
+	@Override
+	public CompletableFuture<MetricQueryServiceGateway> retrieveService(String queryServicePath) {
+		return FutureUtils.completedExceptionally(new FlinkException("Cannot retrieve metric query service for " + queryServicePath + '.'));
+	}
+}
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
index 2e39bafe67f..fbeb579e6ef 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
@@ -55,6 +55,7 @@
 import org.apache.flink.runtime.util.LeaderConnectionInfo;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.apache.flink.runtime.webmonitor.retriever.impl.VoidMetricQueryServiceRetriever;
 import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
 import org.apache.flink.util.TestLogger;
 import org.apache.flink.util.function.CheckedSupplier;
@@ -151,6 +152,7 @@ public void testCancelingOnProcessFailure() throws Exception {
 				new HeartbeatServices(100L, 1000L),
 				NoOpMetricRegistry.INSTANCE,
 				new MemoryArchivedExecutionGraphStore(),
+				VoidMetricQueryServiceRetriever.INSTANCE,
 				fatalErrorHandler);
 
 			final Map<String, String> keyValues = config.toMap();


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services