Skip to content

Commit

Permalink
[FLINK-17308] Extract ExecutionGraphCache interface and rename impl i…
Browse files Browse the repository at this point in the history
…nto DefaultExecutionGraphCache

This commit extracts the ExecutionGraphCache interface from the implementation and renames
the latter into DefaultExecutionGraphCache. Moreover, it introduces the NoOpExecutionGraphCache
implementation which is used for the DocumentingDispatcherRestEndpoint.
  • Loading branch information
tillrohrmann committed Apr 24, 2020
1 parent 3635b54 commit 08025ec
Show file tree
Hide file tree
Showing 12 changed files with 240 additions and 143 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.flink.runtime.leaderelection.LeaderElectionService;
import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration;
import org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache;
import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher;
import org.apache.flink.runtime.rpc.FatalErrorHandler;
Expand Down Expand Up @@ -52,7 +53,7 @@ WebMonitorEndpoint<T> createRestEndpoint(
FatalErrorHandler fatalErrorHandler) throws Exception;

static ExecutionGraphCache createExecutionGraphCache(RestHandlerConfiguration restConfiguration) {
return new ExecutionGraphCache(
return new DefaultExecutionGraphCache(
restConfiguration.getTimeout(),
Time.milliseconds(restConfiguration.getRefreshInterval()));
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,157 @@
/*
* 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.rest.handler.legacy;

import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
import org.apache.flink.runtime.webmonitor.RestfulGateway;
import org.apache.flink.util.Preconditions;

import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Function;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Default implementation of {@link ExecutionGraphCache}.
*/
public class DefaultExecutionGraphCache implements ExecutionGraphCache {

private final Time timeout;

private final Time timeToLive;

private final ConcurrentHashMap<JobID, ExecutionGraphEntry> cachedExecutionGraphs;

private volatile boolean running = true;

public DefaultExecutionGraphCache(
Time timeout,
Time timeToLive) {
this.timeout = checkNotNull(timeout);
this.timeToLive = checkNotNull(timeToLive);

cachedExecutionGraphs = new ConcurrentHashMap<>(4);
}

@Override
public void close() {
running = false;

// clear all cached AccessExecutionGraphs
cachedExecutionGraphs.clear();
}

@Override
public int size() {
return cachedExecutionGraphs.size();
}

@Override
public CompletableFuture<AccessExecutionGraph> getExecutionGraph(JobID jobId, RestfulGateway restfulGateway) {
return getExecutionGraphInternal(jobId, restfulGateway).thenApply(Function.identity());
}

private CompletableFuture<ArchivedExecutionGraph> getExecutionGraphInternal(JobID jobId, RestfulGateway restfulGateway) {
Preconditions.checkState(running, "ExecutionGraphCache is no longer running");

while (true) {
final ExecutionGraphEntry oldEntry = cachedExecutionGraphs.get(jobId);

final long currentTime = System.currentTimeMillis();

if (oldEntry != null && currentTime < oldEntry.getTTL()) {
final CompletableFuture<ArchivedExecutionGraph> executionGraphFuture = oldEntry.getExecutionGraphFuture();
if (!executionGraphFuture.isCompletedExceptionally()) {
return executionGraphFuture;
}
// otherwise it must be completed exceptionally
}

final ExecutionGraphEntry newEntry = new ExecutionGraphEntry(currentTime + timeToLive.toMilliseconds());

final boolean successfulUpdate;

if (oldEntry == null) {
successfulUpdate = cachedExecutionGraphs.putIfAbsent(jobId, newEntry) == null;
} else {
successfulUpdate = cachedExecutionGraphs.replace(jobId, oldEntry, newEntry);
// cancel potentially outstanding futures
oldEntry.getExecutionGraphFuture().cancel(false);
}

if (successfulUpdate) {
final CompletableFuture<ArchivedExecutionGraph> executionGraphFuture = restfulGateway.requestJob(jobId, timeout);

executionGraphFuture.whenComplete(
(ArchivedExecutionGraph executionGraph, Throwable throwable) -> {
if (throwable != null) {
newEntry.getExecutionGraphFuture().completeExceptionally(throwable);

// remove exceptionally completed entry because it doesn't help
cachedExecutionGraphs.remove(jobId, newEntry);
} else {
newEntry.getExecutionGraphFuture().complete(executionGraph);
}
});

if (!running) {
// delete newly added entry in case of a concurrent stopping operation
cachedExecutionGraphs.remove(jobId, newEntry);
}

return newEntry.getExecutionGraphFuture();
}
}
}

@Override
public void cleanup() {
long currentTime = System.currentTimeMillis();

// remove entries which have exceeded their time to live
cachedExecutionGraphs.values().removeIf(
(ExecutionGraphEntry entry) -> currentTime >= entry.getTTL());
}

/**
* Wrapper containing the current execution graph and it's time to live (TTL).
*/
private static final class ExecutionGraphEntry {
private final long ttl;

private final CompletableFuture<ArchivedExecutionGraph> executionGraphFuture;

ExecutionGraphEntry(long ttl) {
this.ttl = ttl;
this.executionGraphFuture = new CompletableFuture<>();
}

public long getTTL() {
return ttl;
}

public CompletableFuture<ArchivedExecutionGraph> getExecutionGraphFuture() {
return executionGraphFuture;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@
* "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
* 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,
Expand All @@ -19,57 +19,24 @@
package org.apache.flink.runtime.rest.handler.legacy;

import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
import org.apache.flink.runtime.webmonitor.RestfulGateway;
import org.apache.flink.util.Preconditions;

import java.io.Closeable;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Function;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Cache for {@link ArchivedExecutionGraph} which are obtained from the Flink cluster. Every cache entry
* has an associated time to live after which a new request will trigger the reloading of the
* {@link ArchivedExecutionGraph} from the cluster.
*/
public class ExecutionGraphCache implements Closeable {

private final Time timeout;

private final Time timeToLive;

private final ConcurrentHashMap<JobID, ExecutionGraphEntry> cachedExecutionGraphs;

private volatile boolean running = true;

public ExecutionGraphCache(
Time timeout,
Time timeToLive) {
this.timeout = checkNotNull(timeout);
this.timeToLive = checkNotNull(timeToLive);

cachedExecutionGraphs = new ConcurrentHashMap<>(4);
}

@Override
public void close() {
running = false;

// clear all cached AccessExecutionGraphs
cachedExecutionGraphs.clear();
}
public interface ExecutionGraphCache extends Closeable {

/**
* Gets the number of cache entries.
*/
public int size() {
return cachedExecutionGraphs.size();
}
int size();

/**
* Gets the {@link AccessExecutionGraph} for the given {@link JobID} and caches it. The
Expand All @@ -80,93 +47,16 @@ public int size() {
* @param restfulGateway to request the {@link ArchivedExecutionGraph} from
* @return Future containing the requested {@link ArchivedExecutionGraph}
*/
public CompletableFuture<AccessExecutionGraph> getExecutionGraph(JobID jobId, RestfulGateway restfulGateway) {
return getExecutionGraphInternal(jobId, restfulGateway).thenApply(Function.identity());
}

private CompletableFuture<ArchivedExecutionGraph> getExecutionGraphInternal(JobID jobId, RestfulGateway restfulGateway) {
Preconditions.checkState(running, "ExecutionGraphCache is no longer running");

while (true) {
final ExecutionGraphEntry oldEntry = cachedExecutionGraphs.get(jobId);

final long currentTime = System.currentTimeMillis();

if (oldEntry != null && currentTime < oldEntry.getTTL()) {
final CompletableFuture<ArchivedExecutionGraph> executionGraphFuture = oldEntry.getExecutionGraphFuture();
if (!executionGraphFuture.isCompletedExceptionally()) {
return executionGraphFuture;
}
// otherwise it must be completed exceptionally
}

final ExecutionGraphEntry newEntry = new ExecutionGraphEntry(currentTime + timeToLive.toMilliseconds());

final boolean successfulUpdate;

if (oldEntry == null) {
successfulUpdate = cachedExecutionGraphs.putIfAbsent(jobId, newEntry) == null;
} else {
successfulUpdate = cachedExecutionGraphs.replace(jobId, oldEntry, newEntry);
// cancel potentially outstanding futures
oldEntry.getExecutionGraphFuture().cancel(false);
}

if (successfulUpdate) {
final CompletableFuture<ArchivedExecutionGraph> executionGraphFuture = restfulGateway.requestJob(jobId, timeout);

executionGraphFuture.whenComplete(
(ArchivedExecutionGraph executionGraph, Throwable throwable) -> {
if (throwable != null) {
newEntry.getExecutionGraphFuture().completeExceptionally(throwable);

// remove exceptionally completed entry because it doesn't help
cachedExecutionGraphs.remove(jobId, newEntry);
} else {
newEntry.getExecutionGraphFuture().complete(executionGraph);
}
});

if (!running) {
// delete newly added entry in case of a concurrent stopping operation
cachedExecutionGraphs.remove(jobId, newEntry);
}

return newEntry.getExecutionGraphFuture();
}
}
}
CompletableFuture<AccessExecutionGraph> getExecutionGraph(JobID jobId, RestfulGateway restfulGateway);

/**
* Perform the cleanup of out dated {@link ExecutionGraphEntry}.
* Perform the cleanup of out dated cache entries.
*/
public void cleanup() {
long currentTime = System.currentTimeMillis();

// remove entries which have exceeded their time to live
cachedExecutionGraphs.values().removeIf(
(ExecutionGraphEntry entry) -> currentTime >= entry.getTTL());
}
void cleanup();

/**
* Wrapper containing the current execution graph and it's time to live (TTL).
* Closes the execution graph cache.
*/
private static final class ExecutionGraphEntry {
private final long ttl;

private final CompletableFuture<ArchivedExecutionGraph> executionGraphFuture;

ExecutionGraphEntry(long ttl) {
this.ttl = ttl;
this.executionGraphFuture = new CompletableFuture<>();
}

public long getTTL() {
return ttl;
}

public CompletableFuture<ArchivedExecutionGraph> getExecutionGraphFuture() {
return executionGraphFuture;
}
}
@Override
void close();
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.rest.handler.HandlerRequest;
import org.apache.flink.runtime.rest.handler.HandlerRequestException;
import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
import org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache;
import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionConfigBuilder;
import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
Expand Down Expand Up @@ -57,7 +57,7 @@ public void handleRequest_executionConfigWithSecretValues_excludesSecretValuesFr
TestingUtils.TIMEOUT(),
Collections.emptyMap(),
JobConfigHeaders.getInstance(),
new ExecutionGraphCache(TestingUtils.TIMEOUT(), TestingUtils.TIMEOUT()),
new DefaultExecutionGraphCache(TestingUtils.TIMEOUT(), TestingUtils.TIMEOUT()),
TestingUtils.defaultExecutor());

final Map<String, String> globalJobParameters = new HashMap<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.rest.handler.HandlerRequest;
import org.apache.flink.runtime.rest.handler.HandlerRequestException;
import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
import org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache;
import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
import org.apache.flink.runtime.rest.messages.JobExceptionsHeaders;
Expand Down Expand Up @@ -65,7 +65,7 @@ public void testGetJobExceptionsInfo() throws HandlerRequestException {
TestingUtils.TIMEOUT(),
Collections.emptyMap(),
JobExceptionsHeaders.getInstance(),
new ExecutionGraphCache(TestingUtils.TIMEOUT(), TestingUtils.TIMEOUT()),
new DefaultExecutionGraphCache(TestingUtils.TIMEOUT(), TestingUtils.TIMEOUT()),
TestingUtils.defaultExecutor());
final int numExceptions = 20;
final AccessExecutionGraph archivedExecutionGraph = createAccessExecutionGraph(numExceptions);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.rest.handler.HandlerRequest;
import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration;
import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
import org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache;
import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher;
import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcherImpl;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
Expand Down Expand Up @@ -125,7 +125,7 @@ public void testHandleRequest() throws Exception {
Time.milliseconds(100),
Collections.emptyMap(),
SubtaskCurrentAttemptDetailsHeaders.getInstance(),
new ExecutionGraphCache(
new DefaultExecutionGraphCache(
restHandlerConfiguration.getTimeout(),
Time.milliseconds(restHandlerConfiguration.getRefreshInterval())),
TestingUtils.defaultExecutor(),
Expand Down
Loading

0 comments on commit 08025ec

Please sign in to comment.