You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/12/15 17:02:28 UTC

[GitHub] [pinot] jugomezv opened a new pull request, #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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

   … in Pinot
   
   Early draft for review, no significant unitest or end to end testing done yet. Missing: code to read ideal state, unitets.
   -Changed code to not remove but verify a partition when CONSUMING to OFFLINE transition comes
   Tests: make sure server, core tests pass.
   -Added support for reading ideal state
   -Added some unitests for tracker object
   -Added more unitests that deal with partition removal. -Removed caching the maximum to simplefy the code. -Use Concurrent hashes instead of synchronized methods. -Rename some variables for consistency
   


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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1054693046


##########
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);
+    _timer.schedule(new TrackingTimerTask("TimerTask" + tableNameWithType, this),
+        _initialTimeThreadDelayMs, _timerThreadTickIntervalMs);
+    // Install callback metric
+    _serverMetrics.addCallbackTableGaugeIfNeeded(_tableNameWithType, ServerGauge.MAX_PINOT_CONSUMPTION_DELAY_MS,
+        () -> (long) getMaxPinotConsumptionDelay());
+  }
+
+  // Constructor that uses default timeout
+  public ConsumptionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager) {
+    this(serverMetrics, tableNameWithType, tableDataManager, INITIAL_TIMER_THREAD_DELAY_MS,
+        TIMER_THREAD_TICK_INTERVAL_MS);
+  }
+
+  /**
+   * Use to set or rest the aging of reported values.
+   * @param enableAging true if we want maximum to be aged as per sample time or false if we do not want to age
+   *                   samples
+   */
+  @VisibleForTesting
+  void setEnableAging(boolean enableAging) {
+    _enableAging = enableAging;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   * If the new sample represents a new Maximum we update the current maximum.
+   * If the new sample was for the partition that was maximum, but delay is not maximum anymore, we must select
+   * a new maximum.
+   *
+   * @param delayInMilliseconds pinot consumption delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void storeConsumptionDelay(long delayInMilliseconds, long sampleTime, long partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    _partitionToDelaySampleMap.put(partitionGroupId, new DelayMeasure(sampleTime, delayInMilliseconds));
+    // If we are consuming we do not need to track this partition for removal.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+  }
+
+  /*
+   * Handle partition removal event. This must be invoked when we stop serving a given partition for
+   * this table in the current server.
+   * This function will be invoked when we receive CONSUMING -> DROPPED / OFFLINE state transitions.
+   *
+   * @param partitionGroupId partition id that we should stop tracking.
+   */
+  public void stopTrackingPartitionConsumptionDelay(long partitionGroupId) {
+    removePartitionId(partitionGroupId);
+  }
+
+  /*
+   * This method is used for timing out inactive partitions, so we don't display their metrics on current server.
+   * When the inactive time exceeds some threshold, we read from ideal state to confirm we still host the partition,
+   * if not we remove the partition from being tracked locally.
+   * This call is to be invoked by a timer thread that will periodically wake up and invoke this function.
+   */
+  public void timeoutInactivePartitions() {
+    List<Integer> partitionsHostedByThisServer = null;
+    try {
+      partitionsHostedByThisServer = getPartitionsHostedByThisServerPerIdealState();
+    } catch (Exception e) {
+      _logger.error("Failed to get partitions hosted by this server for table {}", _tableNameWithType);

Review Comment:
   good point



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060843027


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -613,7 +613,7 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi
         _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);
+      setIngestionDelayToZero();

Review Comment:
   So are you suggesting we change the initial implementation of updateIngestion delay? I guess I can do that but it will then look pretty close to what we have initially: a method that passes a number of arguments. 



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061972872


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java:
##########
@@ -201,4 +201,18 @@ void addOrReplaceSegment(String segmentName, IndexLoadingConfig indexLoadingConf
    * @return List of {@link SegmentErrorInfo}
    */
   Map<String, SegmentErrorInfo> getSegmentErrors();
+
+  /**
+   * Interface to place actions to be performed when a Segment goes from CONSUMING to ONLINE.
+   *
+   * @param segmentNameStr name of segment for which the state change is being handled
+   */
+  default void onConsumingToOnline(String segmentNameStr) { };
+
+  /**
+   * Interface to place actions to be performed when a Segment goes from CONSUMING to DROPPED.
+   *
+   * @param segmentNameStr name of segment for which the state change is being handled
+   */
+  default void onConsumingToDropped(String segmentNameStr) { };
 }

Review Comment:
   We tried this but it made things a little more complicated so the change was rolled back



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060970122


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java:
##########
@@ -201,4 +201,18 @@ void addOrReplaceSegment(String segmentName, IndexLoadingConfig indexLoadingConf
    * @return List of {@link SegmentErrorInfo}
    */
   Map<String, SegmentErrorInfo> getSegmentErrors();
+
+  /**
+   * Interface to place actions to be performed when a Segment goes from CONSUMING to ONLINE.
+   *
+   * @param segmentNameStr name of segment for which the state change is being handled
+   */
+  default void onConsumingToOnline(String segmentNameStr) { };
+
+  /**
+   * Interface to place actions to be performed when a Segment goes from CONSUMING to DROPPED.
+   *
+   * @param segmentNameStr name of segment for which the state change is being handled
+   */
+  default void onConsumingToDropped(String segmentNameStr) { };
 }

Review Comment:
   Seems like a good suggestion to keep the high level interface simpler, working on this.



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1053913365


##########
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;

Review Comment:
   final. please do for other classes as well .. will not repeat.



##########
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;

Review Comment:
   final please



##########
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);
+    _timer.schedule(new TrackingTimerTask("TimerTask" + tableNameWithType, this),
+        _initialTimeThreadDelayMs, _timerThreadTickIntervalMs);
+    // Install callback metric
+    _serverMetrics.addCallbackTableGaugeIfNeeded(_tableNameWithType, ServerGauge.MAX_PINOT_CONSUMPTION_DELAY_MS,
+        () -> (long) getMaxPinotConsumptionDelay());
+  }
+
+  // Constructor that uses default timeout
+  public ConsumptionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager) {
+    this(serverMetrics, tableNameWithType, tableDataManager, INITIAL_TIMER_THREAD_DELAY_MS,
+        TIMER_THREAD_TICK_INTERVAL_MS);
+  }
+
+  /**
+   * Use to set or rest the aging of reported values.
+   * @param enableAging true if we want maximum to be aged as per sample time or false if we do not want to age
+   *                   samples
+   */
+  @VisibleForTesting
+  void setEnableAging(boolean enableAging) {
+    _enableAging = enableAging;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   * If the new sample represents a new Maximum we update the current maximum.
+   * If the new sample was for the partition that was maximum, but delay is not maximum anymore, we must select
+   * a new maximum.
+   *
+   * @param delayInMilliseconds pinot consumption delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void storeConsumptionDelay(long delayInMilliseconds, long sampleTime, long partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    _partitionToDelaySampleMap.put(partitionGroupId, new DelayMeasure(sampleTime, delayInMilliseconds));
+    // If we are consuming we do not need to track this partition for removal.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+  }
+
+  /*
+   * Handle partition removal event. This must be invoked when we stop serving a given partition for
+   * this table in the current server.
+   * This function will be invoked when we receive CONSUMING -> DROPPED / OFFLINE state transitions.
+   *
+   * @param partitionGroupId partition id that we should stop tracking.
+   */
+  public void stopTrackingPartitionConsumptionDelay(long partitionGroupId) {

Review Comment:
   Why is partitionGroupId long? We have it as int everywhere else



##########
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:
   Is this there only for testing, or are we trying to make it configurable whether to enable aging or not?



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064194317


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -611,6 +613,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:
   The outcome is the same: metrics are reported as zero anyway.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1066499436


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.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.time.Clock;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ * TODO: handle bug situations like the one where a partition is not allocated to a given server due to a bug.
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 {

Review Comment:
   Good simplification, I am working on this. I think this complexity came out of first considering measures with and not age in the same code base and then forgetting to simplify, nice catch. Working on this now



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064194028


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMilliseconds + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.TABLE_INGESTION_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: _metricName + partitionGroupId
+   */
+  private String getPerPartitionMetricName(int partitionGroupId) {
+    return _metricName + partitionGroupId;
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  // Custom Constructor
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric)
+      throws RuntimeException {
+    _logger = LoggerFactory.getLogger(getClass().getSimpleName());
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;
+    _realTimeTableDataManager = realtimeTableDataManager;
+    _clock = Clock.systemDefaultZone();

Review Comment:
   Good point @mcvsubbu  addressed in commit ecfcf1b1cdf81f4fe3547e1eec4ac23a1df72c5c



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064235087


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -115,15 +117,24 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
 
   private TableDedupMetadataManager _tableDedupMetadataManager;
   private TableUpsertMetadataManager _tableUpsertMetadataManager;
+  // Object to track ingestion delay for all partitions
+  private IngestionDelayTracker _ingestionDelayTracker;
+  private Supplier<Boolean> _isReadyToServeQueries;
 
   public RealtimeTableDataManager(Semaphore segmentBuildSemaphore) {
+    this(segmentBuildSemaphore, () -> true);
+  }
+
+  public RealtimeTableDataManager(Semaphore segmentBuildSemaphore, Supplier<Boolean> isReadyToServeQueries) {
     _segmentBuildSemaphore = segmentBuildSemaphore;
+    _isReadyToServeQueries = isReadyToServeQueries;

Review Comment:
   This has been addressed in commit 2e4daa35609356522a483f2a9503dde263a2e5ff, please close this at your earliest if this satisfy your request



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065219850


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java:
##########
@@ -183,4 +184,11 @@ void addOrReplaceSegment(String tableNameWithType, String segmentName)
    * Immediately stop consumption and start committing the consuming segments.
    */
   void forceCommit(String tableNameWithType, Set<String> segmentNames);
+
+  /**
+   * Enables the installation of a method to determine if a server is ready to server queries.
+   *
+   * @param isServerReadyToServeQueries supplier to retrieve state of server.
+   */
+  void setSupplierOfIsServerReadyToServeQueries(Supplier<Boolean> isServerReadyToServeQueries);

Review Comment:
   Added default init to avoid the problem you are describing, may be we can refine later if necessary: see commit 3ce7b783c5dc88073106f400f00783ba4ab71281



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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065116929


##########
pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java:
##########
@@ -45,8 +45,9 @@ public enum ServerGauge implements AbstractMetrics.Gauge {
   // Dedup metrics
   DEDUP_PRIMARY_KEYS_COUNT("dedupPrimaryKeysCount", false),
   CONSUMPTION_QUOTA_UTILIZATION("ratio", false),
-  JVM_HEAP_USED_BYTES("bytes", true);
-
+  JVM_HEAP_USED_BYTES("bytes", true),
+  // Ingestion delay metrics
+  TABLE_INGESTION_DELAY_MS("milliseconds", false);

Review Comment:
   Suggest renaming it to `REALTIME_INGESTION_DELAY_MS` since it is not always table level
   (nit) Keep an empty line after this



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -39,37 +39,56 @@ private TableStateUtils() {
   }
 
   /**
-   * Checks if all segments for the given @param tableNameWithType are succesfully loaded
-   * This function will get all segments in IDEALSTATE and CURRENTSTATE for the given table,
-   * and then check if all ONLINE segments in IDEALSTATE match with CURRENTSTATE.
-   * @param helixManager helix manager for the server instance
-   * @param tableNameWithType table name for which segment state is to be checked
-   * @return true if all segments for the given table are succesfully loaded. False otherwise
+   * Returns all segments in a given state for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   * @param state state of the segments to be returned
+   *
+   * @return List of segment names in a given state.
    */
-  public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+  public static Set<String> getSegmentsInGivenStateForThisInstance(HelixManager helixManager, String tableNameWithType,
+      String state) {
     HelixDataAccessor dataAccessor = helixManager.getHelixDataAccessor();
     PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
     IdealState idealState = dataAccessor.getProperty(keyBuilder.idealStates(tableNameWithType));
+    Set<String> segmentsInGivenState = new HashSet<>();
     if (idealState == null) {
       LOGGER.warn("Failed to find ideal state for table: {}", tableNameWithType);
-      return false;
+      return segmentsInGivenState;
     }
 
     // Get all ONLINE segments from idealState
     String instanceName = helixManager.getInstanceName();
-    List<String> onlineSegments = new ArrayList<>();
     Map<String, Map<String, String>> idealStatesMap = idealState.getRecord().getMapFields();
     for (Map.Entry<String, Map<String, String>> entry : idealStatesMap.entrySet()) {
       String segmentName = entry.getKey();
       Map<String, String> instanceStateMap = entry.getValue();
       String expectedState = instanceStateMap.get(instanceName);
       // Only track ONLINE segments assigned to the current instance
-      if (!CommonConstants.Helix.StateModel.SegmentStateModel.ONLINE.equals(expectedState)) {
+      if (!state.equals(expectedState)) {

Review Comment:
   Can be simplified
   ```suggestion
         if (state.equals(instanceStateMap.get(instanceName))) {
           segmentsInGivenState.add(segmentName);
         }
   ```



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -39,37 +39,56 @@ private TableStateUtils() {
   }
 
   /**
-   * Checks if all segments for the given @param tableNameWithType are succesfully loaded
-   * This function will get all segments in IDEALSTATE and CURRENTSTATE for the given table,
-   * and then check if all ONLINE segments in IDEALSTATE match with CURRENTSTATE.
-   * @param helixManager helix manager for the server instance
-   * @param tableNameWithType table name for which segment state is to be checked
-   * @return true if all segments for the given table are succesfully loaded. False otherwise
+   * Returns all segments in a given state for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   * @param state state of the segments to be returned
+   *
+   * @return List of segment names in a given state.
    */
-  public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+  public static Set<String> getSegmentsInGivenStateForThisInstance(HelixManager helixManager, String tableNameWithType,

Review Comment:
   Return a `List<String>` here to avoid the performance/memory overhead.
   Also suggest renaming the method to reflect that the state is read from IS



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.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.time.Clock;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ * TODO: handle bug situations like the one where a partition is not allocated to a given server due to a bug.
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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) {
+      _delayMs = d;
+      _sampleTime = t;
+    }
+    public final long _delayMs;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private final Supplier<Boolean> _isServerReadyToServeQueries;
+
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongst all partitions we are tracking.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (int partitionGroupId : _partitionToDelaySampleMap.keySet()) {

Review Comment:
   Use `_partitionToDelaySampleMap.entrySet()` to avoid extra lookups, same for other map traversals



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.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.time.Clock;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ * TODO: handle bug situations like the one where a partition is not allocated to a given server due to a bug.
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 {

Review Comment:
   Since we always count the diff with `(currentTime - sampleTime) + delayMs`, where `delayMs = sampleTime - ingestionTime`, why not directly tracking `ingestionTime`?



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.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.time.Clock;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ * TODO: handle bug situations like the one where a partition is not allocated to a given server due to a bug.
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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) {
+      _delayMs = d;
+      _sampleTime = t;
+    }
+    public final long _delayMs;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private final Supplier<Boolean> _isServerReadyToServeQueries;
+
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongst all partitions we are tracking.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (int partitionGroupId : _partitionToDelaySampleMap.keySet()) {
+      DelayMeasure currentMeasure = _partitionToDelaySampleMap.get(partitionGroupId);
+      if ((newMax == null)
+          ||
+          (currentMeasure != null) && (currentMeasure._delayMs > newMax._delayMs)) {
+        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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMs + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.removePartitionGauge(_metricName, partitionGroupId, ServerGauge.TABLE_INGESTION_DELAY_MS);
+    }
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric, Supplier<Boolean> isServerReadyToServeQueries)
+      throws RuntimeException {
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;
+    _realTimeTableDataManager = realtimeTableDataManager;
+    _clock = Clock.systemUTC();
+    _isServerReadyToServeQueries = isServerReadyToServeQueries;
+    // Handle negative timer values
+    if (timerThreadTickIntervalMs <= 0) {
+      throw new RuntimeException(String.format("Illegal timer timeout argument, expected > 0, got=%d for table=%s",
+          timerThreadTickIntervalMs, _tableNameWithType));
+    }
+    _enablePerPartitionMetric = enablePerPartitionMetric;
+    _enableAggregateMetric = enableAggregateMetric;
+    _timerThreadTickIntervalMs = timerThreadTickIntervalMs;
+    _timer = new Timer("IngestionDelayTimerThread" + TableNameBuilder.extractRawTableName(tableNameWithType));
+    _timer.schedule(new TimerTask() {
+        @Override
+        public void run() {
+          timeoutInactivePartitions();
+        }
+      }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+    // Install callback metric
+    if (_enableAggregateMetric) {
+      _serverMetrics.addCallbackTableGaugeIfNeeded(_metricName, ServerGauge.TABLE_INGESTION_DELAY_MS,
+          () -> getMaximumIngestionDelay());
+    }
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager, Supplier<Boolean> isServerReadyToServeQueries) {
+    this(serverMetrics, tableNameWithType, tableDataManager, TIMER_THREAD_TICK_INTERVAL_MS,
+        "", true, true, isServerReadyToServeQueries);
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager, String metricNamePrefix,
+      Supplier<Boolean> isServerReadyToServeQueries) {
+    this(serverMetrics, metricNamePrefix + tableNameWithType, tableDataManager,
+        TIMER_THREAD_TICK_INTERVAL_MS, metricNamePrefix, true, true, isServerReadyToServeQueries);
+  }
+
+  /**
+   * Function that enable use to set predictable clocks for testing purposes.
+   *
+   * @param clock clock to be used by the class
+   */
+  @VisibleForTesting
+  void setClock(Clock clock) {
+    _clock = clock;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   *
+   * @param delayMs ingestion delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void updateIngestionDelay(long delayMs, long sampleTime, int partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    // TODO: see if we can install gauges after the server is ready.
+    if (!_isServerReadyToServeQueries.get()) {
+      // Do not update the ingestion delay metrics during server startup period
+      return;
+    }
+    DelayMeasure previousMeasure = _partitionToDelaySampleMap.put(partitionGroupId,
+        new DelayMeasure(sampleTime, delayMs));
+    if ((previousMeasure == null) && _enablePerPartitionMetric) {
+      // First time we start tracking a partition we should start tracking it via metric
+      _serverMetrics.addCallbackPartitionGaugeIfNeeded(_metricName, partitionGroupId,
+          ServerGauge.TABLE_INGESTION_DELAY_MS, () -> getPartitionIngestionDelay(partitionGroupId));
+    }
+    // If we are consuming we do not need to track this partition for removal.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+  }
+
+  /*
+   * Handle partition removal event. This must be invoked when we stop serving a given partition for
+   * this table in the current server.
+   *
+   * @param partitionGroupId partition id that we should stop tracking.
+   */
+  public void stopTrackingPartitionIngestionDelay(int partitionGroupId) {
+    removePartitionId(partitionGroupId);
+  }
+
+  /*
+   * This method is used for timing out inactive partitions, so we don't display their metrics on current server.
+   * When the inactive time exceeds some threshold, we read from ideal state to confirm we still host the partition,
+   * if not we remove the partition from being tracked locally.
+   * This call is to be invoked by a timer thread that will periodically wake up and invoke this function.
+   */
+  public void timeoutInactivePartitions() {
+    Set<Integer> partitionsHostedByThisServer = null;
+    // Check if we have any partition to verify, else don't make the call to check ideal state as that
+    // involves network traffic and may be inefficient.
+    ArrayList<Integer> partitionsToVerify = getPartitionsToBeVerified();
+    if (partitionsToVerify.size() == 0) {
+      // Don't make the call to getHostedPartitionsGroupIds() as it involves checking ideal state.
+      return;
+    }
+    try {
+      partitionsHostedByThisServer = _realTimeTableDataManager.getHostedPartitionsGroupIds();
+    } catch (Exception e) {
+      _logger.error("Failed to get partitions hosted by this server, table={}", _tableNameWithType);
+      return;
+    }
+    for (int partitionGroupId : partitionsToVerify) {
+      if (!partitionsHostedByThisServer.contains(partitionGroupId)) {
+        // Partition is not hosted in this server anymore, stop tracking it
+        removePartitionId(partitionGroupId);
+      }
+    }
+  }
+
+  /*
+   * This function is invoked when a partition goes from CONSUMING to ONLINE, so we can assert whether the
+   * partition is still hosted by this server after some interval of time.
+   *
+   * @param partitionGroupId Partition id that we need confirmed via ideal state as still hosted by this server.
+   */
+  public void markPartitionForVerification(int partitionGroupId) {
+    _partitionsMarkedForVerification.put(partitionGroupId, _clock.millis());
+  }
+
+  /*
+   * This is the function to be invoked when reading the metric.
+   * It reports the maximum ingestion delay for all partitions of this table being served
+   * by current server; it adds the time elapsed since the sample was taken to the measure.
+   * If no measures have been taken, then the reported value is zero.
+   *
+   * @return max ingestion delay in milliseconds.
+   */
+  public long getMaximumIngestionDelay() {
+    DelayMeasure currentMaxDelay = getMaximumDelay();

Review Comment:
   I don't think this is correct because when calculating the `currentMaxDelay`, we didn't count for the difference between current time and sample time. E.g. we might get a message with very small delay initially, but the stream hangs for a long time



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java:
##########
@@ -201,4 +201,18 @@ void addOrReplaceSegment(String segmentName, IndexLoadingConfig indexLoadingConf
    * @return List of {@link SegmentErrorInfo}
    */
   Map<String, SegmentErrorInfo> getSegmentErrors();
+
+  /**
+   * Interface to handle segment state transitions from CONSUMING to DROPPED
+   *
+   * @param segmentNameStr name of segment for which the state change is being handled
+   */
+  default void onConsumingToDropped(String segmentNameStr) { };

Review Comment:
   (code format) same for the other one
   ```suggestion
     default void onConsumingToDropped(String segmentNameStr) {
     };
   ```



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -39,37 +39,56 @@ private TableStateUtils() {
   }
 
   /**
-   * Checks if all segments for the given @param tableNameWithType are succesfully loaded
-   * This function will get all segments in IDEALSTATE and CURRENTSTATE for the given table,
-   * and then check if all ONLINE segments in IDEALSTATE match with CURRENTSTATE.
-   * @param helixManager helix manager for the server instance
-   * @param tableNameWithType table name for which segment state is to be checked
-   * @return true if all segments for the given table are succesfully loaded. False otherwise
+   * Returns all segments in a given state for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   * @param state state of the segments to be returned
+   *
+   * @return List of segment names in a given state.
    */
-  public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+  public static Set<String> getSegmentsInGivenStateForThisInstance(HelixManager helixManager, String tableNameWithType,
+      String state) {
     HelixDataAccessor dataAccessor = helixManager.getHelixDataAccessor();
     PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
     IdealState idealState = dataAccessor.getProperty(keyBuilder.idealStates(tableNameWithType));
+    Set<String> segmentsInGivenState = new HashSet<>();
     if (idealState == null) {
       LOGGER.warn("Failed to find ideal state for table: {}", tableNameWithType);
-      return false;
+      return segmentsInGivenState;
     }
 
     // Get all ONLINE segments from idealState

Review Comment:
   (minor) update the comment



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMilliseconds + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.TABLE_INGESTION_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: _metricName + partitionGroupId
+   */
+  private String getPerPartitionMetricName(int partitionGroupId) {
+    return _metricName + partitionGroupId;
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  // Custom Constructor
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric)
+      throws RuntimeException {
+    _logger = LoggerFactory.getLogger(getClass().getSimpleName());
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;

Review Comment:
   I'd suggest keeping things simple especially for the first version. We can extend it in the future after getting the first version stable. We may consider removing `metricNamePrefix`, `enableAggregateMetric` and `enablePerPartitionMetric` and always emit partition level metric



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java:
##########
@@ -183,4 +184,11 @@ void addOrReplaceSegment(String tableNameWithType, String segmentName)
    * Immediately stop consumption and start committing the consuming segments.
    */
   void forceCommit(String tableNameWithType, Set<String> segmentNames);
+
+  /**
+   * Enables the installation of a method to determine if a server is ready to server queries.
+   *
+   * @param isServerReadyToServeQueries supplier to retrieve state of server.
+   */
+  void setSupplierOfIsServerReadyToServeQueries(Supplier<Boolean> isServerReadyToServeQueries);

Review Comment:
   From ease of use perspective, I feel `notifyStartServingQueries()` is better than passing in a supplier. We may maintain a supplier within the `InstanceDataManager`. Currently if this method is not called, we will get NPE somewhere because `HelixInstanceDataManager._isReadyToServeQueries` is `null`



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.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.time.Clock;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ * TODO: handle bug situations like the one where a partition is not allocated to a given server due to a bug.
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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) {
+      _delayMs = d;
+      _sampleTime = t;
+    }
+    public final long _delayMs;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private final Supplier<Boolean> _isServerReadyToServeQueries;
+
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongst all partitions we are tracking.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (int partitionGroupId : _partitionToDelaySampleMap.keySet()) {
+      DelayMeasure currentMeasure = _partitionToDelaySampleMap.get(partitionGroupId);
+      if ((newMax == null)
+          ||
+          (currentMeasure != null) && (currentMeasure._delayMs > newMax._delayMs)) {
+        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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMs + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.removePartitionGauge(_metricName, partitionGroupId, ServerGauge.TABLE_INGESTION_DELAY_MS);
+    }
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,

Review Comment:
   (minor) Put the constructors above other methods for readability



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065183312


##########
pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java:
##########
@@ -45,8 +45,9 @@ public enum ServerGauge implements AbstractMetrics.Gauge {
   // Dedup metrics
   DEDUP_PRIMARY_KEYS_COUNT("dedupPrimaryKeysCount", false),
   CONSUMPTION_QUOTA_UTILIZATION("ratio", false),
-  JVM_HEAP_USED_BYTES("bytes", true);
-
+  JVM_HEAP_USED_BYTES("bytes", true),
+  // Ingestion delay metrics
+  TABLE_INGESTION_DELAY_MS("milliseconds", false);

Review Comment:
   Done



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064229032


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +223,58 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();
+  }
+
+  /*
+   * Method used by LLRealtimeSegmentManagers to update their partition delays
+   *
+   * @param ingestionDelayMs Ingestion delay being reported.
+   * @param currentTimeMs 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 ingestionDelayMs, long currenTimeMs, int partitionGroupId) {
+    _ingestionDelayTracker.updateIngestionDelay(ingestionDelayMs, currenTimeMs, partitionGroupId);
+  }
+
+  /*
+   * Method to handle CONSUMING -> DROPPED segment state transitions:
+   * We stop tracking partitions whose segments are dropped.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   */
+  @Override
+  public void onConsumingToDropped(String segmentNameStr) {
+    LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr);
+    _ingestionDelayTracker.stopTrackingPartitionIngestionDelay(segmentName.getPartitionGroupId());
+  }
+
+  /*
+   * Method to handle CONSUMING -> ONLINE segment state transitions:
+   * We mark partitions for verification against 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);
+    _ingestionDelayTracker.markPartitionForVerification(segmentName.getPartitionGroupId());
+  }
+
+  /**
+   * Returns all partitionGroupIds for the partitions hosted by this server for current table.
+   * @Note: this involves Zookeeper read and should not be used frequently due to efficiency concerns.
+   */
+  public List<Integer> getHostedPartitionsGroupIds() {
+    ArrayList<Integer> partitionsHostedByThisServer = new ArrayList<>();
+    List<String> segments = TableStateUtils.getOnlineSegmentsForThisInstance(_helixManager, _tableNameWithType);

Review Comment:
   When the table is newly created, it will have only consuming segments.



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


[GitHub] [pinot] jadami10 commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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

   >  I guess first question is what is the stream involved? This should return zero when we are not observing events:messagesAndOffsets.getMessageCount();
   
   We ingest from kafka, but we talk through a proxy that consumes events from multiple kafka clusters. I've confirmed our subclass of `MessageBatch` returns the size of the records list we get, so it should be returning 0.
   
   > Can you check also what is the fetchTimeout config for your stream?
   
   We set it to 500ms. Our stream ingestion plugin is fairly strange. The upstream consume API we use is async, so we request records and wait either 500ms or until we've received 1000 records before returning.
   
   is this feature using offsets at all? I know we've had some weird issues in the past where there's an off by 1 depending on what you consider the "latest" offset


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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061968584


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +216,83 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();
+  }
+
+  /*
+   * Method to handle CONSUMING->DROPPED transition.
+   *
+   * @param partitionGroupId Partition id that we must stop tracking on this server.
+   */
+  private void stopTrackingPartitionDelay(int partitionGroupId) {
+    _ingestionDelayTracker.stopTrackingPartitionIngestionDelay(partitionGroupId);
+  }
+
+  /*
+   * Method to handle CONSUMING->ONLINE transition.
+   * If no new ingestion 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) {
+    _ingestionDelayTracker.markPartitionForConfirmation(partitionGroupId);
+  }
+
+  /*
+   * Method used by LLRealtimeSegmentManagers to update their partition delays
+   *
+   * @param ingestionDelayMs Ingestion delay being reported.
+   * @param currentTimeMs 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 ingestionDelayMs, long currenTimeMs, int partitionGroupId) {
+    _ingestionDelayTracker.updateIngestionDelay(ingestionDelayMs, currenTimeMs, partitionGroupId);
+  }
+
+  /*
+   * Method to handle supported transitions of segments states for this table.
+   * Supported transitions include:
+   *
+   * CONSUMING -> ONLINE:
+   * We mark partitions for verification against 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.
+   * CONSUMING -> DROPPED:
+   * We stop tracking partitions whose segments are dropped.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   * @param fromState state from which the segment is transitioning.
+   * @param toState state to which the segment is transitioning to.
+   */
+  @Override
+  public void onSegmentStateTransition(String segmentNameStr, SegmentState fromState, SegmentState toState) {

Review Comment:
   reverted this change to use two methods



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


[GitHub] [pinot] jadami10 commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +225,59 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();

Review Comment:
   Sorry, forgot to respond with testing results. I was thinking it should be done at the top of the function before the segment data managers are destroyed. But I tested that, and I still saw lag when servers were shutting down. I've since pivoted to tackling some other ingestion lag issues around losing controllers, but I'll update once I get a chance to come back to this.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1063091118


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1554,6 +1557,25 @@ private void createPartitionMetadataProvider(String reason) {
     _partitionMetadataProvider = _streamConsumerFactory.createPartitionMetadataProvider(_clientId, _partitionGroupId);
   }
 
+  /*
+   * Updates the ingestion delay if messages were processed using the time stamp for the last consumed event.
+   *
+   * @param indexedMessagesCount
+   */
+  private void updateIngestionDelay(int indexedMessageCount) {
+    if (_catchingUpPhase) {

Review Comment:
   Code now changed to use server ready signal



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064224309


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -39,25 +39,25 @@ private TableStateUtils() {
   }
 
   /**
-   * Checks if all segments for the given @param tableNameWithType are succesfully loaded
-   * This function will get all segments in IDEALSTATE and CURRENTSTATE for the given table,
-   * and then check if all ONLINE segments in IDEALSTATE match with CURRENTSTATE.
-   * @param helixManager helix manager for the server instance
-   * @param tableNameWithType table name for which segment state is to be checked
-   * @return true if all segments for the given table are succesfully loaded. False otherwise
+   * Returns all online segments for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   *
+   * @return List of ONLINE segment names.
    */
-  public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+  public static List<String> getOnlineSegmentsForThisInstance(HelixManager helixManager, String tableNameWithType) {

Review Comment:
   On second thought, I think I could modify the original code to use a set as well, so reopening this. The only pitfall is that I start modifying code that has been working for some time, but seems simple enough t take the time to do this



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


[GitHub] [pinot] mcvsubbu merged pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu merged PR #9994:
URL: https://github.com/apache/pinot/pull/9994


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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064194282


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  private long getAgedDelay(DelayMeasure currentDelay) {

Review Comment:
   This is higher abstraction than plain delay concept, doing the suggested will also make me pass the clock to delay which is currently very simple. I would rather leave as 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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064194102


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;

Review Comment:
   Yes, has been addressed in commit ecfcf1b1cdf81f4fe3547e1eec4ac23a1df72c5c



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065185467


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java:
##########
@@ -201,4 +201,18 @@ void addOrReplaceSegment(String segmentName, IndexLoadingConfig indexLoadingConf
    * @return List of {@link SegmentErrorInfo}
    */
   Map<String, SegmentErrorInfo> getSegmentErrors();
+
+  /**
+   * Interface to handle segment state transitions from CONSUMING to DROPPED
+   *
+   * @param segmentNameStr name of segment for which the state change is being handled
+   */
+  default void onConsumingToDropped(String segmentNameStr) { };

Review Comment:
   done



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1063043991


##########
pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/ConsumptionDelayTrackerTest.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.metrics.PinotMetricUtils;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class ConsumptionDelayTrackerTest {
+
+  private static final int TIMER_THREAD_TICK_INTERVAL_MS = 100;
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  private void sleepMs(long timeMs) {

Review Comment:
   This has been addressed by commit 9a6b0e5fb705ac00f25b2e41f5ca7690b05ca34e



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065002648


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -39,25 +39,25 @@ private TableStateUtils() {
   }
 
   /**
-   * Checks if all segments for the given @param tableNameWithType are succesfully loaded
-   * This function will get all segments in IDEALSTATE and CURRENTSTATE for the given table,
-   * and then check if all ONLINE segments in IDEALSTATE match with CURRENTSTATE.
-   * @param helixManager helix manager for the server instance
-   * @param tableNameWithType table name for which segment state is to be checked
-   * @return true if all segments for the given table are succesfully loaded. False otherwise
+   * Returns all online segments for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   *
+   * @return List of ONLINE segment names.
    */
-  public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+  public static List<String> getOnlineSegmentsForThisInstance(HelixManager helixManager, String tableNameWithType) {

Review Comment:
   closing as comment was addressed



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060801528


##########
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");

Review Comment:
   This has been addressed



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1054687048


##########
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:
   Why do you think having the "_REALTIME" will help here?



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


[GitHub] [pinot] jadami10 commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jadami10 commented on PR #9994:
URL: https://github.com/apache/pinot/pull/9994#issuecomment-1397833536

   I tried deploying this change to one of our clusters with #10101 or #10121. We currently track ingestion lag via our custom stream ingestion plugin. For the most part these match up. But I still see an issue where partitions that have infrequent events continue to report forever increasing lag. From the code comments it seems that shouldn't be the case, but I want to make sure before I dig into our plugin to see if something isn't implemented as expected.


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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +225,59 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();

Review Comment:
   @jadami10 Once consumption stops, we do tick the measures reported by the metric until the consuming thread is shutdown and yes there may be a ramping delay depending on how long it takes to shutdown things. We can likely improve that by marking shutdown in progress and stopping the ramp in time, this may help with heavily loaded servers to stop ramping during long-lasting shutdown. If that makes sense please file an issue and we can try to reach consensus to adding that feature. 



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1053939812


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +217,63 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shutdown the table.
+    if (_consumptionDelayTracker != null) {
+      _consumptionDelayTracker.shutdown();
+    }
+    // Now that segments can't report metric destroy metric for this table
+    _serverMetrics.removeTableGauge(_tableNameWithType, ServerGauge.MAX_PINOT_CONSUMPTION_DELAY_MS);
+  }
+
+  /*
+   * Method to handle CONSUMING->DROPPED transition.
+   *
+   * @param partitionGroupId Partition id that we must stop tracking on this server.
+   */
+  public void stopTrackingPartitionDelay(long 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.
+   */
+  public void markPartitionForVerification(long 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 updatePinotIngestionDelay(long ingestionDelayMillis, long currenTimeMillis, long partitionGroupId) {
+    if (_consumptionDelayTracker != null) {
+      _consumptionDelayTracker.storeConsumptionDelay(ingestionDelayMillis, currenTimeMillis,
+          partitionGroupId);
+    }
+  }
+
+  /**
+   * Returns all partitionGroupIds for the partitions hosted by this server for current table.
+   */
+  public List<Integer> getTablePartitionsHostedByThisServer() {

Review Comment:
   Since `RealtimeTableDataManager` is on a per-server basis, th `ByThisServer` is redundant. Instead, `getHostedPartitionGroupIds()` may be better?



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +217,63 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shutdown the table.
+    if (_consumptionDelayTracker != null) {
+      _consumptionDelayTracker.shutdown();
+    }
+    // Now that segments can't report metric destroy metric for this table
+    _serverMetrics.removeTableGauge(_tableNameWithType, ServerGauge.MAX_PINOT_CONSUMPTION_DELAY_MS);
+  }
+
+  /*
+   * Method to handle CONSUMING->DROPPED transition.
+   *
+   * @param partitionGroupId Partition id that we must stop tracking on this server.
+   */
+  public void stopTrackingPartitionDelay(long partitionGroupId) {

Review Comment:
   partitionGroupId should be int. In all other places as well. Will not repeat



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

Review Comment:
   nit: Let us move the static declarations up before all the non-static members show up, thanks.



##########
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");

Review Comment:
   Please include in the exception a string that says which argument is illegal, and what the current values are.
   
   Also, is it possible that these are invalid? We are not allowing these to be configurable. Or, do you plan to make these configurable?
   
   Either way, this check belongs in RealtimeTableDataManager (or maybe even before if we are configuring on a per-server basis)



##########
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);

Review Comment:
   Please change `pid` to `partitionGroupId`



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1054696072


##########
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);
+    _timer.schedule(new TrackingTimerTask("TimerTask" + tableNameWithType, this),
+        _initialTimeThreadDelayMs, _timerThreadTickIntervalMs);
+    // Install callback metric
+    _serverMetrics.addCallbackTableGaugeIfNeeded(_tableNameWithType, ServerGauge.MAX_PINOT_CONSUMPTION_DELAY_MS,
+        () -> (long) getMaxPinotConsumptionDelay());
+  }
+
+  // Constructor that uses default timeout
+  public ConsumptionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager) {
+    this(serverMetrics, tableNameWithType, tableDataManager, INITIAL_TIMER_THREAD_DELAY_MS,
+        TIMER_THREAD_TICK_INTERVAL_MS);
+  }
+
+  /**
+   * Use to set or rest the aging of reported values.
+   * @param enableAging true if we want maximum to be aged as per sample time or false if we do not want to age
+   *                   samples
+   */
+  @VisibleForTesting
+  void setEnableAging(boolean enableAging) {
+    _enableAging = enableAging;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   * If the new sample represents a new Maximum we update the current maximum.
+   * If the new sample was for the partition that was maximum, but delay is not maximum anymore, we must select
+   * a new maximum.
+   *
+   * @param delayInMilliseconds pinot consumption delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void storeConsumptionDelay(long delayInMilliseconds, long sampleTime, long partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    _partitionToDelaySampleMap.put(partitionGroupId, new DelayMeasure(sampleTime, delayInMilliseconds));
+    // If we are consuming we do not need to track this partition for removal.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+  }
+
+  /*
+   * Handle partition removal event. This must be invoked when we stop serving a given partition for
+   * this table in the current server.
+   * This function will be invoked when we receive CONSUMING -> DROPPED / OFFLINE state transitions.
+   *
+   * @param partitionGroupId partition id that we should stop tracking.
+   */
+  public void stopTrackingPartitionConsumptionDelay(long partitionGroupId) {
+    removePartitionId(partitionGroupId);
+  }
+
+  /*
+   * This method is used for timing out inactive partitions, so we don't display their metrics on current server.
+   * When the inactive time exceeds some threshold, we read from ideal state to confirm we still host the partition,
+   * if not we remove the partition from being tracked locally.
+   * This call is to be invoked by a timer thread that will periodically wake up and invoke this function.
+   */
+  public void timeoutInactivePartitions() {
+    List<Integer> partitionsHostedByThisServer = null;
+    try {
+      partitionsHostedByThisServer = getPartitionsHostedByThisServerPerIdealState();
+    } catch (Exception e) {
+      _logger.error("Failed to get partitions hosted by this server for table {}", _tableNameWithType);
+      return;
+    }
+    HashSet<Integer> hashSetOfHostedPartitions = new HashSet(partitionsHostedByThisServer);

Review Comment:
   done



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1054696833


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +217,63 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shutdown the table.
+    if (_consumptionDelayTracker != null) {
+      _consumptionDelayTracker.shutdown();
+    }
+    // Now that segments can't report metric destroy metric for this table
+    _serverMetrics.removeTableGauge(_tableNameWithType, ServerGauge.MAX_PINOT_CONSUMPTION_DELAY_MS);
+  }
+
+  /*
+   * Method to handle CONSUMING->DROPPED transition.
+   *
+   * @param partitionGroupId Partition id that we must stop tracking on this server.
+   */
+  public void stopTrackingPartitionDelay(long partitionGroupId) {

Review Comment:
   Yes got this in my env, somehow missed my last push



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1054699322


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -38,6 +38,40 @@ public class TableStateUtils {
   private TableStateUtils() {
   }
 
+  /**
+   * Returns all online segments for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   *
+   * @return List of ONLINE segment names.
+   */
+  public static List<String> getTableOnlineSegments(HelixManager helixManager, String tableNameWithType) {

Review Comment:
   done



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1053779647


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -611,6 +616,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

Review Comment:
   Discussed this in our online meet up: there should be a contract with Stream providers not to provide empty batch if there are events. We currently go to sleep when that happen and this will negatively affect performance for those streams that return empty and do have events. We also discussed options to handle this differently for each stream provider using a method in the stream provider that will indicate if zeroing is valid for the specific stream provider.



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


[GitHub] [pinot] jugomezv commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on PR #9994:
URL: https://github.com/apache/pinot/pull/9994#issuecomment-1353829874

   > 
   
   Our customers want to know how long their events are taking from the time they are posted to the stream service. The use of this knowledge is to determine and diagnose problem in their ingestion pipeline.


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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1055974144


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java:
##########
@@ -91,6 +92,16 @@ public void onBecomeOnlineFromConsuming(Message message, NotificationContext con
 
       TableDataManager tableDataManager = _instanceDataManager.getTableDataManager(realtimeTableName);
       Preconditions.checkNotNull(tableDataManager);
+      if (tableDataManager instanceof RealtimeTableDataManager) {

Review Comment:
   I will address as per our live discussion, see next commit for resolution



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060205249


##########
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;

Review Comment:
   I think I cleaned all instances that I got reversed now.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060219667


##########
pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java:
##########
@@ -45,8 +45,10 @@ public enum ServerGauge implements AbstractMetrics.Gauge {
   // Dedup metrics
   DEDUP_PRIMARY_KEYS_COUNT("dedupPrimaryKeysCount", false),
   CONSUMPTION_QUOTA_UTILIZATION("ratio", false),
-  JVM_HEAP_USED_BYTES("bytes", true);
-
+  JVM_HEAP_USED_BYTES("bytes", true),
+  // Lag metrics
+  MAX_CONSUMPTION_DELAY_MS("milliseconds", false),

Review Comment:
   Yes, makes sense to add the TABLE prefix to both and settle on ingestion delay naming everywhere



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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1062020489


##########
pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java:
##########
@@ -45,8 +45,10 @@ public enum ServerGauge implements AbstractMetrics.Gauge {
   // Dedup metrics
   DEDUP_PRIMARY_KEYS_COUNT("dedupPrimaryKeysCount", false),
   CONSUMPTION_QUOTA_UTILIZATION("ratio", false),
-  JVM_HEAP_USED_BYTES("bytes", true);
-
+  JVM_HEAP_USED_BYTES("bytes", true),
+  // Lag metrics
+  TABLE_MAX_INGESTION_DELAY_MS("milliseconds", false),
+  TABLE_PER_PARTITION_INGESTION_DELAY_MS("milliseconds", false);

Review Comment:
   I think we can use the same gauge for both table level and partition level. For partition level, we will suffix it with `.partitionId`.
   ```suggestion
     MAX_INGESTION_DELAY_MS("milliseconds", false);
   ```



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1554,6 +1557,25 @@ private void createPartitionMetadataProvider(String reason) {
     _partitionMetadataProvider = _streamConsumerFactory.createPartitionMetadataProvider(_clientId, _partitionGroupId);
   }
 
+  /*
+   * Updates the ingestion delay if messages were processed using the time stamp for the last consumed event.
+   *
+   * @param indexedMessagesCount
+   */
+  private void updateIngestionDelay(int indexedMessageCount) {
+    if (_catchingUpPhase) {

Review Comment:
   If this is `false` in the beginning, we also need to update the ingestion delay when it becomes `true`, or the lag will already be propagated.



##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/IngestionBasedConsumptionStatusChecker.java:
##########
@@ -77,6 +77,9 @@ public int getNumConsumingSegmentsNotReachedIngestionCriteria() {
         LLRealtimeSegmentDataManager rtSegmentDataManager = (LLRealtimeSegmentDataManager) segmentDataManager;
         if (isSegmentCaughtUp(segName, rtSegmentDataManager)) {
           _caughtUpSegments.add(segName);
+          rtSegmentDataManager.notifyConsumptionCaughtUp(false);

Review Comment:
   We want to skip reporting delay during server startup because:
   1. Server will need to catch up, which will very likely trigger the alert
   2. Server is not really serving the query, so it is actually false alarm
   
   We do want to report delay when a new consuming segment is created because server is serving queries from it.



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java:
##########
@@ -69,4 +69,13 @@ protected static PinotDataBufferMemoryManager getMemoryManager(String consumerDi
 
   public abstract Map<String, PartitionLagState> getPartitionToLagState(
       Map<String, ConsumerPartitionState> consumerPartitionStateMap);
+
+  /**
+   * The RT segment data manager can handle status change from external components like the ConsumptionStatusChecker
+   * etc. Currently, it acts as a way to signal the RT Segment data manager that the current partition has caught up.
+   *
+   * @param caughtUpWithUpstream Boolean indicating if the partition has caught up with upstream source or not based on
+   *                            the strategy used in the {@literal IngestionBasedConsumptionStatusChecker}
+   */
+  public abstract void notifyConsumptionCaughtUp(boolean caughtUpWithUpstream);

Review Comment:
   This is quite confusing. Per the implementation, seems like passing `false` means the segment is caught up?



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1594,4 +1616,9 @@ public String getSegmentName() {
   public void forceCommit() {
     _forceCommitMessageReceived = true;
   }
+
+  @Override
+  public void notifyConsumptionCaughtUp(boolean catchingUpPhase) {

Review Comment:
   Another way to handle it is to assume it is caught up in the beginning, and let the status checker to set it as not caught up yet. The status checker only monitor the initial consuming segments during restart. New consuming segments won't be tracked, thus will be in caught up phase automatically.
   I think this PR is already taking this approach.



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1063836862


##########
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:
   Can you pass this as 0 and avoid this declaration altogether?



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1062843087


##########
pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java:
##########
@@ -45,8 +45,10 @@ public enum ServerGauge implements AbstractMetrics.Gauge {
   // Dedup metrics
   DEDUP_PRIMARY_KEYS_COUNT("dedupPrimaryKeysCount", false),
   CONSUMPTION_QUOTA_UTILIZATION("ratio", false),
-  JVM_HEAP_USED_BYTES("bytes", true);
-
+  JVM_HEAP_USED_BYTES("bytes", true),
+  // Lag metrics
+  TABLE_MAX_INGESTION_DELAY_MS("milliseconds", false),
+  TABLE_PER_PARTITION_INGESTION_DELAY_MS("milliseconds", false);

Review Comment:
   Yes that makes sense, but then the name TABLE_INGESTION_DELAY_MS makes more sense 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


[GitHub] [pinot] jugomezv commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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

   > I know this image doesn't have a lot of context, but it's definitely in milliseconds, and it seems this affects ~7/48 partitions for this topic.
   
   Thanks a lot, the scale in the y-axis is that hours? days? Let me continue to look in to the consume loop to see if there is other places where consumption could be stuck that would lead to such increases.
   
   I have looked at the consume loop code and have the following suggestions:
   
   Can you enable debug logs: there is a wealth of debug/info traces in that can help us tell the difference of consumption patterns between your partitions and what leads to the ramping up times. Currently there are two places where this code blocks: on fetching a message batch from stream (configurable timeout described above) and right after we get an empty batch where we block for 100milliseconds. 
   
   We also have a number of other interesting metrics which you should correlate with the graph above:
   LLC_PARTITION_CONSUMING should indicate if the partition is actively consuming or not
   HIGHEST_STREAM_OFFSET_CONSUMED
   REALTIME_ROWS_CONSUMED
   INVALID_REALTIME_ROWS_DROPPED
   INCOMPLETE_REALTIME_ROWS_CONSUMED
   
   
   Also another question for you: is there any filtering of messages going on? I noticed that if we do get a batch of messages and all are filtered the metric could reflect the lag for the last unfiltered message
   
   
   
    


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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064238443


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,362 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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) {
+      _delayMs = d;
+      _sampleTime = t;
+    }
+    public final long _delayMs;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private final Supplier<Boolean> _isServerReadyToServeQueries;
+
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongst all partitions we are tracking.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (int partitionGroupId : _partitionToDelaySampleMap.keySet()) {
+      DelayMeasure currentMeasure = _partitionToDelaySampleMap.get(partitionGroupId);
+      if ((newMax == null)
+          ||
+          (currentMeasure != null) && (currentMeasure._delayMs > newMax._delayMs)) {
+        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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMs + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.removePartitionGauge(_metricName, partitionGroupId, ServerGauge.TABLE_INGESTION_DELAY_MS);
+    }
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric, Supplier<Boolean> isServerReadyToServeQueries)
+      throws RuntimeException {
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;
+    _realTimeTableDataManager = realtimeTableDataManager;
+    _clock = Clock.systemUTC();
+    _isServerReadyToServeQueries = isServerReadyToServeQueries;
+    // Handle negative timer values
+    if (timerThreadTickIntervalMs <= 0) {
+      throw new RuntimeException(String.format("Illegal timer timeout argument, expected > 0, got=%d for table=%s",
+          timerThreadTickIntervalMs, _tableNameWithType));
+    }
+    _enablePerPartitionMetric = enablePerPartitionMetric;
+    _enableAggregateMetric = enableAggregateMetric;
+    _timerThreadTickIntervalMs = timerThreadTickIntervalMs;
+    _timer = new Timer("IngestionDelayTimerThread" + TableNameBuilder.extractRawTableName(tableNameWithType));
+    _timer.schedule(new TimerTask() {
+        @Override
+        public void run() {
+          timeoutInactivePartitions();
+        }
+      }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+    // Install callback metric
+    if (_enableAggregateMetric) {
+      _serverMetrics.addCallbackTableGaugeIfNeeded(_metricName, ServerGauge.TABLE_INGESTION_DELAY_MS,
+          () -> getMaximumIngestionDelay());
+    }
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager, Supplier<Boolean> isServerReadyToServeQueries) {
+    this(serverMetrics, tableNameWithType, tableDataManager, TIMER_THREAD_TICK_INTERVAL_MS,
+        "", true, true, isServerReadyToServeQueries);
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager, String metricNamePrefix,
+      Supplier<Boolean> isServerReadyToServeQueries) {
+    this(serverMetrics, metricNamePrefix + tableNameWithType, tableDataManager,
+        TIMER_THREAD_TICK_INTERVAL_MS, metricNamePrefix, true, true, isServerReadyToServeQueries);
+  }
+
+  /**
+   * Function that enable use to set predictable clocks for testing purposes.
+   *
+   * @param clock clock to be used by the class
+   */
+  @VisibleForTesting
+  void setClock(Clock clock) {
+    _clock = clock;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   *
+   * @param delayMs ingestion delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void updateIngestionDelay(long delayMs, long sampleTime, int partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    if (!_isServerReadyToServeQueries.get()) {

Review Comment:
   For completeness and future-proofing, it is better to put this check on all public methods I think.
   
   Another alternative is to put this check on the place where we create the add the callback gauge. That way, all the logic works the same way all the time, except metrics are never emitted until the server is ready to serve queries.
   
   Bonus is that there are only two places in which you create the callback gauge, so the check needs to be only there.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065219405


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -39,37 +39,56 @@ private TableStateUtils() {
   }
 
   /**
-   * Checks if all segments for the given @param tableNameWithType are succesfully loaded
-   * This function will get all segments in IDEALSTATE and CURRENTSTATE for the given table,
-   * and then check if all ONLINE segments in IDEALSTATE match with CURRENTSTATE.
-   * @param helixManager helix manager for the server instance
-   * @param tableNameWithType table name for which segment state is to be checked
-   * @return true if all segments for the given table are succesfully loaded. False otherwise
+   * Returns all segments in a given state for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   * @param state state of the segments to be returned
+   *
+   * @return List of segment names in a given state.
    */
-  public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+  public static Set<String> getSegmentsInGivenStateForThisInstance(HelixManager helixManager, String tableNameWithType,

Review Comment:
   Was list before and changed to a set as we needed it in the other call of this function but rolled back the change, see 3ce7b783c5dc88073106f400f00783ba4ab71281



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


[GitHub] [pinot] navina commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1058711265


##########
pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java:
##########
@@ -45,8 +45,10 @@ public enum ServerGauge implements AbstractMetrics.Gauge {
   // Dedup metrics
   DEDUP_PRIMARY_KEYS_COUNT("dedupPrimaryKeysCount", false),
   CONSUMPTION_QUOTA_UTILIZATION("ratio", false),
-  JVM_HEAP_USED_BYTES("bytes", true);
-
+  JVM_HEAP_USED_BYTES("bytes", true),
+  // Lag metrics
+  MAX_CONSUMPTION_DELAY_MS("milliseconds", false),

Review Comment:
   rename this as `TABLE_MAX_CONSUMPTION_DELAY_MS` or `TABLE_MAX_INGESTION_DELAY_MS` ?



##########
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());

Review Comment:
   nit: cast unnecessary



##########
pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/ConsumptionDelayTrackerTest.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.metrics.PinotMetricUtils;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class ConsumptionDelayTrackerTest {
+
+  private static final int TIMER_THREAD_TICK_INTERVAL_MS = 100;
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  private void sleepMs(long timeMs) {

Review Comment:
   +1 to avoiding sleeps in test. 
   You can extend `java.time.Clock` to define your own ticking clock or use one of the existing clocks like `FixedClock`, `OffsetClock` etc. There are also other implementations like `org.threeten.extra.MutableClock` which is likely what you need. Manually set the clock to a time to advance the clock. 



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


[GitHub] [pinot] jugomezv commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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

   > 
   
   
   
   > I tried deploying this change to one of our clusters with #10101 or #10121. We currently track ingestion lag via our custom stream ingestion plugin. For the most part these match up. But I still see an issue where partitions that have infrequent events continue to report forever increasing lag. From the code comments it seems that shouldn't be the case, but I want to make sure before I dig into our plugin to see if something isn't implemented as expected.
   
   Can you check also what is the fetchTimeout config for your stream? Default seems to be 5 seconds so for an inactive system I would expect a sawtooth graph that peaks at fetch timeout and returns to zero periodically in intervals of fetch timeout, if fetch timeout has been set to larger value then we would expect that larger value to be the peak of the sawtooth graph


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


[GitHub] [pinot] jadami10 commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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

   here's what it looks like. the active partitions report a consistent 200-300ms, but the inactive ones just go up consistently then come down arbitrarily
   
   <img width="1528" alt="image" src="https://user-images.githubusercontent.com/4760722/214177226-28fdbad8-c522-4509-bfb8-a8b44fed5028.png">
   
   I know this image doesn't have a lot of context, but it's definitely in milliseconds, and it seems this affects ~7/48 partitions for this topic.


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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065190160


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -39,37 +39,56 @@ private TableStateUtils() {
   }
 
   /**
-   * Checks if all segments for the given @param tableNameWithType are succesfully loaded
-   * This function will get all segments in IDEALSTATE and CURRENTSTATE for the given table,
-   * and then check if all ONLINE segments in IDEALSTATE match with CURRENTSTATE.
-   * @param helixManager helix manager for the server instance
-   * @param tableNameWithType table name for which segment state is to be checked
-   * @return true if all segments for the given table are succesfully loaded. False otherwise
+   * Returns all segments in a given state for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   * @param state state of the segments to be returned
+   *
+   * @return List of segment names in a given state.
    */
-  public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+  public static Set<String> getSegmentsInGivenStateForThisInstance(HelixManager helixManager, String tableNameWithType,
+      String state) {
     HelixDataAccessor dataAccessor = helixManager.getHelixDataAccessor();
     PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
     IdealState idealState = dataAccessor.getProperty(keyBuilder.idealStates(tableNameWithType));
+    Set<String> segmentsInGivenState = new HashSet<>();
     if (idealState == null) {
       LOGGER.warn("Failed to find ideal state for table: {}", tableNameWithType);
-      return false;
+      return segmentsInGivenState;
     }
 
     // Get all ONLINE segments from idealState

Review Comment:
   done



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065219049


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -39,37 +39,56 @@ private TableStateUtils() {
   }
 
   /**
-   * Checks if all segments for the given @param tableNameWithType are succesfully loaded
-   * This function will get all segments in IDEALSTATE and CURRENTSTATE for the given table,
-   * and then check if all ONLINE segments in IDEALSTATE match with CURRENTSTATE.
-   * @param helixManager helix manager for the server instance
-   * @param tableNameWithType table name for which segment state is to be checked
-   * @return true if all segments for the given table are succesfully loaded. False otherwise
+   * Returns all segments in a given state for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   * @param state state of the segments to be returned
+   *
+   * @return List of segment names in a given state.
    */
-  public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+  public static Set<String> getSegmentsInGivenStateForThisInstance(HelixManager helixManager, String tableNameWithType,
+      String state) {
     HelixDataAccessor dataAccessor = helixManager.getHelixDataAccessor();
     PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
     IdealState idealState = dataAccessor.getProperty(keyBuilder.idealStates(tableNameWithType));
+    Set<String> segmentsInGivenState = new HashSet<>();
     if (idealState == null) {
       LOGGER.warn("Failed to find ideal state for table: {}", tableNameWithType);
-      return false;
+      return segmentsInGivenState;
     }
 
     // Get all ONLINE segments from idealState

Review Comment:
   see commit 3ce7b783c5dc88073106f400f00783ba4ab71281



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


[GitHub] [pinot] mcvsubbu commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on PR #9994:
URL: https://github.com/apache/pinot/pull/9994#issuecomment-1376607259

   Pending comments to be addressed in a subsequent PR.


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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060219457


##########
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());

Review Comment:
   Good catch, I think I am removing these.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060800619


##########
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:
   There is another comment to resolve this, let's not add multiple ones for the same issue.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060810220


##########
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);

Review Comment:
   Good case to use anonymous since it is only used here and very simple override.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060944878


##########
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:
   We need to pass a initial delay for the call.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060941920


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -613,7 +613,7 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi
         _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);
+      setIngestionDelayToZero();

Review Comment:
   We resolved this offline, we will keep things as they were.



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061906746


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1594,4 +1616,9 @@ public String getSegmentName() {
   public void forceCommit() {
     _forceCommitMessageReceived = true;
   }
+
+  @Override
+  public void notifyConsumptionCaughtUp(boolean catchingUpPhase) {

Review Comment:
   The ingestion-based status checkers have a specific purpose -- to answer the question of whether the ingestion has caught up "reasonably" before we can start to field queries after server startup. Once the server is started up, we should be fielding queries no matter whether ingestion is caught up or not. 
   
   The way to handle this situation is to do one of the following:
   - Ignore the startup part -- at least for now. Yes, we may show a lag until it catches up, but it is useful metric to observe as well.  I _strongly_ suggest going this approach in the beginning.
   - My less preferred approach is to plumb the code so as to keep the ingestion delay trackers disabled until the server has started up completely.  I don't feel comfortable handling it as a part of this PR, since we may introduce unintentional circular dependency.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064281475


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -39,25 +39,25 @@ private TableStateUtils() {
   }
 
   /**
-   * Checks if all segments for the given @param tableNameWithType are succesfully loaded
-   * This function will get all segments in IDEALSTATE and CURRENTSTATE for the given table,
-   * and then check if all ONLINE segments in IDEALSTATE match with CURRENTSTATE.
-   * @param helixManager helix manager for the server instance
-   * @param tableNameWithType table name for which segment state is to be checked
-   * @return true if all segments for the given table are succesfully loaded. False otherwise
+   * Returns all online segments for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   *
+   * @return List of ONLINE segment names.
    */
-  public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+  public static List<String> getOnlineSegmentsForThisInstance(HelixManager helixManager, String tableNameWithType) {

Review Comment:
   Addressed by commit e476c1efae63d35835576fbf0a75cf3b6c8ef970, @mcvsubbu please resolve the comment at your earliest



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1050223101


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -585,6 +585,11 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi
             realtimeRowsConsumedMeter =
                 _serverMetrics.addMeteredTableValue(_metricKeyName, ServerMeter.REALTIME_ROWS_CONSUMED, 1,
                     realtimeRowsConsumedMeter);
+            long currentTime = System.currentTimeMillis();
+            long pinotIngestionDelayMs = currentTime - msgMetadata.getRecordIngestionTimeMs();
+            pinotIngestionDelayMs = pinotIngestionDelayMs >= 0 ? pinotIngestionDelayMs : 0;

Review Comment:
   I think there are two possible cases here: something changes the system time manually, and the second is the timestamp introduced by the stream service is using a clock which is drifted from the pinot server time.



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


[GitHub] [pinot] navina commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1050098428


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -585,6 +585,11 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi
             realtimeRowsConsumedMeter =
                 _serverMetrics.addMeteredTableValue(_metricKeyName, ServerMeter.REALTIME_ROWS_CONSUMED, 1,
                     realtimeRowsConsumedMeter);
+            long currentTime = System.currentTimeMillis();
+            long pinotIngestionDelayMs = currentTime - msgMetadata.getRecordIngestionTimeMs();
+            pinotIngestionDelayMs = pinotIngestionDelayMs >= 0 ? pinotIngestionDelayMs : 0;

Review Comment:
   when would `System.currentTimeMillis() - msgMetadata.getRecordIngestionTimeMs()` ever be < 0? 



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -611,6 +616,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

Review Comment:
   How would we know if we are up-to-date without querying the source ? A consumer returning an empty batch doesn't necessarily mean we are caught up to the source. Can you explain why we make this assumption here?



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1054692543


##########
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);
+    _timer.schedule(new TrackingTimerTask("TimerTask" + tableNameWithType, this),
+        _initialTimeThreadDelayMs, _timerThreadTickIntervalMs);
+    // Install callback metric
+    _serverMetrics.addCallbackTableGaugeIfNeeded(_tableNameWithType, ServerGauge.MAX_PINOT_CONSUMPTION_DELAY_MS,
+        () -> (long) getMaxPinotConsumptionDelay());
+  }
+
+  // Constructor that uses default timeout
+  public ConsumptionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager) {
+    this(serverMetrics, tableNameWithType, tableDataManager, INITIAL_TIMER_THREAD_DELAY_MS,
+        TIMER_THREAD_TICK_INTERVAL_MS);
+  }
+
+  /**
+   * Use to set or rest the aging of reported values.
+   * @param enableAging true if we want maximum to be aged as per sample time or false if we do not want to age
+   *                   samples
+   */
+  @VisibleForTesting
+  void setEnableAging(boolean enableAging) {
+    _enableAging = enableAging;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   * If the new sample represents a new Maximum we update the current maximum.
+   * If the new sample was for the partition that was maximum, but delay is not maximum anymore, we must select
+   * a new maximum.
+   *
+   * @param delayInMilliseconds pinot consumption delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void storeConsumptionDelay(long delayInMilliseconds, long sampleTime, long partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    _partitionToDelaySampleMap.put(partitionGroupId, new DelayMeasure(sampleTime, delayInMilliseconds));
+    // If we are consuming we do not need to track this partition for removal.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+  }
+
+  /*
+   * Handle partition removal event. This must be invoked when we stop serving a given partition for
+   * this table in the current server.
+   * This function will be invoked when we receive CONSUMING -> DROPPED / OFFLINE state transitions.
+   *
+   * @param partitionGroupId partition id that we should stop tracking.
+   */
+  public void stopTrackingPartitionConsumptionDelay(long partitionGroupId) {

Review Comment:
   Seems like I missed this in my push yesterday but it is in my dev env.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1054674461


##########
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:
   I think these names are more telling and will help in debugging, so leaving as is. Same for the one below.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060963331


##########
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);

Review Comment:
   Done



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061991555


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 realtime ingestion 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. And individual metrics are associated with
+ *   each partition being tracked.
+ * 6-Delays reported for partitions that do not have events to consume are 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 a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then 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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // 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.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+
+  private boolean _enableAging;
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final 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.TABLE_PER_PARTITION_INGESTION_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 IngestionDelayTracker(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("IngestionDelayTimerThread" + tableNameWithType);
+    _timer.schedule(new TimerTask() {
+        @Override
+        public void run() {
+          timeoutInactivePartitions();
+        }
+      }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+    // Install callback metric
+    if (_enableAggregateMetric) {
+      _serverMetrics.addCallbackTableGaugeIfNeeded(_tableNameWithType, ServerGauge.TABLE_MAX_INGESTION_DELAY_MS,
+          () -> getMaximumIngestionDelay());
+    }
+  }
+
+  // Constructor that uses defaults
+  public IngestionDelayTracker(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 IngestionDelayTracker(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 reset the aging of reported values.
+   * @param enableAging true if we want maximum to be aged as per sample time or false if we do not want to age
+   *                   samples
+   */
+  @VisibleForTesting
+  void setEnableAging(boolean enableAging) {
+    _enableAging = enableAging;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   * If the new sample represents a new Maximum we update the current maximum.
+   * If the new sample was for the partition that was maximum, but delay is not maximum anymore, we must select
+   * a new maximum.
+   *
+   * @param delayInMilliseconds ingestion delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void updateIngestionDelay(long delayInMilliseconds, long sampleTime, int partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    DelayMeasure previousMeasure = _partitionToDelaySampleMap.put(partitionGroupId,
+        new DelayMeasure(sampleTime, delayInMilliseconds));
+    if ((previousMeasure == null) && _enablePerPartitionMetric) {
+      // First time we start tracking a partition we should start tracking it via metric
+      _serverMetrics.addCallbackTableGaugeIfNeeded(getPerPartitionMetricName(partitionGroupId),
+          ServerGauge.TABLE_PER_PARTITION_INGESTION_DELAY_MS,
+          () -> getPartitionIngestionDelay(partitionGroupId));
+    }
+    // If we are consuming we do not need to track this partition for removal.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+  }
+
+  /*
+   * Handle partition removal event. This must be invoked when we stop serving a given partition for
+   * this table in the current server.
+   * This function will be invoked when we receive CONSUMING -> DROPPED / OFFLINE state transitions.
+   *
+   * @param partitionGroupId partition id that we should stop tracking.
+   */
+  public void stopTrackingPartitionIngestionDelay(int partitionGroupId) {
+    removePartitionId(partitionGroupId);
+  }
+
+  /*
+   * This method is used for timing out inactive partitions, so we don't display their metrics on current server.
+   * When the inactive time exceeds some threshold, we read from ideal state to confirm we still host the partition,
+   * if not we remove the partition from being tracked locally.
+   * This call is to be invoked by a timer thread that will periodically wake up and invoke this function.
+   */
+  public void timeoutInactivePartitions() {
+    List<Integer> partitionsHostedByThisServer = null;
+    try {
+      partitionsHostedByThisServer = getPartitionsHostedByThisServerPerIdealState();

Review Comment:
   Done in the las commit



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1062886088


##########
pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java:
##########
@@ -45,8 +45,10 @@ public enum ServerGauge implements AbstractMetrics.Gauge {
   // Dedup metrics
   DEDUP_PRIMARY_KEYS_COUNT("dedupPrimaryKeysCount", false),
   CONSUMPTION_QUOTA_UTILIZATION("ratio", false),
-  JVM_HEAP_USED_BYTES("bytes", true);
-
+  JVM_HEAP_USED_BYTES("bytes", true),
+  // Lag metrics
+  TABLE_MAX_INGESTION_DELAY_MS("milliseconds", false),
+  TABLE_PER_PARTITION_INGESTION_DELAY_MS("milliseconds", false);

Review Comment:
   Addressed in commit 3d1fc795ff83ae648b407c301dd5f4e8c1b4b471



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1063090872


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java:
##########
@@ -69,4 +69,13 @@ protected static PinotDataBufferMemoryManager getMemoryManager(String consumerDi
 
   public abstract Map<String, PartitionLagState> getPartitionToLagState(
       Map<String, ConsumerPartitionState> consumerPartitionStateMap);
+
+  /**
+   * The RT segment data manager can handle status change from external components like the ConsumptionStatusChecker
+   * etc. Currently, it acts as a way to signal the RT Segment data manager that the current partition has caught up.
+   *
+   * @param caughtUpWithUpstream Boolean indicating if the partition has caught up with upstream source or not based on
+   *                            the strategy used in the {@literal IngestionBasedConsumptionStatusChecker}
+   */
+  public abstract void notifyConsumptionCaughtUp(boolean caughtUpWithUpstream);

Review Comment:
   Code now changed to use server ready signal



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065218428


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.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.time.Clock;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ * TODO: handle bug situations like the one where a partition is not allocated to a given server due to a bug.
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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) {
+      _delayMs = d;
+      _sampleTime = t;
+    }
+    public final long _delayMs;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private final Supplier<Boolean> _isServerReadyToServeQueries;
+
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongst all partitions we are tracking.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (int partitionGroupId : _partitionToDelaySampleMap.keySet()) {
+      DelayMeasure currentMeasure = _partitionToDelaySampleMap.get(partitionGroupId);
+      if ((newMax == null)
+          ||
+          (currentMeasure != null) && (currentMeasure._delayMs > newMax._delayMs)) {
+        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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMs + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.removePartitionGauge(_metricName, partitionGroupId, ServerGauge.TABLE_INGESTION_DELAY_MS);
+    }
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric, Supplier<Boolean> isServerReadyToServeQueries)
+      throws RuntimeException {
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;
+    _realTimeTableDataManager = realtimeTableDataManager;
+    _clock = Clock.systemUTC();
+    _isServerReadyToServeQueries = isServerReadyToServeQueries;
+    // Handle negative timer values
+    if (timerThreadTickIntervalMs <= 0) {
+      throw new RuntimeException(String.format("Illegal timer timeout argument, expected > 0, got=%d for table=%s",
+          timerThreadTickIntervalMs, _tableNameWithType));
+    }
+    _enablePerPartitionMetric = enablePerPartitionMetric;
+    _enableAggregateMetric = enableAggregateMetric;
+    _timerThreadTickIntervalMs = timerThreadTickIntervalMs;
+    _timer = new Timer("IngestionDelayTimerThread" + TableNameBuilder.extractRawTableName(tableNameWithType));
+    _timer.schedule(new TimerTask() {
+        @Override
+        public void run() {
+          timeoutInactivePartitions();
+        }
+      }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+    // Install callback metric
+    if (_enableAggregateMetric) {
+      _serverMetrics.addCallbackTableGaugeIfNeeded(_metricName, ServerGauge.TABLE_INGESTION_DELAY_MS,
+          () -> getMaximumIngestionDelay());
+    }
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager, Supplier<Boolean> isServerReadyToServeQueries) {
+    this(serverMetrics, tableNameWithType, tableDataManager, TIMER_THREAD_TICK_INTERVAL_MS,
+        "", true, true, isServerReadyToServeQueries);
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager, String metricNamePrefix,
+      Supplier<Boolean> isServerReadyToServeQueries) {
+    this(serverMetrics, metricNamePrefix + tableNameWithType, tableDataManager,
+        TIMER_THREAD_TICK_INTERVAL_MS, metricNamePrefix, true, true, isServerReadyToServeQueries);
+  }
+
+  /**
+   * Function that enable use to set predictable clocks for testing purposes.
+   *
+   * @param clock clock to be used by the class
+   */
+  @VisibleForTesting
+  void setClock(Clock clock) {
+    _clock = clock;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   *
+   * @param delayMs ingestion delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void updateIngestionDelay(long delayMs, long sampleTime, int partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    // TODO: see if we can install gauges after the server is ready.
+    if (!_isServerReadyToServeQueries.get()) {
+      // Do not update the ingestion delay metrics during server startup period
+      return;
+    }
+    DelayMeasure previousMeasure = _partitionToDelaySampleMap.put(partitionGroupId,
+        new DelayMeasure(sampleTime, delayMs));
+    if ((previousMeasure == null) && _enablePerPartitionMetric) {
+      // First time we start tracking a partition we should start tracking it via metric
+      _serverMetrics.addCallbackPartitionGaugeIfNeeded(_metricName, partitionGroupId,
+          ServerGauge.TABLE_INGESTION_DELAY_MS, () -> getPartitionIngestionDelay(partitionGroupId));
+    }
+    // If we are consuming we do not need to track this partition for removal.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+  }
+
+  /*
+   * Handle partition removal event. This must be invoked when we stop serving a given partition for
+   * this table in the current server.
+   *
+   * @param partitionGroupId partition id that we should stop tracking.
+   */
+  public void stopTrackingPartitionIngestionDelay(int partitionGroupId) {
+    removePartitionId(partitionGroupId);
+  }
+
+  /*
+   * This method is used for timing out inactive partitions, so we don't display their metrics on current server.
+   * When the inactive time exceeds some threshold, we read from ideal state to confirm we still host the partition,
+   * if not we remove the partition from being tracked locally.
+   * This call is to be invoked by a timer thread that will periodically wake up and invoke this function.
+   */
+  public void timeoutInactivePartitions() {
+    Set<Integer> partitionsHostedByThisServer = null;
+    // Check if we have any partition to verify, else don't make the call to check ideal state as that
+    // involves network traffic and may be inefficient.
+    ArrayList<Integer> partitionsToVerify = getPartitionsToBeVerified();
+    if (partitionsToVerify.size() == 0) {
+      // Don't make the call to getHostedPartitionsGroupIds() as it involves checking ideal state.
+      return;
+    }
+    try {
+      partitionsHostedByThisServer = _realTimeTableDataManager.getHostedPartitionsGroupIds();
+    } catch (Exception e) {
+      _logger.error("Failed to get partitions hosted by this server, table={}", _tableNameWithType);
+      return;
+    }
+    for (int partitionGroupId : partitionsToVerify) {
+      if (!partitionsHostedByThisServer.contains(partitionGroupId)) {
+        // Partition is not hosted in this server anymore, stop tracking it
+        removePartitionId(partitionGroupId);
+      }
+    }
+  }
+
+  /*
+   * This function is invoked when a partition goes from CONSUMING to ONLINE, so we can assert whether the
+   * partition is still hosted by this server after some interval of time.
+   *
+   * @param partitionGroupId Partition id that we need confirmed via ideal state as still hosted by this server.
+   */
+  public void markPartitionForVerification(int partitionGroupId) {
+    _partitionsMarkedForVerification.put(partitionGroupId, _clock.millis());
+  }
+
+  /*
+   * This is the function to be invoked when reading the metric.
+   * It reports the maximum ingestion delay for all partitions of this table being served
+   * by current server; it adds the time elapsed since the sample was taken to the measure.
+   * If no measures have been taken, then the reported value is zero.
+   *
+   * @return max ingestion delay in milliseconds.
+   */
+  public long getMaximumIngestionDelay() {
+    DelayMeasure currentMaxDelay = getMaximumDelay();

Review Comment:
   removed aggregate metric, see commit 3ce7b783c5dc88073106f400f00783ba4ab71281



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065218789


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.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.time.Clock;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ * TODO: handle bug situations like the one where a partition is not allocated to a given server due to a bug.
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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) {
+      _delayMs = d;
+      _sampleTime = t;
+    }
+    public final long _delayMs;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private final Supplier<Boolean> _isServerReadyToServeQueries;
+
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongst all partitions we are tracking.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (int partitionGroupId : _partitionToDelaySampleMap.keySet()) {

Review Comment:
   done across the class: see commit 3ce7b783c5dc88073106f400f00783ba4ab71281



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064179022


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -39,25 +39,25 @@ private TableStateUtils() {
   }
 
   /**
-   * Checks if all segments for the given @param tableNameWithType are succesfully loaded
-   * This function will get all segments in IDEALSTATE and CURRENTSTATE for the given table,
-   * and then check if all ONLINE segments in IDEALSTATE match with CURRENTSTATE.
-   * @param helixManager helix manager for the server instance
-   * @param tableNameWithType table name for which segment state is to be checked
-   * @return true if all segments for the given table are succesfully loaded. False otherwise
+   * Returns all online segments for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   *
+   * @return List of ONLINE segment names.
    */
-  public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+  public static List<String> getOnlineSegmentsForThisInstance(HelixManager helixManager, String tableNameWithType) {

Review Comment:
   Because the code that used this earlier needed a list.



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


[GitHub] [pinot] jugomezv commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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

   > if it is not already there, can you add comments in the interface that the timestamps should be in UTC? Thanks.
   
   Good point @mcvsubbu also in user facing doc, I have to add a text for the new metric so will edit this one too!


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


[GitHub] [pinot] jugomezv commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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

   > > 
   > 
   > Hi @jadami10 for a partition that has infrequent events we should be displaying lags that stay at zero or near zero, the expectation is that the stream will return message count zero and we will set the delay zero. I guess first question is what is the stream involved? This should return zero when we are not observing events:messagesAndOffsets.getMessageCount();
   
   @jadami10 
   I will look deeper into the code and see if consumption can get stuck without reporting empty batch for long, if so I may need to add a timer refresh there. Thanks for bringing this up to my attention!


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


[GitHub] [pinot] jadami10 commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +225,59 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();

Review Comment:
   filed https://github.com/apache/pinot/issues/11533. I'll try to get our team to chase this down



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1054688550


##########
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:
   you never know when we may intro a new timer for the other part 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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1054675484


##########
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:
   I think this is better for debug purposes, leaving as 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


[GitHub] [pinot] codecov-commenter commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #9994:
URL: https://github.com/apache/pinot/pull/9994#issuecomment-1353904178

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/9994?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#9994](https://codecov.io/gh/apache/pinot/pull/9994?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (327585c) into [master](https://codecov.io/gh/apache/pinot/commit/7eba70c0ed8159ecf2bd6b152a2d0374916cc068?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (7eba70c) will **decrease** coverage by `45.34%`.
   > The diff coverage is `46.96%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #9994       +/-   ##
   =============================================
   - Coverage     70.42%   25.08%   -45.35%     
   + Complexity     5662       44     -5618     
   =============================================
     Files          1991     1980       -11     
     Lines        107251   107012      -239     
     Branches      16302    16277       -25     
   =============================================
   - Hits          75533    26841    -48692     
   - Misses        26444    77457    +51013     
   + Partials       5274     2714     -2560     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `25.08% <46.96%> (-0.09%)` | :arrow_down: |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `?` | |
   
   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=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/9994?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...egment/local/utils/tablestate/TableStateUtils.java](https://codecov.io/gh/apache/pinot/pull/9994/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy90YWJsZXN0YXRlL1RhYmxlU3RhdGVVdGlscy5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...r/helix/SegmentOnlineOfflineStateModelFactory.java](https://codecov.io/gh/apache/pinot/pull/9994/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VydmVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zZXJ2ZXIvc3RhcnRlci9oZWxpeC9TZWdtZW50T25saW5lT2ZmbGluZVN0YXRlTW9kZWxGYWN0b3J5LmphdmE=) | `51.72% <20.00%> (-6.70%)` | :arrow_down: |
   | [.../realtime/MaximumPinotConsumptionDelayTracker.java](https://codecov.io/gh/apache/pinot/pull/9994/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvTWF4aW11bVBpbm90Q29uc3VtcHRpb25EZWxheVRyYWNrZXIuamF2YQ==) | `57.35% <57.35%> (ø)` | |
   | [...ata/manager/realtime/RealtimeTableDataManager.java](https://codecov.io/gh/apache/pinot/pull/9994/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVUYWJsZURhdGFNYW5hZ2VyLmphdmE=) | `41.69% <71.42%> (-27.89%)` | :arrow_down: |
   | [...manager/realtime/LLRealtimeSegmentDataManager.java](https://codecov.io/gh/apache/pinot/pull/9994/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvTExSZWFsdGltZVNlZ21lbnREYXRhTWFuYWdlci5qYXZh) | `57.49% <80.00%> (-13.08%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/metrics/ServerGauge.java](https://codecov.io/gh/apache/pinot/pull/9994/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9TZXJ2ZXJHYXVnZS5qYXZh) | `96.42% <100.00%> (+0.13%)` | :arrow_up: |
   | [...ain/java/org/apache/pinot/spi/utils/LoopUtils.java](https://codecov.io/gh/apache/pinot/pull/9994/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvTG9vcFV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...in/java/org/apache/pinot/spi/utils/BytesUtils.java](https://codecov.io/gh/apache/pinot/pull/9994/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQnl0ZXNVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/spi/trace/BaseRecording.java](https://codecov.io/gh/apache/pinot/pull/9994/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdHJhY2UvQmFzZVJlY29yZGluZy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...java/org/apache/pinot/spi/trace/NoOpRecording.java](https://codecov.io/gh/apache/pinot/pull/9994/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdHJhY2UvTm9PcFJlY29yZGluZy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1452 more](https://codecov.io/gh/apache/pinot/pull/9994/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1055973205


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -47,6 +81,10 @@ private TableStateUtils() {
    * @return true if all segments for the given table are succesfully loaded. False otherwise
    */
   public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+    HelixDataAccessor dataAccessor = helixManager.getHelixDataAccessor();
+    PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
+    String instanceName = helixManager.getInstanceName();
+    /*

Review Comment:
   Removed



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065199612


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.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.time.Clock;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ * TODO: handle bug situations like the one where a partition is not allocated to a given server due to a bug.
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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) {
+      _delayMs = d;
+      _sampleTime = t;
+    }
+    public final long _delayMs;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private final Supplier<Boolean> _isServerReadyToServeQueries;
+
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongst all partitions we are tracking.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (int partitionGroupId : _partitionToDelaySampleMap.keySet()) {
+      DelayMeasure currentMeasure = _partitionToDelaySampleMap.get(partitionGroupId);
+      if ((newMax == null)
+          ||
+          (currentMeasure != null) && (currentMeasure._delayMs > newMax._delayMs)) {
+        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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMs + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.removePartitionGauge(_metricName, partitionGroupId, ServerGauge.TABLE_INGESTION_DELAY_MS);
+    }
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric, Supplier<Boolean> isServerReadyToServeQueries)
+      throws RuntimeException {
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;
+    _realTimeTableDataManager = realtimeTableDataManager;
+    _clock = Clock.systemUTC();
+    _isServerReadyToServeQueries = isServerReadyToServeQueries;
+    // Handle negative timer values
+    if (timerThreadTickIntervalMs <= 0) {
+      throw new RuntimeException(String.format("Illegal timer timeout argument, expected > 0, got=%d for table=%s",
+          timerThreadTickIntervalMs, _tableNameWithType));
+    }
+    _enablePerPartitionMetric = enablePerPartitionMetric;
+    _enableAggregateMetric = enableAggregateMetric;
+    _timerThreadTickIntervalMs = timerThreadTickIntervalMs;
+    _timer = new Timer("IngestionDelayTimerThread" + TableNameBuilder.extractRawTableName(tableNameWithType));
+    _timer.schedule(new TimerTask() {
+        @Override
+        public void run() {
+          timeoutInactivePartitions();
+        }
+      }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+    // Install callback metric
+    if (_enableAggregateMetric) {
+      _serverMetrics.addCallbackTableGaugeIfNeeded(_metricName, ServerGauge.TABLE_INGESTION_DELAY_MS,
+          () -> getMaximumIngestionDelay());
+    }
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager, Supplier<Boolean> isServerReadyToServeQueries) {
+    this(serverMetrics, tableNameWithType, tableDataManager, TIMER_THREAD_TICK_INTERVAL_MS,
+        "", true, true, isServerReadyToServeQueries);
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager, String metricNamePrefix,
+      Supplier<Boolean> isServerReadyToServeQueries) {
+    this(serverMetrics, metricNamePrefix + tableNameWithType, tableDataManager,
+        TIMER_THREAD_TICK_INTERVAL_MS, metricNamePrefix, true, true, isServerReadyToServeQueries);
+  }
+
+  /**
+   * Function that enable use to set predictable clocks for testing purposes.
+   *
+   * @param clock clock to be used by the class
+   */
+  @VisibleForTesting
+  void setClock(Clock clock) {
+    _clock = clock;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   *
+   * @param delayMs ingestion delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void updateIngestionDelay(long delayMs, long sampleTime, int partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    // TODO: see if we can install gauges after the server is ready.
+    if (!_isServerReadyToServeQueries.get()) {
+      // Do not update the ingestion delay metrics during server startup period
+      return;
+    }
+    DelayMeasure previousMeasure = _partitionToDelaySampleMap.put(partitionGroupId,
+        new DelayMeasure(sampleTime, delayMs));
+    if ((previousMeasure == null) && _enablePerPartitionMetric) {
+      // First time we start tracking a partition we should start tracking it via metric
+      _serverMetrics.addCallbackPartitionGaugeIfNeeded(_metricName, partitionGroupId,
+          ServerGauge.TABLE_INGESTION_DELAY_MS, () -> getPartitionIngestionDelay(partitionGroupId));
+    }
+    // If we are consuming we do not need to track this partition for removal.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+  }
+
+  /*
+   * Handle partition removal event. This must be invoked when we stop serving a given partition for
+   * this table in the current server.
+   *
+   * @param partitionGroupId partition id that we should stop tracking.
+   */
+  public void stopTrackingPartitionIngestionDelay(int partitionGroupId) {
+    removePartitionId(partitionGroupId);
+  }
+
+  /*
+   * This method is used for timing out inactive partitions, so we don't display their metrics on current server.
+   * When the inactive time exceeds some threshold, we read from ideal state to confirm we still host the partition,
+   * if not we remove the partition from being tracked locally.
+   * This call is to be invoked by a timer thread that will periodically wake up and invoke this function.
+   */
+  public void timeoutInactivePartitions() {
+    Set<Integer> partitionsHostedByThisServer = null;
+    // Check if we have any partition to verify, else don't make the call to check ideal state as that
+    // involves network traffic and may be inefficient.
+    ArrayList<Integer> partitionsToVerify = getPartitionsToBeVerified();
+    if (partitionsToVerify.size() == 0) {
+      // Don't make the call to getHostedPartitionsGroupIds() as it involves checking ideal state.
+      return;
+    }
+    try {
+      partitionsHostedByThisServer = _realTimeTableDataManager.getHostedPartitionsGroupIds();
+    } catch (Exception e) {
+      _logger.error("Failed to get partitions hosted by this server, table={}", _tableNameWithType);
+      return;
+    }
+    for (int partitionGroupId : partitionsToVerify) {
+      if (!partitionsHostedByThisServer.contains(partitionGroupId)) {
+        // Partition is not hosted in this server anymore, stop tracking it
+        removePartitionId(partitionGroupId);
+      }
+    }
+  }
+
+  /*
+   * This function is invoked when a partition goes from CONSUMING to ONLINE, so we can assert whether the
+   * partition is still hosted by this server after some interval of time.
+   *
+   * @param partitionGroupId Partition id that we need confirmed via ideal state as still hosted by this server.
+   */
+  public void markPartitionForVerification(int partitionGroupId) {
+    _partitionsMarkedForVerification.put(partitionGroupId, _clock.millis());
+  }
+
+  /*
+   * This is the function to be invoked when reading the metric.
+   * It reports the maximum ingestion delay for all partitions of this table being served
+   * by current server; it adds the time elapsed since the sample was taken to the measure.
+   * If no measures have been taken, then the reported value is zero.
+   *
+   * @return max ingestion delay in milliseconds.
+   */
+  public long getMaximumIngestionDelay() {
+    DelayMeasure currentMaxDelay = getMaximumDelay();

Review Comment:
   I see your point, as suggested I will remove this to start with



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060225704


##########
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:
   That method has a different semantic for the argument: it is not the time but number of events processed in the batch, we use that to decided whether we record or not delay. But since this may be a bit hard to read as one liner I have created the method setIngestionDelayToZero() to make code more readable.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060833470


##########
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:
   I think you are referring to [this](https://github.com/apache/pinot/issues/10049), I am not addressing this here as this change is already large enough and it is not good idea to mix with other issues. The TOD has been placed in the code and the issue opened. I will address in a different PR. If this was not what you are referring please elaborate and reopen the comment.



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060827634


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -613,7 +613,7 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi
         _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);
+      setIngestionDelayToZero();

Review Comment:
   Other than adding an extra method call, is there any change here?
   
   Instead, if we call `updateIngestionDelay(0, currentTime, partiongroupId`,  can we not handle all the ingstion delay specific logic inside that method?
   



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061039352


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +216,83 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();
+  }
+
+  /*
+   * Method to handle CONSUMING->DROPPED transition.
+   *
+   * @param partitionGroupId Partition id that we must stop tracking on this server.
+   */
+  private void stopTrackingPartitionDelay(int partitionGroupId) {
+    _ingestionDelayTracker.stopTrackingPartitionIngestionDelay(partitionGroupId);
+  }
+
+  /*
+   * Method to handle CONSUMING->ONLINE transition.
+   * If no new ingestion 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) {
+    _ingestionDelayTracker.markPartitionForConfirmation(partitionGroupId);
+  }
+
+  /*
+   * Method used by LLRealtimeSegmentManagers to update their partition delays
+   *
+   * @param ingestionDelayMs Ingestion delay being reported.
+   * @param currentTimeMs 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 ingestionDelayMs, long currenTimeMs, int partitionGroupId) {
+    _ingestionDelayTracker.updateIngestionDelay(ingestionDelayMs, currenTimeMs, partitionGroupId);
+  }
+
+  /*
+   * Method to handle supported transitions of segments states for this table.
+   * Supported transitions include:
+   *
+   * CONSUMING -> ONLINE:
+   * We mark partitions for verification against 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.
+   * CONSUMING -> DROPPED:
+   * We stop tracking partitions whose segments are dropped.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   * @param fromState state from which the segment is transitioning.
+   * @param toState state to which the segment is transitioning to.
+   */
+  @Override
+  public void onSegmentStateTransition(String segmentNameStr, SegmentState fromState, SegmentState toState) {

Review Comment:
   Let us have individual methods for each state transition. Makes things cleaner, easier to read, with no if statements, thanks 



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 realtime ingestion 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. And individual metrics are associated with
+ *   each partition being tracked.
+ * 6-Delays reported for partitions that do not have events to consume are 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 a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then 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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // 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.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+
+  private boolean _enableAging;
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final 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.TABLE_PER_PARTITION_INGESTION_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 IngestionDelayTracker(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("IngestionDelayTimerThread" + tableNameWithType);
+    _timer.schedule(new TimerTask() {
+        @Override
+        public void run() {
+          timeoutInactivePartitions();
+        }
+      }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+    // Install callback metric
+    if (_enableAggregateMetric) {
+      _serverMetrics.addCallbackTableGaugeIfNeeded(_tableNameWithType, ServerGauge.TABLE_MAX_INGESTION_DELAY_MS,
+          () -> getMaximumIngestionDelay());
+    }
+  }
+
+  // Constructor that uses defaults
+  public IngestionDelayTracker(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 IngestionDelayTracker(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 reset the aging of reported values.
+   * @param enableAging true if we want maximum to be aged as per sample time or false if we do not want to age
+   *                   samples
+   */
+  @VisibleForTesting
+  void setEnableAging(boolean enableAging) {
+    _enableAging = enableAging;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   * If the new sample represents a new Maximum we update the current maximum.
+   * If the new sample was for the partition that was maximum, but delay is not maximum anymore, we must select
+   * a new maximum.
+   *
+   * @param delayInMilliseconds ingestion delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void updateIngestionDelay(long delayInMilliseconds, long sampleTime, int partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    DelayMeasure previousMeasure = _partitionToDelaySampleMap.put(partitionGroupId,
+        new DelayMeasure(sampleTime, delayInMilliseconds));
+    if ((previousMeasure == null) && _enablePerPartitionMetric) {
+      // First time we start tracking a partition we should start tracking it via metric
+      _serverMetrics.addCallbackTableGaugeIfNeeded(getPerPartitionMetricName(partitionGroupId),
+          ServerGauge.TABLE_PER_PARTITION_INGESTION_DELAY_MS,
+          () -> getPartitionIngestionDelay(partitionGroupId));
+    }
+    // If we are consuming we do not need to track this partition for removal.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+  }
+
+  /*
+   * Handle partition removal event. This must be invoked when we stop serving a given partition for
+   * this table in the current server.
+   * This function will be invoked when we receive CONSUMING -> DROPPED / OFFLINE state transitions.
+   *
+   * @param partitionGroupId partition id that we should stop tracking.
+   */
+  public void stopTrackingPartitionIngestionDelay(int partitionGroupId) {
+    removePartitionId(partitionGroupId);
+  }
+
+  /*
+   * This method is used for timing out inactive partitions, so we don't display their metrics on current server.
+   * When the inactive time exceeds some threshold, we read from ideal state to confirm we still host the partition,
+   * if not we remove the partition from being tracked locally.
+   * This call is to be invoked by a timer thread that will periodically wake up and invoke this function.
+   */
+  public void timeoutInactivePartitions() {
+    List<Integer> partitionsHostedByThisServer = null;
+    try {
+      partitionsHostedByThisServer = getPartitionsHostedByThisServerPerIdealState();

Review Comment:
   Important: Do not make this call if there are no partitions marked for verification (mostly the case)



##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/IngestionBasedConsumptionStatusChecker.java:
##########
@@ -77,6 +77,9 @@ public int getNumConsumingSegmentsNotReachedIngestionCriteria() {
         LLRealtimeSegmentDataManager rtSegmentDataManager = (LLRealtimeSegmentDataManager) segmentDataManager;
         if (isSegmentCaughtUp(segName, rtSegmentDataManager)) {
           _caughtUpSegments.add(segName);
+          rtSegmentDataManager.notifyConsumptionCaughtUp(false);

Review Comment:
   I believe there is a problem here. 
   
   The catching up that happens in this phase is only when a server starts up, not for every consuming segment that is created when a server is already up and running.
   
   My suggestion is to NOT wait for the server to catch up in order to report delays.
   
   if we _do_ want to wait, then this wait should be at a table level. 
   



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1594,4 +1616,9 @@ public String getSegmentName() {
   public void forceCommit() {
     _forceCommitMessageReceived = true;
   }
+
+  @Override
+  public void notifyConsumptionCaughtUp(boolean catchingUpPhase) {

Review Comment:
   This call will come in only for the first few consuming segments created during server startup. After that, we will never receive this call.
   
   For example, if a server hosts two partitions, and we start with segment p1_6 and p2_8, then we will get the calls for these two segments after the server has completed its startup.
   
   Now, when p1_6 and p_8 go online, and we create new p1_7 and p2_9 segments, we will never get this call, and the catchup phase will be on forever.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java:
##########
@@ -44,6 +44,12 @@
 @ThreadSafe
 public interface TableDataManager {
 
+  enum SegmentState {

Review Comment:
   These are already defined.
   Also, we don't need the definitions if we had individual methods for state transitions



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060839870


##########
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:
   Done



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1063872683


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMilliseconds + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.TABLE_INGESTION_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: _metricName + partitionGroupId
+   */
+  private String getPerPartitionMetricName(int partitionGroupId) {
+    return _metricName + partitionGroupId;
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  // Custom Constructor
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric)
+      throws RuntimeException {
+    _logger = LoggerFactory.getLogger(getClass().getSimpleName());
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;
+    _realTimeTableDataManager = realtimeTableDataManager;
+    _clock = Clock.systemDefaultZone();

Review Comment:
   We should use `Clock.systemUTC()`



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  private long getAgedDelay(DelayMeasure currentDelay) {

Review Comment:
   Suggestion:
   You can remove this method, in favor of a method inside the DelayMeasure class that computes the agedDelay. You can do the null check outside this method in the one case where it may happen that it is null.



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;

Review Comment:
   Can be a static final. Not a member variable.



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -115,15 +117,24 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
 
   private TableDedupMetadataManager _tableDedupMetadataManager;
   private TableUpsertMetadataManager _tableUpsertMetadataManager;
+  // Object to track ingestion delay for all partitions
+  private IngestionDelayTracker _ingestionDelayTracker;
+  private Supplier<Boolean> _isReadyToServeQueries;
 
   public RealtimeTableDataManager(Semaphore segmentBuildSemaphore) {
+    this(segmentBuildSemaphore, () -> true);
+  }
+
+  public RealtimeTableDataManager(Semaphore segmentBuildSemaphore, Supplier<Boolean> isReadyToServeQueries) {
     _segmentBuildSemaphore = segmentBuildSemaphore;
+    _isReadyToServeQueries = isReadyToServeQueries;

Review Comment:
   I suggest that the `isReadyToServeQueries` either be handled in the table data manager (by blocking all calls to ingestion delay tracker), or in the ingestion delay tracker itself.  Let us not involve the segment data manager with another state that it really does not need to know about



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -39,25 +39,25 @@ private TableStateUtils() {
   }
 
   /**
-   * Checks if all segments for the given @param tableNameWithType are succesfully loaded
-   * This function will get all segments in IDEALSTATE and CURRENTSTATE for the given table,
-   * and then check if all ONLINE segments in IDEALSTATE match with CURRENTSTATE.
-   * @param helixManager helix manager for the server instance
-   * @param tableNameWithType table name for which segment state is to be checked
-   * @return true if all segments for the given table are succesfully loaded. False otherwise
+   * Returns all online segments for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   *
+   * @return List of ONLINE segment names.
    */
-  public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+  public static List<String> getOnlineSegmentsForThisInstance(HelixManager helixManager, String tableNameWithType) {

Review Comment:
   Why not return this as a set?



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -611,6 +613,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:
   Looks like this call is made even if the server is not ready to serve queries



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +223,58 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();
+  }
+
+  /*
+   * Method used by LLRealtimeSegmentManagers to update their partition delays
+   *
+   * @param ingestionDelayMs Ingestion delay being reported.
+   * @param currentTimeMs 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 ingestionDelayMs, long currenTimeMs, int partitionGroupId) {
+    _ingestionDelayTracker.updateIngestionDelay(ingestionDelayMs, currenTimeMs, partitionGroupId);
+  }
+
+  /*
+   * Method to handle CONSUMING -> DROPPED segment state transitions:
+   * We stop tracking partitions whose segments are dropped.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   */
+  @Override
+  public void onConsumingToDropped(String segmentNameStr) {
+    LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr);
+    _ingestionDelayTracker.stopTrackingPartitionIngestionDelay(segmentName.getPartitionGroupId());
+  }
+
+  /*
+   * Method to handle CONSUMING -> ONLINE segment state transitions:
+   * We mark partitions for verification against 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);
+    _ingestionDelayTracker.markPartitionForVerification(segmentName.getPartitionGroupId());
+  }
+
+  /**
+   * Returns all partitionGroupIds for the partitions hosted by this server for current table.
+   * @Note: this involves Zookeeper read and should not be used frequently due to efficiency concerns.
+   */
+  public List<Integer> getHostedPartitionsGroupIds() {
+    ArrayList<Integer> partitionsHostedByThisServer = new ArrayList<>();
+    List<String> segments = TableStateUtils.getOnlineSegmentsForThisInstance(_helixManager, _tableNameWithType);

Review Comment:
   Major: You want to get the consuming segments for this instance, not online segments from the idealtstate.



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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)) {

Review Comment:
   I thought there was a comment to conform to `Ms` everywhere instead of "millis" or "millseconds", etc.?



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


[GitHub] [pinot] navina commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061765187


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -611,6 +616,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

Review Comment:
   I think the interface should be in the consumer as that is the one which is polling the source and knows the state of the upstream partition. I would like to point out that only the consumer (or subscriber) will know its own state that relative to its upstream. 
   
   In general, I urge the team to think about solutions that can be cleanly extended/adapted to all other streaming systems. 



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


[GitHub] [pinot] navina commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061765187


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -611,6 +616,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

Review Comment:
   I think the interface should be in the consumer as that is the one which is polling the source and knows the state of the upstream partition. I would like to point out that only the consumer (or subscriber) will know its own state that is relative to its upstream. 
   
   In general, I urge the team to think about solutions that can be cleanly extended/adapted to all other streaming systems. 



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


[GitHub] [pinot] navina commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061817534


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1594,4 +1616,9 @@ public String getSegmentName() {
   public void forceCommit() {
     _forceCommitMessageReceived = true;
   }
+
+  @Override
+  public void notifyConsumptionCaughtUp(boolean catchingUpPhase) {

Review Comment:
   This might be the case because the ingestion based status checkers are not aware of the new segments that were not around when the server started. 
   Wouldn't that indicate an issue with the ingestion based status checkers itself, then?  Shouldn't it also detect new segments if the segment rolls over during catch-up? 



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064194398


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java:
##########
@@ -183,4 +184,7 @@ void addOrReplaceSegment(String tableNameWithType, String segmentName)
    * Immediately stop consumption and start committing the consuming segments.
    */
   void forceCommit(String tableNameWithType, Set<String> segmentNames);
+
+  void setSupplierOfReadyToServeQueries(Supplier<Boolean> isReadyToServeQueries);

Review Comment:
   done in commit ecfcf1b1cdf81f4fe3547e1eec4ac23a1df72c5c



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +225,59 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();

Review Comment:
   OK what would be the proposal here: where do you suggest we do the shutdown?



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1050223757


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -611,6 +616,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

Review Comment:
   At least at this point, we know we have no other events then we sleep. 



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1053912680


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java:
##########
@@ -91,6 +92,16 @@ public void onBecomeOnlineFromConsuming(Message message, NotificationContext con
 
       TableDataManager tableDataManager = _instanceDataManager.getTableDataManager(realtimeTableName);
       Preconditions.checkNotNull(tableDataManager);
+      if (tableDataManager instanceof RealtimeTableDataManager) {

Review Comment:
   Please add the desired method in the `BaseTableDataManager` class with a default implementation that does nothing, and override it on the `RealtimeTableDataManager`.



##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java:
##########
@@ -138,6 +149,19 @@ 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();
+      String segmentNameStr = message.getPartitionName();
+      TableDataManager tableDataManager = _instanceDataManager.getTableDataManager(realtimeTableName);
+      Preconditions.checkNotNull(tableDataManager);
+      if (tableDataManager instanceof RealtimeTableDataManager) {

Review Comment:
   Same here, let us just add a method in the base class



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -38,6 +38,40 @@ public class TableStateUtils {
   private TableStateUtils() {
   }
 
+  /**
+   * Returns all online segments for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   *
+   * @return List of ONLINE segment names.
+   */
+  public static List<String> getTableOnlineSegments(HelixManager helixManager, String tableNameWithType) {

Review Comment:
   Suggest rename to `getOnlineSegmentsForThisInstance()` 



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -611,6 +616,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

Review Comment:
   @Jackie-Jiang to propose appropriate interface. I suggest an interface in the stream metadata like (just as an example) `emptyReturnIndicatesNoEventsAvailable()`. I think what Jackie had in mind was to change the poll API to indicate whether there are more events. That could be a return in the metadata, for example. I am open to either approach.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -47,6 +81,10 @@ private TableStateUtils() {
    * @return true if all segments for the given table are succesfully loaded. False otherwise
    */
   public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+    HelixDataAccessor dataAccessor = helixManager.getHelixDataAccessor();
+    PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
+    String instanceName = helixManager.getInstanceName();
+    /*

Review Comment:
   I suppose you are going to remove the commented code, right? It is here just for review. Please add a TODO or some clear indication that these lines will be removed before you merge



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1053957742


##########
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);
+   }

Review Comment:
   nit : extra space



##########
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);
+    _timer.schedule(new TrackingTimerTask("TimerTask" + tableNameWithType, this),
+        _initialTimeThreadDelayMs, _timerThreadTickIntervalMs);
+    // Install callback metric
+    _serverMetrics.addCallbackTableGaugeIfNeeded(_tableNameWithType, ServerGauge.MAX_PINOT_CONSUMPTION_DELAY_MS,
+        () -> (long) getMaxPinotConsumptionDelay());
+  }
+
+  // Constructor that uses default timeout
+  public ConsumptionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager) {
+    this(serverMetrics, tableNameWithType, tableDataManager, INITIAL_TIMER_THREAD_DELAY_MS,
+        TIMER_THREAD_TICK_INTERVAL_MS);
+  }
+
+  /**
+   * Use to set or rest the aging of reported values.
+   * @param enableAging true if we want maximum to be aged as per sample time or false if we do not want to age
+   *                   samples
+   */
+  @VisibleForTesting
+  void setEnableAging(boolean enableAging) {
+    _enableAging = enableAging;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   * If the new sample represents a new Maximum we update the current maximum.
+   * If the new sample was for the partition that was maximum, but delay is not maximum anymore, we must select
+   * a new maximum.
+   *
+   * @param delayInMilliseconds pinot consumption delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void storeConsumptionDelay(long delayInMilliseconds, long sampleTime, long partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    _partitionToDelaySampleMap.put(partitionGroupId, new DelayMeasure(sampleTime, delayInMilliseconds));
+    // If we are consuming we do not need to track this partition for removal.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+  }
+
+  /*
+   * Handle partition removal event. This must be invoked when we stop serving a given partition for
+   * this table in the current server.
+   * This function will be invoked when we receive CONSUMING -> DROPPED / OFFLINE state transitions.
+   *
+   * @param partitionGroupId partition id that we should stop tracking.
+   */
+  public void stopTrackingPartitionConsumptionDelay(long partitionGroupId) {
+    removePartitionId(partitionGroupId);
+  }
+
+  /*
+   * This method is used for timing out inactive partitions, so we don't display their metrics on current server.
+   * When the inactive time exceeds some threshold, we read from ideal state to confirm we still host the partition,
+   * if not we remove the partition from being tracked locally.
+   * This call is to be invoked by a timer thread that will periodically wake up and invoke this function.
+   */
+  public void timeoutInactivePartitions() {
+    List<Integer> partitionsHostedByThisServer = null;
+    try {
+      partitionsHostedByThisServer = getPartitionsHostedByThisServerPerIdealState();
+    } catch (Exception e) {
+      _logger.error("Failed to get partitions hosted by this server for table {}", _tableNameWithType);

Review Comment:
   Dont we already set the logger to contain 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());

Review Comment:
   raw table name should suffice. 



##########
pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/ConsumptionDelayTrackerTest.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.metrics.PinotMetricUtils;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class ConsumptionDelayTrackerTest {
+
+  private static final int TIMER_THREAD_TICK_INTERVAL_MS = 100;
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  private void sleepMs(long timeMs) {

Review Comment:
   Please avoid sleeps in test. Use ways to modify clock time as needed. I think there are a couple of different examples in the code base for this.



##########
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);
+    _timer.schedule(new TrackingTimerTask("TimerTask" + tableNameWithType, this),
+        _initialTimeThreadDelayMs, _timerThreadTickIntervalMs);
+    // Install callback metric
+    _serverMetrics.addCallbackTableGaugeIfNeeded(_tableNameWithType, ServerGauge.MAX_PINOT_CONSUMPTION_DELAY_MS,
+        () -> (long) getMaxPinotConsumptionDelay());
+  }
+
+  // Constructor that uses default timeout
+  public ConsumptionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager) {
+    this(serverMetrics, tableNameWithType, tableDataManager, INITIAL_TIMER_THREAD_DELAY_MS,
+        TIMER_THREAD_TICK_INTERVAL_MS);
+  }
+
+  /**
+   * Use to set or rest the aging of reported values.
+   * @param enableAging true if we want maximum to be aged as per sample time or false if we do not want to age
+   *                   samples
+   */
+  @VisibleForTesting
+  void setEnableAging(boolean enableAging) {
+    _enableAging = enableAging;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   * If the new sample represents a new Maximum we update the current maximum.
+   * If the new sample was for the partition that was maximum, but delay is not maximum anymore, we must select
+   * a new maximum.
+   *
+   * @param delayInMilliseconds pinot consumption delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void storeConsumptionDelay(long delayInMilliseconds, long sampleTime, long partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    _partitionToDelaySampleMap.put(partitionGroupId, new DelayMeasure(sampleTime, delayInMilliseconds));
+    // If we are consuming we do not need to track this partition for removal.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+  }
+
+  /*
+   * Handle partition removal event. This must be invoked when we stop serving a given partition for
+   * this table in the current server.
+   * This function will be invoked when we receive CONSUMING -> DROPPED / OFFLINE state transitions.
+   *
+   * @param partitionGroupId partition id that we should stop tracking.
+   */
+  public void stopTrackingPartitionConsumptionDelay(long partitionGroupId) {
+    removePartitionId(partitionGroupId);
+  }
+
+  /*
+   * This method is used for timing out inactive partitions, so we don't display their metrics on current server.
+   * When the inactive time exceeds some threshold, we read from ideal state to confirm we still host the partition,
+   * if not we remove the partition from being tracked locally.
+   * This call is to be invoked by a timer thread that will periodically wake up and invoke this function.
+   */
+  public void timeoutInactivePartitions() {
+    List<Integer> partitionsHostedByThisServer = null;
+    try {
+      partitionsHostedByThisServer = getPartitionsHostedByThisServerPerIdealState();
+    } catch (Exception e) {
+      _logger.error("Failed to get partitions hosted by this server for table {}", _tableNameWithType);
+      return;
+    }
+    HashSet<Integer> hashSetOfHostedPartitions = new HashSet(partitionsHostedByThisServer);

Review Comment:
   suggest rename `hostedPartitions`



##########
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:
   raw table name should be good enough in this line as well as next



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1054698326


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +217,63 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shutdown the table.
+    if (_consumptionDelayTracker != null) {
+      _consumptionDelayTracker.shutdown();
+    }
+    // Now that segments can't report metric destroy metric for this table
+    _serverMetrics.removeTableGauge(_tableNameWithType, ServerGauge.MAX_PINOT_CONSUMPTION_DELAY_MS);
+  }
+
+  /*
+   * Method to handle CONSUMING->DROPPED transition.
+   *
+   * @param partitionGroupId Partition id that we must stop tracking on this server.
+   */
+  public void stopTrackingPartitionDelay(long 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.
+   */
+  public void markPartitionForVerification(long 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 updatePinotIngestionDelay(long ingestionDelayMillis, long currenTimeMillis, long partitionGroupId) {
+    if (_consumptionDelayTracker != null) {
+      _consumptionDelayTracker.storeConsumptionDelay(ingestionDelayMillis, currenTimeMillis,
+          partitionGroupId);
+    }
+  }
+
+  /**
+   * Returns all partitionGroupIds for the partitions hosted by this server for current table.
+   */
+  public List<Integer> getTablePartitionsHostedByThisServer() {

Review Comment:
   makes sense



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1062844784


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java:
##########
@@ -69,4 +69,13 @@ protected static PinotDataBufferMemoryManager getMemoryManager(String consumerDi
 
   public abstract Map<String, PartitionLagState> getPartitionToLagState(
       Map<String, ConsumerPartitionState> consumerPartitionStateMap);
+
+  /**
+   * The RT segment data manager can handle status change from external components like the ConsumptionStatusChecker
+   * etc. Currently, it acts as a way to signal the RT Segment data manager that the current partition has caught up.
+   *
+   * @param caughtUpWithUpstream Boolean indicating if the partition has caught up with upstream source or not based on
+   *                            the strategy used in the {@literal IngestionBasedConsumptionStatusChecker}
+   */
+  public abstract void notifyConsumptionCaughtUp(boolean caughtUpWithUpstream);

Review Comment:
   Agreed--we likely can change things as you suggest above but there is some other things Subbu brings up in some of the other comments which likely makes it useful for us to have a different approach for muting the metrics in this period, let's discuss in zoom meeting this afternoon.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064194514


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMilliseconds + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.TABLE_INGESTION_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: _metricName + partitionGroupId
+   */
+  private String getPerPartitionMetricName(int partitionGroupId) {
+    return _metricName + partitionGroupId;
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  // Custom Constructor
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric)
+      throws RuntimeException {
+    _logger = LoggerFactory.getLogger(getClass().getSimpleName());
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;
+    _realTimeTableDataManager = realtimeTableDataManager;
+    _clock = Clock.systemDefaultZone();
+    // Handle negative timer values
+    if (timerThreadTickIntervalMs <= 0) {
+      throw new RuntimeException(String.format("Illegal timer timeout argument, expected > 0, got=%d for table=%s",
+          timerThreadTickIntervalMs, _tableNameWithType));
+    }
+    _enablePerPartitionMetric = enablePerPartitionMetric;
+    _enableAggregateMetric = enableAggregateMetric;
+    _timerThreadTickIntervalMs = timerThreadTickIntervalMs;
+    _timer = new Timer("IngestionDelayTimerThread" + TableNameBuilder.extractRawTableName(tableNameWithType));
+    _timer.schedule(new TimerTask() {
+        @Override
+        public void run() {
+          timeoutInactivePartitions();
+        }
+      }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+    // Install callback metric
+    if (_enableAggregateMetric) {
+      _serverMetrics.addCallbackTableGaugeIfNeeded(_metricName, ServerGauge.TABLE_INGESTION_DELAY_MS,
+          () -> getMaximumIngestionDelay());
+    }
+  }
+
+  // Constructor that uses defaults
+  public IngestionDelayTracker(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 IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType, String metricNamePrefix,
+      RealtimeTableDataManager tableDataManager) {
+    this(serverMetrics, metricNamePrefix + tableNameWithType, tableDataManager,
+        TIMER_THREAD_TICK_INTERVAL_MS, metricNamePrefix, true, true);
+  }
+
+  /**
+   * Function that enable use to set predictable clocks for testing purposes.
+   *
+   * @param clock clock to be used by the class
+   */
+  @VisibleForTesting
+  void setClock(Clock clock) {
+    _clock = clock;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   *
+   * @param delayInMilliseconds ingestion delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void updateIngestionDelay(long delayInMilliseconds, long sampleTime, int partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    DelayMeasure previousMeasure = _partitionToDelaySampleMap.put(partitionGroupId,
+        new DelayMeasure(sampleTime, delayInMilliseconds));
+    if ((previousMeasure == null) && _enablePerPartitionMetric) {
+      // First time we start tracking a partition we should start tracking it via metric
+      _serverMetrics.addCallbackTableGaugeIfNeeded(getPerPartitionMetricName(partitionGroupId),

Review Comment:
   Good suggestion, addressed in ecfcf1b1cdf81f4fe3547e1eec4ac23a1df72c5c



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1063091637


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/IngestionBasedConsumptionStatusChecker.java:
##########
@@ -77,6 +77,9 @@ public int getNumConsumingSegmentsNotReachedIngestionCriteria() {
         LLRealtimeSegmentDataManager rtSegmentDataManager = (LLRealtimeSegmentDataManager) segmentDataManager;
         if (isSegmentCaughtUp(segName, rtSegmentDataManager)) {
           _caughtUpSegments.add(segName);
+          rtSegmentDataManager.notifyConsumptionCaughtUp(false);

Review Comment:
   Code now changed to use server ready signal



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1063091381


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1594,4 +1616,9 @@ public String getSegmentName() {
   public void forceCommit() {
     _forceCommitMessageReceived = true;
   }
+
+  @Override
+  public void notifyConsumptionCaughtUp(boolean catchingUpPhase) {

Review Comment:
   Code changed to use server ready signal now



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064194217


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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)) {

Review Comment:
   Thanks for pointing this out, there is no clear convention through the full code but it is good to keep consistency through one file at least, addressed in commit ecfcf1b1cdf81f4fe3547e1eec4ac23a1df72c5c



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060227882


##########
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:
   Added a note to warn users about this



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


[GitHub] [pinot] jugomezv commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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

   > 
   
   Hi @jadami10 for a partition that has infrequent events we should be displaying lags that stay at zero or near zero, the expectation is that the stream will return message count zero and we will set the delay zero. I guess first question is what is the stream involved? This should return zero when we are not observing events:messagesAndOffsets.getMessageCount();


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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065218949


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.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.time.Clock;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ * TODO: handle bug situations like the one where a partition is not allocated to a given server due to a bug.
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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) {
+      _delayMs = d;
+      _sampleTime = t;
+    }
+    public final long _delayMs;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private final Supplier<Boolean> _isServerReadyToServeQueries;
+
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongst all partitions we are tracking.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (int partitionGroupId : _partitionToDelaySampleMap.keySet()) {
+      DelayMeasure currentMeasure = _partitionToDelaySampleMap.get(partitionGroupId);
+      if ((newMax == null)
+          ||
+          (currentMeasure != null) && (currentMeasure._delayMs > newMax._delayMs)) {
+        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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMs + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.removePartitionGauge(_metricName, partitionGroupId, ServerGauge.TABLE_INGESTION_DELAY_MS);
+    }
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,

Review Comment:
   done see commit 3ce7b783c5dc88073106f400f00783ba4ab71281



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065197699


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.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.time.Clock;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ * TODO: handle bug situations like the one where a partition is not allocated to a given server due to a bug.
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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) {
+      _delayMs = d;
+      _sampleTime = t;
+    }
+    public final long _delayMs;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private final Supplier<Boolean> _isServerReadyToServeQueries;
+
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongst all partitions we are tracking.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (int partitionGroupId : _partitionToDelaySampleMap.keySet()) {

Review Comment:
   done



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065219504


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java:
##########
@@ -201,4 +201,18 @@ void addOrReplaceSegment(String segmentName, IndexLoadingConfig indexLoadingConf
    * @return List of {@link SegmentErrorInfo}
    */
   Map<String, SegmentErrorInfo> getSegmentErrors();
+
+  /**
+   * Interface to handle segment state transitions from CONSUMING to DROPPED
+   *
+   * @param segmentNameStr name of segment for which the state change is being handled
+   */
+  default void onConsumingToDropped(String segmentNameStr) { };

Review Comment:
   see commit 3ce7b783c5dc88073106f400f00783ba4ab71281



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061968962


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java:
##########
@@ -44,6 +44,12 @@
 @ThreadSafe
 public interface TableDataManager {
 
+  enum SegmentState {

Review Comment:
   This has been reverted as described in related comment



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1055750820


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java:
##########
@@ -91,6 +92,16 @@ public void onBecomeOnlineFromConsuming(Message message, NotificationContext con
 
       TableDataManager tableDataManager = _instanceDataManager.getTableDataManager(realtimeTableName);
       Preconditions.checkNotNull(tableDataManager);
+      if (tableDataManager instanceof RealtimeTableDataManager) {

Review Comment:
   You can add a method in TableDataManager interface with a default implementation.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060205182


##########
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) {

Review Comment:
   Renamed this but note we may need to open an issue to clean many instances where Millis is used instead, i.e. check LLRealtimeSegmentDataManager.java



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


[GitHub] [pinot] navina commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061812908


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/IngestionBasedConsumptionStatusChecker.java:
##########
@@ -77,6 +77,9 @@ public int getNumConsumingSegmentsNotReachedIngestionCriteria() {
         LLRealtimeSegmentDataManager rtSegmentDataManager = (LLRealtimeSegmentDataManager) segmentDataManager;
         if (isSegmentCaughtUp(segName, rtSegmentDataManager)) {
           _caughtUpSegments.add(segName);
+          rtSegmentDataManager.notifyConsumptionCaughtUp(false);

Review Comment:
   > only when a server starts up
   
   This was the scenario we wanted to handle because often there is lot of catch-up to do after restarting servers. Not for every consuming segment on an already running server. 
   
   >if we do want to wait, then this wait should be at a table level.
   
   What is your rationale behind this? 
   



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061886497


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +216,83 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();
+  }
+
+  /*
+   * Method to handle CONSUMING->DROPPED transition.
+   *
+   * @param partitionGroupId Partition id that we must stop tracking on this server.
+   */
+  private void stopTrackingPartitionDelay(int partitionGroupId) {
+    _ingestionDelayTracker.stopTrackingPartitionIngestionDelay(partitionGroupId);
+  }
+
+  /*
+   * Method to handle CONSUMING->ONLINE transition.
+   * If no new ingestion 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) {
+    _ingestionDelayTracker.markPartitionForConfirmation(partitionGroupId);
+  }
+
+  /*
+   * Method used by LLRealtimeSegmentManagers to update their partition delays
+   *
+   * @param ingestionDelayMs Ingestion delay being reported.
+   * @param currentTimeMs 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 ingestionDelayMs, long currenTimeMs, int partitionGroupId) {
+    _ingestionDelayTracker.updateIngestionDelay(ingestionDelayMs, currenTimeMs, partitionGroupId);
+  }
+
+  /*
+   * Method to handle supported transitions of segments states for this table.
+   * Supported transitions include:
+   *
+   * CONSUMING -> ONLINE:
+   * We mark partitions for verification against 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.
+   * CONSUMING -> DROPPED:
+   * We stop tracking partitions whose segments are dropped.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   * @param fromState state from which the segment is transitioning.
+   * @param toState state to which the segment is transitioning to.
+   */
+  @Override
+  public void onSegmentStateTransition(String segmentNameStr, SegmentState fromState, SegmentState toState) {

Review Comment:
   After implementing both approaches I think we are likely better off with individual methods, reason being are the following:
   1.-There is precedent on handling transitions in individual methods. See SegmentOnlineOfflineStateModeFactory.
   2.-At this point we are adding only two interfaces, if we have to add more we can revisit then.
   3.-Keeping a single method for transition makes the code simple, no throw, checks on states etc.
   
   So I will roll this change back, @navina hopefully you are ok with this.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061974248


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,350 @@
+/**
+ * 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 realtime ingestion 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. And individual metrics are associated with
+ *   each partition being tracked.
+ * 6-Delays reported for partitions that do not have events to consume are 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 a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then 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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // 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.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+
+  private boolean _enableAging;
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final 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.TABLE_PER_PARTITION_INGESTION_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 IngestionDelayTracker(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("IngestionDelayTimerThread" + tableNameWithType);
+    _timer.schedule(new TimerTask() {
+        @Override
+        public void run() {
+          timeoutInactivePartitions();
+        }
+      }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+    // Install callback metric
+    if (_enableAggregateMetric) {
+      _serverMetrics.addCallbackTableGaugeIfNeeded(_tableNameWithType, ServerGauge.TABLE_MAX_INGESTION_DELAY_MS,
+          () -> getMaximumIngestionDelay());
+    }
+  }
+
+  // Constructor that uses defaults
+  public IngestionDelayTracker(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 IngestionDelayTracker(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 reset the aging of reported values.
+   * @param enableAging true if we want maximum to be aged as per sample time or false if we do not want to age
+   *                   samples
+   */
+  @VisibleForTesting
+  void setEnableAging(boolean enableAging) {
+    _enableAging = enableAging;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   * If the new sample represents a new Maximum we update the current maximum.
+   * If the new sample was for the partition that was maximum, but delay is not maximum anymore, we must select
+   * a new maximum.
+   *
+   * @param delayInMilliseconds ingestion delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void updateIngestionDelay(long delayInMilliseconds, long sampleTime, int partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    DelayMeasure previousMeasure = _partitionToDelaySampleMap.put(partitionGroupId,
+        new DelayMeasure(sampleTime, delayInMilliseconds));
+    if ((previousMeasure == null) && _enablePerPartitionMetric) {
+      // First time we start tracking a partition we should start tracking it via metric
+      _serverMetrics.addCallbackTableGaugeIfNeeded(getPerPartitionMetricName(partitionGroupId),
+          ServerGauge.TABLE_PER_PARTITION_INGESTION_DELAY_MS,
+          () -> getPartitionIngestionDelay(partitionGroupId));
+    }
+    // If we are consuming we do not need to track this partition for removal.
+    _partitionsMarkedForVerification.remove(partitionGroupId);
+  }
+
+  /*
+   * Handle partition removal event. This must be invoked when we stop serving a given partition for
+   * this table in the current server.
+   * This function will be invoked when we receive CONSUMING -> DROPPED / OFFLINE state transitions.
+   *
+   * @param partitionGroupId partition id that we should stop tracking.
+   */
+  public void stopTrackingPartitionIngestionDelay(int partitionGroupId) {
+    removePartitionId(partitionGroupId);
+  }
+
+  /*
+   * This method is used for timing out inactive partitions, so we don't display their metrics on current server.
+   * When the inactive time exceeds some threshold, we read from ideal state to confirm we still host the partition,
+   * if not we remove the partition from being tracked locally.
+   * This call is to be invoked by a timer thread that will periodically wake up and invoke this function.
+   */
+  public void timeoutInactivePartitions() {
+    List<Integer> partitionsHostedByThisServer = null;
+    try {
+      partitionsHostedByThisServer = getPartitionsHostedByThisServerPerIdealState();

Review Comment:
   Good optimization, let me address this right away



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064194360


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +223,58 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();
+  }
+
+  /*
+   * Method used by LLRealtimeSegmentManagers to update their partition delays
+   *
+   * @param ingestionDelayMs Ingestion delay being reported.
+   * @param currentTimeMs 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 ingestionDelayMs, long currenTimeMs, int partitionGroupId) {
+    _ingestionDelayTracker.updateIngestionDelay(ingestionDelayMs, currenTimeMs, partitionGroupId);
+  }
+
+  /*
+   * Method to handle CONSUMING -> DROPPED segment state transitions:
+   * We stop tracking partitions whose segments are dropped.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   */
+  @Override
+  public void onConsumingToDropped(String segmentNameStr) {
+    LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr);
+    _ingestionDelayTracker.stopTrackingPartitionIngestionDelay(segmentName.getPartitionGroupId());
+  }
+
+  /*
+   * Method to handle CONSUMING -> ONLINE segment state transitions:
+   * We mark partitions for verification against 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);
+    _ingestionDelayTracker.markPartitionForVerification(segmentName.getPartitionGroupId());
+  }
+
+  /**
+   * Returns all partitionGroupIds for the partitions hosted by this server for current table.
+   * @Note: this involves Zookeeper read and should not be used frequently due to efficiency concerns.
+   */
+  public List<Integer> getHostedPartitionsGroupIds() {
+    ArrayList<Integer> partitionsHostedByThisServer = new ArrayList<>();
+    List<String> segments = TableStateUtils.getOnlineSegmentsForThisInstance(_helixManager, _tableNameWithType);

Review Comment:
   I don't think this is the case. Remember all we want to do is see if the partition is hosted by the server: having online segments indicates 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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
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-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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060800029


##########
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;

Review Comment:
   Done



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060802089


##########
pinot-core/src/test/java/org/apache/pinot/core/data/manager/realtime/ConsumptionDelayTrackerTest.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.metrics.PinotMetricUtils;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class ConsumptionDelayTrackerTest {
+
+  private static final int TIMER_THREAD_TICK_INTERVAL_MS = 100;
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  private void sleepMs(long timeMs) {

Review Comment:
   ACK, I will work on this but likely towards the end once all code issues are closed.



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060841124


##########
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:
   Being tracked in another open comment, please do not duplicate



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060963152


##########
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:
   This is addressed in the last commit. 



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064933014


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,362 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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) {
+      _delayMs = d;
+      _sampleTime = t;
+    }
+    public final long _delayMs;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private final Supplier<Boolean> _isServerReadyToServeQueries;
+
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongst all partitions we are tracking.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (int partitionGroupId : _partitionToDelaySampleMap.keySet()) {
+      DelayMeasure currentMeasure = _partitionToDelaySampleMap.get(partitionGroupId);
+      if ((newMax == null)
+          ||
+          (currentMeasure != null) && (currentMeasure._delayMs > newMax._delayMs)) {
+        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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMs + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.removePartitionGauge(_metricName, partitionGroupId, ServerGauge.TABLE_INGESTION_DELAY_MS);
+    }
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric, Supplier<Boolean> isServerReadyToServeQueries)
+      throws RuntimeException {
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;
+    _realTimeTableDataManager = realtimeTableDataManager;
+    _clock = Clock.systemUTC();
+    _isServerReadyToServeQueries = isServerReadyToServeQueries;
+    // Handle negative timer values
+    if (timerThreadTickIntervalMs <= 0) {
+      throw new RuntimeException(String.format("Illegal timer timeout argument, expected > 0, got=%d for table=%s",
+          timerThreadTickIntervalMs, _tableNameWithType));
+    }
+    _enablePerPartitionMetric = enablePerPartitionMetric;
+    _enableAggregateMetric = enableAggregateMetric;
+    _timerThreadTickIntervalMs = timerThreadTickIntervalMs;
+    _timer = new Timer("IngestionDelayTimerThread" + TableNameBuilder.extractRawTableName(tableNameWithType));
+    _timer.schedule(new TimerTask() {
+        @Override
+        public void run() {
+          timeoutInactivePartitions();
+        }
+      }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+    // Install callback metric
+    if (_enableAggregateMetric) {
+      _serverMetrics.addCallbackTableGaugeIfNeeded(_metricName, ServerGauge.TABLE_INGESTION_DELAY_MS,
+          () -> getMaximumIngestionDelay());
+    }
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager, Supplier<Boolean> isServerReadyToServeQueries) {
+    this(serverMetrics, tableNameWithType, tableDataManager, TIMER_THREAD_TICK_INTERVAL_MS,
+        "", true, true, isServerReadyToServeQueries);
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager, String metricNamePrefix,
+      Supplier<Boolean> isServerReadyToServeQueries) {
+    this(serverMetrics, metricNamePrefix + tableNameWithType, tableDataManager,
+        TIMER_THREAD_TICK_INTERVAL_MS, metricNamePrefix, true, true, isServerReadyToServeQueries);
+  }
+
+  /**
+   * Function that enable use to set predictable clocks for testing purposes.
+   *
+   * @param clock clock to be used by the class
+   */
+  @VisibleForTesting
+  void setClock(Clock clock) {
+    _clock = clock;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   *
+   * @param delayMs ingestion delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void updateIngestionDelay(long delayMs, long sampleTime, int partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    if (!_isServerReadyToServeQueries.get()) {

Review Comment:
   I don't think the first option is needed at this point, I looked at all other public methods and they will work fine if we are muting updates.
   
   To implement the second option we will need some infra that tracks what gauges have been installed so we don't keep calling the install over and over. Likely a synchronize method or atomic variable to install global max gauge and a concurrent hash to mark what gauges have been installed.
   
   May be as an future enhancement: Added comment in the code. 



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065220139


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMilliseconds + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.TABLE_INGESTION_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: _metricName + partitionGroupId
+   */
+  private String getPerPartitionMetricName(int partitionGroupId) {
+    return _metricName + partitionGroupId;
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  // Custom Constructor
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric)
+      throws RuntimeException {
+    _logger = LoggerFactory.getLogger(getClass().getSimpleName());
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;

Review Comment:
   significantly simplified all this, see 3ce7b783c5dc88073106f400f00783ba4ab71281



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065219973


##########
pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java:
##########
@@ -45,8 +45,9 @@ public enum ServerGauge implements AbstractMetrics.Gauge {
   // Dedup metrics
   DEDUP_PRIMARY_KEYS_COUNT("dedupPrimaryKeysCount", false),
   CONSUMPTION_QUOTA_UTILIZATION("ratio", false),
-  JVM_HEAP_USED_BYTES("bytes", true);
-
+  JVM_HEAP_USED_BYTES("bytes", true),
+  // Ingestion delay metrics
+  TABLE_INGESTION_DELAY_MS("milliseconds", false);

Review Comment:
   see commit 3ce7b783c5dc88073106f400f00783ba4ab71281



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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065260703


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/tablestate/TableStateUtils.java:
##########
@@ -39,37 +39,56 @@ private TableStateUtils() {
   }
 
   /**
-   * Checks if all segments for the given @param tableNameWithType are succesfully loaded
-   * This function will get all segments in IDEALSTATE and CURRENTSTATE for the given table,
-   * and then check if all ONLINE segments in IDEALSTATE match with CURRENTSTATE.
-   * @param helixManager helix manager for the server instance
-   * @param tableNameWithType table name for which segment state is to be checked
-   * @return true if all segments for the given table are succesfully loaded. False otherwise
+   * Returns all segments in a given state for a given table.
+   *
+   * @param helixManager instance of Helix manager
+   * @param tableNameWithType table for which we are obtaining ONLINE segments
+   * @param state state of the segments to be returned
+   *
+   * @return List of segment names in a given state.
    */
-  public static boolean isAllSegmentsLoaded(HelixManager helixManager, String tableNameWithType) {
+  public static Set<String> getSegmentsInGivenStateForThisInstance(HelixManager helixManager, String tableNameWithType,
+      String state) {
     HelixDataAccessor dataAccessor = helixManager.getHelixDataAccessor();
     PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
     IdealState idealState = dataAccessor.getProperty(keyBuilder.idealStates(tableNameWithType));
+    Set<String> segmentsInGivenState = new HashSet<>();
     if (idealState == null) {
       LOGGER.warn("Failed to find ideal state for table: {}", tableNameWithType);
-      return false;
+      return segmentsInGivenState;
     }
 
     // Get all ONLINE segments from idealState
     String instanceName = helixManager.getInstanceName();
-    List<String> onlineSegments = new ArrayList<>();
     Map<String, Map<String, String>> idealStatesMap = idealState.getRecord().getMapFields();
     for (Map.Entry<String, Map<String, String>> entry : idealStatesMap.entrySet()) {
       String segmentName = entry.getKey();
       Map<String, String> instanceStateMap = entry.getValue();
       String expectedState = instanceStateMap.get(instanceName);
       // Only track ONLINE segments assigned to the current instance
-      if (!CommonConstants.Helix.StateModel.SegmentStateModel.ONLINE.equals(expectedState)) {
+      if (!state.equals(expectedState)) {

Review Comment:
   ^^



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java:
##########
@@ -183,4 +184,11 @@ void addOrReplaceSegment(String tableNameWithType, String segmentName)
    * Immediately stop consumption and start committing the consuming segments.
    */
   void forceCommit(String tableNameWithType, Set<String> segmentNames);
+
+  /**
+   * Enables the installation of a method to determine if a server is ready to server queries.
+   *
+   * @param isServerReadyToServeQueries supplier to retrieve state of server.
+   */
+  void setSupplierOfIsServerReadyToServeQueries(Supplier<Boolean> isServerReadyToServeQueries);

Review Comment:
   ^^



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.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.time.Clock;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ * TODO: handle bug situations like the one where a partition is not allocated to a given server due to a bug.
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 {

Review Comment:
   ^^



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


[GitHub] [pinot] mcvsubbu commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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

   > > We set it to 500ms. Our stream ingestion plugin is fairly strange. The upstream consume API we use is async, so we request records and wait either 500ms or until we've received 1000 records before returning.
   > 
   > OK if this is the case, for an inactive stream we should see that 0 is reported and then the value linearly increases to about 500ms and then falls to zero, creating a sawtooth shape when your system is not consuming: is that what you see? if not can you share the graphs you are observing on an inactive stream? Also are the timestamps in your events UTC? This metric assumed times are in UTC, if not then the graph will be offset by the hours diff with UTC in milliseconds
   
   @jugomezv if it is not already there, can you add comments in the interface that the timestamps should be in UTC? 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-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


[GitHub] [pinot] jadami-stripe commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by "jadami-stripe (via GitHub)" <gi...@apache.org>.
jadami-stripe commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1312419841


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +225,59 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();

Review Comment:
   is there any reason we don't shut this down first? I'm testing now, but I feel like this causes lag to artificially go up while a server is shutting down



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064230639


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +223,58 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();
+  }
+
+  /*
+   * Method used by LLRealtimeSegmentManagers to update their partition delays
+   *
+   * @param ingestionDelayMs Ingestion delay being reported.
+   * @param currentTimeMs 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 ingestionDelayMs, long currenTimeMs, int partitionGroupId) {
+    _ingestionDelayTracker.updateIngestionDelay(ingestionDelayMs, currenTimeMs, partitionGroupId);
+  }
+
+  /*
+   * Method to handle CONSUMING -> DROPPED segment state transitions:
+   * We stop tracking partitions whose segments are dropped.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   */
+  @Override
+  public void onConsumingToDropped(String segmentNameStr) {
+    LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr);
+    _ingestionDelayTracker.stopTrackingPartitionIngestionDelay(segmentName.getPartitionGroupId());
+  }
+
+  /*
+   * Method to handle CONSUMING -> ONLINE segment state transitions:
+   * We mark partitions for verification against 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);
+    _ingestionDelayTracker.markPartitionForVerification(segmentName.getPartitionGroupId());
+  }
+
+  /**
+   * Returns all partitionGroupIds for the partitions hosted by this server for current table.
+   * @Note: this involves Zookeeper read and should not be used frequently due to efficiency concerns.
+   */
+  public List<Integer> getHostedPartitionsGroupIds() {
+    ArrayList<Integer> partitionsHostedByThisServer = new ArrayList<>();
+    List<String> segments = TableStateUtils.getOnlineSegmentsForThisInstance(_helixManager, _tableNameWithType);

Review Comment:
   I see, but then the possibility of having only ONLINE at the time you are committing has to be addressed 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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064230722


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +223,58 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();
+  }
+
+  /*
+   * Method used by LLRealtimeSegmentManagers to update their partition delays
+   *
+   * @param ingestionDelayMs Ingestion delay being reported.
+   * @param currentTimeMs 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 ingestionDelayMs, long currenTimeMs, int partitionGroupId) {
+    _ingestionDelayTracker.updateIngestionDelay(ingestionDelayMs, currenTimeMs, partitionGroupId);
+  }
+
+  /*
+   * Method to handle CONSUMING -> DROPPED segment state transitions:
+   * We stop tracking partitions whose segments are dropped.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   */
+  @Override
+  public void onConsumingToDropped(String segmentNameStr) {
+    LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr);
+    _ingestionDelayTracker.stopTrackingPartitionIngestionDelay(segmentName.getPartitionGroupId());
+  }
+
+  /*
+   * Method to handle CONSUMING -> ONLINE segment state transitions:
+   * We mark partitions for verification against 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);
+    _ingestionDelayTracker.markPartitionForVerification(segmentName.getPartitionGroupId());
+  }
+
+  /**
+   * Returns all partitionGroupIds for the partitions hosted by this server for current table.
+   * @Note: this involves Zookeeper read and should not be used frequently due to efficiency concerns.
+   */
+  public List<Integer> getHostedPartitionsGroupIds() {
+    ArrayList<Integer> partitionsHostedByThisServer = new ArrayList<>();
+    List<String> segments = TableStateUtils.getOnlineSegmentsForThisInstance(_helixManager, _tableNameWithType);

Review Comment:
   so I should be considering CONSUMING *and* ONLINE segments 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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065003417


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,362 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import org.apache.pinot.common.metrics.ServerGauge;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+  private static final Logger _logger = LoggerFactory.getLogger(IngestionDelayTracker.class.getSimpleName());
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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) {
+      _delayMs = d;
+      _sampleTime = t;
+    }
+    public final long _delayMs;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private final Supplier<Boolean> _isServerReadyToServeQueries;
+
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongst all partitions we are tracking.
+   */
+  private DelayMeasure getMaximumDelay() {
+    DelayMeasure newMax = null;
+    for (int partitionGroupId : _partitionToDelaySampleMap.keySet()) {
+      DelayMeasure currentMeasure = _partitionToDelaySampleMap.get(partitionGroupId);
+      if ((newMax == null)
+          ||
+          (currentMeasure != null) && (currentMeasure._delayMs > newMax._delayMs)) {
+        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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMs + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.removePartitionGauge(_metricName, partitionGroupId, ServerGauge.TABLE_INGESTION_DELAY_MS);
+    }
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric, Supplier<Boolean> isServerReadyToServeQueries)
+      throws RuntimeException {
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;
+    _realTimeTableDataManager = realtimeTableDataManager;
+    _clock = Clock.systemUTC();
+    _isServerReadyToServeQueries = isServerReadyToServeQueries;
+    // Handle negative timer values
+    if (timerThreadTickIntervalMs <= 0) {
+      throw new RuntimeException(String.format("Illegal timer timeout argument, expected > 0, got=%d for table=%s",
+          timerThreadTickIntervalMs, _tableNameWithType));
+    }
+    _enablePerPartitionMetric = enablePerPartitionMetric;
+    _enableAggregateMetric = enableAggregateMetric;
+    _timerThreadTickIntervalMs = timerThreadTickIntervalMs;
+    _timer = new Timer("IngestionDelayTimerThread" + TableNameBuilder.extractRawTableName(tableNameWithType));
+    _timer.schedule(new TimerTask() {
+        @Override
+        public void run() {
+          timeoutInactivePartitions();
+        }
+      }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+    // Install callback metric
+    if (_enableAggregateMetric) {
+      _serverMetrics.addCallbackTableGaugeIfNeeded(_metricName, ServerGauge.TABLE_INGESTION_DELAY_MS,
+          () -> getMaximumIngestionDelay());
+    }
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager, Supplier<Boolean> isServerReadyToServeQueries) {
+    this(serverMetrics, tableNameWithType, tableDataManager, TIMER_THREAD_TICK_INTERVAL_MS,
+        "", true, true, isServerReadyToServeQueries);
+  }
+
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager tableDataManager, String metricNamePrefix,
+      Supplier<Boolean> isServerReadyToServeQueries) {
+    this(serverMetrics, metricNamePrefix + tableNameWithType, tableDataManager,
+        TIMER_THREAD_TICK_INTERVAL_MS, metricNamePrefix, true, true, isServerReadyToServeQueries);
+  }
+
+  /**
+   * Function that enable use to set predictable clocks for testing purposes.
+   *
+   * @param clock clock to be used by the class
+   */
+  @VisibleForTesting
+  void setClock(Clock clock) {
+    _clock = clock;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   *
+   * @param delayMs ingestion delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void updateIngestionDelay(long delayMs, long sampleTime, int partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    if (!_isServerReadyToServeQueries.get()) {

Review Comment:
   Closing, tagged as future enhacement



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065001747


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +223,58 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();
+  }
+
+  /*
+   * Method used by LLRealtimeSegmentManagers to update their partition delays
+   *
+   * @param ingestionDelayMs Ingestion delay being reported.
+   * @param currentTimeMs 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 ingestionDelayMs, long currenTimeMs, int partitionGroupId) {
+    _ingestionDelayTracker.updateIngestionDelay(ingestionDelayMs, currenTimeMs, partitionGroupId);
+  }
+
+  /*
+   * Method to handle CONSUMING -> DROPPED segment state transitions:
+   * We stop tracking partitions whose segments are dropped.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   */
+  @Override
+  public void onConsumingToDropped(String segmentNameStr) {
+    LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr);
+    _ingestionDelayTracker.stopTrackingPartitionIngestionDelay(segmentName.getPartitionGroupId());
+  }
+
+  /*
+   * Method to handle CONSUMING -> ONLINE segment state transitions:
+   * We mark partitions for verification against 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);
+    _ingestionDelayTracker.markPartitionForVerification(segmentName.getPartitionGroupId());
+  }
+
+  /**
+   * Returns all partitionGroupIds for the partitions hosted by this server for current table.
+   * @Note: this involves Zookeeper read and should not be used frequently due to efficiency concerns.
+   */
+  public List<Integer> getHostedPartitionsGroupIds() {
+    ArrayList<Integer> partitionsHostedByThisServer = new ArrayList<>();
+    List<String> segments = TableStateUtils.getOnlineSegmentsForThisInstance(_helixManager, _tableNameWithType);

Review Comment:
   Closing after online discussion with @mcvsubbu 



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064194470


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMilliseconds + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.TABLE_INGESTION_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: _metricName + partitionGroupId
+   */
+  private String getPerPartitionMetricName(int partitionGroupId) {
+    return _metricName + partitionGroupId;
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  // Custom Constructor
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric)
+      throws RuntimeException {
+    _logger = LoggerFactory.getLogger(getClass().getSimpleName());
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;

Review Comment:
   this is a bit forwarding looking to extend this work to create another metric that report end to end ingestion delays as our offline discussion



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


[GitHub] [pinot] navina commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064062339


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMilliseconds + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.TABLE_INGESTION_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: _metricName + partitionGroupId
+   */
+  private String getPerPartitionMetricName(int partitionGroupId) {
+    return _metricName + partitionGroupId;
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  // Custom Constructor
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric)
+      throws RuntimeException {
+    _logger = LoggerFactory.getLogger(getClass().getSimpleName());
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;
+    _realTimeTableDataManager = realtimeTableDataManager;
+    _clock = Clock.systemDefaultZone();
+    // Handle negative timer values
+    if (timerThreadTickIntervalMs <= 0) {
+      throw new RuntimeException(String.format("Illegal timer timeout argument, expected > 0, got=%d for table=%s",
+          timerThreadTickIntervalMs, _tableNameWithType));
+    }
+    _enablePerPartitionMetric = enablePerPartitionMetric;
+    _enableAggregateMetric = enableAggregateMetric;
+    _timerThreadTickIntervalMs = timerThreadTickIntervalMs;
+    _timer = new Timer("IngestionDelayTimerThread" + TableNameBuilder.extractRawTableName(tableNameWithType));
+    _timer.schedule(new TimerTask() {
+        @Override
+        public void run() {
+          timeoutInactivePartitions();
+        }
+      }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
+    // Install callback metric
+    if (_enableAggregateMetric) {
+      _serverMetrics.addCallbackTableGaugeIfNeeded(_metricName, ServerGauge.TABLE_INGESTION_DELAY_MS,
+          () -> getMaximumIngestionDelay());
+    }
+  }
+
+  // Constructor that uses defaults
+  public IngestionDelayTracker(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 IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType, String metricNamePrefix,
+      RealtimeTableDataManager tableDataManager) {
+    this(serverMetrics, metricNamePrefix + tableNameWithType, tableDataManager,
+        TIMER_THREAD_TICK_INTERVAL_MS, metricNamePrefix, true, true);
+  }
+
+  /**
+   * Function that enable use to set predictable clocks for testing purposes.
+   *
+   * @param clock clock to be used by the class
+   */
+  @VisibleForTesting
+  void setClock(Clock clock) {
+    _clock = clock;
+  }
+
+  /*
+   * Called by LLRealTimeSegmentDataManagers to post delay updates to this tracker class.
+   *
+   * @param delayInMilliseconds ingestion delay being recorded.
+   * @param sampleTime sample time.
+   * @param partitionGroupId partition ID for which this delay is being recorded.
+   */
+  public void updateIngestionDelay(long delayInMilliseconds, long sampleTime, int partitionGroupId) {
+    // Store new measure and wipe old one for this partition
+    DelayMeasure previousMeasure = _partitionToDelaySampleMap.put(partitionGroupId,
+        new DelayMeasure(sampleTime, delayInMilliseconds));
+    if ((previousMeasure == null) && _enablePerPartitionMetric) {
+      // First time we start tracking a partition we should start tracking it via metric
+      _serverMetrics.addCallbackTableGaugeIfNeeded(getPerPartitionMetricName(partitionGroupId),

Review Comment:
   You can add method `addPartitionCallbackGauge` in `AbstractMetrics` and making it a standard to append partitionId to the metric name. Then, you can get rid of the `getPerPartitionMetricName` method here. 



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMilliseconds + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.TABLE_INGESTION_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: _metricName + partitionGroupId
+   */
+  private String getPerPartitionMetricName(int partitionGroupId) {
+    return _metricName + partitionGroupId;
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  // Custom Constructor
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric)
+      throws RuntimeException {
+    _logger = LoggerFactory.getLogger(getClass().getSimpleName());
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;

Review Comment:
    Iiuc, `ServerMetrics` already has a metric prefix that can be configured. Each partition level metric includes the `partitionGroupId`.  how will this `metricNamePrefix` will be used? Do we really need it?
    



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/InstanceDataManager.java:
##########
@@ -183,4 +184,7 @@ void addOrReplaceSegment(String tableNameWithType, String segmentName)
    * Immediately stop consumption and start committing the consuming segments.
    */
   void forceCommit(String tableNameWithType, Set<String> segmentNames);
+
+  void setSupplierOfReadyToServeQueries(Supplier<Boolean> isReadyToServeQueries);

Review Comment:
   please add javadocs for interface methods



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1064281319


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +223,58 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();
+  }
+
+  /*
+   * Method used by LLRealtimeSegmentManagers to update their partition delays
+   *
+   * @param ingestionDelayMs Ingestion delay being reported.
+   * @param currentTimeMs 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 ingestionDelayMs, long currenTimeMs, int partitionGroupId) {
+    _ingestionDelayTracker.updateIngestionDelay(ingestionDelayMs, currenTimeMs, partitionGroupId);
+  }
+
+  /*
+   * Method to handle CONSUMING -> DROPPED segment state transitions:
+   * We stop tracking partitions whose segments are dropped.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   */
+  @Override
+  public void onConsumingToDropped(String segmentNameStr) {
+    LLCSegmentName segmentName = new LLCSegmentName(segmentNameStr);
+    _ingestionDelayTracker.stopTrackingPartitionIngestionDelay(segmentName.getPartitionGroupId());
+  }
+
+  /*
+   * Method to handle CONSUMING -> ONLINE segment state transitions:
+   * We mark partitions for verification against 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);
+    _ingestionDelayTracker.markPartitionForVerification(segmentName.getPartitionGroupId());
+  }
+
+  /**
+   * Returns all partitionGroupIds for the partitions hosted by this server for current table.
+   * @Note: this involves Zookeeper read and should not be used frequently due to efficiency concerns.
+   */
+  public List<Integer> getHostedPartitionsGroupIds() {
+    ArrayList<Integer> partitionsHostedByThisServer = new ArrayList<>();
+    List<String> segments = TableStateUtils.getOnlineSegmentsForThisInstance(_helixManager, _tableNameWithType);

Review Comment:
   Addressed by commit e476c1efae63d35835576fbf0a75cf3b6c8ef970 please resolve if you agree @mcvsubbu 



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1065182907


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/IngestionDelayTracker.java:
##########
@@ -0,0 +1,371 @@
+/**
+ * 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.time.Clock;
+import java.util.ArrayList;
+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.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Class to track realtime ingestion 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-A Metric is derived from reading the maximum tracked by this class. In addition, individual metrics are associated
+ *   with each partition being tracked.
+ * 5-Delays reported for partitions that do not have events to consume are reported as zero.
+ * 6-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.
+ * 7-Partitions whose Segments go from CONSUMING to DROPPED state stop being tracked so their delays do not cloud
+ *   delays of active partitions.
+ * 8-When a segment goes from CONSUMING to ONLINE, we start a timeout for the corresponding partition.
+ *   If no consumption is noticed after the timeout, we then read ideal state to confirm the server still hosts the
+ *   partition. If not, we stop tracking the respective partition.
+ * 9-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)
+ *            |                         |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 0}}
+ *            |                         |
+ * ___________V_________________________V_
+ * |           (Table X)                |<-updateIngestionDelay()-{LLRealtimeSegmentDataManager(Partition 1}}
+ * | IngestionDelayTracker              |           ...
+ * |____________________________________|<-updateIngestionDelay()-{LLRealtimeSegmentDataManager (Partition n}}
+ *              ^                      ^
+ *              |                       \
+ *   timeoutInactivePartitions()    stopTrackingPartitionIngestionDelay(partitionId)
+ *    _________|__________                \
+ *   | TimerTrackingTask |          (CONSUMING -> DROPPED state change)
+ *   |___________________|
+ *
+ */
+
+public class IngestionDelayTracker {
+
+  // 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 amount of time after starting timer
+  private static final int INITIAL_TIMER_THREAD_DELAY_MS = 100;
+
+  /*
+   * Class to keep an ingestion delay measure and the time when the sample was taken (i.e. sample time)
+   * We will use the sample time to increase ingestion 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 final long _delayMilliseconds;
+    public final long _sampleTime;
+  }
+
+  // HashMap used to store delay measures for all partitions active for the current table.
+  private final 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.
+  private final ConcurrentHashMap<Integer, Long> _partitionsMarkedForVerification = new ConcurrentHashMap<>();
+
+  final int _timerThreadTickIntervalMs;
+  // Timer task to check partitions that are inactive against ideal state.
+  private final Timer _timer;
+
+  private final ServerMetrics _serverMetrics;
+  private final String _tableNameWithType;
+  private final String _metricName;
+
+  private final boolean _enablePerPartitionMetric;
+  private final boolean _enableAggregateMetric;
+  private final Logger _logger;
+
+  private final RealtimeTableDataManager _realTimeTableDataManager;
+  private Clock _clock;
+
+  /*
+   * Returns the maximum ingestion delay amongs all partitions we are tracking.
+   */
+  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.
+  *
+  * @param currentDelay original sample delay to which we will add the age of the measure.
+   */
+  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 = _clock.millis() - currentDelay._sampleTime;
+    // Correct to zero for any time shifts due to NTP or time reset.
+    measureAgeInMs = Math.max(measureAgeInMs, 0);
+    return currentDelay._delayMilliseconds + measureAgeInMs;
+  }
+
+  /*
+   * Helper function to be called when we should stop tracking a given partition. Removes the partition from
+   * all our maps.
+   *
+   * @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.TABLE_INGESTION_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: _metricName + partitionGroupId
+   */
+  private String getPerPartitionMetricName(int partitionGroupId) {
+    return _metricName + partitionGroupId;
+  }
+
+  /*
+   * Helper functions that creates a list of all the partitions that are marked for verification and whose
+   * timeouts are expired. This helps us optimize checks of the ideal state.
+   */
+  private ArrayList<Integer> getPartitionsToBeVerified() {
+    ArrayList<Integer> partitionsToVerify = new ArrayList<>();
+    for (int partitionGroupId : _partitionsMarkedForVerification.keySet()) {
+      long markTime = _partitionsMarkedForVerification.get(partitionGroupId);
+      long timeMarked = _clock.millis() - markTime;
+      if (timeMarked > PARTITION_TIMEOUT_MS) {
+        // Partition must be verified
+        partitionsToVerify.add(partitionGroupId);
+      }
+    }
+    return partitionsToVerify;
+  }
+
+  // Custom Constructor
+  public IngestionDelayTracker(ServerMetrics serverMetrics, String tableNameWithType,
+      RealtimeTableDataManager realtimeTableDataManager, int timerThreadTickIntervalMs, String metricNamePrefix,
+      boolean enableAggregateMetric, boolean enablePerPartitionMetric)
+      throws RuntimeException {
+    _logger = LoggerFactory.getLogger(getClass().getSimpleName());
+    _serverMetrics = serverMetrics;
+    _tableNameWithType = tableNameWithType;
+    _metricName = metricNamePrefix + tableNameWithType;

Review Comment:
   Done.



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


[GitHub] [pinot] jugomezv commented on pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

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

   > We set it to 500ms. Our stream ingestion plugin is fairly strange. The upstream consume API we use is async, so we request records and wait either 500ms or until we've received 1000 records before returning.
   
   OK if this is the case, for an inactive stream we should see that 0 is reported and then the value linearly increases to about 500ms and then falls to zero, creating a sawtooth shape when your system is not consuming: is that what you see? if not can you share the graphs you are observing on an inactive stream? Also are the timestamps in your events UTC? This metric assumed times are in UTC, if not then the graph will be offset by the hours diff with UTC in milliseconds


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


[GitHub] [pinot] navina commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1058167406


##########
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) {

Review Comment:
   nit: s/currenTimeMillis/currentTimeMs/ ?
   
   in general, can we use Ms for milliseconds to keep the variable names to a reasonable length? 



##########
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);

Review Comment:
   Do we really need a `TrackingTimerTask` class defined for this? Can we just use anonymous class like the following and get rid of yet another private inner class?
   
   ```
   _timer.schedule(new TimerTask() {
         @Override
         public void run() {
           timeoutInactivePartitions();
         }
       }, INITIAL_TIMER_THREAD_DELAY_MS, _timerThreadTickIntervalMs);
       ```



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java:
##########
@@ -201,4 +201,18 @@ void addOrReplaceSegment(String segmentName, IndexLoadingConfig indexLoadingConf
    * @return List of {@link SegmentErrorInfo}
    */
   Map<String, SegmentErrorInfo> getSegmentErrors();
+
+  /**
+   * Interface to place actions to be performed when a Segment goes from CONSUMING to ONLINE.
+   *
+   * @param segmentNameStr name of segment for which the state change is being handled
+   */
+  default void onConsumingToOnline(String segmentNameStr) { };
+
+  /**
+   * Interface to place actions to be performed when a Segment goes from CONSUMING to DROPPED.
+   *
+   * @param segmentNameStr name of segment for which the state change is being handled
+   */
+  default void onConsumingToDropped(String segmentNameStr) { };
 }

Review Comment:
   Can we use a single method for handling state transition notifications?
   ```
   default void onSegmentStateTransition(String segmentName, String fromState, String toState) { }
   ```
   It will be easier to enhance it rather than add more interface methods?
   
   



##########
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;

Review Comment:
   Its a convention in this codebase (and most Java repos that I have seen) to use `private final` rather than `final private`. Can you please fix these? 



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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1055627527


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java:
##########
@@ -91,6 +92,16 @@ public void onBecomeOnlineFromConsuming(Message message, NotificationContext con
 
       TableDataManager tableDataManager = _instanceDataManager.getTableDataManager(realtimeTableName);
       Preconditions.checkNotNull(tableDataManager);
+      if (tableDataManager instanceof RealtimeTableDataManager) {

Review Comment:
   I can do this but I don't like the way the code ends up looking like: since we are talking about a TableDataManager and not a BaseTableDataManager, I will still have to cast TableDataManager to BaseTableDataManager before I invoke the method. (Note the hierarchy is as follows TableDataManager->BaseTableDataManager->RealTimeTableDataManager)  As an alternative I can bring these interfaces up to TableDataManager and override in BaseDataManager as nop function and then with the implementation I need in RealTimeTableDataManager. This has the problem that we start exposing low level interfaces in what should be the high level TableDataManager. So I think it is best to leave the code as 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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1055628430


##########
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentOnlineOfflineStateModelFactory.java:
##########
@@ -138,6 +149,19 @@ 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();
+      String segmentNameStr = message.getPartitionName();
+      TableDataManager tableDataManager = _instanceDataManager.getTableDataManager(realtimeTableName);
+      Preconditions.checkNotNull(tableDataManager);
+      if (tableDataManager instanceof RealtimeTableDataManager) {

Review Comment:
   See my response above, this option seems better than bubbling up a realtime specific method to tabledata manager



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


[GitHub] [pinot] mcvsubbu commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060894007


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -613,7 +613,7 @@ private boolean processStreamEvents(MessageBatch messagesAndOffsets, long idlePi
         _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);
+      setIngestionDelayToZero();

Review Comment:
   Discussed offline, thanks for the explanation. Your original call to realtimeTableDatamanager is good. Let us revert to 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


[GitHub] [pinot] jugomezv commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
jugomezv commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1060947825


##########
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:
   Yes, but I think you are correct a lot of other stuff will break before this break so it does not make sense to keep these changes here, I will remove.



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


[GitHub] [pinot] navina commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1061806303


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java:
##########
@@ -212,6 +216,83 @@ protected void doShutdown() {
     if (_leaseExtender != null) {
       _leaseExtender.shutDown();
     }
+    // Make sure we do metric cleanup when we shut down the table.
+    _ingestionDelayTracker.shutdown();
+  }
+
+  /*
+   * Method to handle CONSUMING->DROPPED transition.
+   *
+   * @param partitionGroupId Partition id that we must stop tracking on this server.
+   */
+  private void stopTrackingPartitionDelay(int partitionGroupId) {
+    _ingestionDelayTracker.stopTrackingPartitionIngestionDelay(partitionGroupId);
+  }
+
+  /*
+   * Method to handle CONSUMING->ONLINE transition.
+   * If no new ingestion 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) {
+    _ingestionDelayTracker.markPartitionForConfirmation(partitionGroupId);
+  }
+
+  /*
+   * Method used by LLRealtimeSegmentManagers to update their partition delays
+   *
+   * @param ingestionDelayMs Ingestion delay being reported.
+   * @param currentTimeMs 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 ingestionDelayMs, long currenTimeMs, int partitionGroupId) {
+    _ingestionDelayTracker.updateIngestionDelay(ingestionDelayMs, currenTimeMs, partitionGroupId);
+  }
+
+  /*
+   * Method to handle supported transitions of segments states for this table.
+   * Supported transitions include:
+   *
+   * CONSUMING -> ONLINE:
+   * We mark partitions for verification against 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.
+   * CONSUMING -> DROPPED:
+   * We stop tracking partitions whose segments are dropped.
+   *
+   * @param segmentNameStr name of segment which is transitioning state.
+   * @param fromState state from which the segment is transitioning.
+   * @param toState state to which the segment is transitioning to.
+   */
+  @Override
+  public void onSegmentStateTransition(String segmentNameStr, SegmentState fromState, SegmentState toState) {

Review Comment:
   I don't think we should be adding a new method for each state transition. Then, the number of methods here would potentially grow to the number of state transitions.
   
   There are alternatives if you want to avoid if statements (throwing it out there: define constants for each transition like `CONSUMING_TO_ONLINE` , `DROPPED_ENDSTATE` ). It will improve readability while keeping a check on the number of new interface methods.  



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


[GitHub] [pinot] navina commented on a diff in pull request #9994: [feature] Add a Tracker class to support aggregate-worst case consumption delay…

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9994:
URL: https://github.com/apache/pinot/pull/9994#discussion_r1062197948


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1594,4 +1616,9 @@ public String getSegmentName() {
   public void forceCommit() {
     _forceCommitMessageReceived = true;
   }
+
+  @Override
+  public void notifyConsumptionCaughtUp(boolean catchingUpPhase) {

Review Comment:
   >Another way to handle it is to assume it is caught up in the beginning, and let the status checker to set it as not caught up yet. 
   
   I think that's what my PR (and this one) does. 



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