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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]