flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-7974) AbstractServerBase#shutdown does not wait for shutdown completion
Date Fri, 10 Nov 2017 13:15:10 GMT

    [ https://issues.apache.org/jira/browse/FLINK-7974?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16247471#comment-16247471
] 

ASF GitHub Bot commented on FLINK-7974:
---------------------------------------

Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4993#discussion_r150225215
  
    --- Diff: flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java
---
    @@ -260,25 +262,47 @@ private boolean attemptToBind(final int port) throws Throwable {
     	/**
     	 * Shuts down the server and all related thread pools.
     	 */
    -	public void shutdown() {
    -		LOG.info("Shutting down server {} @ {}", serverName, serverAddress);
    -
    -		if (handler != null) {
    -			handler.shutdown();
    -			handler = null;
    -		}
    -
    -		if (queryExecutor != null) {
    -			queryExecutor.shutdown();
    -		}
    +	public CompletableFuture<?> shutdownServer(Time timeout) throws InterruptedException
{
    +		log.info("Shutting down {} @ {}", serverName, serverAddress);
    +
    +		final CompletableFuture<Boolean> queryExecShutdownFuture = CompletableFuture.supplyAsync(()
-> {
    +				try {
    +					if (queryExecutor != null && !queryExecutor.isShutdown()) {
    +						queryExecutor.shutdown();
    +						queryExecutor.awaitTermination(timeout.toMilliseconds(), TimeUnit.MILLISECONDS);
    +						return true;
    +					}
    +				} catch (InterruptedException e) {
    +					log.warn("Failed to shutdown {}: ", serverName, e);
    +					return false;
    +				}
    +				return false;
    +			});
    +
    +		final CompletableFuture<Boolean> groupShutdownFuture = new CompletableFuture<>();
    +		final CompletableFuture<Boolean> handlerShutdownFuture = new CompletableFuture<>();
    +
    +		queryExecShutdownFuture.thenRun(() -> {
    +			if (bootstrap != null) {
    +				EventLoopGroup group = bootstrap.group();
    +				if (group != null) {
    +					group.shutdownGracefully(0L, timeout.toMilliseconds(), TimeUnit.MILLISECONDS)
    +							.addListener(finished -> groupShutdownFuture.complete(null));
    +				} else {
    +					groupShutdownFuture.complete(null);
    +				}
    +			}
     
    -		if (bootstrap != null) {
    -			EventLoopGroup group = bootstrap.group();
    -			if (group != null) {
    -				group.shutdownGracefully(0L, 10L, TimeUnit.SECONDS);
    +			if (handler != null) {
    +				handler.shutdown().thenRun(() -> {
    +					handler = null;
    --- End diff --
    
    I would actually refrain from state changes from an aysnchronous callback thread.


> AbstractServerBase#shutdown does not wait for shutdown completion
> -----------------------------------------------------------------
>
>                 Key: FLINK-7974
>                 URL: https://issues.apache.org/jira/browse/FLINK-7974
>             Project: Flink
>          Issue Type: Bug
>          Components: Queryable State
>    Affects Versions: 1.4.0
>            Reporter: Till Rohrmann
>            Assignee: Kostas Kloudas
>            Priority: Critical
>
> The {{AbstractServerBase}} does not wait for the completion of its shutdown when calling
{{AbstractServerBase#shutdown}}. This is problematic since it leads to resource leaks and
instable tests such as the {{AbstractServerTest}}. I propose to let the {{AbstractServerBase#shutdown}}
return a termination future which is completed upon shutdown completion.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message