chia7712 commented on code in PR #20481:
URL: https://github.com/apache/kafka/pull/20481#discussion_r2478970202


##########
core/src/main/scala/kafka/server/KafkaRequestHandler.scala:
##########
@@ -192,22 +197,55 @@ class KafkaRequestHandler(
 
 }
 
+/**
+ * Factory for creating KafkaRequestHandlerPool instances with shared 
aggregate metrics.
+ * All pools created by the same factory share the same aggregateThreads 
counter.
+ */
+class KafkaRequestHandlerPoolFactory {
+  private[this] val aggregateThreads = new AtomicInteger(0)
+  private[this] val RequestHandlerAvgIdleMetricName = 
"RequestHandlerAvgIdlePercent"
+  
+  def createPool(
+    brokerId: Int,
+    requestChannel: RequestChannel,
+    apis: ApiRequestHandler,
+    time: Time,
+    numThreads: Int,
+    nodeName: String
+  ): KafkaRequestHandlerPool = {
+    new KafkaRequestHandlerPool(aggregateThreads, 
RequestHandlerAvgIdleMetricName, brokerId, requestChannel, apis, time, 
numThreads, nodeName)
+  }
+
+  // Only used for test purpose
+  def aggregateThreadCount: Int = aggregateThreads.get()
+}
+
 class KafkaRequestHandlerPool(
+  val aggregateThreads: AtomicInteger,
+  val requestHandlerAvgIdleMetricName: String,
   val brokerId: Int,
   val requestChannel: RequestChannel,
   val apis: ApiRequestHandler,
   time: Time,
   numThreads: Int,
-  requestHandlerAvgIdleMetricName: String,
-  nodeName: String = "broker"
+  nodeName: String
 ) extends Logging {
   // Changing the package or class name may cause incompatibility with 
existing code and metrics configuration
   private val metricsPackage = "kafka.server"
   private val metricsClassName = "KafkaRequestHandlerPool"
   private val metricsGroup = new KafkaMetricsGroup(metricsPackage, 
metricsClassName)
 
   val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads)
-  /* a meter to track the average free capacity of the request handlers */
+  private val perPoolIdleMeterName = if (nodeName == "broker") {

Review Comment:
   How about using the `ProcessRole` type instead of `String`? It would be more 
readable in this case



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to