You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "gortiz (via GitHub)" <gi...@apache.org> on 2024/02/12 19:35:04 UTC

[PR] Remove all controller table metrics when a table is deleted in the controller [pinot]

gortiz opened a new pull request, #12403:
URL: https://github.com/apache/pinot/pull/12403

   When the controller steps back and stops being the leader of a table, all metrics should be deleted. Previously just a list of them were deleted. This is a very error prone process. Some metrics were not listed there and some other were removed twice for some reason. The metrics that were not deleted at that point did not disappear until the controller was restarted.
   
   This PR changes the way metrics are removed so we iterate over all enum values and delete any of them that is not global.
   In case we want to keep some specific metric, we can just explicitly ignore them in `SegmentStatusChecker`


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #12403:
URL: https://github.com/apache/pinot/pull/12403#discussion_r1497321255


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentMessageHandlerFactory.java:
##########
@@ -173,6 +177,17 @@ public HelixTaskResult handleMessage()
       try {
         _instanceDataManager.deleteTable(_tableNameWithType);
         helixTaskResult.setSuccess(true);
+        Arrays.stream(ServerMeter.values())
+            .filter(m -> !m.isGlobal())
+            .forEach(m -> _metrics.removeTableMeter(_tableNameWithType, m));
+        Arrays.stream(ServerGauge.values())
+            .filter(g -> !g.isGlobal())
+            .forEach(g -> _metrics.removeTableGauge(_tableNameWithType, g));
+        Arrays.stream(ServerTimer.values())
+            .filter(t -> !t.isGlobal())
+            .forEach(t -> _metrics.removeTableTimer(_tableNameWithType, t));
+        Arrays.stream(ServerQueryPhase.values())
+            .forEach(p -> _metrics.removePhaseTiming(_tableNameWithType, p));
       } catch (Exception e) {
         _metrics.addMeteredTableValue(_tableNameWithType, ServerMeter.DELETE_TABLE_FAILURES, 1);

Review Comment:
   I'm going to remove this code because that is not correctly deleting the server metrics. It seems even after `        _instanceDataManager.deleteTable(_tableNameWithType);` is called other threads are updating these metrics, so some of the deleted metrics here are created again. Specifically the ones that survive are:
   ```
   "org.apache.pinot.common.metrics":name="pinot.server.endToEndRealtimeIngestionDelayMs.meetupRsvp_REALTIME.X",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.highestStreamOffsetConsumed.meetupRsvp_REALTIME-meetupRSVPEvents-X",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.lastRealtimeSegmentCompletionDurationSeconds.meetupRsvp_REALTIME-meetupRSVPEvents-X",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.lastRealtimeSegmentCreationDurationSeconds.meetupRsvp_REALTIME-meetupRSVPEvents-X",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.lastRealtimeSegmentInitialConsumptionDurationSeconds.meetupRsvp_REALTIME-meetupRSVPEvents-X",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.meetupRsvp.segmentUploadSuccess",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.meetupRsvp.segmentUploadTimeMs",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.meetupRsvp_REALTIME-meetupRSVPEvents-X.realtimeRowsConsumed",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.realtimeOffheapMemoryUsed.meetupRsvp",type="ServerMetrics"
   
   ```



-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "xiangfu0 (via GitHub)" <gi...@apache.org>.
xiangfu0 commented on code in PR #12403:
URL: https://github.com/apache/pinot/pull/12403#discussion_r1493752165


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentMessageHandlerFactory.java:
##########
@@ -173,6 +177,17 @@ public HelixTaskResult handleMessage()
       try {
         _instanceDataManager.deleteTable(_tableNameWithType);
         helixTaskResult.setSuccess(true);
+        Arrays.stream(ServerMeter.values())
+            .filter(m -> !m.isGlobal())
+            .forEach(m -> _metrics.removeTableMeter(_tableNameWithType, m));
+        Arrays.stream(ServerGauge.values())
+            .filter(g -> !g.isGlobal())
+            .forEach(g -> _metrics.removeTableGauge(_tableNameWithType, g));
+        Arrays.stream(ServerTimer.values())
+            .filter(t -> !t.isGlobal())
+            .forEach(t -> _metrics.removeTableTimer(_tableNameWithType, t));
+        Arrays.stream(ServerQueryPhase.values())
+            .forEach(p -> _metrics.removePhaseTiming(_tableNameWithType, p));
       } catch (Exception e) {
         _metrics.addMeteredTableValue(_tableNameWithType, ServerMeter.DELETE_TABLE_FAILURES, 1);

Review Comment:
   this `ServerMeter.DELETE_TABLE_FAILURES` is also a table level metrics right?



-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on PR #12403:
URL: https://github.com/apache/pinot/pull/12403#issuecomment-1956502139

   I'm still not sure how/what to create a test here. Some metrics take a while to appear (as explained in https://github.com/apache/pinot/issues/12344, we need to wait some minutes to see some metrics). Some other will never appear until we actually do something with the table (like upload them somewhere).
   
   What the suggested test will be checking is:
   * Whether the lifetime endpoints are being called
   * Whether deletion methods in metrics registry are working
   
   Both things are out of the scope of this PR and it isn't clear to me how to write these tests. Unless you think this is a blocker, I would prefer to merge it as it is.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #12403:
URL: https://github.com/apache/pinot/pull/12403#discussion_r1494127142


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentMessageHandlerFactory.java:
##########
@@ -173,6 +177,17 @@ public HelixTaskResult handleMessage()
       try {
         _instanceDataManager.deleteTable(_tableNameWithType);
         helixTaskResult.setSuccess(true);
+        Arrays.stream(ServerMeter.values())
+            .filter(m -> !m.isGlobal())
+            .forEach(m -> _metrics.removeTableMeter(_tableNameWithType, m));
+        Arrays.stream(ServerGauge.values())
+            .filter(g -> !g.isGlobal())
+            .forEach(g -> _metrics.removeTableGauge(_tableNameWithType, g));
+        Arrays.stream(ServerTimer.values())
+            .filter(t -> !t.isGlobal())
+            .forEach(t -> _metrics.removeTableTimer(_tableNameWithType, t));
+        Arrays.stream(ServerQueryPhase.values())
+            .forEach(p -> _metrics.removePhaseTiming(_tableNameWithType, p));
       } catch (Exception e) {
         _metrics.addMeteredTableValue(_tableNameWithType, ServerMeter.DELETE_TABLE_FAILURES, 1);

Review Comment:
   Yes, it is. A sequence like:
   1. Create table A
   2. Fail to delete table A
   3. Successfully delete table A
   
   Will show DELETE_TABLE_FAILURES with value 1 for table A point 2 and 3, but won't show that value after 3 is correctly executed. 



-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on PR #12403:
URL: https://github.com/apache/pinot/pull/12403#issuecomment-1956486692

   It seems this PR is not enough to remove all server metrics. Specifically, the following metrics survive:
   ```
   name="pinot.server.endToEndRealtimeIngestionDelayMs.meetupRsvp_REALTIME.X",type="ServerMetrics"
   name="pinot.server.highestStreamOffsetConsumed.meetupRsvp_REALTIME-meetupRSVPEvents-X",type="ServerMetrics"
   name="pinot.server.lastRealtimeSegmentCompletionDurationSeconds.meetupRsvp_REALTIME-meetupRSVPEvents-X",type="ServerMetrics"
   name="pinot.server.lastRealtimeSegmentCreationDurationSeconds.meetupRsvp_REALTIME-meetupRSVPEvents-X",type="ServerMetrics"
   name="pinot.server.lastRealtimeSegmentInitialConsumptionDurationSeconds.meetupRsvp_REALTIME-meetupRSVPEvents-X",type="ServerMetrics"
   name="pinot.server.meetupRsvp.segmentUploadSuccess",type="ServerMetrics"
   name="pinot.server.meetupRsvp.segmentUploadTimeMs",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME-meetupRSVPEvents-X.realtimeRowsConsumed",type="ServerMetrics"
   name="pinot.server.realtimeOffheapMemoryUsed.meetupRsvp",type="ServerMetrics"
   ```
   
   As you can see most of the metrics that survive were either indexed by table name (`meetupRsvp_REALTIME-meetupRSVPEvents-X` is the name of the kafka client where X is the partition) or the name without type (like `pinot.server.meetupRsvp.segmentUploadTimeMs`).
   
   `pinot.server.endToEndRealtimeIngestionDelayMs.meetupRsvp_REALTIME.X` should be deleted in `IngestionDelayTracker.removePartitionId`. I've added a commit to delete it.
   
   
   While the following were deleted:
   ```
   name="pinot.server.meetupRsvp_REALTIME.buildQueryPlan",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.deletedSegmentCount",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.freshnessLagMs",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.nettyConnectionSendResponseLatency",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.numDocsScanned",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.numEntriesScannedPostFilter",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.numResizes",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.numSegmentsMatched",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.numSegmentsProcessed",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.numSegmentsQueried",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.queryPlanExecution",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.queryProcessing",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.requestDeserialization",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.responseSerialization",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.schedulerWait",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.segmentPruning",type="ServerMetrics"
   name="pinot.server.meetupRsvp_REALTIME.totalQueryTime",type="ServerMetrics"
   name="pinot.server.segmentCount.meetupRsvp_REALTIME",type="ServerMetrics"
   ```


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12403:
URL: https://github.com/apache/pinot/pull/12403#discussion_r1496246707


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java:
##########
@@ -379,12 +384,11 @@ private void setStatusToDefault() {
   }
 
   private void resetTableMetrics(String tableName) {
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.NUMBER_OF_REPLICAS, Long.MIN_VALUE);
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.PERCENT_OF_REPLICAS, Long.MIN_VALUE);
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.SEGMENTS_IN_ERROR_STATE, Long.MIN_VALUE);
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS,
-        Long.MIN_VALUE);
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.PERCENT_SEGMENTS_AVAILABLE, Long.MIN_VALUE);
+    for (ControllerGauge gauge : ControllerGauge.values()) {

Review Comment:
   Not introduced in this PR, but this logic doesn't look correct to me. Can you help check the history of this resetting logic, why not removing them but resetting instead?



##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentMessageHandlerFactory.java:
##########
@@ -173,6 +177,17 @@ public HelixTaskResult handleMessage()
       try {
         _instanceDataManager.deleteTable(_tableNameWithType);
         helixTaskResult.setSuccess(true);
+        Arrays.stream(ServerMeter.values())
+            .filter(m -> !m.isGlobal())
+            .forEach(m -> _metrics.removeTableMeter(_tableNameWithType, m));
+        Arrays.stream(ServerGauge.values())
+            .filter(g -> !g.isGlobal())
+            .forEach(g -> _metrics.removeTableGauge(_tableNameWithType, g));
+        Arrays.stream(ServerTimer.values())
+            .filter(t -> !t.isGlobal())
+            .forEach(t -> _metrics.removeTableTimer(_tableNameWithType, t));
+        Arrays.stream(ServerQueryPhase.values())
+            .forEach(p -> _metrics.removePhaseTiming(_tableNameWithType, p));
       } catch (Exception e) {
         _metrics.addMeteredTableValue(_tableNameWithType, ServerMeter.DELETE_TABLE_FAILURES, 1);

Review Comment:
   Will the next run of `SegmentStatusChecker` clear this metric accidentally? I guess we want to keep it, or change it to a global metric?



-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12403:
URL: https://github.com/apache/pinot/pull/12403#discussion_r1500108209


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java:
##########
@@ -379,12 +384,11 @@ private void setStatusToDefault() {
   }
 
   private void resetTableMetrics(String tableName) {
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.NUMBER_OF_REPLICAS, Long.MIN_VALUE);
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.PERCENT_OF_REPLICAS, Long.MIN_VALUE);
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.SEGMENTS_IN_ERROR_STATE, Long.MIN_VALUE);
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS,
-        Long.MIN_VALUE);
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.PERCENT_SEGMENTS_AVAILABLE, Long.MIN_VALUE);
+    for (ControllerGauge gauge : ControllerGauge.values()) {

Review Comment:
   I went over the history, and seems we should just remove the metrics, instead of resetting them



-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #12403:
URL: https://github.com/apache/pinot/pull/12403#discussion_r1502209924


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java:
##########
@@ -354,39 +356,33 @@ protected void nonLeaderCleanup(List<String> tableNamesWithType) {
 
   private void removeMetricsForTable(String tableNameWithType) {
     LOGGER.info("Removing metrics from {} given it is not a table known by Helix", tableNameWithType);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.NUMBER_OF_REPLICAS);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.PERCENT_OF_REPLICAS);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.PERCENT_SEGMENTS_AVAILABLE);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.IDEALSTATE_ZNODE_SIZE);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.IDEALSTATE_ZNODE_BYTE_SIZE);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.SEGMENT_COUNT);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.SEGMENT_COUNT_INCLUDING_REPLACED);
-
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.SEGMENTS_IN_ERROR_STATE);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.PERCENT_SEGMENTS_AVAILABLE);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.TABLE_DISABLED);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.TABLE_CONSUMPTION_PAUSED);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.TABLE_REBALANCE_IN_PROGRESS);
+    for (ControllerGauge metric : ControllerGauge.values()) {
+      if (!metric.isGlobal()) {
+        _controllerMetrics.removeTableGauge(tableNameWithType, metric);
+      }
+    }
+
+    for (ControllerMeter metric : ControllerMeter.values()) {
+      if (!metric.isGlobal()) {
+        _controllerMetrics.removeTableMeter(tableNameWithType, metric);
+      }
+    }
+
+    for (ControllerTimer metric : ControllerTimer.values()) {
+      if (!metric.isGlobal()) {
+        _controllerMetrics.removeTableTimer(tableNameWithType, metric);
+      }
+    }
   }
 
   private void setStatusToDefault() {

Review Comment:
   Then we can just remove the method, right?



-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang merged PR #12403:
URL: https://github.com/apache/pinot/pull/12403


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on PR #12403:
URL: https://github.com/apache/pinot/pull/12403#issuecomment-1956413778

   > One assumption in this PR is that all the metrics are managed by SegmentStatusChecker. Is that always the case?
   
   Here we are assuming that after calling `SegmentStatusChecker.nonLeaderCleanup` there are not going to be other threads modifying the controller metrics for this table. In case this assumption ends up being not correct, the metric will be deleted here and then added again by the rogue thread.
   
   `SegmentStatusChecker` is called by default every 5 mins. Specifically `SegmentStatusChecker.nonLeaderCleanup` is called when `SegmentStatusChecker` is executed and there are tables that during the previous iteration were led by this controller but in this iteration they are not (as long as helix knows).
   
   I'm not sure if some rogue thread may be modifying the metrics after `SegmentStatusChecker.nonLeaderCleanup` is called, but doesn't seem probable given the frequency on which we call `SegmentStatusChecker` and in case that happens, it won't be super problematic. In fact it would not be worse than what we had before where some metrics were not deleted. Also, in case that happens, we should manually remove metrics in these services that continue to be running and the deletion here should be seen as a protection in case we forget to remove them somewhere else.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12403:
URL: https://github.com/apache/pinot/pull/12403#discussion_r1501220074


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java:
##########
@@ -354,39 +356,33 @@ protected void nonLeaderCleanup(List<String> tableNamesWithType) {
 
   private void removeMetricsForTable(String tableNameWithType) {
     LOGGER.info("Removing metrics from {} given it is not a table known by Helix", tableNameWithType);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.NUMBER_OF_REPLICAS);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.PERCENT_OF_REPLICAS);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.PERCENT_SEGMENTS_AVAILABLE);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.IDEALSTATE_ZNODE_SIZE);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.IDEALSTATE_ZNODE_BYTE_SIZE);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.SEGMENT_COUNT);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.SEGMENT_COUNT_INCLUDING_REPLACED);
-
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.SEGMENTS_IN_ERROR_STATE);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.PERCENT_SEGMENTS_AVAILABLE);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.TABLE_DISABLED);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.TABLE_CONSUMPTION_PAUSED);
-    _controllerMetrics.removeTableGauge(tableNameWithType, ControllerGauge.TABLE_REBALANCE_IN_PROGRESS);
+    for (ControllerGauge metric : ControllerGauge.values()) {
+      if (!metric.isGlobal()) {
+        _controllerMetrics.removeTableGauge(tableNameWithType, metric);
+      }
+    }
+
+    for (ControllerMeter metric : ControllerMeter.values()) {
+      if (!metric.isGlobal()) {
+        _controllerMetrics.removeTableMeter(tableNameWithType, metric);
+      }
+    }
+
+    for (ControllerTimer metric : ControllerTimer.values()) {
+      if (!metric.isGlobal()) {
+        _controllerMetrics.removeTableTimer(tableNameWithType, metric);
+      }
+    }
   }
 
   private void setStatusToDefault() {

Review Comment:
   Let's rename this method to `removeMetricsForAllTables()`.
   We should not call this method in `setUpTask()` because it might clean up metrics updated by other services. Maybe we should not call it in `cleanUpTask()` as well.



-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on PR #12403:
URL: https://github.com/apache/pinot/pull/12403#issuecomment-1951902506

   Given we had a conflict when controller gauges are being reset, I've added a commit to define gauges as _reseteable_. Therefore in the future we just need to define the gauge as _reseteable_ in order to reset, which will make more difficult to forget about that.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #12403:
URL: https://github.com/apache/pinot/pull/12403#discussion_r1497393058


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentMessageHandlerFactory.java:
##########
@@ -173,6 +177,17 @@ public HelixTaskResult handleMessage()
       try {
         _instanceDataManager.deleteTable(_tableNameWithType);
         helixTaskResult.setSuccess(true);
+        Arrays.stream(ServerMeter.values())
+            .filter(m -> !m.isGlobal())
+            .forEach(m -> _metrics.removeTableMeter(_tableNameWithType, m));
+        Arrays.stream(ServerGauge.values())
+            .filter(g -> !g.isGlobal())
+            .forEach(g -> _metrics.removeTableGauge(_tableNameWithType, g));
+        Arrays.stream(ServerTimer.values())
+            .filter(t -> !t.isGlobal())
+            .forEach(t -> _metrics.removeTableTimer(_tableNameWithType, t));
+        Arrays.stream(ServerQueryPhase.values())
+            .forEach(p -> _metrics.removePhaseTiming(_tableNameWithType, p));
       } catch (Exception e) {
         _metrics.addMeteredTableValue(_tableNameWithType, ServerMeter.DELETE_TABLE_FAILURES, 1);

Review Comment:
   Nope, `SegmentStatusChecker` cleans the controller metrics. This is a server metric.
   
   Anyway I've changed the code a bit. Semantic is almost the same, but it will be cleaner that `ServerMeter.DELETE_TABLE_FAILURES` will not be deleted in case there is an exception.



-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all controller table metrics when a table is deleted in the controller [pinot]

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #12403:
URL: https://github.com/apache/pinot/pull/12403#issuecomment-1939498364

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/12403?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   Attention: `13 lines` in your changes are missing coverage. Please review.
   > Comparison is base [(`43dadbf`)](https://app.codecov.io/gh/apache/pinot/commit/43dadbfd96a70c19a9ac83bb6c0c35f3fa58bffb?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) 61.73% compared to head [(`e2847de`)](https://app.codecov.io/gh/apache/pinot/pull/12403?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) 34.85%.
   
   | [Files](https://app.codecov.io/gh/apache/pinot/pull/12403?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Patch % | Lines |
   |---|---|---|
   | [...e/pinot/controller/helix/SegmentStatusChecker.java](https://app.codecov.io/gh/apache/pinot/pull/12403?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9TZWdtZW50U3RhdHVzQ2hlY2tlci5qYXZh) | 0.00% | [9 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12403?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   | [...g/apache/pinot/common/metrics/AbstractMetrics.java](https://app.codecov.io/gh/apache/pinot/pull/12403?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9BYnN0cmFjdE1ldHJpY3MuamF2YQ==) | 0.00% | [4 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12403?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   
   <details><summary>Additional details and impacted files</summary>
   
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #12403       +/-   ##
   =============================================
   - Coverage     61.73%   34.85%   -26.89%     
   =============================================
     Files          2428     2352       -76     
     Lines        132828   129083     -3745     
     Branches      20545    19993      -552     
   =============================================
   - Hits          82007    44990    -37017     
   - Misses        44811    80861    +36050     
   + Partials       6010     3232     -2778     
   ```
   
   | [Flag](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flags&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [custom-integration1](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [integration](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `0.00% <0.00%> (-0.01%)` | :arrow_down: |
   | [integration1](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [integration2](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `0.00% <0.00%> (ø)` | |
   | [java-11](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `0.00% <0.00%> (-61.67%)` | :arrow_down: |
   | [java-21](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `34.85% <0.00%> (-26.77%)` | :arrow_down: |
   | [skip-bytebuffers-false](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `34.85% <0.00%> (-26.86%)` | :arrow_down: |
   | [skip-bytebuffers-true](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `0.00% <0.00%> (-61.59%)` | :arrow_down: |
   | [temurin](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `34.85% <0.00%> (-26.89%)` | :arrow_down: |
   | [unittests](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.72% <0.00%> (-15.01%)` | :arrow_down: |
   | [unittests1](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.72% <0.00%> (-0.20%)` | :arrow_down: |
   | [unittests2](https://app.codecov.io/gh/apache/pinot/pull/12403/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   
   </details>
   
   [:umbrella: View full report in Codecov by Sentry](https://app.codecov.io/gh/apache/pinot/pull/12403?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).   
   :loudspeaker: Have feedback on the report? [Share it here](https://about.codecov.io/codecov-pr-comment-feedback/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).
   


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on PR #12403:
URL: https://github.com/apache/pinot/pull/12403#issuecomment-1951884656

   > Can we add an integration test to create table and check for metrics existence then delete table and check for the metrics are gone?
   
   I'm not sure how to do that. Do we have any test like that in Apache Pinot to get some inspiration?


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "xiangfu0 (via GitHub)" <gi...@apache.org>.
xiangfu0 commented on PR #12403:
URL: https://github.com/apache/pinot/pull/12403#issuecomment-1952618113

   > > Can we add an integration test to create table and check for metrics existence then delete table and check for the metrics are gone?
   > 
   > I'm not sure how to do that. Do we have any test like that in Apache Pinot to get some inspiration?
   
   You can add a few lines in the integration tests after addTable and removeTable.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #12403:
URL: https://github.com/apache/pinot/pull/12403#discussion_r1497321255


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentMessageHandlerFactory.java:
##########
@@ -173,6 +177,17 @@ public HelixTaskResult handleMessage()
       try {
         _instanceDataManager.deleteTable(_tableNameWithType);
         helixTaskResult.setSuccess(true);
+        Arrays.stream(ServerMeter.values())
+            .filter(m -> !m.isGlobal())
+            .forEach(m -> _metrics.removeTableMeter(_tableNameWithType, m));
+        Arrays.stream(ServerGauge.values())
+            .filter(g -> !g.isGlobal())
+            .forEach(g -> _metrics.removeTableGauge(_tableNameWithType, g));
+        Arrays.stream(ServerTimer.values())
+            .filter(t -> !t.isGlobal())
+            .forEach(t -> _metrics.removeTableTimer(_tableNameWithType, t));
+        Arrays.stream(ServerQueryPhase.values())
+            .forEach(p -> _metrics.removePhaseTiming(_tableNameWithType, p));
       } catch (Exception e) {
         _metrics.addMeteredTableValue(_tableNameWithType, ServerMeter.DELETE_TABLE_FAILURES, 1);

Review Comment:
   I'm going to remove this code because that is not correctly deleting the server metrics. It seems even after `        _instanceDataManager.deleteTable(_tableNameWithType);` is called other threads are updating these metrics, so some of the deleted metrics here are created again. Specifically the ones that survive are:
   ```
   "org.apache.pinot.common.metrics":name="pinot.server.endToEndRealtimeIngestionDelayMs.meetupRsvp_REALTIME.X",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.highestStreamOffsetConsumed.meetupRsvp_REALTIME-meetupRSVPEvents-X",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.lastRealtimeSegmentCompletionDurationSeconds.meetupRsvp_REALTIME-meetupRSVPEvents-X",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.lastRealtimeSegmentCreationDurationSeconds.meetupRsvp_REALTIME-meetupRSVPEvents-X",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.lastRealtimeSegmentInitialConsumptionDurationSeconds.meetupRsvp_REALTIME-meetupRSVPEvents-X",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.meetupRsvp.segmentUploadSuccess",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.meetupRsvp.segmentUploadTimeMs",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.meetupRsvp_REALTIME-meetupRSVPEvents-X.realtimeRowsConsumed",type="ServerMetrics"
   "org.apache.pinot.common.metrics":name="pinot.server.realtimeOffheapMemoryUsed.meetupRsvp",type="ServerMetrics"
   
   ```



-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] Remove all table metrics when a table is deleted [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #12403:
URL: https://github.com/apache/pinot/pull/12403#discussion_r1500602405


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/SegmentStatusChecker.java:
##########
@@ -379,12 +384,11 @@ private void setStatusToDefault() {
   }
 
   private void resetTableMetrics(String tableName) {
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.NUMBER_OF_REPLICAS, Long.MIN_VALUE);
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.PERCENT_OF_REPLICAS, Long.MIN_VALUE);
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.SEGMENTS_IN_ERROR_STATE, Long.MIN_VALUE);
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.SEGMENTS_WITH_LESS_REPLICAS,
-        Long.MIN_VALUE);
-    _controllerMetrics.setValueOfTableGauge(tableName, ControllerGauge.PERCENT_SEGMENTS_AVAILABLE, Long.MIN_VALUE);
+    for (ControllerGauge gauge : ControllerGauge.values()) {

Review Comment:
   I've just changed all usages of `resetTableMetrics` with `removeMetricsForTable`



-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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