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

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

zentol closed pull request #6786: [FLINK-10282][rest] Separate REST and 
Dispatcher RPC thread pools
URL: https://github.com/apache/flink/pull/6786
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/docs/_includes/generated/rest_configuration.html 
b/docs/_includes/generated/rest_configuration.html
index 25da9cfb067..1aa963fb3e2 100644
--- a/docs/_includes/generated/rest_configuration.html
+++ b/docs/_includes/generated/rest_configuration.html
@@ -57,5 +57,10 @@
             <td style="word-wrap: break-word;">104857600</td>
             <td>The maximum content length in bytes that the server will 
handle.</td>
         </tr>
+        <tr>
+            <td><h5>rest.server.numThreads</h5></td>
+            <td style="word-wrap: break-word;">4</td>
+            <td>The number of threads for the asynchronous processing of 
requests.</td>
+        </tr>
     </tbody>
 </table>
diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java 
b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java
index c834483d7d0..edfd39be808 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java
@@ -117,4 +117,8 @@
                        .defaultValue(104_857_600)
                        .withDescription("The maximum content length in bytes 
that the client will handle.");
 
+       public static final ConfigOption<Integer> SERVER_NUM_THREADS =
+               key("rest.server.numThreads")
+                       .defaultValue(4)
+                       .withDescription("The number of threads for the 
asynchronous processing of requests.");
 }
diff --git 
a/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java 
b/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java
index 4df1d6ee71b..47a5725387e 100644
--- 
a/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java
+++ 
b/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java
@@ -22,7 +22,6 @@
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.RestOptions;
-import org.apache.flink.runtime.concurrent.Executors;
 import org.apache.flink.runtime.dispatcher.DispatcherGateway;
 import org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint;
 import org.apache.flink.runtime.leaderelection.LeaderContender;
@@ -69,7 +68,7 @@
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
 import java.util.stream.Collectors;
 
 import static org.apache.flink.docs.util.Utils.escapeCharacters;
@@ -323,7 +322,6 @@ public SerializableString getEscapeSequence(int i) {
                private static final Configuration config;
                private static final RestServerEndpointConfiguration restConfig;
                private static final RestHandlerConfiguration handlerConfig;
-               private static final Executor executor;
                private static final GatewayRetriever<DispatcherGateway> 
dispatcherGatewayRetriever;
                private static final GatewayRetriever<ResourceManagerGateway> 
resourceManagerGatewayRetriever;
                private static final MetricQueryServiceRetriever 
metricQueryServiceRetriever;
@@ -339,7 +337,6 @@ public SerializableString getEscapeSequence(int i) {
                                throw new RuntimeException("Implementation 
error. RestServerEndpointConfiguration#fromConfiguration failed for default 
configuration.");
                        }
                        handlerConfig = 
RestHandlerConfiguration.fromConfiguration(config);
-                       executor = Executors.directExecutor();
 
                        dispatcherGatewayRetriever = () -> null;
                        resourceManagerGatewayRetriever = () -> null;
@@ -354,7 +351,7 @@ private DocumentingDispatcherRestEndpoint() throws 
IOException {
                                handlerConfig,
                                resourceManagerGatewayRetriever,
                                NoOpTransientBlobService.INSTANCE,
-                               executor,
+                               Executors.newFixedThreadPool(1),
                                metricQueryServiceRetriever,
                                NoOpElectionService.INSTANCE,
                                NoOpFatalErrorHandler.INSTANCE);
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
index ba080c6fda9..1bd6ad9a7ef 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java
@@ -43,7 +43,7 @@
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
 
 /**
  * REST endpoint for the {@link Dispatcher} component.
@@ -59,7 +59,7 @@ public DispatcherRestEndpoint(
                        RestHandlerConfiguration restConfiguration,
                        GatewayRetriever<ResourceManagerGateway> 
resourceManagerRetriever,
                        TransientBlobService transientBlobService,
-                       Executor executor,
+                       ExecutorService executor,
                        MetricQueryServiceRetriever metricQueryServiceRetriever,
                        LeaderElectionService leaderElectionService,
                        FatalErrorHandler fatalErrorHandler) throws IOException 
{
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java
index 0a374117841..043ccecf422 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java
@@ -21,6 +21,7 @@
 import org.apache.flink.api.common.time.Time;
 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;
@@ -138,7 +139,9 @@ public AbstractDispatcherResourceManagerComponentFactory(
                                dispatcherGatewayRetriever,
                                resourceManagerGatewayRetriever,
                                blobServer,
-                               rpcService.getExecutor(),
+                               WebMonitorEndpoint.createExecutorService(
+                                       
configuration.getInteger(RestOptions.SERVER_NUM_THREADS),
+                                       "DispatcherRestEndpoint"),
                                new AkkaQueryServiceRetriever(actorSystem, 
timeout),
                                
highAvailabilityServices.getWebMonitorLeaderElectionService(),
                                fatalErrorHandler);
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java
index b31c46a2901..dae9d8e5625 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java
@@ -32,7 +32,7 @@
 import 
org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
 
 import java.io.IOException;
-import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
 
 /**
  * REST endpoint for the {@link JobClusterEntrypoint}.
@@ -46,7 +46,7 @@ public MiniDispatcherRestEndpoint(
                        RestHandlerConfiguration restConfiguration,
                        GatewayRetriever<ResourceManagerGateway> 
resourceManagerRetriever,
                        TransientBlobService transientBlobService,
-                       Executor executor,
+                       ExecutorService executor,
                        MetricQueryServiceRetriever metricQueryServiceRetriever,
                        LeaderElectionService leaderElectionService,
                        FatalErrorHandler fatalErrorHandler) throws IOException 
{
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index bbdb099ae0a..8a6bb956453 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -25,6 +25,7 @@
 import org.apache.flink.api.common.time.Time;
 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.akka.AkkaUtils;
 import org.apache.flink.runtime.blob.BlobCacheService;
@@ -70,6 +71,7 @@
 import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
 import org.apache.flink.runtime.taskexecutor.TaskExecutor;
 import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
+import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint;
 import 
org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever;
 import org.apache.flink.runtime.webmonitor.retriever.impl.RpcGatewayRetriever;
 import org.apache.flink.util.AutoCloseableAsync;
@@ -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, 1),
+                                               "DispatcherRestEndpoint"),
                                        new AkkaQueryServiceRetriever(
                                                actorSystem,
                                                
Time.milliseconds(configuration.getLong(WebOptions.TIMEOUT))),
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/JobRestEndpointFactory.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/JobRestEndpointFactory.java
index da4b0633f40..9bfc9acac47 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/JobRestEndpointFactory.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/JobRestEndpointFactory.java
@@ -31,7 +31,7 @@
 import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever;
 import 
org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
 
-import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
 
 /**
  * {@link RestEndpointFactory} which creates a {@link 
MiniDispatcherRestEndpoint}.
@@ -45,7 +45,7 @@
                        LeaderGatewayRetriever<DispatcherGateway> 
dispatcherGatewayRetriever,
                        LeaderGatewayRetriever<ResourceManagerGateway> 
resourceManagerGatewayRetriever,
                        TransientBlobService transientBlobService,
-                       Executor executor,
+                       ExecutorService executor,
                        MetricQueryServiceRetriever metricQueryServiceRetriever,
                        LeaderElectionService leaderElectionService,
                        FatalErrorHandler fatalErrorHandler) throws Exception {
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestEndpointFactory.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestEndpointFactory.java
index ffdc0cbc39e..64750e7485c 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestEndpointFactory.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestEndpointFactory.java
@@ -29,7 +29,7 @@
 import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever;
 import 
org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
 
-import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
 
 /**
  * {@link WebMonitorEndpoint} factory.
@@ -43,7 +43,7 @@
                LeaderGatewayRetriever<DispatcherGateway> 
dispatcherGatewayRetriever,
                LeaderGatewayRetriever<ResourceManagerGateway> 
resourceManagerGatewayRetriever,
                TransientBlobService transientBlobService,
-               Executor executor,
+               ExecutorService executor,
                MetricQueryServiceRetriever metricQueryServiceRetriever,
                LeaderElectionService leaderElectionService,
                FatalErrorHandler fatalErrorHandler) throws Exception;
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/SessionRestEndpointFactory.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/SessionRestEndpointFactory.java
index 359efbfa18e..4669745b66e 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/SessionRestEndpointFactory.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/SessionRestEndpointFactory.java
@@ -30,7 +30,7 @@
 import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever;
 import 
org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
 
-import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
 
 /**
  * {@link RestEndpointFactory} which creates a {@link DispatcherRestEndpoint}.
@@ -44,7 +44,7 @@
                        LeaderGatewayRetriever<DispatcherGateway> 
dispatcherGatewayRetriever,
                        LeaderGatewayRetriever<ResourceManagerGateway> 
resourceManagerGatewayRetriever,
                        TransientBlobService transientBlobService,
-                       Executor executor,
+                       ExecutorService executor,
                        MetricQueryServiceRetriever metricQueryServiceRetriever,
                        LeaderElectionService leaderElectionService,
                        FatalErrorHandler fatalErrorHandler) throws Exception {
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
index 7673111d2d1..5ee1bcfde9c 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.util;
 
+import javax.annotation.Nullable;
+
 import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -51,6 +53,9 @@
 
        private final String namePrefix;
 
+       private final int threadPriority;
+
+       @Nullable
        private final UncaughtExceptionHandler exceptionHandler;
 
        // 
------------------------------------------------------------------------
@@ -81,14 +86,20 @@ public ExecutorThreadFactory(String poolName) {
         * @param exceptionHandler The uncaught exception handler for the 
threads
         */
        public ExecutorThreadFactory(String poolName, UncaughtExceptionHandler 
exceptionHandler) {
-               checkNotNull(poolName, "poolName");
+               this(poolName, Thread.NORM_PRIORITY, exceptionHandler);
+       }
+
+       ExecutorThreadFactory(
+                       final String poolName,
+                       final int threadPriority,
+                       @Nullable final UncaughtExceptionHandler 
exceptionHandler) {
+               this.namePrefix = checkNotNull(poolName, "poolName") + 
"-thread-";
+               this.threadPriority = threadPriority;
+               this.exceptionHandler = exceptionHandler;
 
                SecurityManager securityManager = System.getSecurityManager();
                this.group = (securityManager != null) ? 
securityManager.getThreadGroup() :
-                               Thread.currentThread().getThreadGroup();
-
-               this.namePrefix = poolName + "-thread-";
-               this.exceptionHandler = exceptionHandler;
+                       Thread.currentThread().getThreadGroup();
        }
 
        // 
------------------------------------------------------------------------
@@ -98,10 +109,7 @@ public Thread newThread(Runnable runnable) {
                Thread t = new Thread(group, runnable, namePrefix + 
threadNumber.getAndIncrement());
                t.setDaemon(true);
 
-               // normalize the priority
-               if (t.getPriority() != Thread.NORM_PRIORITY) {
-                       t.setPriority(Thread.NORM_PRIORITY);
-               }
+               t.setPriority(threadPriority);
 
                // optional handler for uncaught exceptions
                if (exceptionHandler != null) {
@@ -113,4 +121,28 @@ public Thread newThread(Runnable runnable) {
 
        // 
--------------------------------------------------------------------------------------------
 
+       public static final class Builder {
+               private String poolName;
+               private int priority = Thread.NORM_PRIORITY;
+               private UncaughtExceptionHandler exceptionHandler = 
FatalExitExceptionHandler.INSTANCE;
+
+               public Builder setPoolName(final String poolName) {
+                       this.poolName = poolName;
+                       return this;
+               }
+               
+               public Builder setThreadPriority(final int priority) {
+                       this.priority = priority;
+                       return this;
+               }
+               
+               public Builder setExceptionHandler(final 
UncaughtExceptionHandler exceptionHandler) {
+                       this.exceptionHandler = exceptionHandler;
+                       return this;
+               }
+               
+               public ExecutorThreadFactory build() {
+                       return new ExecutorThreadFactory(poolName, priority, 
exceptionHandler);
+               }
+       }
 }
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java
index f9b2fa89814..02d92dc54fb 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java
@@ -114,11 +114,13 @@
 import 
org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerStdoutFileHeaders;
 import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders;
 import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
 import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
 import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
 import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
 import 
org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
 import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
 import org.apache.flink.util.FileUtils;
 import org.apache.flink.util.Preconditions;
 
@@ -134,7 +136,9 @@
 import java.util.Optional;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Rest endpoint which serves the web frontend REST calls.
@@ -148,7 +152,7 @@
        protected final RestHandlerConfiguration restConfiguration;
        private final GatewayRetriever<ResourceManagerGateway> 
resourceManagerRetriever;
        private final TransientBlobService transientBlobService;
-       protected final Executor executor;
+       protected final ExecutorService executor;
 
        private final ExecutionGraphCache executionGraphCache;
        private final CheckpointStatsCache checkpointStatsCache;
@@ -170,7 +174,7 @@ public WebMonitorEndpoint(
                        RestHandlerConfiguration restConfiguration,
                        GatewayRetriever<ResourceManagerGateway> 
resourceManagerRetriever,
                        TransientBlobService transientBlobService,
-                       Executor executor,
+                       ExecutorService executor,
                        MetricQueryServiceRetriever metricQueryServiceRetriever,
                        LeaderElectionService leaderElectionService,
                        FatalErrorHandler fatalErrorHandler) throws IOException 
{
@@ -715,7 +719,9 @@ public void startInternal() throws Exception {
        protected CompletableFuture<Void> shutDownInternal() {
                executionGraphCache.close();
 
-               final CompletableFuture<Void> shutdownFuture = 
super.shutDownInternal();
+               final CompletableFuture<Void> shutdownFuture = 
FutureUtils.runAfterwards(
+                       super.shutDownInternal(),
+                       () -> ExecutorUtils.gracefulShutdown(10, 
TimeUnit.SECONDS, executor));
 
                final File webUiDir = restConfiguration.getWebUiDir();
 
@@ -776,4 +782,13 @@ public void handleError(final Exception exception) {
                }
                return archivedJson;
        }
+
+       public static ExecutorService createExecutorService(int numThreads, 
String componentName) {
+               return Executors.newFixedThreadPool(
+                       numThreads,
+                       new ExecutorThreadFactory.Builder()
+                               .setThreadPriority(Thread.MIN_PRIORITY)
+                               .setPoolName("Flink-" + componentName)
+                               .build());
+       }
 }
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointConfigurationTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointConfigurationTest.java
new file mode 100644
index 00000000000..67f951b0cb1
--- /dev/null
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointConfigurationTest.java
@@ -0,0 +1,65 @@
+/*
+ * 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://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;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.WebOptions;
+import org.apache.flink.util.ConfigurationException;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.hamcrest.CoreMatchers.containsString;
+
+/**
+ * Tests for the {@link RestServerEndpointConfiguration}.
+ */
+public class RestServerEndpointConfigurationTest extends TestLogger {
+
+       private static final String ADDRESS = "123.123.123.123";
+       private static final String BIND_ADDRESS = "023.023.023.023";
+       private static final int PORT = 7282;
+       private static final int CONTENT_LENGTH = 1234;
+
+       @Rule
+       public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+       @Test
+       public void testBasicMapping() throws ConfigurationException {
+               Configuration originalConfig = new Configuration();
+               originalConfig.setString(RestOptions.ADDRESS, ADDRESS);
+               originalConfig.setString(RestOptions.BIND_ADDRESS, 
BIND_ADDRESS);
+               originalConfig.setInteger(RestOptions.PORT, PORT);
+               
originalConfig.setInteger(RestOptions.SERVER_MAX_CONTENT_LENGTH, 
CONTENT_LENGTH);
+               originalConfig.setString(WebOptions.TMP_DIR, 
temporaryFolder.getRoot().getAbsolutePath());
+
+               final RestServerEndpointConfiguration result = 
RestServerEndpointConfiguration.fromConfiguration(originalConfig);
+               Assert.assertEquals(ADDRESS, result.getRestAddress());
+               Assert.assertEquals(BIND_ADDRESS, result.getRestBindAddress());
+               Assert.assertEquals(PORT, result.getRestBindPort());
+               Assert.assertEquals(CONTENT_LENGTH, 
result.getMaxContentLength());
+               Assert.assertThat(
+                       result.getUploadDir().toAbsolutePath().toString(),
+                       
containsString(temporaryFolder.getRoot().getAbsolutePath()));
+       }
+}


 

----------------------------------------------------------------
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)

Reply via email to