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 2021/09/13 22:31:07 UTC

[GitHub] [pinot] mcvsubbu commented on a change in pull request #7267: Offset based realtime consumption status checker

mcvsubbu commented on a change in pull request #7267:
URL: https://github.com/apache/pinot/pull/7267#discussion_r707757823



##########
File path: pinot-server/src/main/java/org/apache/pinot/server/starter/helix/OffsetBasedConsumptionStatusChecker.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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.server.starter.helix;
+
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.core.data.manager.InstanceDataManager;
+import org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager;
+import org.apache.pinot.segment.local.data.manager.SegmentDataManager;
+import org.apache.pinot.segment.local.data.manager.TableDataManager;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.stream.StreamPartitionMsgOffset;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This class is used at startup time to have a more accurate estimate of the catchup period in which no query execution
+ * happens and consumers try to catch up to the latest messages available in streams.
+ * To achieve this, every time status check is called, {@link #haveAllConsumingSegmentsReachedStreamLatestOffset},
+ * for each consuming segment, we check if segment's latest ingested offset has reached the latest stream offset.
+ * To prevent chasing a moving target, once the latest stream offset is fetched, it will not be fetched again and
+ * subsequent status check calls compare latest ingested offset with the already fetched stream offset.
+ */
+public class OffsetBasedConsumptionStatusChecker {
+  private static final Logger LOGGER = LoggerFactory.getLogger(OffsetBasedConsumptionStatusChecker.class);
+
+  // constructor parameters
+  private final InstanceDataManager _instanceDataManager;
+  private final Set<String> _consumingSegments;
+
+  // helper variable
+  private final Set<String> _caughtUpSegments = new HashSet<>();
+
+  public OffsetBasedConsumptionStatusChecker(InstanceDataManager instanceDataManager, Set<String> consumingSegments) {
+    _instanceDataManager = instanceDataManager;
+    _consumingSegments = consumingSegments;
+  }
+
+  public boolean haveAllConsumingSegmentsReachedStreamLatestOffset() {
+    boolean allSegsReachedLatest = true;
+    for (String segName : _consumingSegments) {
+      if (_caughtUpSegments.contains(segName)) {
+        continue;
+      }
+      TableDataManager tableDataManager = getTableDataManager(segName);
+      if (tableDataManager == null) {
+        LOGGER.info("TableDataManager is not yet setup for segment {}. Will check consumption status later", segName);
+        return false;
+      }
+      SegmentDataManager segmentDataManager = tableDataManager.acquireSegment(segName);
+      if (segmentDataManager == null) {
+        LOGGER.info("SegmentDataManager is not yet setup for segment {}. Will check consumption status later", segName);
+        return false;
+      }
+      if (!(segmentDataManager instanceof LLRealtimeSegmentDataManager)) {
+        // There's a possibility that a consuming segment has converted to a committed segment. If that's the case,
+        // segment data manager will not be of type LLRealtime.
+        LOGGER.info("Segment {} is already committed and is considered caught up.", segName);
+        _caughtUpSegments.add(segName);
+        releaseSegment(tableDataManager, segmentDataManager);
+        continue;
+      }
+      LLRealtimeSegmentDataManager rtSegmentDataManager = (LLRealtimeSegmentDataManager) segmentDataManager;
+      StreamPartitionMsgOffset latestIngestedOffset = rtSegmentDataManager.getCurrentOffset();
+      StreamPartitionMsgOffset latestStreamOffset = rtSegmentDataManager.getLatestStreamOffsetAtStartupTime();
+      releaseSegment(tableDataManager, segmentDataManager);

Review comment:
       Isnt it better to do release in a try/finally, so that in case there are some exceptions, we still release the segment?

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/ServiceStatus.java
##########
@@ -236,13 +242,24 @@ public synchronized Status getServiceStatus() {
         return _serviceStatus;
       }
       long now = System.currentTimeMillis();
-      if (now < _endWaitTime) {
-        _statusDescription =
-            String.format("Waiting for consuming segments to catchup, timeRemaining=%dms", _endWaitTime - now);
-        return Status.STARTING;
+      if (now >= _endWaitTime) {
+        _statusDescription = String.format("Consuming segments status GOOD since %dms", _endWaitTime);
+        return Status.GOOD;
       }
-      _statusDescription = String.format("Consuming segments status GOOD since %dms", _endWaitTime);
-      return Status.GOOD;
+      if (_consumptionNotYetCaughtUp && _allConsumingSegmentsHaveReachedLatestOffset.get()) {
+        // TODO: Once the performance of offset based consumption checker is validated:
+        //      - remove the log line
+        //      - uncomment the status & statusDescription lines
+        //      - remove boolean flag _consumptionNotYetCaughtUp
+        _consumptionNotYetCaughtUp = false;
+        LOGGER.info("All consuming segments have reached their latest offsets! "
+            + "Finished {} msec earlier than time threshold.", _endWaitTime - now);
+//      _statusDescription = "Consuming segments status GOOD as all consuming segments have reached the latest offset";
+//      return Status.GOOD;
+      }
+      _statusDescription =
+          String.format("Waiting for consuming segments to catchup, timeRemaining=%dms", _endWaitTime - now);

Review comment:
       suggest adding the value of _consumptionNotYetCaughtUp to the status description. Or, even the number of segments yet to catch up.

##########
File path: pinot-server/src/main/java/org/apache/pinot/server/starter/helix/OffsetBasedConsumptionStatusChecker.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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.server.starter.helix;
+
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.core.data.manager.InstanceDataManager;
+import org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager;
+import org.apache.pinot.segment.local.data.manager.SegmentDataManager;
+import org.apache.pinot.segment.local.data.manager.TableDataManager;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.stream.StreamPartitionMsgOffset;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This class is used at startup time to have a more accurate estimate of the catchup period in which no query execution
+ * happens and consumers try to catch up to the latest messages available in streams.
+ * To achieve this, every time status check is called, {@link #haveAllConsumingSegmentsReachedStreamLatestOffset},
+ * for each consuming segment, we check if segment's latest ingested offset has reached the latest stream offset.
+ * To prevent chasing a moving target, once the latest stream offset is fetched, it will not be fetched again and
+ * subsequent status check calls compare latest ingested offset with the already fetched stream offset.
+ */
+public class OffsetBasedConsumptionStatusChecker {
+  private static final Logger LOGGER = LoggerFactory.getLogger(OffsetBasedConsumptionStatusChecker.class);
+
+  // constructor parameters
+  private final InstanceDataManager _instanceDataManager;
+  private final Set<String> _consumingSegments;
+
+  // helper variable
+  private final Set<String> _caughtUpSegments = new HashSet<>();
+
+  public OffsetBasedConsumptionStatusChecker(InstanceDataManager instanceDataManager, Set<String> consumingSegments) {
+    _instanceDataManager = instanceDataManager;
+    _consumingSegments = consumingSegments;
+  }
+
+  public boolean haveAllConsumingSegmentsReachedStreamLatestOffset() {

Review comment:
       Maybe returning the number of segments yet to reach offset will give us more information? Just a thought. But that means you cannot return early, so it is a double-edged sword.

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/ServiceStatus.java
##########
@@ -236,13 +242,24 @@ public synchronized Status getServiceStatus() {
         return _serviceStatus;
       }
       long now = System.currentTimeMillis();
-      if (now < _endWaitTime) {
-        _statusDescription =
-            String.format("Waiting for consuming segments to catchup, timeRemaining=%dms", _endWaitTime - now);
-        return Status.STARTING;
+      if (now >= _endWaitTime) {
+        _statusDescription = String.format("Consuming segments status GOOD since %dms", _endWaitTime);

Review comment:
       suggest adding the value of `_consumptionNotYetCaughtUp` to the status description. Or, even the number of segments yet to catch up.

##########
File path: pinot-server/src/test/java/org/apache/pinot/server/starter/helix/OffsetBasedConsumptionStatusCheckerTest.java
##########
@@ -0,0 +1,252 @@
+/**
+ * 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.server.starter.helix;
+
+import com.google.common.collect.ImmutableSet;
+import java.util.Set;
+import org.apache.pinot.core.data.manager.InstanceDataManager;
+import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager;
+import org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager;
+import org.apache.pinot.segment.local.data.manager.TableDataManager;
+import org.apache.pinot.spi.stream.LongMsgOffset;
+import org.testng.annotations.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.*;
+
+
+public class OffsetBasedConsumptionStatusCheckerTest {
+
+  @Test
+  public void regularCase() {
+
+    String segA0 = "tableA__0__0__123Z";
+    String segA1 = "tableA__1__0__123Z";
+    String segB0 = "tableB__0__0__123Z";
+    Set<String> consumingSegments = ImmutableSet.of(segA0, segA1, segB0);
+    InstanceDataManager instanceDataManager = mock(InstanceDataManager.class);
+    OffsetBasedConsumptionStatusChecker statusChecker =
+        new OffsetBasedConsumptionStatusChecker(instanceDataManager, consumingSegments);
+
+    // setup TableDataMangers
+    TableDataManager tableDataManagerA = mock(TableDataManager.class);
+    TableDataManager tableDataManagerB = mock(TableDataManager.class);
+    when(instanceDataManager.getTableDataManager("tableA_REALTIME")).thenReturn(tableDataManagerA);
+    when(instanceDataManager.getTableDataManager("tableB_REALTIME")).thenReturn(tableDataManagerB);
+
+    // setup SegmentDataManagers
+    LLRealtimeSegmentDataManager segMngrA0 = mock(LLRealtimeSegmentDataManager.class);
+    LLRealtimeSegmentDataManager segMngrA1 = mock(LLRealtimeSegmentDataManager.class);
+    LLRealtimeSegmentDataManager segMngrB0 = mock(LLRealtimeSegmentDataManager.class);
+    when(tableDataManagerA.acquireSegment(segA0)).thenReturn(segMngrA0);
+    when(tableDataManagerA.acquireSegment(segA1)).thenReturn(segMngrA1);
+    when(tableDataManagerB.acquireSegment(segB0)).thenReturn(segMngrB0);
+    when(segMngrA0.getLatestStreamOffsetAtStartupTime()).thenReturn(new LongMsgOffset(15));
+    when(segMngrA1.getLatestStreamOffsetAtStartupTime()).thenReturn(new LongMsgOffset(150));
+    when(segMngrB0.getLatestStreamOffsetAtStartupTime()).thenReturn(new LongMsgOffset(1500));
+
+    //           latest ingested offset    latest stream offset
+    // segA0              10                       15
+    // segA1              100                      150
+    // segB0              1000                     1500
+    when(segMngrA0.getCurrentOffset()).thenReturn(new LongMsgOffset(10));
+    when(segMngrA1.getCurrentOffset()).thenReturn(new LongMsgOffset(100));
+    when(segMngrB0.getCurrentOffset()).thenReturn(new LongMsgOffset(1000));
+    assertFalse(statusChecker.haveAllConsumingSegmentsReachedStreamLatestOffset());

Review comment:
       will be nice to also call the `haveAll` method when one or two segments have reached offset but another has not.
   
   but i see you have a case like that below, so it is ok

##########
File path: pinot-server/src/main/java/org/apache/pinot/server/starter/helix/OffsetBasedConsumptionStatusChecker.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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.server.starter.helix;
+
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.core.data.manager.InstanceDataManager;
+import org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager;
+import org.apache.pinot.segment.local.data.manager.SegmentDataManager;
+import org.apache.pinot.segment.local.data.manager.TableDataManager;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.stream.StreamPartitionMsgOffset;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This class is used at startup time to have a more accurate estimate of the catchup period in which no query execution
+ * happens and consumers try to catch up to the latest messages available in streams.
+ * To achieve this, every time status check is called, {@link #haveAllConsumingSegmentsReachedStreamLatestOffset},
+ * for each consuming segment, we check if segment's latest ingested offset has reached the latest stream offset.
+ * To prevent chasing a moving target, once the latest stream offset is fetched, it will not be fetched again and
+ * subsequent status check calls compare latest ingested offset with the already fetched stream offset.
+ */
+public class OffsetBasedConsumptionStatusChecker {
+  private static final Logger LOGGER = LoggerFactory.getLogger(OffsetBasedConsumptionStatusChecker.class);
+
+  // constructor parameters
+  private final InstanceDataManager _instanceDataManager;
+  private final Set<String> _consumingSegments;
+
+  // helper variable
+  private final Set<String> _caughtUpSegments = new HashSet<>();
+
+  public OffsetBasedConsumptionStatusChecker(InstanceDataManager instanceDataManager, Set<String> consumingSegments) {
+    _instanceDataManager = instanceDataManager;
+    _consumingSegments = consumingSegments;
+  }
+
+  public boolean haveAllConsumingSegmentsReachedStreamLatestOffset() {
+    boolean allSegsReachedLatest = true;
+    for (String segName : _consumingSegments) {
+      if (_caughtUpSegments.contains(segName)) {
+        continue;
+      }
+      TableDataManager tableDataManager = getTableDataManager(segName);
+      if (tableDataManager == null) {
+        LOGGER.info("TableDataManager is not yet setup for segment {}. Will check consumption status later", segName);
+        return false;
+      }
+      SegmentDataManager segmentDataManager = tableDataManager.acquireSegment(segName);
+      if (segmentDataManager == null) {
+        LOGGER.info("SegmentDataManager is not yet setup for segment {}. Will check consumption status later", segName);
+        return false;
+      }
+      if (!(segmentDataManager instanceof LLRealtimeSegmentDataManager)) {
+        // There's a possibility that a consuming segment has converted to a committed segment. If that's the case,
+        // segment data manager will not be of type LLRealtime.
+        LOGGER.info("Segment {} is already committed and is considered caught up.", segName);
+        _caughtUpSegments.add(segName);
+        releaseSegment(tableDataManager, segmentDataManager);
+        continue;
+      }
+      LLRealtimeSegmentDataManager rtSegmentDataManager = (LLRealtimeSegmentDataManager) segmentDataManager;
+      StreamPartitionMsgOffset latestIngestedOffset = rtSegmentDataManager.getCurrentOffset();
+      StreamPartitionMsgOffset latestStreamOffset = rtSegmentDataManager.getLatestStreamOffsetAtStartupTime();
+      releaseSegment(tableDataManager, segmentDataManager);
+      if (latestStreamOffset == null || latestIngestedOffset == null) {
+        LOGGER.info("Null offset found for segment {} - latest stream offset: {}, latest ingested offset: {}. "
+            + "Will check consumption status later", segName, latestStreamOffset, latestIngestedOffset);
+        return false;
+      }
+      if (latestIngestedOffset.compareTo(latestStreamOffset) < 0) {
+        LOGGER.info("Latest ingested offset {} in segment {} is smaller than stream latest available offset {} ",
+            latestIngestedOffset, segName, latestStreamOffset);
+        allSegsReachedLatest = false;
+        continue;
+      }
+      LOGGER.info("Segment {} with latest ingested offset {} has caught up to the latest stream offset {}", segName,
+            latestIngestedOffset, latestStreamOffset);
+      _caughtUpSegments.add(segName);
+    }
+    return allSegsReachedLatest;
+  }
+
+  void releaseSegment(TableDataManager tableDataManager, SegmentDataManager segmentDataManager) {

Review comment:
       why a separate 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