Skip to content

Commit

Permalink
[FLINK-10602] Use metric's ActorSystem in MetricFetcher
Browse files Browse the repository at this point in the history
Use the metric system's ActorSystem in the MetricFetcher. This makes the fetching
of metrics run in a separate ActorSystem from the one used in the RpcService. This
should decouple Flink from the metric system further and allows to individually
configure both systems.
  • Loading branch information
tillrohrmann committed Oct 18, 2018
1 parent bbdf153 commit f6a7100
Show file tree
Hide file tree
Showing 5 changed files with 50 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;

Expand Down Expand Up @@ -102,6 +99,7 @@ public DispatcherResourceManagerComponent<T> create(
HeartbeatServices heartbeatServices,
MetricRegistry metricRegistry,
ArchivedExecutionGraphStore archivedExecutionGraphStore,
MetricQueryServiceRetriever metricQueryServiceRetriever,
FatalErrorHandler fatalErrorHandler) throws Exception {

LeaderRetrievalService dispatcherLeaderRetrievalService = null;
Expand Down Expand Up @@ -130,10 +128,6 @@ public DispatcherResourceManagerComponent<T> create(
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,
Expand All @@ -143,7 +137,7 @@ public DispatcherResourceManagerComponent<T> create(
configuration.getInteger(RestOptions.SERVER_NUM_THREADS),
configuration.getInteger(RestOptions.SERVER_THREAD_PRIORITY),
"DispatcherRestEndpoint"),
new AkkaQueryServiceRetriever(actorSystem, timeout),
metricQueryServiceRetriever,
highAvailabilityServices.getWebMonitorLeaderElectionService(),
fatalErrorHandler);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}.
Expand All @@ -41,5 +42,6 @@ DispatcherResourceManagerComponent<T> create(
HeartbeatServices heartbeatServices,
MetricRegistry metricRegistry,
ArchivedExecutionGraphStore archivedExecutionGraphStore,
MetricQueryServiceRetriever metricQueryServiceRetriever,
FatalErrorHandler fatalErrorHandler) throws Exception;
}
Original file line number Diff line number Diff line change
@@ -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:https://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 + '.'));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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();
Expand Down

0 comments on commit f6a7100

Please sign in to comment.