You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ne...@apache.org on 2019/01/28 17:54:18 UTC

[incubator-pinot] branch metric_after_periodic_task_run updated: Keep only 1 ControllerGauge value and use suffix for task name

This is an automated email from the ASF dual-hosted git repository.

nehapawar pushed a commit to branch metric_after_periodic_task_run
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git


The following commit(s) were added to refs/heads/metric_after_periodic_task_run by this push:
     new 3998bd2  Keep only 1 ControllerGauge value and use suffix for task name
3998bd2 is described below

commit 3998bd2a23fde61f5b7da76377e4252ea971d063
Author: Neha Pawar <np...@linkedin.com>
AuthorDate: Mon Jan 28 09:54:04 2019 -0800

    Keep only 1 ControllerGauge value and use suffix for task name
---
 .../pinot/common/metrics/AbstractMetrics.java      | 55 +++++++++++++---------
 .../pinot/common/metrics/ControllerGauge.java      |  8 +---
 .../controller/helix/SegmentStatusChecker.java     |  5 +-
 .../helix/core/minion/PinotTaskManager.java        |  6 +--
 .../core/periodictask/ControllerPeriodicTask.java  | 10 +++-
 .../core/relocation/RealtimeSegmentRelocator.java  |  5 +-
 .../helix/core/retention/RetentionManager.java     |  5 +-
 .../BrokerResourceValidationManager.java           |  6 +--
 .../validation/OfflineSegmentIntervalChecker.java  |  6 +--
 .../RealtimeSegmentValidationManager.java          |  6 +--
 .../periodictask/ControllerPeriodicTaskTest.java   | 18 +++++--
 .../tasks/SegmentStatusCheckerIntegrationTest.java |  9 ++--
 12 files changed, 74 insertions(+), 65 deletions(-)

diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java
index 6f78211..7857f57 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java
@@ -327,23 +327,22 @@ public abstract class AbstractMetrics<QP extends AbstractMetrics.QueryPhase, M e
     String gaugeName = gauge.getGaugeName();
     fullGaugeName = gaugeName + "." + getTableName(tableName);
 
-    if (!_gaugeValues.containsKey(fullGaugeName)) {
-      synchronized (_gaugeValues) {
-        if(!_gaugeValues.containsKey(fullGaugeName)) {
-          _gaugeValues.put(fullGaugeName, new AtomicLong(value));
-          addCallbackGauge(fullGaugeName, new Callable<Long>() {
-            @Override
-            public Long call() throws Exception {
-              return _gaugeValues.get(fullGaugeName).get();
-            }
-          });
-        } else {
-          _gaugeValues.get(fullGaugeName).set(value);
-        }
-      }
-    } else {
-      _gaugeValues.get(fullGaugeName).set(value);
-    }
+    setValueOfGauge(value, fullGaugeName);
+  }
+
+  /**
+   * Sets the value of a custom global gauge.
+   *
+   * @param suffix The suffix to attach to the gauge name
+   * @param gauge The gauge to use
+   * @param value The value to set the gauge to
+   */
+  public void setValueOfGlobalGauge(final G gauge, final String suffix, final long value) {
+    final String fullGaugeName;
+    String gaugeName = gauge.getGaugeName();
+    fullGaugeName = gaugeName + "." + suffix;
+
+    setValueOfGauge(value, fullGaugeName);
   }
 
   /**
@@ -355,16 +354,15 @@ public abstract class AbstractMetrics<QP extends AbstractMetrics.QueryPhase, M e
   public void setValueOfGlobalGauge(final G gauge, final long value) {
     final String gaugeName = gauge.getGaugeName();
 
+    setValueOfGauge(value, gaugeName);
+  }
+
+  private void setValueOfGauge(long value, String gaugeName) {
     if (!_gaugeValues.containsKey(gaugeName)) {
       synchronized (_gaugeValues) {
         if(!_gaugeValues.containsKey(gaugeName)) {
           _gaugeValues.put(gaugeName, new AtomicLong(value));
-          addCallbackGauge(gaugeName, new Callable<Long>() {
-            @Override
-            public Long call() throws Exception {
-              return _gaugeValues.get(gaugeName).get();
-            }
-          });
+          addCallbackGauge(gaugeName, () -> _gaugeValues.get(gaugeName).get());
         } else {
           _gaugeValues.get(gaugeName).set(value);
         }
@@ -412,6 +410,17 @@ public abstract class AbstractMetrics<QP extends AbstractMetrics.QueryPhase, M e
     }
   }
 
+
+  @VisibleForTesting
+  public long getValueOfGlobalGauge(final G gauge, String suffix) {
+    String fullGaugeName = gauge.getGaugeName() + "." + suffix;
+    if (!_gaugeValues.containsKey(fullGaugeName)) {
+      return 0;
+    } else {
+      return _gaugeValues.get(fullGaugeName).get();
+    }
+  }
+
   /**
    * Gets the value of a table gauge.
    *
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java
index a745814..0c18040 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java
@@ -38,13 +38,7 @@ public enum ControllerGauge implements AbstractMetrics.Gauge {
   OFFLINE_TABLE_COUNT("TableCount", true),
   DISABLED_TABLE_COUNT("TableCount", true),
 
-  SEGMENT_STATUS_CHECKER_NUM_TABLES_PROCESSED("SegmentStatusCheckerNumTablesProcessed", true),
-  REALTIME_SEGMENTS_VALIDATION_NUM_TABLES_PROCESSED("RealtimeSegmentsValidationNumTablesProcessed", true),
-  OFFLINE_SEGMENT_INTERVAL_CHECKER_NUM_TABLES_PROCESSED("OfflineSegmentIntervalCheckerNumTablesProcessed", true),
-  BROKER_RESOURCE_VALIDATION_NUM_TABLES_PROCESSED("BrokerResourceValidationNumTablesProcessed", true),
-  SEGMENT_RELOCATOR_NUM_TABLES_PROCESSED("SegmentRelocatorNumTablesProcessed", true),
-  RETENTION_NUM_TABLES_PROCESSED("RetentionNumTablesProcessed", true),
-  PINOT_TASK_MANAGER_NUM_TABLES_PROCESSED("PinotTaskManagerNumTablesProcessed", true),
+  PERIODIC_TASK_NUM_TABLES_PROCESSED("PeriodicTaskNumTablesProcessed", true),
 
   SHORT_OF_LIVE_INSTANCES("ShortOfLiveInstances", false), // Number of extra live instances needed.
 
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java
index 464fa48..d971e1e 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java
@@ -77,7 +77,7 @@ public class SegmentStatusChecker extends ControllerPeriodicTask {
 
   @Override
   protected void preprocess() {
-    _numTablesProcessed = 0;
+    super.preprocess();
     _realTimeTableCount = 0;
     _offlineTableCount = 0;
     _disabledTableCount = 0;
@@ -107,8 +107,7 @@ public class SegmentStatusChecker extends ControllerPeriodicTask {
 
   @Override
   protected void postprocess() {
-    _metricsRegistry.setValueOfGlobalGauge(ControllerGauge.SEGMENT_STATUS_CHECKER_NUM_TABLES_PROCESSED,
-        _numTablesProcessed);
+    super.postprocess();
     _metricsRegistry.setValueOfGlobalGauge(ControllerGauge.REALTIME_TABLE_COUNT, _realTimeTableCount);
     _metricsRegistry.setValueOfGlobalGauge(ControllerGauge.OFFLINE_TABLE_COUNT, _offlineTableCount);
     _metricsRegistry.setValueOfGlobalGauge(ControllerGauge.DISABLED_TABLE_COUNT, _disabledTableCount);
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java
index 81df97e..dea29c0 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java
@@ -27,7 +27,6 @@ import javax.annotation.Nonnull;
 import org.apache.pinot.common.config.PinotTaskConfig;
 import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.config.TableTaskConfig;
-import org.apache.pinot.common.metrics.ControllerGauge;
 import org.apache.pinot.common.metrics.ControllerMeter;
 import org.apache.pinot.common.metrics.ControllerMetrics;
 import org.apache.pinot.controller.ControllerConf;
@@ -103,7 +102,7 @@ public class PinotTaskManager extends ControllerPeriodicTask {
 
   @Override
   protected void preprocess() {
-    _numTablesProcessed = 0;
+    super.preprocess();
     _metricsRegistry.addMeteredGlobalValue(ControllerMeter.NUMBER_TIMES_SCHEDULE_TASKS_CALLED, 1L);
 
     _taskTypes = _taskGeneratorRegistry.getAllTaskTypes();
@@ -151,8 +150,7 @@ public class PinotTaskManager extends ControllerPeriodicTask {
         _metricsRegistry.addMeteredTableValue(taskType, ControllerMeter.NUMBER_TASKS_SUBMITTED, numTasks);
       }
     }
-    _metricsRegistry.setValueOfGlobalGauge(ControllerGauge.PINOT_TASK_MANAGER_NUM_TABLES_PROCESSED,
-        _numTablesProcessed);
+    super.postprocess();
   }
 
   /**
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java
index adb8b42..dc06af0 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java
@@ -20,6 +20,7 @@ package org.apache.pinot.controller.helix.core.periodictask;
 
 import com.google.common.annotations.VisibleForTesting;
 import java.util.List;
+import org.apache.pinot.common.metrics.ControllerGauge;
 import org.apache.pinot.common.metrics.ControllerMetrics;
 import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
 import org.apache.pinot.core.periodictask.BasePeriodicTask;
@@ -140,7 +141,9 @@ public abstract class ControllerPeriodicTask extends BasePeriodicTask {
   /**
    * This method runs before processing all tables
    */
-  protected abstract void preprocess();
+  protected void preprocess() {
+    _numTablesProcessed = 0;
+  }
 
   /**
    * Execute the controller periodic task for the given table
@@ -151,7 +154,10 @@ public abstract class ControllerPeriodicTask extends BasePeriodicTask {
   /**
    * This method runs after processing all tables
    */
-  protected abstract void postprocess();
+  protected void postprocess() {
+    _metricsRegistry.setValueOfGlobalGauge(ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED, getTaskName(),
+        _numTablesProcessed);
+  }
 
   @VisibleForTesting
   protected boolean shouldStopPeriodicTask() {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/relocation/RealtimeSegmentRelocator.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/relocation/RealtimeSegmentRelocator.java
index 99ede85..deb03a9 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/relocation/RealtimeSegmentRelocator.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/relocation/RealtimeSegmentRelocator.java
@@ -33,7 +33,6 @@ import org.apache.helix.model.IdealState;
 import org.apache.pinot.common.config.RealtimeTagConfig;
 import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.config.TableNameBuilder;
-import org.apache.pinot.common.metrics.ControllerGauge;
 import org.apache.pinot.common.metrics.ControllerMetrics;
 import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.common.utils.helix.HelixHelper;
@@ -70,7 +69,7 @@ public class RealtimeSegmentRelocator extends ControllerPeriodicTask {
 
   @Override
   protected void preprocess() {
-    _numTablesProcessed = 0;
+    super.preprocess();
   }
 
   @Override
@@ -88,7 +87,7 @@ public class RealtimeSegmentRelocator extends ControllerPeriodicTask {
 
   @Override
   protected void postprocess() {
-    _metricsRegistry.setValueOfGlobalGauge(ControllerGauge.SEGMENT_RELOCATOR_NUM_TABLES_PROCESSED, _numTablesProcessed);
+    super.postprocess();
   }
 
   /**
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/retention/RetentionManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/retention/RetentionManager.java
index 5b3e42b..4e493cc 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/retention/RetentionManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/retention/RetentionManager.java
@@ -30,7 +30,6 @@ import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.config.TableNameBuilder;
 import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
 import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
-import org.apache.pinot.common.metrics.ControllerGauge;
 import org.apache.pinot.common.metrics.ControllerMetrics;
 import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.common.utils.CommonConstants.Segment.Realtime.Status;
@@ -72,7 +71,7 @@ public class RetentionManager extends ControllerPeriodicTask {
 
   @Override
   protected void preprocess() {
-    _numTablesProcessed = 0;
+    super.preprocess();
   }
 
   @Override
@@ -90,7 +89,7 @@ public class RetentionManager extends ControllerPeriodicTask {
   protected void postprocess() {
     LOGGER.info("Removing aged (more than {} days) deleted segments for all tables", _deletedSegmentsRetentionInDays);
     _pinotHelixResourceManager.getSegmentDeletionManager().removeAgedDeletedSegments(_deletedSegmentsRetentionInDays);
-    _metricsRegistry.setValueOfGlobalGauge(ControllerGauge.RETENTION_NUM_TABLES_PROCESSED, _numTablesProcessed);
+    super.postprocess();
   }
 
   private void manageRetentionForTable(String tableNameWithType) {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/BrokerResourceValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/BrokerResourceValidationManager.java
index baa7ec2..ebcc5b6 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/BrokerResourceValidationManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/BrokerResourceValidationManager.java
@@ -22,7 +22,6 @@ import java.util.List;
 import java.util.Set;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.pinot.common.config.TableConfig;
-import org.apache.pinot.common.metrics.ControllerGauge;
 import org.apache.pinot.common.metrics.ControllerMetrics;
 import org.apache.pinot.controller.ControllerConf;
 import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
@@ -47,7 +46,7 @@ public class BrokerResourceValidationManager extends ControllerPeriodicTask {
 
   @Override
   protected void preprocess() {
-    _numTablesProcessed = 0;
+    super.preprocess();
     _instanceConfigs = _pinotHelixResourceManager.getAllHelixInstanceConfigs();
   }
 
@@ -73,8 +72,7 @@ public class BrokerResourceValidationManager extends ControllerPeriodicTask {
 
   @Override
   protected void postprocess() {
-    _metricsRegistry.setValueOfGlobalGauge(ControllerGauge.BROKER_RESOURCE_VALIDATION_NUM_TABLES_PROCESSED,
-        _numTablesProcessed);
+    super.postprocess();
   }
 
   @Override
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/OfflineSegmentIntervalChecker.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/OfflineSegmentIntervalChecker.java
index ad4085f..8610347 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/OfflineSegmentIntervalChecker.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/OfflineSegmentIntervalChecker.java
@@ -26,7 +26,6 @@ import org.apache.pinot.common.config.SegmentsValidationAndRetentionConfig;
 import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.config.TableNameBuilder;
 import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
-import org.apache.pinot.common.metrics.ControllerGauge;
 import org.apache.pinot.common.metrics.ControllerMetrics;
 import org.apache.pinot.common.metrics.ValidationMetrics;
 import org.apache.pinot.common.utils.CommonConstants;
@@ -58,7 +57,7 @@ public class OfflineSegmentIntervalChecker extends ControllerPeriodicTask {
 
   @Override
   protected void preprocess() {
-    _numTablesProcessed = 0;
+    super.preprocess();
   }
 
   @Override
@@ -216,8 +215,7 @@ public class OfflineSegmentIntervalChecker extends ControllerPeriodicTask {
 
   @Override
   protected void postprocess() {
-    _metricsRegistry.setValueOfGlobalGauge(ControllerGauge.OFFLINE_SEGMENT_INTERVAL_CHECKER_NUM_TABLES_PROCESSED,
-        _numTablesProcessed);
+    super.postprocess();
   }
 
   @Override
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java
index 2ee7991..70fdbe9 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java
@@ -26,7 +26,6 @@ import java.util.concurrent.TimeUnit;
 import org.apache.pinot.common.config.TableConfig;
 import org.apache.pinot.common.config.TableNameBuilder;
 import org.apache.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
-import org.apache.pinot.common.metrics.ControllerGauge;
 import org.apache.pinot.common.metrics.ControllerMetrics;
 import org.apache.pinot.common.metrics.ValidationMetrics;
 import org.apache.pinot.common.utils.CommonConstants;
@@ -68,7 +67,7 @@ public class RealtimeSegmentValidationManager extends ControllerPeriodicTask {
 
   @Override
   protected void preprocess() {
-    _numTablesProcessed = 0;
+    super.preprocess();
     // Update realtime document counts only if certain time has passed after previous run
     _updateRealtimeDocumentCount = false;
     long currentTimeMs = System.currentTimeMillis();
@@ -156,8 +155,7 @@ public class RealtimeSegmentValidationManager extends ControllerPeriodicTask {
 
   @Override
   protected void postprocess() {
-    _metricsRegistry.setValueOfGlobalGauge(ControllerGauge.REALTIME_SEGMENTS_VALIDATION_NUM_TABLES_PROCESSED,
-        _numTablesProcessed);
+    super.postprocess();
   }
 
   @Override
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTaskTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTaskTest.java
index 9a6e9cb..5c59221 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTaskTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTaskTest.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.IntStream;
+import org.apache.pinot.common.metrics.ControllerGauge;
 import org.apache.pinot.common.metrics.ControllerMetrics;
 import org.apache.pinot.controller.ControllerConf;
 import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
@@ -48,8 +49,9 @@ public class ControllerPeriodicTaskTest {
   private final AtomicBoolean _processCalled = new AtomicBoolean();
   private final AtomicInteger _tablesProcessed = new AtomicInteger();
   private final int _numTables = 3;
+  private static final String TASK_NAME = "TestTask";
 
-  private final MockControllerPeriodicTask _task = new MockControllerPeriodicTask("TestTask", RUN_FREQUENCY_IN_SECONDS,
+  private final MockControllerPeriodicTask _task = new MockControllerPeriodicTask(TASK_NAME, RUN_FREQUENCY_IN_SECONDS,
       _controllerConf.getPeriodicTaskInitialDelayInSeconds(), _resourceManager, _controllerMetrics) {
 
     @Override
@@ -71,6 +73,7 @@ public class ControllerPeriodicTaskTest {
     @Override
     public void processTable(String tableNameWithType) {
       _tablesProcessed.getAndIncrement();
+      _numTablesProcessed ++;
     }
   };
 
@@ -86,6 +89,7 @@ public class ControllerPeriodicTaskTest {
     _stopTaskCalled.set(false);
     _processCalled.set(false);
     _tablesProcessed.set(0);
+    _controllerMetrics.setValueOfGlobalGauge(ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED, TASK_NAME,0);
   }
 
   @Test
@@ -108,6 +112,8 @@ public class ControllerPeriodicTaskTest {
     assertEquals(_tablesProcessed.get(), 0);
     assertFalse(_stopTaskCalled.get());
     assertFalse(_task.shouldStopPeriodicTask());
+    assertEquals(_controllerMetrics.getValueOfGlobalGauge(ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED, TASK_NAME),
+        0);
 
     // run task - leadership gained
     resetState();
@@ -115,6 +121,8 @@ public class ControllerPeriodicTaskTest {
     assertFalse(_initTaskCalled.get());
     assertTrue(_processCalled.get());
     assertEquals(_tablesProcessed.get(), _numTables);
+    assertEquals(_controllerMetrics.getValueOfGlobalGauge(ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED, TASK_NAME),
+        _numTables);
     assertFalse(_stopTaskCalled.get());
     assertFalse(_task.shouldStopPeriodicTask());
 
@@ -124,6 +132,8 @@ public class ControllerPeriodicTaskTest {
     assertFalse(_initTaskCalled.get());
     assertFalse(_processCalled.get());
     assertEquals(_tablesProcessed.get(), 0);
+    assertEquals(_controllerMetrics.getValueOfGlobalGauge(ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED, TASK_NAME),
+        0);
     assertTrue(_stopTaskCalled.get());
     assertTrue(_task.shouldStopPeriodicTask());
 
@@ -134,6 +144,8 @@ public class ControllerPeriodicTaskTest {
     assertFalse(_initTaskCalled.get());
     assertTrue(_processCalled.get());
     assertEquals(_tablesProcessed.get(), _numTables);
+    assertEquals(_controllerMetrics.getValueOfGlobalGauge(ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED, TASK_NAME),
+        _numTables);
     assertFalse(_stopTaskCalled.get());
 
   }
@@ -152,7 +164,7 @@ public class ControllerPeriodicTaskTest {
 
     @Override
     protected void preprocess() {
-
+      super.preprocess();
     }
 
     @Override
@@ -162,7 +174,7 @@ public class ControllerPeriodicTaskTest {
 
     @Override
     public void postprocess() {
-
+      super.postprocess();
     }
 
 
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/controller/periodic/tasks/SegmentStatusCheckerIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/controller/periodic/tasks/SegmentStatusCheckerIntegrationTest.java
index b009ffa..9966840 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/controller/periodic/tasks/SegmentStatusCheckerIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/controller/periodic/tasks/SegmentStatusCheckerIntegrationTest.java
@@ -175,8 +175,8 @@ public class SegmentStatusCheckerIntegrationTest extends BaseClusterIntegrationT
     ControllerMetrics controllerMetrics = _controllerStarter.getControllerMetrics();
 
     long millisToWait = TimeUnit.MILLISECONDS.convert(2, TimeUnit.MINUTES);
-    while (controllerMetrics.getValueOfGlobalGauge(ControllerGauge.SEGMENT_STATUS_CHECKER_NUM_TABLES_PROCESSED)
-        < NUM_TABLES && millisToWait > 0) {
+    while (controllerMetrics.getValueOfGlobalGauge(ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED,
+        "SegmentStatusChecker") < NUM_TABLES && millisToWait > 0) {
       try {
         Thread.sleep(1000);
         millisToWait -= 1000;
@@ -185,9 +185,8 @@ public class SegmentStatusCheckerIntegrationTest extends BaseClusterIntegrationT
       }
     }
 
-    Assert.assertEquals(
-        controllerMetrics.getValueOfGlobalGauge(ControllerGauge.SEGMENT_STATUS_CHECKER_NUM_TABLES_PROCESSED),
-        NUM_TABLES);
+    Assert.assertEquals(controllerMetrics.getValueOfGlobalGauge(ControllerGauge.PERIODIC_TASK_NUM_TABLES_PROCESSED,
+        "SegmentStatusChecker"), NUM_TABLES);
 
     // empty table - table1_OFFLINE
     // num replicas set from ideal state


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org