Jackie-Jiang commented on code in PR #15261:
URL: https://github.com/apache/pinot/pull/15261#discussion_r2014842432


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumerCoordinator.java:
##########
@@ -0,0 +1,302 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.Map;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.annotation.Nullable;
+import org.apache.helix.model.IdealState;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.metrics.ServerTimer;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.segment.local.data.manager.SegmentDataManager;
+import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The ConsumerCoordinator coordinates the offline->consuming helix 
transitions.
+ */
+public class ConsumerCoordinator {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(ConsumerCoordinator.class);
+
+  private final Semaphore _semaphore;
+  private final boolean _enforceConsumptionInOrder;
+  private final Condition _condition;
+  private final Lock _lock;
+  private final ServerMetrics _serverMetrics;
+  private volatile int _maxSegmentSeqNumRegistered = -1;
+  private final boolean _alwaysRelyOnIdealState;
+  private final RealtimeTableDataManager _realtimeTableDataManager;
+  private final AtomicBoolean _isFirstTransitionProcessed;
+  private static final long WAIT_INTERVAL_MS = TimeUnit.MINUTES.toMillis(3);
+
+  public ConsumerCoordinator(boolean enforceConsumptionInOrder, 
RealtimeTableDataManager realtimeTableDataManager) {
+    _semaphore = new Semaphore(1);
+    _lock = new ReentrantLock();
+    _condition = _lock.newCondition();
+    _enforceConsumptionInOrder = enforceConsumptionInOrder;
+    _realtimeTableDataManager = realtimeTableDataManager;
+    StreamIngestionConfig streamIngestionConfig = 
realtimeTableDataManager.getStreamIngestionConfig();
+    if (streamIngestionConfig != null) {
+      // if isUseIdealStateToCalculatePreviousSegment is true, server relies 
on ideal state to fetch previous segment
+      // to a segment for all helix transitions.
+      _alwaysRelyOnIdealState = 
streamIngestionConfig.isUseIdealStateToCalculatePreviousSegment();
+    } else {
+      _alwaysRelyOnIdealState = false;
+    }
+    _isFirstTransitionProcessed = new AtomicBoolean(false);
+    _serverMetrics = ServerMetrics.get();
+  }
+
+  void acquire(LLCSegmentName llcSegmentName)
+      throws InterruptedException {
+    long startTimeMs;
+
+    if (_enforceConsumptionInOrder) {
+      startTimeMs = System.currentTimeMillis();
+
+      waitForPrevSegment(llcSegmentName);
+
+      
_serverMetrics.addTimedTableValue(_realtimeTableDataManager.getTableName(),
+          ServerTimer.PREV_SEGMENT_WAIT_DURATION_MS, 
System.currentTimeMillis() - startTimeMs, TimeUnit.MILLISECONDS);
+    }
+
+    startTimeMs = System.currentTimeMillis();
+    while (!_semaphore.tryAcquire(WAIT_INTERVAL_MS, TimeUnit.MILLISECONDS)) {
+      LOGGER.warn("Failed to acquire partitionGroup consumer semaphore in: {} 
ms. Retrying.",
+          System.currentTimeMillis() - startTimeMs);
+    }
+  }
+
+  void release() {
+    _semaphore.release();
+  }
+
+  @VisibleForTesting
+  Semaphore getSemaphore() {
+    return _semaphore;
+  }
+
+  void trackSegment(LLCSegmentName llcSegmentName) {
+    _lock.lock();
+    try {
+      if (!_alwaysRelyOnIdealState) {
+        _maxSegmentSeqNumRegistered = Math.max(_maxSegmentSeqNumRegistered, 
llcSegmentName.getSequenceNumber());
+      }
+      // notify all helix threads waiting for their offline -> consuming 
segment's prev segment to be loaded
+      _condition.signalAll();
+    } finally {
+      _lock.unlock();
+    }
+  }
+
+  private void waitForPrevSegment(LLCSegmentName currSegment)
+      throws InterruptedException {
+
+    if (_alwaysRelyOnIdealState || !_isFirstTransitionProcessed.get()) {
+      // if _alwaysRelyOnIdealState or no offline -> consuming transition has 
been processed, it means rely on
+      // ideal state to fetch previous segment.
+      acquireSegmentRelyingOnIdealState(currSegment);
+
+      // the first transition will always be prone to error, consider edge 
case where segment previous to current
+      // helix transition's segment was deleted and this server came alive 
after successful deletion. the prev
+      // segment will not exist, hence first transition is handled using 
isFirstTransitionSuccessful.
+      _isFirstTransitionProcessed.compareAndSet(false, true);
+      return;
+    }
+
+    // rely on _maxSegmentSeqNumRegistered watermark for previous segment.
+    if (awaitForPreviousSegmentSequenceNumber(currSegment, WAIT_INTERVAL_MS)) {
+      return;
+    }
+
+    // tried using prevSegSeqNumber watermark, but could not acquire the 
previous segment.
+    // fallback to acquire prev segment from ideal state.
+    acquireSegmentRelyingOnIdealState(currSegment);
+  }
+
+  private void acquireSegmentRelyingOnIdealState(LLCSegmentName currSegment)
+      throws InterruptedException {
+    LLCSegmentName previousLLCSegmentName = getPreviousSegment(currSegment);
+    if (previousLLCSegmentName == null) {
+      // previous segment can only be null if either all the previous segments 
are deleted or this is the starting
+      // sequence segment of the partition Group.
+      return;
+    }
+    String previousSegment = previousLLCSegmentName.getSegmentName();
+
+    SegmentDataManager segmentDataManager = 
_realtimeTableDataManager.acquireSegment(previousSegment);
+    try {
+      long startTimeMs = System.currentTimeMillis();
+      _lock.lock();
+      try {
+        while (segmentDataManager == null) {
+          // if segmentDataManager == null, it means segment is not loaded in 
the server.
+          // wait until it's loaded.
+          while (!_condition.await(WAIT_INTERVAL_MS, TimeUnit.MILLISECONDS)) {
+            LOGGER.warn("Semaphore access denied to segment: {}. Waiting on 
previous segment: {} since: {} ms.",
+                currSegment.getSegmentName(), previousSegment, 
System.currentTimeMillis() - startTimeMs);
+            // waited until timeout, fetch previous segment again from ideal 
state as previous segment might be
+            // changed in ideal state.
+            previousLLCSegmentName = getPreviousSegment(currSegment);
+            if (previousLLCSegmentName == null) {
+              return;
+            }
+            previousSegment = previousLLCSegmentName.getSegmentName();
+          }
+          segmentDataManager = 
_realtimeTableDataManager.acquireSegment(previousSegment);
+        }
+      } finally {
+        _lock.unlock();
+      }
+    } finally {
+      if (segmentDataManager != null) {
+        _realtimeTableDataManager.releaseSegment(segmentDataManager);
+      }
+    }
+  }
+
+  /***
+   * @param currSegment is the segment of current helix transition.
+   * @param timeoutMs is max time to wait in millis
+   * @return true if previous Segment was registered to the server, else false.
+   * @throws InterruptedException
+   */
+  @VisibleForTesting
+  boolean awaitForPreviousSegmentSequenceNumber(LLCSegmentName currSegment, 
long timeoutMs)
+      throws InterruptedException {
+    long startTimeMs = System.currentTimeMillis();
+    int prevSeqNum = currSegment.getSequenceNumber() - 1;
+    _lock.lock();
+    try {
+      while (_maxSegmentSeqNumRegistered < prevSeqNum) {
+        // it means all segments until _maxSegmentSeqNumRegistered is not 
loaded in the server. Wait until it's loaded.
+        if (!_condition.await(timeoutMs, TimeUnit.MILLISECONDS)) {
+          LOGGER.warn("Semaphore access denied to segment: {}."
+                  + " Waiting on previous segment with sequence number: {} 
since: {} ms.", currSegment.getSegmentName(),
+              prevSeqNum, System.currentTimeMillis() - startTimeMs);
+          // waited until the timeout. Rely on ideal state now.
+          return false;
+        }
+      }
+
+      return (_maxSegmentSeqNumRegistered >= prevSeqNum);
+    } finally {
+      _lock.unlock();
+    }
+  }
+
+  @VisibleForTesting
+  @Nullable
+  LLCSegmentName getPreviousSegment(LLCSegmentName currSegment) {

Review Comment:
   (minor) Make it more specific
   ```suggestion
     LLCSegmentName getPreviousSegmentFromIdealState(LLCSegmentName 
currSegment) {
   ```



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumerCoordinator.java:
##########
@@ -0,0 +1,302 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.Map;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.annotation.Nullable;
+import org.apache.helix.model.IdealState;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.metrics.ServerTimer;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.segment.local.data.manager.SegmentDataManager;
+import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The ConsumerCoordinator coordinates the offline->consuming helix 
transitions.
+ */
+public class ConsumerCoordinator {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(ConsumerCoordinator.class);
+
+  private final Semaphore _semaphore;
+  private final boolean _enforceConsumptionInOrder;
+  private final Condition _condition;
+  private final Lock _lock;
+  private final ServerMetrics _serverMetrics;
+  private volatile int _maxSegmentSeqNumRegistered = -1;

Review Comment:
   (minor) Re-order variables. Put constant first, followed by `final`, then 
this one



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumerCoordinator.java:
##########
@@ -0,0 +1,302 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.Map;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.annotation.Nullable;
+import org.apache.helix.model.IdealState;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.metrics.ServerTimer;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.segment.local.data.manager.SegmentDataManager;
+import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The ConsumerCoordinator coordinates the offline->consuming helix 
transitions.
+ */
+public class ConsumerCoordinator {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(ConsumerCoordinator.class);
+
+  private final Semaphore _semaphore;
+  private final boolean _enforceConsumptionInOrder;
+  private final Condition _condition;
+  private final Lock _lock;
+  private final ServerMetrics _serverMetrics;
+  private volatile int _maxSegmentSeqNumRegistered = -1;
+  private final boolean _alwaysRelyOnIdealState;
+  private final RealtimeTableDataManager _realtimeTableDataManager;
+  private final AtomicBoolean _isFirstTransitionProcessed;
+  private static final long WAIT_INTERVAL_MS = TimeUnit.MINUTES.toMillis(3);
+
+  public ConsumerCoordinator(boolean enforceConsumptionInOrder, 
RealtimeTableDataManager realtimeTableDataManager) {
+    _semaphore = new Semaphore(1);
+    _lock = new ReentrantLock();
+    _condition = _lock.newCondition();
+    _enforceConsumptionInOrder = enforceConsumptionInOrder;
+    _realtimeTableDataManager = realtimeTableDataManager;
+    StreamIngestionConfig streamIngestionConfig = 
realtimeTableDataManager.getStreamIngestionConfig();
+    if (streamIngestionConfig != null) {
+      // if isUseIdealStateToCalculatePreviousSegment is true, server relies 
on ideal state to fetch previous segment
+      // to a segment for all helix transitions.
+      _alwaysRelyOnIdealState = 
streamIngestionConfig.isUseIdealStateToCalculatePreviousSegment();
+    } else {
+      _alwaysRelyOnIdealState = false;
+    }
+    _isFirstTransitionProcessed = new AtomicBoolean(false);
+    _serverMetrics = ServerMetrics.get();
+  }
+
+  void acquire(LLCSegmentName llcSegmentName)
+      throws InterruptedException {
+    long startTimeMs;
+
+    if (_enforceConsumptionInOrder) {
+      startTimeMs = System.currentTimeMillis();
+
+      waitForPrevSegment(llcSegmentName);
+
+      
_serverMetrics.addTimedTableValue(_realtimeTableDataManager.getTableName(),
+          ServerTimer.PREV_SEGMENT_WAIT_DURATION_MS, 
System.currentTimeMillis() - startTimeMs, TimeUnit.MILLISECONDS);
+    }
+
+    startTimeMs = System.currentTimeMillis();
+    while (!_semaphore.tryAcquire(WAIT_INTERVAL_MS, TimeUnit.MILLISECONDS)) {
+      LOGGER.warn("Failed to acquire partitionGroup consumer semaphore in: {} 
ms. Retrying.",
+          System.currentTimeMillis() - startTimeMs);
+    }
+  }
+
+  void release() {
+    _semaphore.release();
+  }
+
+  @VisibleForTesting
+  Semaphore getSemaphore() {
+    return _semaphore;
+  }
+
+  void trackSegment(LLCSegmentName llcSegmentName) {
+    _lock.lock();
+    try {
+      if (!_alwaysRelyOnIdealState) {
+        _maxSegmentSeqNumRegistered = Math.max(_maxSegmentSeqNumRegistered, 
llcSegmentName.getSequenceNumber());
+      }
+      // notify all helix threads waiting for their offline -> consuming 
segment's prev segment to be loaded
+      _condition.signalAll();
+    } finally {
+      _lock.unlock();
+    }
+  }
+
+  private void waitForPrevSegment(LLCSegmentName currSegment)
+      throws InterruptedException {
+
+    if (_alwaysRelyOnIdealState || !_isFirstTransitionProcessed.get()) {
+      // if _alwaysRelyOnIdealState or no offline -> consuming transition has 
been processed, it means rely on
+      // ideal state to fetch previous segment.
+      acquireSegmentRelyingOnIdealState(currSegment);
+
+      // the first transition will always be prone to error, consider edge 
case where segment previous to current
+      // helix transition's segment was deleted and this server came alive 
after successful deletion. the prev
+      // segment will not exist, hence first transition is handled using 
isFirstTransitionSuccessful.
+      _isFirstTransitionProcessed.compareAndSet(false, true);

Review Comment:
   (minor) `set()` should be enough



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumerCoordinator.java:
##########
@@ -0,0 +1,302 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.Map;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.annotation.Nullable;
+import org.apache.helix.model.IdealState;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.metrics.ServerTimer;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.segment.local.data.manager.SegmentDataManager;
+import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The ConsumerCoordinator coordinates the offline->consuming helix 
transitions.
+ */
+public class ConsumerCoordinator {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(ConsumerCoordinator.class);
+
+  private final Semaphore _semaphore;
+  private final boolean _enforceConsumptionInOrder;
+  private final Condition _condition;
+  private final Lock _lock;
+  private final ServerMetrics _serverMetrics;
+  private volatile int _maxSegmentSeqNumRegistered = -1;
+  private final boolean _alwaysRelyOnIdealState;
+  private final RealtimeTableDataManager _realtimeTableDataManager;
+  private final AtomicBoolean _isFirstTransitionProcessed;
+  private static final long WAIT_INTERVAL_MS = TimeUnit.MINUTES.toMillis(3);
+
+  public ConsumerCoordinator(boolean enforceConsumptionInOrder, 
RealtimeTableDataManager realtimeTableDataManager) {
+    _semaphore = new Semaphore(1);
+    _lock = new ReentrantLock();
+    _condition = _lock.newCondition();
+    _enforceConsumptionInOrder = enforceConsumptionInOrder;
+    _realtimeTableDataManager = realtimeTableDataManager;
+    StreamIngestionConfig streamIngestionConfig = 
realtimeTableDataManager.getStreamIngestionConfig();
+    if (streamIngestionConfig != null) {
+      // if isUseIdealStateToCalculatePreviousSegment is true, server relies 
on ideal state to fetch previous segment
+      // to a segment for all helix transitions.
+      _alwaysRelyOnIdealState = 
streamIngestionConfig.isUseIdealStateToCalculatePreviousSegment();
+    } else {
+      _alwaysRelyOnIdealState = false;
+    }
+    _isFirstTransitionProcessed = new AtomicBoolean(false);
+    _serverMetrics = ServerMetrics.get();
+  }
+
+  void acquire(LLCSegmentName llcSegmentName)
+      throws InterruptedException {
+    long startTimeMs;
+
+    if (_enforceConsumptionInOrder) {
+      startTimeMs = System.currentTimeMillis();
+
+      waitForPrevSegment(llcSegmentName);
+
+      
_serverMetrics.addTimedTableValue(_realtimeTableDataManager.getTableName(),
+          ServerTimer.PREV_SEGMENT_WAIT_DURATION_MS, 
System.currentTimeMillis() - startTimeMs, TimeUnit.MILLISECONDS);
+    }
+
+    startTimeMs = System.currentTimeMillis();
+    while (!_semaphore.tryAcquire(WAIT_INTERVAL_MS, TimeUnit.MILLISECONDS)) {
+      LOGGER.warn("Failed to acquire partitionGroup consumer semaphore in: {} 
ms. Retrying.",
+          System.currentTimeMillis() - startTimeMs);
+    }
+  }
+
+  void release() {
+    _semaphore.release();
+  }
+
+  @VisibleForTesting
+  Semaphore getSemaphore() {
+    return _semaphore;
+  }
+
+  void trackSegment(LLCSegmentName llcSegmentName) {
+    _lock.lock();
+    try {
+      if (!_alwaysRelyOnIdealState) {
+        _maxSegmentSeqNumRegistered = Math.max(_maxSegmentSeqNumRegistered, 
llcSegmentName.getSequenceNumber());
+      }
+      // notify all helix threads waiting for their offline -> consuming 
segment's prev segment to be loaded
+      _condition.signalAll();
+    } finally {
+      _lock.unlock();
+    }
+  }
+
+  private void waitForPrevSegment(LLCSegmentName currSegment)
+      throws InterruptedException {
+
+    if (_alwaysRelyOnIdealState || !_isFirstTransitionProcessed.get()) {
+      // if _alwaysRelyOnIdealState or no offline -> consuming transition has 
been processed, it means rely on
+      // ideal state to fetch previous segment.
+      acquireSegmentRelyingOnIdealState(currSegment);
+
+      // the first transition will always be prone to error, consider edge 
case where segment previous to current
+      // helix transition's segment was deleted and this server came alive 
after successful deletion. the prev
+      // segment will not exist, hence first transition is handled using 
isFirstTransitionSuccessful.
+      _isFirstTransitionProcessed.compareAndSet(false, true);
+      return;
+    }
+
+    // rely on _maxSegmentSeqNumRegistered watermark for previous segment.
+    if (awaitForPreviousSegmentSequenceNumber(currSegment, WAIT_INTERVAL_MS)) {
+      return;
+    }
+
+    // tried using prevSegSeqNumber watermark, but could not acquire the 
previous segment.
+    // fallback to acquire prev segment from ideal state.
+    acquireSegmentRelyingOnIdealState(currSegment);
+  }
+
+  private void acquireSegmentRelyingOnIdealState(LLCSegmentName currSegment)
+      throws InterruptedException {
+    LLCSegmentName previousLLCSegmentName = getPreviousSegment(currSegment);
+    if (previousLLCSegmentName == null) {
+      // previous segment can only be null if either all the previous segments 
are deleted or this is the starting
+      // sequence segment of the partition Group.
+      return;
+    }
+    String previousSegment = previousLLCSegmentName.getSegmentName();
+
+    SegmentDataManager segmentDataManager = 
_realtimeTableDataManager.acquireSegment(previousSegment);
+    try {
+      long startTimeMs = System.currentTimeMillis();
+      _lock.lock();
+      try {
+        while (segmentDataManager == null) {
+          // if segmentDataManager == null, it means segment is not loaded in 
the server.
+          // wait until it's loaded.
+          while (!_condition.await(WAIT_INTERVAL_MS, TimeUnit.MILLISECONDS)) {
+            LOGGER.warn("Semaphore access denied to segment: {}. Waiting on 
previous segment: {} since: {} ms.",
+                currSegment.getSegmentName(), previousSegment, 
System.currentTimeMillis() - startTimeMs);
+            // waited until timeout, fetch previous segment again from ideal 
state as previous segment might be
+            // changed in ideal state.
+            previousLLCSegmentName = getPreviousSegment(currSegment);
+            if (previousLLCSegmentName == null) {
+              return;
+            }
+            previousSegment = previousLLCSegmentName.getSegmentName();
+          }
+          segmentDataManager = 
_realtimeTableDataManager.acquireSegment(previousSegment);
+        }
+      } finally {
+        _lock.unlock();
+      }
+    } finally {
+      if (segmentDataManager != null) {
+        _realtimeTableDataManager.releaseSegment(segmentDataManager);
+      }
+    }
+  }
+
+  /***
+   * @param currSegment is the segment of current helix transition.
+   * @param timeoutMs is max time to wait in millis
+   * @return true if previous Segment was registered to the server, else false.
+   * @throws InterruptedException
+   */
+  @VisibleForTesting
+  boolean awaitForPreviousSegmentSequenceNumber(LLCSegmentName currSegment, 
long timeoutMs)
+      throws InterruptedException {
+    long startTimeMs = System.currentTimeMillis();
+    int prevSeqNum = currSegment.getSequenceNumber() - 1;
+    _lock.lock();
+    try {
+      while (_maxSegmentSeqNumRegistered < prevSeqNum) {

Review Comment:
   We should perform check twice, once before the wait and once after the wait. 
Currently it only checks once



##########
pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/StreamIngestionConfig.java:
##########
@@ -43,6 +43,13 @@ public class StreamIngestionConfig extends BaseJsonConfig {
   @JsonPropertyDescription("Whether pauseless consumption is enabled for the 
table")
   private boolean _pauselessConsumptionEnabled = false;
 
+  @JsonPropertyDescription("Enforce consumption of segments in order of 
segment creation by the controller")
+  private boolean _enforceConsumptionInOrder = false;
+
+  @JsonPropertyDescription("If enabled, Server always relies on ideal state to 
get previous segment. Else server uses"

Review Comment:
   If disabled, server uses sequence id - 1



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumerCoordinator.java:
##########
@@ -0,0 +1,302 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.Map;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.annotation.Nullable;
+import org.apache.helix.model.IdealState;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.metrics.ServerTimer;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.segment.local.data.manager.SegmentDataManager;
+import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The ConsumerCoordinator coordinates the offline->consuming helix 
transitions.
+ */
+public class ConsumerCoordinator {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(ConsumerCoordinator.class);
+
+  private final Semaphore _semaphore;
+  private final boolean _enforceConsumptionInOrder;
+  private final Condition _condition;
+  private final Lock _lock;
+  private final ServerMetrics _serverMetrics;
+  private volatile int _maxSegmentSeqNumRegistered = -1;
+  private final boolean _alwaysRelyOnIdealState;
+  private final RealtimeTableDataManager _realtimeTableDataManager;
+  private final AtomicBoolean _isFirstTransitionProcessed;

Review Comment:
   (nit)
   ```suggestion
     private final AtomicBoolean _firstTransitionProcessed;
   ```



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumerCoordinator.java:
##########
@@ -0,0 +1,302 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.Map;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.annotation.Nullable;
+import org.apache.helix.model.IdealState;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.metrics.ServerTimer;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.segment.local.data.manager.SegmentDataManager;
+import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The ConsumerCoordinator coordinates the offline->consuming helix 
transitions.
+ */
+public class ConsumerCoordinator {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(ConsumerCoordinator.class);
+
+  private final Semaphore _semaphore;
+  private final boolean _enforceConsumptionInOrder;
+  private final Condition _condition;
+  private final Lock _lock;
+  private final ServerMetrics _serverMetrics;
+  private volatile int _maxSegmentSeqNumRegistered = -1;
+  private final boolean _alwaysRelyOnIdealState;
+  private final RealtimeTableDataManager _realtimeTableDataManager;
+  private final AtomicBoolean _isFirstTransitionProcessed;
+  private static final long WAIT_INTERVAL_MS = TimeUnit.MINUTES.toMillis(3);
+
+  public ConsumerCoordinator(boolean enforceConsumptionInOrder, 
RealtimeTableDataManager realtimeTableDataManager) {
+    _semaphore = new Semaphore(1);
+    _lock = new ReentrantLock();
+    _condition = _lock.newCondition();
+    _enforceConsumptionInOrder = enforceConsumptionInOrder;
+    _realtimeTableDataManager = realtimeTableDataManager;
+    StreamIngestionConfig streamIngestionConfig = 
realtimeTableDataManager.getStreamIngestionConfig();
+    if (streamIngestionConfig != null) {
+      // if isUseIdealStateToCalculatePreviousSegment is true, server relies 
on ideal state to fetch previous segment
+      // to a segment for all helix transitions.
+      _alwaysRelyOnIdealState = 
streamIngestionConfig.isUseIdealStateToCalculatePreviousSegment();
+    } else {
+      _alwaysRelyOnIdealState = false;
+    }
+    _isFirstTransitionProcessed = new AtomicBoolean(false);
+    _serverMetrics = ServerMetrics.get();
+  }
+
+  void acquire(LLCSegmentName llcSegmentName)
+      throws InterruptedException {
+    long startTimeMs;
+
+    if (_enforceConsumptionInOrder) {
+      startTimeMs = System.currentTimeMillis();
+
+      waitForPrevSegment(llcSegmentName);
+
+      
_serverMetrics.addTimedTableValue(_realtimeTableDataManager.getTableName(),
+          ServerTimer.PREV_SEGMENT_WAIT_DURATION_MS, 
System.currentTimeMillis() - startTimeMs, TimeUnit.MILLISECONDS);
+    }
+
+    startTimeMs = System.currentTimeMillis();
+    while (!_semaphore.tryAcquire(WAIT_INTERVAL_MS, TimeUnit.MILLISECONDS)) {
+      LOGGER.warn("Failed to acquire partitionGroup consumer semaphore in: {} 
ms. Retrying.",
+          System.currentTimeMillis() - startTimeMs);
+    }
+  }
+
+  void release() {
+    _semaphore.release();
+  }
+
+  @VisibleForTesting
+  Semaphore getSemaphore() {
+    return _semaphore;
+  }
+
+  void trackSegment(LLCSegmentName llcSegmentName) {
+    _lock.lock();
+    try {
+      if (!_alwaysRelyOnIdealState) {
+        _maxSegmentSeqNumRegistered = Math.max(_maxSegmentSeqNumRegistered, 
llcSegmentName.getSequenceNumber());
+      }
+      // notify all helix threads waiting for their offline -> consuming 
segment's prev segment to be loaded
+      _condition.signalAll();
+    } finally {
+      _lock.unlock();
+    }
+  }
+
+  private void waitForPrevSegment(LLCSegmentName currSegment)
+      throws InterruptedException {
+
+    if (_alwaysRelyOnIdealState || !_isFirstTransitionProcessed.get()) {
+      // if _alwaysRelyOnIdealState or no offline -> consuming transition has 
been processed, it means rely on
+      // ideal state to fetch previous segment.
+      acquireSegmentRelyingOnIdealState(currSegment);
+
+      // the first transition will always be prone to error, consider edge 
case where segment previous to current
+      // helix transition's segment was deleted and this server came alive 
after successful deletion. the prev
+      // segment will not exist, hence first transition is handled using 
isFirstTransitionSuccessful.
+      _isFirstTransitionProcessed.compareAndSet(false, true);
+      return;
+    }
+
+    // rely on _maxSegmentSeqNumRegistered watermark for previous segment.
+    if (awaitForPreviousSegmentSequenceNumber(currSegment, WAIT_INTERVAL_MS)) {
+      return;
+    }
+
+    // tried using prevSegSeqNumber watermark, but could not acquire the 
previous segment.
+    // fallback to acquire prev segment from ideal state.
+    acquireSegmentRelyingOnIdealState(currSegment);
+  }
+
+  private void acquireSegmentRelyingOnIdealState(LLCSegmentName currSegment)
+      throws InterruptedException {
+    LLCSegmentName previousLLCSegmentName = getPreviousSegment(currSegment);
+    if (previousLLCSegmentName == null) {
+      // previous segment can only be null if either all the previous segments 
are deleted or this is the starting
+      // sequence segment of the partition Group.
+      return;
+    }
+    String previousSegment = previousLLCSegmentName.getSegmentName();
+
+    SegmentDataManager segmentDataManager = 
_realtimeTableDataManager.acquireSegment(previousSegment);
+    try {
+      long startTimeMs = System.currentTimeMillis();
+      _lock.lock();
+      try {
+        while (segmentDataManager == null) {
+          // if segmentDataManager == null, it means segment is not loaded in 
the server.
+          // wait until it's loaded.
+          while (!_condition.await(WAIT_INTERVAL_MS, TimeUnit.MILLISECONDS)) {
+            LOGGER.warn("Semaphore access denied to segment: {}. Waiting on 
previous segment: {} since: {} ms.",
+                currSegment.getSegmentName(), previousSegment, 
System.currentTimeMillis() - startTimeMs);
+            // waited until timeout, fetch previous segment again from ideal 
state as previous segment might be
+            // changed in ideal state.
+            previousLLCSegmentName = getPreviousSegment(currSegment);
+            if (previousLLCSegmentName == null) {
+              return;
+            }
+            previousSegment = previousLLCSegmentName.getSegmentName();
+          }
+          segmentDataManager = 
_realtimeTableDataManager.acquireSegment(previousSegment);
+        }
+      } finally {
+        _lock.unlock();
+      }
+    } finally {
+      if (segmentDataManager != null) {
+        _realtimeTableDataManager.releaseSegment(segmentDataManager);
+      }
+    }
+  }
+
+  /***
+   * @param currSegment is the segment of current helix transition.
+   * @param timeoutMs is max time to wait in millis
+   * @return true if previous Segment was registered to the server, else false.
+   * @throws InterruptedException
+   */
+  @VisibleForTesting
+  boolean awaitForPreviousSegmentSequenceNumber(LLCSegmentName currSegment, 
long timeoutMs)
+      throws InterruptedException {
+    long startTimeMs = System.currentTimeMillis();
+    int prevSeqNum = currSegment.getSequenceNumber() - 1;
+    _lock.lock();
+    try {
+      while (_maxSegmentSeqNumRegistered < prevSeqNum) {
+        // it means all segments until _maxSegmentSeqNumRegistered is not 
loaded in the server. Wait until it's loaded.
+        if (!_condition.await(timeoutMs, TimeUnit.MILLISECONDS)) {
+          LOGGER.warn("Semaphore access denied to segment: {}."
+                  + " Waiting on previous segment with sequence number: {} 
since: {} ms.", currSegment.getSegmentName(),
+              prevSeqNum, System.currentTimeMillis() - startTimeMs);
+          // waited until the timeout. Rely on ideal state now.
+          return false;
+        }
+      }
+
+      return (_maxSegmentSeqNumRegistered >= prevSeqNum);
+    } finally {
+      _lock.unlock();
+    }
+  }
+
+  @VisibleForTesting
+  @Nullable
+  LLCSegmentName getPreviousSegment(LLCSegmentName currSegment) {
+    long startTimeMs = System.currentTimeMillis();
+    // if seq num of current segment is 102, maxSequenceNumBelowCurrentSegment 
must be highest seq num of any segment
+    // created before current segment
+    int maxSequenceNumBelowCurrentSegment = -1;
+    LLCSegmentName previousSegment = null;
+    int currPartitionGroupId = currSegment.getPartitionGroupId();
+    int currSequenceNum = currSegment.getSequenceNumber();
+    Map<String, Map<String, String>> segmentAssignment = 
getSegmentAssignment();
+    String currentServerInstanceId = 
_realtimeTableDataManager.getServerInstance();
+
+    for (Map.Entry<String, Map<String, String>> entry : 
segmentAssignment.entrySet()) {
+      String segmentName = entry.getKey();
+      Map<String, String> instanceStateMap = entry.getValue();
+      String state = instanceStateMap.get(currentServerInstanceId);
+
+      if 
(!CommonConstants.Helix.StateModel.SegmentStateModel.ONLINE.equals(state)) {
+        // if server is looking for previous segment to current transition's 
segment, it means the previous segment
+        // has to be online in the instance. If all previous segments are not 
online, we just allow the current helix
+        // transition to go ahead.
+        continue;
+      }
+
+      LLCSegmentName llcSegmentName = LLCSegmentName.of(segmentName);
+      if (llcSegmentName == null) {
+        // can't compare with this segment, hence skip.

Review Comment:
   (minor)
   ```suggestion
           // ignore uploaded segments
   ```



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/ConsumerCoordinator.java:
##########
@@ -0,0 +1,302 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import java.util.Map;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.annotation.Nullable;
+import org.apache.helix.model.IdealState;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.metrics.ServerTimer;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+import org.apache.pinot.segment.local.data.manager.SegmentDataManager;
+import org.apache.pinot.spi.config.table.ingestion.StreamIngestionConfig;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The ConsumerCoordinator coordinates the offline->consuming helix 
transitions.
+ */
+public class ConsumerCoordinator {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(ConsumerCoordinator.class);
+
+  private final Semaphore _semaphore;
+  private final boolean _enforceConsumptionInOrder;
+  private final Condition _condition;
+  private final Lock _lock;
+  private final ServerMetrics _serverMetrics;
+  private volatile int _maxSegmentSeqNumRegistered = -1;
+  private final boolean _alwaysRelyOnIdealState;
+  private final RealtimeTableDataManager _realtimeTableDataManager;
+  private final AtomicBoolean _isFirstTransitionProcessed;
+  private static final long WAIT_INTERVAL_MS = TimeUnit.MINUTES.toMillis(3);
+
+  public ConsumerCoordinator(boolean enforceConsumptionInOrder, 
RealtimeTableDataManager realtimeTableDataManager) {
+    _semaphore = new Semaphore(1);
+    _lock = new ReentrantLock();
+    _condition = _lock.newCondition();
+    _enforceConsumptionInOrder = enforceConsumptionInOrder;
+    _realtimeTableDataManager = realtimeTableDataManager;
+    StreamIngestionConfig streamIngestionConfig = 
realtimeTableDataManager.getStreamIngestionConfig();
+    if (streamIngestionConfig != null) {
+      // if isUseIdealStateToCalculatePreviousSegment is true, server relies 
on ideal state to fetch previous segment
+      // to a segment for all helix transitions.
+      _alwaysRelyOnIdealState = 
streamIngestionConfig.isUseIdealStateToCalculatePreviousSegment();
+    } else {
+      _alwaysRelyOnIdealState = false;
+    }
+    _isFirstTransitionProcessed = new AtomicBoolean(false);
+    _serverMetrics = ServerMetrics.get();
+  }
+
+  void acquire(LLCSegmentName llcSegmentName)

Review Comment:
   Change the public facing method to `public`



##########
pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerTimer.java:
##########
@@ -63,6 +63,12 @@ public enum ServerTimer implements AbstractMetrics.Timer {
   SECONDARY_Q_WAIT_TIME_MS("milliseconds", false,
       "Time spent waiting in the secondary queue when BinaryWorkloadScheduler 
is used."),
 
+  PREV_SEGMENT_FETCH_IDEAL_STATE_DURATION_MS("milliseconds", false,
+      "Time spent while fetching previous segment from ideal state for any 
segment."),
+
+  PREV_SEGMENT_WAIT_DURATION_MS("milliseconds", false,
+      "Time spent while waiting on previous segment to be registered."),

Review Comment:
   For consistency:
   ```suggestion
     PREV_SEGMENT_FETCH_IDEAL_STATE_TIME_MS("milliseconds", false,
         "Time spent while fetching previous segment from ideal state for any 
segment."),
   
     PREV_SEGMENT_WAIT_TIME_MS("milliseconds", false,
         "Time spent while waiting on previous segment to be registered."),
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

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


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


Reply via email to