sajjad-moradi commented on code in PR #9244: URL: https://github.com/apache/pinot/pull/9244#discussion_r950995089
########## pinot-server/src/main/java/org/apache/pinot/server/starter/helix/FreshnessBasedConsumptionStatusChecker.java: ########## @@ -0,0 +1,154 @@ +/** + * 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.LongMsgOffset; +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 #getNumConsumingSegmentsNotReachedMinFreshness} - + * for each consuming segment, we check if either: + * - the segment's latest ingested offset has reached the current stream offset that's + * - the last ingested message is within {@link #_minFreshnessMs} of the current system time + */ +public class FreshnessBasedConsumptionStatusChecker { + private static final Logger LOGGER = LoggerFactory.getLogger(FreshnessBasedConsumptionStatusChecker.class); + + // constructor parameters + private final InstanceDataManager _instanceDataManager; + private final Set<String> _consumingSegments; + private final Long _minFreshnessMs; + + // helper variable + private final Set<String> _caughtUpSegments = new HashSet<>(); + + public FreshnessBasedConsumptionStatusChecker(InstanceDataManager instanceDataManager, Set<String> consumingSegments, + Long minFreshnessMs) { + _instanceDataManager = instanceDataManager; + _consumingSegments = consumingSegments; + _minFreshnessMs = minFreshnessMs; + } + + private boolean isOffsetCaughtUp(StreamPartitionMsgOffset currentOffset, StreamPartitionMsgOffset latestOffset) { + if (currentOffset != null && latestOffset != null) { + if (currentOffset.compareTo(latestOffset) == 0) { + return true; + } + long currentOffsetLong = ((LongMsgOffset) currentOffset).getOffset(); + long latestOffsetLong = ((LongMsgOffset) latestOffset).getOffset(); Review Comment: This we can't do. Not all streams have long offsets like Kafka does. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-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