[ https://issues.apache.org/jira/browse/FLINK-10247?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16635495#comment-16635495 ]
ASF GitHub Bot commented on FLINK-10247: ---------------------------------------- tillrohrmann commented on a change in pull request #6759: [FLINK-10247][Metrics] Run MetricQueryService in a dedicated actor system URL: https://github.com/apache/flink/pull/6759#discussion_r221952097 ########## File path: flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala ########## @@ -205,23 +242,34 @@ object AkkaUtils { val supervisorStrategy = classOf[StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy] .getCanonicalName - val forkJoinExecutorParallelismFactor = - configuration.getDouble(AkkaOptions.FORK_JOIN_EXECUTOR_PARALLELISM_FACTOR) - - val forkJoinExecutorParallelismMin = - configuration.getInteger(AkkaOptions.FORK_JOIN_EXECUTOR_PARALLELISM_MIN) - - val forkJoinExecutorParallelismMax = - configuration.getInteger(AkkaOptions.FORK_JOIN_EXECUTOR_PARALLELISM_MAX) - - val forkJoinExecutorConfig = - s""" - | fork-join-executor { - | parallelism-factor = $forkJoinExecutorParallelismFactor - | parallelism-min = $forkJoinExecutorParallelismMin - | parallelism-max = $forkJoinExecutorParallelismMax - | } - """.stripMargin + val executorConfig= executorMode match { + case AkkaExecutorMode.FORK_JOIN_EXECUTOR => + val forkJoinExecutorParallelismFactor = + configuration.getDouble(AkkaOptions.FORK_JOIN_EXECUTOR_PARALLELISM_FACTOR) + + val forkJoinExecutorParallelismMin = + configuration.getInteger(AkkaOptions.FORK_JOIN_EXECUTOR_PARALLELISM_MIN) + + val forkJoinExecutorParallelismMax = + configuration.getInteger(AkkaOptions.FORK_JOIN_EXECUTOR_PARALLELISM_MAX) + + s""" + | fork-join-executor { + | parallelism-factor = $forkJoinExecutorParallelismFactor + | parallelism-min = $forkJoinExecutorParallelismMin + | parallelism-max = $forkJoinExecutorParallelismMax + | } + """.stripMargin + + case AkkaExecutorMode.SINGLE_THREAD_EXECUTOR => + s""" + | single-thread-executor { + | executor = "thread-pool-executor" + | type = PinnedDispatcher + | threads-priority = ${Thread.MIN_PRIORITY} + | } + """.stripMargin + } Review comment: Another benefit would be that we did not have to forward the `AkkaExecutorMode` parameter all the way down. ---------------------------------------------------------------- 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 > Run MetricQueryService in separate thread pool > ---------------------------------------------- > > Key: FLINK-10247 > URL: https://issues.apache.org/jira/browse/FLINK-10247 > Project: Flink > Issue Type: Sub-task > Components: Metrics > Affects Versions: 1.5.3, 1.6.0, 1.7.0 > Reporter: Till Rohrmann > Assignee: Shimin Yang > Priority: Critical > Labels: pull-request-available > Fix For: 1.7.0, 1.6.2, 1.5.5 > > > In order to make the {{MetricQueryService}} run independently of the main > Flink components, it should get its own dedicated thread pool assigned. -- This message was sent by Atlassian JIRA (v7.6.3#76005)