[ https://issues.apache.org/jira/browse/FLINK-10282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16642995#comment-16642995 ]
ASF GitHub Bot commented on FLINK-10282: ---------------------------------------- tillrohrmann commented on a change in pull request #6786: [FLINK-10282][rest] Separate REST and Dispatcher RPC thread pools URL: https://github.com/apache/flink/pull/6786#discussion_r223612401 ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java ########## @@ -341,7 +343,9 @@ public void start() throws Exception { RestHandlerConfiguration.fromConfiguration(configuration), resourceManagerGatewayRetriever, blobServer.getTransientBlobService(), - commonRpcService.getExecutor(), + WebMonitorEndpoint.createExecutorService( + configuration.getInteger(RestOptions.SERVER_NUM_THREADS), + "DispatcherRestEndpoint"), Review comment: Almost. I think it should have a field `terminationFuture`: ``` private static class CloseIgnoringExecutorService extends AbstractExecutorService { private final Executor executor; private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>(); public CloseIgnoringExecutorService(Executor executor) { this.executor = executor; } @Override public void shutdown() { terminationFuture.complete(null); } @Override public List<Runnable> shutdownNow() { shutdown(); return Collections.emptyList(); } @Override public boolean isShutdown() { return terminationFuture.isDone(); } @Override public boolean isTerminated() { return terminationFuture.isDone(); } @Override public boolean awaitTermination(long timeout, TimeUnit unit) { try { terminationFuture.get(timeout, unit); return true; } catch (TimeoutException e) { return false; } } @Override public void execute(Runnable command) { if (terminationFuture.isDone()) { throw new RejectedExecutionException(); } else { executor.execute(command); } } } ``` ---------------------------------------------------------------- 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: us...@infra.apache.org > Provide separate thread-pool for REST endpoint > ---------------------------------------------- > > Key: FLINK-10282 > URL: https://issues.apache.org/jira/browse/FLINK-10282 > Project: Flink > Issue Type: Improvement > Components: Local Runtime, REST > Affects Versions: 1.5.1, 1.6.0, 1.7.0 > Reporter: Chesnay Schepler > Assignee: Chesnay Schepler > Priority: Major > Labels: pull-request-available > Fix For: 1.7.0, 1.6.2, 1.5.5 > > > The REST endpoints currently share their thread-pools with the RPC system, > which can cause the Dispatcher to become unresponsive if the REST parts are > overloaded. -- This message was sent by Atlassian JIRA (v7.6.3#76005)