[ https://issues.apache.org/jira/browse/FLINK-10247?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16631435#comment-16631435 ]
ASF GitHub Bot commented on FLINK-10247: ---------------------------------------- yanghua 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_r221157937 ########## File path: flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala ########## @@ -223,37 +278,87 @@ object AkkaUtils { | } """.stripMargin + getBasicAkkaConfigWithParticularExecutor(configuration, forkJoinExecutorConfig) + } + + /** + * Gets the basic Akka config with single thread executor which is shared by remote + * and local actor systems. + * + * @param configuration instance which contains the user specified values for the configuration + * @return Flink's basic Akka config + */ + private def getSingleThreadExecutorBasicAkkaConfig(configuration: Configuration): Config = { + val singleThreadExecutorConfig = + s""" + | single-thread-executor { + | executor = "thread-pool-executor" + | type = PinnedDispatcher + | threads-priority = ${Thread.MIN_PRIORITY} + | } + """.stripMargin + + getBasicAkkaConfigWithParticularExecutor(configuration, singleThreadExecutorConfig) + } + + /** + * Gets the basic Akka config which is shared by remote and local actor systems. + * + * @param configuration instance which contains the user specified values for the configuration + * @param executorConfig the akka config for particular executor + * @return Flink's basic Akka config + */ + private def getBasicAkkaConfigWithParticularExecutor( + configuration: Configuration, + executorConfig: String): Config = { + val akkaThroughput = configuration.getInteger(AkkaOptions.DISPATCHER_THROUGHPUT) + val lifecycleEvents = configuration.getBoolean(AkkaOptions.LOG_LIFECYCLE_EVENTS) + + val jvmExitOnFatalError = if ( + configuration.getBoolean(AkkaOptions.JVM_EXIT_ON_FATAL_ERROR)){ + "on" + } else { + "off" + } + + val logLifecycleEvents = if (lifecycleEvents) "on" else "off" + + val logLevel = getLogLevel + + val supervisorStrategy = classOf[StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy] + .getCanonicalName + val config = s""" - |akka { - | daemonic = off - | + |akka { + | daemonic = off + | | loggers = ["akka.event.slf4j.Slf4jLogger"] - | logging-filter = "akka.event.slf4j.Slf4jLoggingFilter" - | log-config-on-start = off - | + | logging-filter = "akka.event.slf4j.Slf4jLoggingFilter" + | log-config-on-start = off + | | jvm-exit-on-fatal-error = $jvmExitOnFatalError - | + | | serialize-messages = off - | + | | loglevel = $logLevel - | stdout-loglevel = OFF - | + | stdout-loglevel = OFF + | | log-dead-letters = $logLifecycleEvents - | log-dead-letters-during-shutdown = $logLifecycleEvents - | + | log-dead-letters-during-shutdown = $logLifecycleEvents + | | actor { - | guardian-supervisor-strategy = $supervisorStrategy - | + | guardian-supervisor-strategy = $supervisorStrategy + | | warn-about-java-serializer-usage = off - | + | | default-dispatcher { - | throughput = $akkaThroughput - | - | $forkJoinExecutorConfig - | } - | } - |} + | throughput = $akkaThroughput + | + | $executorConfig + | } + | } + |} Review comment: Here you did a format for the string template? I'd better revert it. ---------------------------------------------------------------- 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)