jackjlli commented on a change in pull request #7267: URL: https://github.com/apache/pinot/pull/7267#discussion_r712379434
########## File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java ########## @@ -1363,6 +1369,16 @@ public LLRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableCo } _state = State.INITIAL_CONSUMING; + // fetch latest stream offset + try (StreamMetadataProvider metadataProvider = _streamConsumerFactory + .createPartitionMetadataProvider(_clientId, _partitionGroupId)) { + _latestStreamOffsetAtStartupTime = metadataProvider + .fetchStreamPartitionOffset(OffsetCriteria.LARGEST_OFFSET_CRITERIA, /*maxWaitTimeMs*/5000); + } catch (Exception e) { + _segmentLogger.warn("Cannot fetch latest stream offset for clientId {} and partitionGroupId {}", _clientId, + _partitionGroupId); Review comment: would it be good to log the exception here? ########## File path: pinot-server/src/main/java/org/apache/pinot/server/starter/helix/OffsetBasedConsumptionStatusChecker.java ########## @@ -0,0 +1,114 @@ +/** + * 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 #getNumConsumingSegmentsNotReachedTheirLatestOffset} - + * for each consuming segment, we check if segment's latest ingested offset has reached the latest stream offset that's + * fetched once at startup time. + */ +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 int getNumConsumingSegmentsNotReachedTheirLatestOffset() { + 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); + continue; + } + SegmentDataManager segmentDataManager = null; + try { + segmentDataManager = tableDataManager.acquireSegment(segName); + if (segmentDataManager == null) { + LOGGER + .info("SegmentDataManager is not yet setup for segment {}. Will check consumption status later", segName); + continue; + } + 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); + continue; + } + LLRealtimeSegmentDataManager rtSegmentDataManager = (LLRealtimeSegmentDataManager) segmentDataManager; + StreamPartitionMsgOffset latestIngestedOffset = rtSegmentDataManager.getCurrentOffset(); + StreamPartitionMsgOffset latestStreamOffset = rtSegmentDataManager.getLatestStreamOffsetAtStartupTime(); + if (latestStreamOffset == null || latestIngestedOffset == null) { + LOGGER.info("Null offset found for segment {} - latest stream offset: {}, latest ingested offset: {}. " Review comment: It seems these message will be printed per segment. How about collecting all the segment names and group them into different categories, so that they will be printed once once? Another approaches is to change the log level. ########## File path: pinot-server/src/test/java/org/apache/pinot/server/starter/helix/OffsetBasedConsumptionStatusCheckerTest.java ########## @@ -0,0 +1,251 @@ +/** + * 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)); + assertEquals(statusChecker.getNumConsumingSegmentsNotReachedTheirLatestOffset(), 3); + + // latest ingested offset latest stream offset + // segA0 20 15 + // segA1 200 150 + // segB0 2000 1500 + when(segMngrA0.getCurrentOffset()).thenReturn(new LongMsgOffset(20)); + when(segMngrA1.getCurrentOffset()).thenReturn(new LongMsgOffset(200)); + when(segMngrB0.getCurrentOffset()).thenReturn(new LongMsgOffset(2000)); + assertEquals(statusChecker.getNumConsumingSegmentsNotReachedTheirLatestOffset(), 0); + } + + @Test + public void dataMangersBeingSetup() { + + 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); + + // TableDataManager is not set up yet + assertEquals(statusChecker.getNumConsumingSegmentsNotReachedTheirLatestOffset(), 3); + + // 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 some SegmentDataManagers + LLRealtimeSegmentDataManager segMngrA0 = mock(LLRealtimeSegmentDataManager.class); + LLRealtimeSegmentDataManager segMngrA1 = mock(LLRealtimeSegmentDataManager.class); + when(tableDataManagerA.acquireSegment(segA0)).thenReturn(segMngrA0); + when(tableDataManagerA.acquireSegment(segA1)).thenReturn(segMngrA1); + + // latest ingested offset latest stream offset + // segA0 10 15 + // segA1 100 150 + // segB0 not setup yet 1500 + when(segMngrA0.getCurrentOffset()).thenReturn(new LongMsgOffset(10)); + when(segMngrA1.getCurrentOffset()).thenReturn(new LongMsgOffset(100)); + when(segMngrA0.getLatestStreamOffsetAtStartupTime()).thenReturn(new LongMsgOffset(15)); + when(segMngrA1.getLatestStreamOffsetAtStartupTime()).thenReturn(new LongMsgOffset(150)); + assertEquals(statusChecker.getNumConsumingSegmentsNotReachedTheirLatestOffset(), 3); + + // setup the remaining SegmentDataManager + LLRealtimeSegmentDataManager segMngrB0 = mock(LLRealtimeSegmentDataManager.class); + when(tableDataManagerB.acquireSegment(segB0)).thenReturn(segMngrB0); + + // latest ingested offset latest stream offset + // segA0 20 15 + // segA1 200 150 + // segB0 1000 1500 + when(segMngrA0.getCurrentOffset()).thenReturn(new LongMsgOffset(20)); + when(segMngrA1.getCurrentOffset()).thenReturn(new LongMsgOffset(200)); + when(segMngrB0.getCurrentOffset()).thenReturn(new LongMsgOffset(1000)); + when(segMngrB0.getLatestStreamOffsetAtStartupTime()).thenReturn(new LongMsgOffset(1500)); + assertEquals(statusChecker.getNumConsumingSegmentsNotReachedTheirLatestOffset(), 1); + + // latest ingested offset latest stream offset + // segA0 30 15 + // segA1 300 150 + // segB0 2000 1500 + when(segMngrA0.getCurrentOffset()).thenReturn(new LongMsgOffset(30)); + when(segMngrA1.getCurrentOffset()).thenReturn(new LongMsgOffset(300)); + when(segMngrB0.getCurrentOffset()).thenReturn(new LongMsgOffset(2000)); + assertEquals(statusChecker.getNumConsumingSegmentsNotReachedTheirLatestOffset(), 0); + } + + @Test + public void segmentsBeingCommitted() { + + 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); + + // 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)); + when(segMngrA0.getLatestStreamOffsetAtStartupTime()).thenReturn(new LongMsgOffset(15)); + when(segMngrA1.getLatestStreamOffsetAtStartupTime()).thenReturn(new LongMsgOffset(150)); + when(segMngrB0.getLatestStreamOffsetAtStartupTime()).thenReturn(new LongMsgOffset(1500)); + assertEquals(statusChecker.getNumConsumingSegmentsNotReachedTheirLatestOffset(), 3); + + // segB0 is now committed; ImmutableSegmentDataManager is returned by table data manager + ImmutableSegmentDataManager immSegMngrB0 = mock(ImmutableSegmentDataManager.class); + when(tableDataManagerB.acquireSegment(segB0)).thenReturn(immSegMngrB0); + + // latest ingested offset latest stream offset + // segA0 20 15 + // segA1 200 150 + // segB0 committed at 1200 1500 + when(segMngrA0.getCurrentOffset()).thenReturn(new LongMsgOffset(20)); + when(segMngrA1.getCurrentOffset()).thenReturn(new LongMsgOffset(200)); + assertEquals(statusChecker.getNumConsumingSegmentsNotReachedTheirLatestOffset(), 0); + } + + @Test + public void cannotGetLatestStreamOffset() { + + 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); + + // latest ingested offset latest stream offset + // segA0 10 15 + // segA1 100 150 + // segB0 1000 null - could not get the latest offset from stream at startup + when(segMngrA0.getCurrentOffset()).thenReturn(new LongMsgOffset(10)); + when(segMngrA1.getCurrentOffset()).thenReturn(new LongMsgOffset(100)); + when(segMngrB0.getCurrentOffset()).thenReturn(new LongMsgOffset(1000)); + when(segMngrA0.getLatestStreamOffsetAtStartupTime()).thenReturn(new LongMsgOffset(15)); + when(segMngrA1.getLatestStreamOffsetAtStartupTime()).thenReturn(new LongMsgOffset(150)); + when(segMngrB0.getLatestStreamOffsetAtStartupTime()).thenReturn(null); + assertEquals(statusChecker.getNumConsumingSegmentsNotReachedTheirLatestOffset(), 3); + + // latest ingested offset latest stream offset + // segA0 20 15 + // segA1 200 150 + // segB0 2000 null - could not get the latest offset from stream at startup + when(segMngrA0.getCurrentOffset()).thenReturn(new LongMsgOffset(20)); + when(segMngrA1.getCurrentOffset()).thenReturn(new LongMsgOffset(200)); + when(segMngrB0.getCurrentOffset()).thenReturn(new LongMsgOffset(2000)); + assertEquals(statusChecker.getNumConsumingSegmentsNotReachedTheirLatestOffset(), 1); + + // latest ingested offset latest stream offset + // segA0 30 15 + // segA1 300 150 + // segB0 3000 null - could not get the latest offset from stream at startup + when(segMngrA0.getCurrentOffset()).thenReturn(new LongMsgOffset(30)); + when(segMngrA1.getCurrentOffset()).thenReturn(new LongMsgOffset(300)); + when(segMngrB0.getCurrentOffset()).thenReturn(new LongMsgOffset(3000)); + assertEquals(statusChecker.getNumConsumingSegmentsNotReachedTheirLatestOffset(), 1); + } +} Review comment: Missing tail empty line. ########## File path: pinot-common/src/main/java/org/apache/pinot/common/utils/ServiceStatus.java ########## @@ -236,13 +242,27 @@ 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; + int numConsumingSegmentsNotCaughtUp = _getNumConsumingSegmentsNotReachedTheirLatestOffset.get(); + if (now >= _endWaitTime) { + _statusDescription = String.format("Consuming segments status GOOD since %dms " + + "(numConsumingSegmentsNotCaughtUp=%d)", _endWaitTime, numConsumingSegmentsNotCaughtUp); + return Status.GOOD; } - _statusDescription = String.format("Consuming segments status GOOD since %dms", _endWaitTime); - return Status.GOOD; + if (_consumptionNotYetCaughtUp && numConsumingSegmentsNotCaughtUp > 0) { + // TODO: Once the performance of offset based consumption checker is validated: + // - remove the log line + // - uncomment the status & statusDescription lines + // - remove variable _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; Review comment: Shouldn't we uncomment it 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-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