Jackie-Jiang commented on code in PR #17198:
URL: https://github.com/apache/pinot/pull/17198#discussion_r2519762268


##########
pinot-spi/src/main/java/org/apache/pinot/spi/executor/HardLimitExecutor.java:
##########
@@ -38,12 +39,21 @@ public class HardLimitExecutor extends 
DecoratorExecutorService {
   private final AtomicInteger _running;
   private final int _max;
   private final QueryThreadExceedStrategy _exceedStrategy;
+  private final Consumer<Integer> _currentUsageGaugeUpdater;
 
-  public HardLimitExecutor(int max, ExecutorService executorService, 
QueryThreadExceedStrategy exceedStrategy) {
+  public HardLimitExecutor(int max, ExecutorService executorService, 
QueryThreadExceedStrategy exceedStrategy,
+      Consumer<Integer> maxUsageGaugeUpdater, Consumer<Integer> 
currentUsageGaugeUpdater) {
     super(executorService);
     _running = new AtomicInteger(0);
     _max = max;
     _exceedStrategy = exceedStrategy;
+    _currentUsageGaugeUpdater = currentUsageGaugeUpdater;
+    maxUsageGaugeUpdater.accept(max);
+    _currentUsageGaugeUpdater.accept(0);
+  }
+
+  public HardLimitExecutor(int max, ExecutorService executorService, 
QueryThreadExceedStrategy exceedStrategy) {
+    this(max, executorService, exceedStrategy, max1 -> { }, current -> { });

Review Comment:
   (format) Seems this is not following pinot style. Same for other places



##########
pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java:
##########
@@ -153,7 +153,11 @@ public enum ServerGauge implements AbstractMetrics.Gauge {
 
   // ThrottleOnCriticalHeapUsageExecutor metrics
   THROTTLE_EXECUTOR_QUEUE_SIZE("count", true,
-      "Current number of tasks in the throttle executor queue");
+      "Current number of tasks in the throttle executor queue"),
+
+  // Multi-stage executor thread usage metrics
+  MSE_THREAD_USAGE_MAX("threads", true, "Maximum allowed threads for 
multi-stage executor"),

Review Comment:
   Since this is fixed, I feel it is not worth adding given the overhead of 
processing gauge. We can log it when HardLimitExecutor is initialized



##########
pinot-spi/src/main/java/org/apache/pinot/spi/executor/HardLimitExecutor.java:
##########
@@ -111,11 +123,13 @@ protected Runnable decorate(Runnable task) {
     checkTaskAllowed();
     return () -> {
       checkTaskAllowed();
-      _running.getAndIncrement();
+      int currentCount = _running.getAndIncrement();
+      _currentUsageGaugeUpdater.accept(currentCount + 1);

Review Comment:
   Ideally this should be a callback gauge. You may add a method in this class 
to return the current thread usage



##########
pinot-spi/src/test/java/org/apache/pinot/spi/executor/HardLimitExecutorTest.java:
##########
@@ -127,4 +130,48 @@ public void testGetMultiStageExecutorHardLimit() {
     PinotConfiguration config5 = new PinotConfiguration(configMap5);
     assertEquals(HardLimitExecutor.getMultiStageExecutorHardLimit(config5), 
-1);
   }
+
+  @Test
+  public void testGaugeTracking()
+      throws Exception {
+    AtomicInteger maxGauge = new AtomicInteger(-1);
+    AtomicInteger currentGauge = new AtomicInteger(-1);
+
+    HardLimitExecutor ex = new HardLimitExecutor(2, 
Executors.newCachedThreadPool(),
+        QueryThreadExceedStrategy.ERROR,
+        max -> maxGauge.set(max),
+        current -> currentGauge.set(current));
+
+    CyclicBarrier barrier = new CyclicBarrier(3);
+
+    try {
+      assertEquals(maxGauge.get(), 2);
+      assertEquals(currentGauge.get(), 0);
+
+      ex.execute(() -> {
+        try {
+          barrier.await();
+          Thread.sleep(Long.MAX_VALUE);

Review Comment:
   If this is not required, let's remove it. I believe we want to test that 
barrier is not reached, and sleep prevents the purpose



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to