mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1059510030


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java:
##########
@@ -138,6 +139,11 @@ public void onBecomeOfflineFromConsuming(Message message, 
NotificationContext co
     @Transition(from = "CONSUMING", to = "DROPPED")
     public void onBecomeDroppedFromConsuming(Message message, 
NotificationContext context) {
       
_logger.info("SegmentOnlineOfflineStateModel.onBecomeDroppedFromConsuming() : " 
+ message);
+      String realtimeTableName = message.getResourceName();

Review Comment:
   Please add a TODO somewhere to clean up other occurrences of checking for 
realtime table data manager, etc. This class should not be holding logic 
intended for table data manager.



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumptionDelayTracker.java:
##########
@@ -0,0 +1,295 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.core.data.manager.realtime;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track maximum realtime delay for a given table on a given server.
+ * Highlights:
+ * 1-An object of this class is hosted by each RealtimeTableDataManager.
+ * 2-The object tracks ingestion delays for all partitions hosted by the 
current server for the given Realtime table.
+ * 3-Partition delays are updated by all LLRealtimeSegmentDataManager objects 
hosted in the corresponding
+ *   RealtimeTableDataManager.
+ * 4-The class tracks the maximum of all ingestion delays observed for all 
partitions of the given table.
+ * 5-A Metric is derived from reading the maximum tracked by this class.
+ * 6-Delay reported for partitions that do not have events to consume is 
reported as zero.
+ * 7-We track the time at which each delay sample was collected so that delay 
can be increased when partition stops
+ *   consuming for any reason other than no events being available for 
consumption.
+ * 8-Segments that go from CONSUMING to DROPPED states stop being tracked so 
their delays do not cloud
+ *   delays of active partitions.
+ * 9-When a segment goes from CONSUMING to ONLINE, we start ticking time for 
the corresponding partition.
+ *   If no consumption is noticed after a timeout, then we read ideal state to 
confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 10-A timer thread is started by this object to track timeouts of partitions 
and drive the reading of their ideal
+ *    state.
+ *
+ *  The following diagram illustrates the object interactions with main 
external APIs
+ *
+ *     (CONSUMING -> ONLINE state change)
+ *             |
+ *      markPartition(partitionId)      
|<-recordPinotConsumptionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                
|<-recordPinotConsumptionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | ConsumptionDelayTracker|           ...
+ * 
|____________________________________|<-recordPinotConsumptionDelay()-{LLRealtimeSegmentDataManager
 (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    
stopTrackingPinotConsumptionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class ConsumptionDelayTracker {
+  /*
+   * Class to handle timer thread that will track inactive partitions
+   */
+  private class TrackingTimerTask extends TimerTask {
+    private String _name;
+    private ConsumptionDelayTracker _tracker;
+
+    public TrackingTimerTask(String name, ConsumptionDelayTracker tracker) {
+      _name = name;
+      _tracker = tracker;
+    }
+
+    @Override
+    public void run() {
+      // tick inactive partitions every interval of time to keep tracked 
partitions up to date
+      _tracker.timeoutInactivePartitions();
+    }
+  }
+
+  /*
+   * Class to keep a Pinot Consumption Delay measure and the time when the 
sample was taken (i.e. sample time)
+   * We will use the sample time to increase consumption delay when a 
partition stops consuming: the time
+   * difference between the sample time and current time will be added to the 
metric when read.
+   */
+  static private class DelayMeasure {
+    public DelayMeasure(long t, long d) {
+      _delayMilliseconds = d;
+      _sampleTime = t;
+    }
+    public long _delayMilliseconds;
+    public long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the 
current table.
+  // _partitionsToDelaySampleMap<Key=PartitionGroupId,Value=DelaySample>
+  private ConcurrentHashMap<Long, DelayMeasure> _partitionToDelaySampleMap = 
new ConcurrentHashMap<>();
+  // We mark partitions that go from CONSUMING to ONLINE in 
_partitionsMarkedForVerification: if they do not
+  // go back to CONSUMING in some period of time, we confirm whether they are 
still hosted in this server by reading
+  // ideal state. This is done with the goal of minimizing reading ideal state 
for efficiency reasons.
+  // 
_partitionsMarkedForVerification<Key=PartitionGroupId,Value=TimePartitionWasMarkedForVerificationMilliseconds>
+  private ConcurrentHashMap<Long, Long> _partitionsMarkedForVerification = new 
ConcurrentHashMap<>();
+  // Sleep interval for timer thread that triggers read of ideal state
+  private static final int TIMER_THREAD_TICK_INTERVAL_MS = 300000; // 5 
minutes +/- precision in timeouts
+  // Once a partition is marked for verification, we wait 10 minutes to pull 
its ideal state.
+  private static final int PARTITION_TIMEOUT_MS = 600000;          // 10 
minutes timeouts
+  // Delay Timer thread for this time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 1000;
+  // Mutable versions of the above constants so we can test with smaller delays
+  final int _timerThreadTickIntervalMs;
+  final int _initialTimeThreadDelayMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  final private Timer _timer;
+
+  final private ServerMetrics _serverMetrics;
+  final private String _tableNameWithType;
+
+  private boolean _enableAging;

Review Comment:
   Please find a way to remove this variable



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -611,6 +611,8 @@ private boolean processStreamEvents(MessageBatch 
messagesAndOffsets, long idlePi
       if (_segmentLogger.isDebugEnabled()) {
         _segmentLogger.debug("empty batch received - sleeping for {}ms", 
idlePipeSleepTimeMillis);
       }
+      // Record Pinot ingestion delay as zero since we are up-to-date and no 
new events
+      _realtimeTableDataManager.updateIngestionDelay(0, 
System.currentTimeMillis(), _partitionGroupId);

Review Comment:
   Let us invoke `updateIngestionDelay(0)` here, and put all the logic inside 
of that.



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +217,85 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    if (_consumptionDelayTracker != null) {
+      _consumptionDelayTracker.shutdown();
+    }
+  }
+
+  /*
+   * Method to handle CONSUMING->DROPPED transition.
+   *
+   * @param partitionGroupId Partition id that we must stop tracking on this 
server.
+   */
+  private void stopTrackingPartitionDelay(int partitionGroupId) {
+    if (_consumptionDelayTracker != null) {
+      
_consumptionDelayTracker.stopTrackingPartitionConsumptionDelay(partitionGroupId);
+    }
+  }
+
+  /*
+   * Method to handle CONSUMING->ONLINE transition.
+   * If no new consumption is noticed for this segment in some timeout, we 
will read
+   * ideal state to verify the partition is still hosted in this server.
+   *
+   * @param partitionGroupId partition id of partition to be verified as 
hosted by this server.
+   */
+  private void markPartitionForVerification(int partitionGroupId) {
+    if (_consumptionDelayTracker != null) {
+      _consumptionDelayTracker.markPartitionForConfirmation(partitionGroupId);
+    }
+  }
+
+  /*
+   * Method used by LLRealtimeSegmentManagers to update their partition delays
+   *
+   * @param ingestionDelayMillis Ingestion delay being reported.
+   * @param currentTimeMillis Timestamp of the measure being provided, i.e. 
when this delay was computed.
+   * @param partitionGroupId Partition ID for which delay is being updated.
+   */
+  public void updateIngestionDelay(long ingestionDelayMillis, long 
currenTimeMillis, int partitionGroupId) {
+    if (_consumptionDelayTracker != null) {
+      _consumptionDelayTracker.storeConsumptionDelay(ingestionDelayMillis, 
currenTimeMillis, partitionGroupId);
+    }
+  }
+
+  /*
+   * Method ta handle CONSUMING to ONLINE transitions of segments in this 
table.
+   * We mark partitions for verification with ideal state when we do not see a 
consuming segment for some time
+   * for that partition. The idea is to remove the related metrics when the 
partition moves from the current server.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   */
+  @Override
+  public void onConsumingToOnline(String segmentNameStr) {
+    LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr);
+    markPartitionForVerification(segmentName.getPartitionGroupId());
+  }
+
+  /*
+   * Method ta handle CONSUMING to DROPPED transitions of segments in this 
table.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   */
+  @Override
+  public void onConsumingToDropped(String segmentNameStr) {
+    LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr);
+    // We stop tracking ingestion delay partitions for which their segments go 
into DROPPED state.
+    stopTrackingPartitionDelay(segmentName.getPartitionGroupId());
+  }
+
+  /**
+   * Returns all partitionGroupIds for the partitions hosted by this server 
for current table.

Review Comment:
   Can you add a comment that this is a zookeeper read? 
   
   Since this is a public method, future contributors may start calling it 
without knowing the implications



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumptionDelayTracker.java:
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.core.data.manager.realtime;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track maximum realtime delay for a given table on a given server.
+ * Highlights:
+ * 1-An object of this class is hosted by each RealtimeTableDataManager.
+ * 2-The object tracks ingestion delays for all partitions hosted by the 
current server for the given Realtime table.
+ * 3-Partition delays are updated by all LLRealtimeSegmentDataManager objects 
hosted in the corresponding
+ *   RealtimeTableDataManager.
+ * 4-The class tracks the maximum of all ingestion delays observed for all 
partitions of the given table.
+ * 5-A Metric is derived from reading the maximum tracked by this class.
+ * 6-Delay reported for partitions that do not have events to consume is 
reported as zero.
+ * 7-We track the time at which each delay sample was collected so that delay 
can be increased when partition stops
+ *   consuming for any reason other than no events being available for 
consumption.
+ * 8-Segments that go from CONSUMING to DROPPED states stop being tracked so 
their delays do not cloud
+ *   delays of active partitions.
+ * 9-When a segment goes from CONSUMING to ONLINE, we start ticking time for 
the corresponding partition.
+ *   If no consumption is noticed after a timeout, then we read ideal state to 
confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 10-A timer thread is started by this object to track timeouts of partitions 
and drive the reading of their ideal
+ *    state.
+ *
+ *  The following diagram illustrates the object interactions with main 
external APIs
+ *
+ *     (CONSUMING -> ONLINE state change)
+ *             |
+ *      markPartitionForConfirmation(partitionId)
+ *            |                         
|<-storeConsumptionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                
|<-storeConsumptionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | ConsumptionDelayTracker            |           ...
+ * 
|____________________________________|<-storeConsumptionDelay()-{LLRealtimeSegmentDataManager
 (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    
stopTrackingPartitionConsumptionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class ConsumptionDelayTracker {
+
+  // Sleep interval for timer thread that triggers read of ideal state
+  private static final int TIMER_THREAD_TICK_INTERVAL_MS = 300000; // 5 
minutes +/- precision in timeouts
+  // Once a partition is marked for verification, we wait 10 minutes to pull 
its ideal state.
+  private static final int PARTITION_TIMEOUT_MS = 600000;          // 10 
minutes timeouts
+  // Delay Timer thread for this time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;

Review Comment:
   Is this initial delay even needed?



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +217,85 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    if (_consumptionDelayTracker != null) {
+      _consumptionDelayTracker.shutdown();
+    }
+  }
+
+  /*
+   * Method to handle CONSUMING->DROPPED transition.
+   *
+   * @param partitionGroupId Partition id that we must stop tracking on this 
server.
+   */
+  private void stopTrackingPartitionDelay(int partitionGroupId) {
+    if (_consumptionDelayTracker != null) {

Review Comment:
   Are we checking this non-null condition because we think doInit() won't be 
called? IIRC a lot of things will break if doInit() is not called .. consuming 
segments cannot be created at all, I think.  You may want to remove the check



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumptionDelayTracker.java:
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.core.data.manager.realtime;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track maximum realtime delay for a given table on a given server.
+ * Highlights:
+ * 1-An object of this class is hosted by each RealtimeTableDataManager.
+ * 2-The object tracks ingestion delays for all partitions hosted by the 
current server for the given Realtime table.
+ * 3-Partition delays are updated by all LLRealtimeSegmentDataManager objects 
hosted in the corresponding
+ *   RealtimeTableDataManager.
+ * 4-The class tracks the maximum of all ingestion delays observed for all 
partitions of the given table.
+ * 5-A Metric is derived from reading the maximum tracked by this class.
+ * 6-Delay reported for partitions that do not have events to consume is 
reported as zero.
+ * 7-We track the time at which each delay sample was collected so that delay 
can be increased when partition stops
+ *   consuming for any reason other than no events being available for 
consumption.
+ * 8-Segments that go from CONSUMING to DROPPED states stop being tracked so 
their delays do not cloud
+ *   delays of active partitions.
+ * 9-When a segment goes from CONSUMING to ONLINE, we start ticking time for 
the corresponding partition.
+ *   If no consumption is noticed after a timeout, then we read ideal state to 
confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 10-A timer thread is started by this object to track timeouts of partitions 
and drive the reading of their ideal
+ *    state.
+ *
+ *  The following diagram illustrates the object interactions with main 
external APIs
+ *
+ *     (CONSUMING -> ONLINE state change)
+ *             |
+ *      markPartitionForConfirmation(partitionId)
+ *            |                         
|<-storeConsumptionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                
|<-storeConsumptionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | ConsumptionDelayTracker            |           ...
+ * 
|____________________________________|<-storeConsumptionDelay()-{LLRealtimeSegmentDataManager
 (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    
stopTrackingPartitionConsumptionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class ConsumptionDelayTracker {
+
+  // Sleep interval for timer thread that triggers read of ideal state
+  private static final int TIMER_THREAD_TICK_INTERVAL_MS = 300000; // 5 
minutes +/- precision in timeouts
+  // Once a partition is marked for verification, we wait 10 minutes to pull 
its ideal state.
+  private static final int PARTITION_TIMEOUT_MS = 600000;          // 10 
minutes timeouts
+  // Delay Timer thread for this time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep a Pinot Consumption Delay measure and the time when the 
sample was taken (i.e. sample time)
+   * We will use the sample time to increase consumption delay when a 
partition stops consuming: the time
+   * difference between the sample time and current time will be added to the 
metric when read.
+   */
+  static private class DelayMeasure {
+    public DelayMeasure(long t, long d) {
+      _delayMilliseconds = d;
+      _sampleTime = t;
+    }
+    public long _delayMilliseconds;
+    public long _sampleTime;
+  }
+
+  /*
+   * Class to handle timer thread that will track inactive partitions
+   */
+  private class TrackingTimerTask extends TimerTask {
+    final private ConsumptionDelayTracker _tracker;
+
+    public TrackingTimerTask(ConsumptionDelayTracker tracker) {
+      _tracker = tracker;
+    }
+
+    @Override
+    public void run() {
+      // tick inactive partitions every interval of time to keep tracked 
partitions up to date
+      _tracker.timeoutInactivePartitions();
+    }
+  }
+
+  // HashMap used to store delay measures for all partitions active for the 
current table.
+  // _partitionsToDelaySampleMap<Key=PartitionGroupId,Value=DelaySample>
+  private ConcurrentHashMap<Integer, DelayMeasure> _partitionToDelaySampleMap 
= new ConcurrentHashMap<>();
+  // We mark partitions that go from CONSUMING to ONLINE in 
_partitionsMarkedForVerification: if they do not
+  // go back to CONSUMING in some period of time, we confirm whether they are 
still hosted in this server by reading
+  // ideal state. This is done with the goal of minimizing reading ideal state 
for efficiency reasons.
+  // 
_partitionsMarkedForVerification<Key=PartitionGroupId,Value=TimePartitionWasMarkedForVerificationMilliseconds>
+  private ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = 
new ConcurrentHashMap<>();
+  // Mutable versions of timer constants so we can test with smaller delays
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  final private Timer _timer;
+
+  final private ServerMetrics _serverMetrics;
+  final private String _tableNameWithType;
+
+  private boolean _enableAging;
+  private boolean _enablePerPartitionMetric = true;
+  private boolean _enableAggregateMetric = true;
+  final private Logger _logger;
+
+  final private RealtimeTableDataManager _realTimeTableDataManager;
+
+  /*
+   * Helper function to update the maximum when the current maximum is removed 
or updated.
+   * If no samples left we set maximum to minimum so new samples can be 
recorded.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (int partitionGroupId : _partitionToDelaySampleMap.keySet()) {
+      DelayMeasure currentMeasure = 
_partitionToDelaySampleMap.get(partitionGroupId);
+      if ((newMax == null)
+          ||
+          (currentMeasure != null) && (currentMeasure._delayMilliseconds > 
newMax._delayMilliseconds)) {
+        newMax = currentMeasure;
+      }
+    }
+    return newMax;
+  }
+
+  /*
+   * Helper function to age a delay measure. Aging means adding the time 
elapsed since the measure was
+   * taken till the measure is being reported.
+   */
+  private long getAgedDelay(DelayMeasure currentDelay) {
+    if (currentDelay == null) {
+      return 0; // return 0 when not initialized
+    }
+    // Add age of measure to the reported value
+    long measureAgeInMs = _enableAging ? (System.currentTimeMillis() - 
currentDelay._sampleTime) : 0;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMilliseconds + measureAgeInMs;
+  }
+
+  private List<Integer> getPartitionsHostedByThisServerPerIdealState() {
+    return _realTimeTableDataManager.getHostedPartitionsGroupIds();
+  }
+  /*
+   * Helper function to be called when we should stop tracking a given 
partition. Removes the partition from
+   * all our maps, it also updates the maximum if the tracked partition was 
the previous maximum.
+   *
+   * @param partitionGroupId partition ID which we should stop tracking.
+   */
+  private void removePartitionId(int partitionGroupId) {
+    _partitionToDelaySampleMap.remove(partitionGroupId);
+    // If we are removing a partition we should stop reading its ideal state.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+    if (_enablePerPartitionMetric) {
+      
_serverMetrics.removeTableGauge(getPerPartitionMetricName(partitionGroupId),
+          ServerGauge.PER_PARTITION_CONSUMPTION_DELAY_MS);
+    }
+  }
+
+  /*
+   * Helper function to generate a per partition metric name.
+   *
+   * @param partitionGroupId the partition group id to be appended to the 
table name so we
+   *        can differentiate between metrics for various partitions.
+   *
+   * @return a metric name with the following structure: tableNameWithType + 
partitionGroupId
+   */
+  private String getPerPartitionMetricName(int partitionGroupId) {
+    return _tableNameWithType + partitionGroupId;
+  }
+
+  // Custom Constructor
+  public ConsumptionDelayTracker(ServerMetrics serverMetrics, String 
tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int 
timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric)
+      throws RuntimeException {
+    _logger = LoggerFactory.getLogger(tableNameWithType + "-" + 
getClass().getSimpleName());
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = metricNamePrefix + tableNameWithType;
+    _realTimeTableDataManager = realtimeTableDataManager;
+    // Handle negative timer values
+    if (timerThreadTickIntervalMs <= 0) {
+      throw new RuntimeException("Illegal timer timeout argument, expected > 
0, got=" + timerThreadTickIntervalMs);
+    }
+    _enableAging = true;
+    _enablePerPartitionMetric = enablePerPartitionMetric;
+    _enableAggregateMetric = enableAggregateMetric;
+    _timerThreadTickIntervalMs = timerThreadTickIntervalMs;
+    _timer = new Timer("ConsumptionDelayTimerThread" + tableNameWithType);
+    _timer.schedule(new TrackingTimerTask(this), 
INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+    // Install callback metric
+    if (_enableAggregateMetric) {
+      _serverMetrics.addCallbackTableGaugeIfNeeded(_tableNameWithType, 
ServerGauge.MAX_CONSUMPTION_DELAY_MS,
+          () -> (long) getMaxConsumptionDelay());
+    }
+  }
+
+  // Constructor that uses defaults
+  public ConsumptionDelayTracker(ServerMetrics serverMetrics, String 
tableNameWithType,
+      RealtimeTableDataManager tableDataManager) {
+    this(serverMetrics, tableNameWithType, tableDataManager, 
TIMER_THREAD_TICK_INTERVAL_MS,
+        "", true, true);
+  }
+
+  // Constructor that takes a prefix to name the metric, so we can keep 
multiple trackers for the same table
+  public ConsumptionDelayTracker(ServerMetrics serverMetrics, String 
tableNameWithType, String metricNamePrefix,
+      RealtimeTableDataManager tableDataManager) {
+    this(serverMetrics, metricNamePrefix + tableNameWithType, tableDataManager,
+        TIMER_THREAD_TICK_INTERVAL_MS, metricNamePrefix, true, true);
+  }
+
+
+  /**
+   * Use to set or rest the aging of reported values.

Review Comment:
   Let us remove this call and adjust tests accordingly.



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumptionDelayTracker.java:
##########
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.core.data.manager.realtime;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track maximum realtime delay for a given table on a given server.
+ * Highlights:
+ * 1-An object of this class is hosted by each RealtimeTableDataManager.
+ * 2-The object tracks ingestion delays for all partitions hosted by the 
current server for the given Realtime table.
+ * 3-Partition delays are updated by all LLRealtimeSegmentDataManager objects 
hosted in the corresponding
+ *   RealtimeTableDataManager.
+ * 4-The class tracks the maximum of all ingestion delays observed for all 
partitions of the given table.
+ * 5-A Metric is derived from reading the maximum tracked by this class.
+ * 6-Delay reported for partitions that do not have events to consume is 
reported as zero.
+ * 7-We track the time at which each delay sample was collected so that delay 
can be increased when partition stops
+ *   consuming for any reason other than no events being available for 
consumption.
+ * 8-Segments that go from CONSUMING to DROPPED states stop being tracked so 
their delays do not cloud
+ *   delays of active partitions.
+ * 9-When a segment goes from CONSUMING to ONLINE, we start ticking time for 
the corresponding partition.
+ *   If no consumption is noticed after a timeout, then we read ideal state to 
confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 10-A timer thread is started by this object to track timeouts of partitions 
and drive the reading of their ideal
+ *    state.
+ *
+ *  The following diagram illustrates the object interactions with main 
external APIs
+ *
+ *     (CONSUMING -> ONLINE state change)
+ *             |
+ *      markPartitionForConfirmation(partitionId)
+ *            |                         
|<-storeConsumptionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                
|<-storeConsumptionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | ConsumptionDelayTracker            |           ...
+ * 
|____________________________________|<-storeConsumptionDelay()-{LLRealtimeSegmentDataManager
 (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    
stopTrackingPartitionConsumptionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class ConsumptionDelayTracker {
+
+  // Sleep interval for timer thread that triggers read of ideal state
+  private static final int TIMER_THREAD_TICK_INTERVAL_MS = 300000; // 5 
minutes +/- precision in timeouts
+  // Once a partition is marked for verification, we wait 10 minutes to pull 
its ideal state.
+  private static final int PARTITION_TIMEOUT_MS = 600000;          // 10 
minutes timeouts
+  // Delay Timer thread for this time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep a Pinot Consumption Delay measure and the time when the 
sample was taken (i.e. sample time)
+   * We will use the sample time to increase consumption delay when a 
partition stops consuming: the time
+   * difference between the sample time and current time will be added to the 
metric when read.
+   */
+  static private class DelayMeasure {
+    public DelayMeasure(long t, long d) {
+      _delayMilliseconds = d;
+      _sampleTime = t;
+    }
+    public long _delayMilliseconds;
+    public long _sampleTime;
+  }
+
+  /*
+   * Class to handle timer thread that will track inactive partitions
+   */
+  private class TrackingTimerTask extends TimerTask {
+    final private ConsumptionDelayTracker _tracker;
+
+    public TrackingTimerTask(ConsumptionDelayTracker tracker) {
+      _tracker = tracker;
+    }
+
+    @Override
+    public void run() {
+      // tick inactive partitions every interval of time to keep tracked 
partitions up to date
+      _tracker.timeoutInactivePartitions();
+    }
+  }
+
+  // HashMap used to store delay measures for all partitions active for the 
current table.
+  // _partitionsToDelaySampleMap<Key=PartitionGroupId,Value=DelaySample>
+  private ConcurrentHashMap<Integer, DelayMeasure> _partitionToDelaySampleMap 
= new ConcurrentHashMap<>();
+  // We mark partitions that go from CONSUMING to ONLINE in 
_partitionsMarkedForVerification: if they do not
+  // go back to CONSUMING in some period of time, we confirm whether they are 
still hosted in this server by reading
+  // ideal state. This is done with the goal of minimizing reading ideal state 
for efficiency reasons.
+  // 
_partitionsMarkedForVerification<Key=PartitionGroupId,Value=TimePartitionWasMarkedForVerificationMilliseconds>
+  private ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = 
new ConcurrentHashMap<>();
+  // Mutable versions of timer constants so we can test with smaller delays
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  final private Timer _timer;
+
+  final private ServerMetrics _serverMetrics;
+  final private String _tableNameWithType;
+
+  private boolean _enableAging;
+  private boolean _enablePerPartitionMetric = true;

Review Comment:
   make these final, and don't set the values here



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumptionDelayTracker.java:
##########
@@ -0,0 +1,295 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.core.data.manager.realtime;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track maximum realtime delay for a given table on a given server.
+ * Highlights:
+ * 1-An object of this class is hosted by each RealtimeTableDataManager.
+ * 2-The object tracks ingestion delays for all partitions hosted by the 
current server for the given Realtime table.
+ * 3-Partition delays are updated by all LLRealtimeSegmentDataManager objects 
hosted in the corresponding
+ *   RealtimeTableDataManager.
+ * 4-The class tracks the maximum of all ingestion delays observed for all 
partitions of the given table.
+ * 5-A Metric is derived from reading the maximum tracked by this class.
+ * 6-Delay reported for partitions that do not have events to consume is 
reported as zero.
+ * 7-We track the time at which each delay sample was collected so that delay 
can be increased when partition stops
+ *   consuming for any reason other than no events being available for 
consumption.
+ * 8-Segments that go from CONSUMING to DROPPED states stop being tracked so 
their delays do not cloud
+ *   delays of active partitions.
+ * 9-When a segment goes from CONSUMING to ONLINE, we start ticking time for 
the corresponding partition.
+ *   If no consumption is noticed after a timeout, then we read ideal state to 
confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 10-A timer thread is started by this object to track timeouts of partitions 
and drive the reading of their ideal
+ *    state.
+ *
+ *  The following diagram illustrates the object interactions with main 
external APIs
+ *
+ *     (CONSUMING -> ONLINE state change)
+ *             |
+ *      markPartition(partitionId)      
|<-recordPinotConsumptionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                
|<-recordPinotConsumptionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | ConsumptionDelayTracker|           ...
+ * 
|____________________________________|<-recordPinotConsumptionDelay()-{LLRealtimeSegmentDataManager
 (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    
stopTrackingPinotConsumptionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class ConsumptionDelayTracker {
+  /*
+   * Class to handle timer thread that will track inactive partitions
+   */
+  private class TrackingTimerTask extends TimerTask {
+    private String _name;
+    private ConsumptionDelayTracker _tracker;
+
+    public TrackingTimerTask(String name, ConsumptionDelayTracker tracker) {
+      _name = name;
+      _tracker = tracker;
+    }
+
+    @Override
+    public void run() {
+      // tick inactive partitions every interval of time to keep tracked 
partitions up to date
+      _tracker.timeoutInactivePartitions();
+    }
+  }
+
+  /*
+   * Class to keep a Pinot Consumption Delay measure and the time when the 
sample was taken (i.e. sample time)
+   * We will use the sample time to increase consumption delay when a 
partition stops consuming: the time
+   * difference between the sample time and current time will be added to the 
metric when read.
+   */
+  static private class DelayMeasure {
+    public DelayMeasure(long t, long d) {
+      _delayMilliseconds = d;
+      _sampleTime = t;
+    }
+    public long _delayMilliseconds;
+    public long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the 
current table.
+  // _partitionsToDelaySampleMap<Key=PartitionGroupId,Value=DelaySample>
+  private ConcurrentHashMap<Long, DelayMeasure> _partitionToDelaySampleMap = 
new ConcurrentHashMap<>();
+  // We mark partitions that go from CONSUMING to ONLINE in 
_partitionsMarkedForVerification: if they do not
+  // go back to CONSUMING in some period of time, we confirm whether they are 
still hosted in this server by reading
+  // ideal state. This is done with the goal of minimizing reading ideal state 
for efficiency reasons.
+  // 
_partitionsMarkedForVerification<Key=PartitionGroupId,Value=TimePartitionWasMarkedForVerificationMilliseconds>
+  private ConcurrentHashMap<Long, Long> _partitionsMarkedForVerification = new 
ConcurrentHashMap<>();
+  // Sleep interval for timer thread that triggers read of ideal state
+  private static final int TIMER_THREAD_TICK_INTERVAL_MS = 300000; // 5 
minutes +/- precision in timeouts
+  // Once a partition is marked for verification, we wait 10 minutes to pull 
its ideal state.
+  private static final int PARTITION_TIMEOUT_MS = 600000;          // 10 
minutes timeouts
+  // Delay Timer thread for this time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 1000;
+  // Mutable versions of the above constants so we can test with smaller delays
+  final int _timerThreadTickIntervalMs;
+  final int _initialTimeThreadDelayMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  final private Timer _timer;
+
+  final private ServerMetrics _serverMetrics;
+  final private String _tableNameWithType;
+
+  private boolean _enableAging;
+  final private Logger _logger;
+
+  final private RealtimeTableDataManager _realTimeTableDataManager;
+
+  /*
+   * Helper function to update the maximum when the current maximum is removed 
or updated.
+   * If no samples left we set maximum to minimum so new samples can be 
recorded.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (long pid : _partitionToDelaySampleMap.keySet()) {
+      DelayMeasure currentMeasure = _partitionToDelaySampleMap.get(pid);
+      if ((newMax == null) || (currentMeasure._delayMilliseconds > 
newMax._delayMilliseconds)) {
+        newMax = currentMeasure;
+      }
+    }
+    return newMax;
+  }
+
+  private List<Integer> getPartitionsHostedByThisServerPerIdealState() {
+    return _realTimeTableDataManager.getTablePartitionsHostedByThisServer();
+  }
+  /*
+   * Helper function to be called when we should stop tracking a given 
partition. Removes the partition from
+   * all our maps, it also updates the maximum if the tracked partition was 
the previous maximum.
+   *
+   * @param partitionGroupId partition ID which we should stop tracking.
+   */
+  private void removePartitionId(long partitionGroupId) {
+    _partitionToDelaySampleMap.remove(partitionGroupId);
+    // If we are removing a partition we should stop reading its ideal state.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+   }
+
+  // Custom Constructor
+  public ConsumptionDelayTracker(ServerMetrics serverMetrics, String 
tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int 
initialTimeThreadDelayMs, int timerThreadTickIntervalMs)
+      throws RuntimeException {
+    _logger = LoggerFactory.getLogger(tableNameWithType + "-" + 
getClass().getSimpleName());

Review Comment:
   Log lines show up being really long with threadname. This class will not be 
associated with anything in the offline table. If at a later day, we do see the 
need for it, we can change the logger name.
   please use raw table name



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumptionDelayTracker.java:
##########
@@ -0,0 +1,295 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.core.data.manager.realtime;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track maximum realtime delay for a given table on a given server.
+ * Highlights:
+ * 1-An object of this class is hosted by each RealtimeTableDataManager.
+ * 2-The object tracks ingestion delays for all partitions hosted by the 
current server for the given Realtime table.
+ * 3-Partition delays are updated by all LLRealtimeSegmentDataManager objects 
hosted in the corresponding
+ *   RealtimeTableDataManager.
+ * 4-The class tracks the maximum of all ingestion delays observed for all 
partitions of the given table.
+ * 5-A Metric is derived from reading the maximum tracked by this class.
+ * 6-Delay reported for partitions that do not have events to consume is 
reported as zero.
+ * 7-We track the time at which each delay sample was collected so that delay 
can be increased when partition stops
+ *   consuming for any reason other than no events being available for 
consumption.
+ * 8-Segments that go from CONSUMING to DROPPED states stop being tracked so 
their delays do not cloud
+ *   delays of active partitions.
+ * 9-When a segment goes from CONSUMING to ONLINE, we start ticking time for 
the corresponding partition.
+ *   If no consumption is noticed after a timeout, then we read ideal state to 
confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 10-A timer thread is started by this object to track timeouts of partitions 
and drive the reading of their ideal
+ *    state.
+ *
+ *  The following diagram illustrates the object interactions with main 
external APIs
+ *
+ *     (CONSUMING -> ONLINE state change)
+ *             |
+ *      markPartition(partitionId)      
|<-recordPinotConsumptionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                
|<-recordPinotConsumptionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | ConsumptionDelayTracker|           ...
+ * 
|____________________________________|<-recordPinotConsumptionDelay()-{LLRealtimeSegmentDataManager
 (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    
stopTrackingPinotConsumptionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class ConsumptionDelayTracker {
+  /*
+   * Class to handle timer thread that will track inactive partitions
+   */
+  private class TrackingTimerTask extends TimerTask {
+    private String _name;
+    private ConsumptionDelayTracker _tracker;
+
+    public TrackingTimerTask(String name, ConsumptionDelayTracker tracker) {
+      _name = name;
+      _tracker = tracker;
+    }
+
+    @Override
+    public void run() {
+      // tick inactive partitions every interval of time to keep tracked 
partitions up to date
+      _tracker.timeoutInactivePartitions();
+    }
+  }
+
+  /*
+   * Class to keep a Pinot Consumption Delay measure and the time when the 
sample was taken (i.e. sample time)
+   * We will use the sample time to increase consumption delay when a 
partition stops consuming: the time
+   * difference between the sample time and current time will be added to the 
metric when read.
+   */
+  static private class DelayMeasure {
+    public DelayMeasure(long t, long d) {
+      _delayMilliseconds = d;
+      _sampleTime = t;
+    }
+    public long _delayMilliseconds;
+    public long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the 
current table.
+  // _partitionsToDelaySampleMap<Key=PartitionGroupId,Value=DelaySample>
+  private ConcurrentHashMap<Long, DelayMeasure> _partitionToDelaySampleMap = 
new ConcurrentHashMap<>();
+  // We mark partitions that go from CONSUMING to ONLINE in 
_partitionsMarkedForVerification: if they do not
+  // go back to CONSUMING in some period of time, we confirm whether they are 
still hosted in this server by reading
+  // ideal state. This is done with the goal of minimizing reading ideal state 
for efficiency reasons.
+  // 
_partitionsMarkedForVerification<Key=PartitionGroupId,Value=TimePartitionWasMarkedForVerificationMilliseconds>
+  private ConcurrentHashMap<Long, Long> _partitionsMarkedForVerification = new 
ConcurrentHashMap<>();
+  // Sleep interval for timer thread that triggers read of ideal state
+  private static final int TIMER_THREAD_TICK_INTERVAL_MS = 300000; // 5 
minutes +/- precision in timeouts
+  // Once a partition is marked for verification, we wait 10 minutes to pull 
its ideal state.
+  private static final int PARTITION_TIMEOUT_MS = 600000;          // 10 
minutes timeouts
+  // Delay Timer thread for this time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 1000;
+  // Mutable versions of the above constants so we can test with smaller delays
+  final int _timerThreadTickIntervalMs;
+  final int _initialTimeThreadDelayMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  final private Timer _timer;
+
+  final private ServerMetrics _serverMetrics;
+  final private String _tableNameWithType;
+
+  private boolean _enableAging;
+  final private Logger _logger;
+
+  final private RealtimeTableDataManager _realTimeTableDataManager;
+
+  /*
+   * Helper function to update the maximum when the current maximum is removed 
or updated.
+   * If no samples left we set maximum to minimum so new samples can be 
recorded.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (long pid : _partitionToDelaySampleMap.keySet()) {
+      DelayMeasure currentMeasure = _partitionToDelaySampleMap.get(pid);
+      if ((newMax == null) || (currentMeasure._delayMilliseconds > 
newMax._delayMilliseconds)) {
+        newMax = currentMeasure;
+      }
+    }
+    return newMax;
+  }
+
+  private List<Integer> getPartitionsHostedByThisServerPerIdealState() {
+    return _realTimeTableDataManager.getTablePartitionsHostedByThisServer();
+  }
+  /*
+   * Helper function to be called when we should stop tracking a given 
partition. Removes the partition from
+   * all our maps, it also updates the maximum if the tracked partition was 
the previous maximum.
+   *
+   * @param partitionGroupId partition ID which we should stop tracking.
+   */
+  private void removePartitionId(long partitionGroupId) {
+    _partitionToDelaySampleMap.remove(partitionGroupId);
+    // If we are removing a partition we should stop reading its ideal state.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+   }
+
+  // Custom Constructor
+  public ConsumptionDelayTracker(ServerMetrics serverMetrics, String 
tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int 
initialTimeThreadDelayMs, int timerThreadTickIntervalMs)
+      throws RuntimeException {
+    _logger = LoggerFactory.getLogger(tableNameWithType + "-" + 
getClass().getSimpleName());
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _realTimeTableDataManager = realtimeTableDataManager;
+    // Handle negative timer values
+    if ((initialTimeThreadDelayMs < 0) || (timerThreadTickIntervalMs <= 0)) {
+      throw new RuntimeException("Illegal timer arguments");
+    }
+    _enableAging = true;
+    _initialTimeThreadDelayMs = initialTimeThreadDelayMs;
+    _timerThreadTickIntervalMs = timerThreadTickIntervalMs;
+    _timer = new Timer("ConsumptionDelayTimerThread" + tableNameWithType);

Review Comment:
   same as before, Long names show up in thread dumps, etc. We should be ok 
with raw table name, please remove, thanks



-- 
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-unsubscr...@pinot.apache.org

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


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

Reply via email to