Skip to content

Commit

Permalink
[FLINK-7974][QS] Wait for QS abstract server to shutdown.
Browse files Browse the repository at this point in the history
  • Loading branch information
kl0u committed Dec 6, 2017
1 parent 5760677 commit 74d052b
Show file tree
Hide file tree
Showing 19 changed files with 306 additions and 158 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointTriggerHeaders;
import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointTriggerResponseBody;
import org.apache.flink.runtime.util.ExecutorThreadFactory;
import org.apache.flink.util.ExecutorUtils;

import javax.annotation.Nullable;

Expand Down Expand Up @@ -90,7 +91,7 @@ public void shutdown() {
log.error("An error occurred during the client shutdown.", e);
}
this.restClient.shutdown(Time.seconds(5));
org.apache.flink.runtime.concurrent.Executors.gracefulShutdown(5, TimeUnit.SECONDS, this.executorService);
ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, this.executorService);
}

@Override
Expand Down
77 changes: 77 additions & 0 deletions flink-core/src/main/java/org/apache/flink/util/ExecutorUtils.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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.util;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;

/**
* Utilities for {@link java.util.concurrent.Executor Executors}.
*/
public class ExecutorUtils {

private static final Logger LOG = LoggerFactory.getLogger(ExecutorUtils.class);

/**
* Gracefully shutdown the given {@link ExecutorService}. The call waits the given timeout that
* all ExecutorServices terminate. If the ExecutorServices do not terminate in this time,
* they will be shut down hard.
*
* @param timeout to wait for the termination of all ExecutorServices
* @param unit of the timeout
* @param executorServices to shut down
*/
public static void gracefulShutdown(long timeout, TimeUnit unit, ExecutorService... executorServices) {
for (ExecutorService executorService: executorServices) {
executorService.shutdown();
}

boolean wasInterrupted = false;
final long endTime = unit.toMillis(timeout) + System.currentTimeMillis();
long timeLeft = unit.toMillis(timeout);
boolean hasTimeLeft = timeLeft > 0L;

for (ExecutorService executorService: executorServices) {
if (wasInterrupted || !hasTimeLeft) {
executorService.shutdownNow();
} else {
try {
if (!executorService.awaitTermination(timeLeft, TimeUnit.MILLISECONDS)) {
LOG.warn("ExecutorService did not terminate in time. Shutting it down now.");
executorService.shutdownNow();
}
} catch (InterruptedException e) {
LOG.warn("Interrupted while shutting down executor services. Shutting all " +
"remaining ExecutorServices down now.", e);
executorService.shutdownNow();

wasInterrupted = true;

Thread.currentThread().interrupt();
}

timeLeft = endTime - System.currentTimeMillis();
hasTimeLeft = timeLeft > 0L;
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@
import org.apache.flink.runtime.webmonitor.WebMonitor;
import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaJobManagerRetriever;
import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever;
import org.apache.flink.util.ExecutorUtils;

import akka.actor.ActorRef;
import akka.actor.ActorSystem;
Expand Down Expand Up @@ -439,7 +440,7 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie
}
}

org.apache.flink.runtime.concurrent.Executors.gracefulShutdown(
ExecutorUtils.gracefulShutdown(
AkkaUtils.getTimeout(config).toMillis(),
TimeUnit.MILLISECONDS,
futureExecutor,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.queryablestate.network.messages.MessageBody;
import org.apache.flink.util.ExecutorUtils;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.Preconditions;

Expand All @@ -45,10 +46,12 @@
import java.util.HashSet;
import java.util.Iterator;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;

/**
* The base class for every server in the queryable state module.
Expand Down Expand Up @@ -83,6 +86,9 @@ public abstract class AbstractServerBase<REQ extends MessageBody, RESP extends M
/** The number of threads to be used for query serving. */
private final int numQueryThreads;

/** Atomic shut down future. */
private final AtomicReference<CompletableFuture<Void>> serverShutdownFuture = new AtomicReference<>(null);

/** Netty's ServerBootstrap. */
private ServerBootstrap bootstrap;

Expand Down Expand Up @@ -179,8 +185,8 @@ public InetSocketAddress getServerAddress() {
* @throws Exception If something goes wrong during the bind operation.
*/
public void start() throws Throwable {
Preconditions.checkState(serverAddress == null,
serverName + " is already running @ " + serverAddress + '.');
Preconditions.checkState(serverAddress == null && serverShutdownFuture.get() == null,
serverName + " is already running @ " + serverAddress + ". ");

Iterator<Integer> portIterator = bindPortRange.iterator();
while (portIterator.hasNext() && !attemptToBind(portIterator.next())) {}
Expand Down Expand Up @@ -251,34 +257,85 @@ private boolean attemptToBind(final int port) throws Throwable {
throw future.cause();
} catch (BindException e) {
log.debug("Failed to start {} on port {}: {}.", serverName, port, e.getMessage());
shutdown();
try {
// we shutdown the server but we reset the future every time because in
// case of failure to bind, we will call attemptToBind() here, and not resetting
// the flag will interfere with future shutdown attempts.

shutdownServer()
.whenComplete((ignoredV, ignoredT) -> serverShutdownFuture.getAndSet(null))
.get();
} catch (Exception r) {

// Here we were seeing this problem:
// https://github.com/netty/netty/issues/4357 if we do a get().
// this is why we now simply wait a bit so that everything is shut down.

log.warn("Problem while shutting down {}: {}", serverName, r.getMessage());
}
}
// any other type of exception we let it bubble up.
return false;
}

/**
* Shuts down the server and all related thread pools.
* @return A {@link CompletableFuture} that will be completed upon termination of the shutdown process.
*/
public void shutdown() {
log.info("Shutting down {} @ {}", serverName, serverAddress);

if (handler != null) {
handler.shutdown();
handler = null;
}

if (queryExecutor != null) {
queryExecutor.shutdown();
}
public CompletableFuture<Void> shutdownServer() {
CompletableFuture<Void> shutdownFuture = new CompletableFuture<>();
if (serverShutdownFuture.compareAndSet(null, shutdownFuture)) {
log.info("Shutting down {} @ {}", serverName, serverAddress);

final CompletableFuture<Void> groupShutdownFuture = new CompletableFuture<>();
if (bootstrap != null) {
EventLoopGroup group = bootstrap.group();
if (group != null && !group.isShutdown()) {
group.shutdownGracefully(0L, 0L, TimeUnit.MILLISECONDS)
.addListener(finished -> {
if (finished.isSuccess()) {
groupShutdownFuture.complete(null);
} else {
groupShutdownFuture.completeExceptionally(finished.cause());
}
});
} else {
groupShutdownFuture.complete(null);
}
} else {
groupShutdownFuture.complete(null);
}

if (bootstrap != null) {
EventLoopGroup group = bootstrap.group();
if (group != null) {
group.shutdownGracefully(0L, 10L, TimeUnit.SECONDS);
final CompletableFuture<Void> handlerShutdownFuture = new CompletableFuture<>();
if (handler == null) {
handlerShutdownFuture.complete(null);
} else {
handler.shutdown().whenComplete((result, throwable) -> {
if (throwable != null) {
handlerShutdownFuture.completeExceptionally(throwable);
} else {
handlerShutdownFuture.complete(null);
}
});
}

final CompletableFuture<Void> queryExecShutdownFuture = CompletableFuture.runAsync(() -> {
if (queryExecutor != null) {
ExecutorUtils.gracefulShutdown(10L, TimeUnit.MINUTES, queryExecutor);
}
});

CompletableFuture.allOf(
queryExecShutdownFuture, groupShutdownFuture, handlerShutdownFuture
).whenComplete((result, throwable) -> {
if (throwable != null) {
shutdownFuture.completeExceptionally(throwable);
} else {
shutdownFuture.complete(null);
}
});
}
serverAddress = null;
return serverShutdownFuture.get();
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -183,9 +183,16 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E
public abstract CompletableFuture<RESP> handleRequest(final long requestId, final REQ request);

/**
* Shuts down any handler specific resources, e.g. thread pools etc.
* Shuts down any handler-specific resources, e.g. thread pools etc and returns
* a {@link CompletableFuture}.
*
* <p>If an exception is thrown during the shutdown process, then that exception
* will be included in the returned future.
*
* @return A {@link CompletableFuture} that will be completed when the shutdown
* process actually finishes.
*/
public abstract void shutdown();
public abstract CompletableFuture<Void> shutdown();

/**
* Task to execute the actual query against the state instance.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,9 @@
import org.apache.flink.queryablestate.network.stats.DisabledKvStateRequestStats;
import org.apache.flink.queryablestate.network.stats.KvStateRequestStats;
import org.apache.flink.queryablestate.server.KvStateServerImpl;
import org.apache.flink.runtime.concurrent.Executors;
import org.apache.flink.runtime.concurrent.FutureUtils;
import org.apache.flink.runtime.messages.FlinkJobNotFoundException;
import org.apache.flink.runtime.query.KvStateClientProxy;
import org.apache.flink.runtime.query.KvStateLocation;
import org.apache.flink.runtime.query.KvStateMessage;
Expand All @@ -42,6 +44,7 @@

import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;

import akka.dispatch.OnComplete;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -141,6 +144,7 @@ private void executeActionAsync(
// KvStateLocation. Therefore we retry this query and
// force look up the location.

LOG.debug("Retrying after failing to retrieve state due to: {}.", throwable.getCause().getMessage());
executeActionAsync(result, request, true);
} else {
result.completeExceptionally(throwable);
Expand Down Expand Up @@ -203,20 +207,34 @@ private CompletableFuture<KvStateLocation> getKvStateLookupInfo(

LOG.debug("Retrieving location for state={} of job={} from the job manager.", jobId, queryableStateName);

final CompletableFuture<KvStateLocation> location = new CompletableFuture<>();
lookupCache.put(cacheKey, location);
return proxy.getJobManagerFuture().thenComposeAsync(
jobManagerGateway -> {
final Object msg = new KvStateMessage.LookupKvStateLocation(jobId, queryableStateName);
final CompletableFuture<KvStateLocation> locationFuture = FutureUtils.toJava(
jobManagerGateway.ask(msg, FiniteDuration.apply(1000L, TimeUnit.MILLISECONDS))
.mapTo(ClassTag$.MODULE$.<KvStateLocation>apply(KvStateLocation.class)));

lookupCache.put(cacheKey, locationFuture);
return locationFuture;
jobManagerGateway.ask(msg, FiniteDuration.apply(1000L, TimeUnit.MILLISECONDS))
.mapTo(ClassTag$.MODULE$.<KvStateLocation>apply(KvStateLocation.class))
.onComplete(new OnComplete<KvStateLocation>() {

@Override
public void onComplete(Throwable failure, KvStateLocation loc) throws Throwable {
if (failure != null) {
if (failure instanceof FlinkJobNotFoundException) {
// if the jobId was wrong, remove the entry from the cache.
lookupCache.remove(cacheKey);
}
location.completeExceptionally(failure);
} else {
location.complete(loc);
}
}
}, Executors.directExecutionContext());
return location;
}, queryExecutor);
}

@Override
public void shutdown() {
kvStateClient.shutdown();
public CompletableFuture<Void> shutdown() {
return kvStateClient.shutdown();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import java.net.InetSocketAddress;
import java.util.Iterator;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;

/**
* The default implementation of the {@link KvStateClientProxy}.
Expand Down Expand Up @@ -96,7 +97,12 @@ public void start() throws Throwable {

@Override
public void shutdown() {
super.shutdown();
try {
shutdownServer().get(10L, TimeUnit.SECONDS);
log.info("{} was shutdown successfully.", getServerName());
} catch (Exception e) {
log.warn("{} shutdown failed: {}", getServerName(), e);
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ public CompletableFuture<KvStateResponse> handleRequest(final long requestId, fi
}

@Override
public void shutdown() {
// do nothing
public CompletableFuture<Void> shutdown() {
return CompletableFuture.completedFuture(null);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.util.Iterator;
import java.util.concurrent.TimeUnit;

/**
* The default implementation of the {@link KvStateServer}.
Expand Down Expand Up @@ -101,6 +102,11 @@ public InetSocketAddress getServerAddress() {

@Override
public void shutdown() {
super.shutdown();
try {
shutdownServer().get(10L, TimeUnit.SECONDS);
log.info("{} was shutdown successfully.", getServerName());
} catch (Exception e) {
log.warn("{} shutdown failed: {}", getServerName(), e);
}
}
}
Loading

0 comments on commit 74d052b

Please sign in to comment.