Jackie-Jiang commented on code in PR #12451: URL: https://github.com/apache/pinot/pull/12451#discussion_r1498371551
########## pinot-common/src/main/java/org/apache/pinot/common/utils/SegmentUtils.java: ########## @@ -49,6 +49,24 @@ public static Integer getRealtimeSegmentPartitionId(String segmentName, String r ZKMetadataProvider.getSegmentZKMetadata(helixManager.getHelixPropertyStore(), realtimeTableName, segmentName); Preconditions.checkState(segmentZKMetadata != null, "Failed to find segment ZK metadata for segment: %s of table: %s", segmentName, realtimeTableName); + return getRealtimeSegmentPartitionId(segmentZKMetadata, partitionColumn); + } + + @Nullable + public static Integer getRealtimeSegmentPartitionId(String segmentName, SegmentZKMetadata segmentZKMetadata, + @Nullable String partitionColumn) { + // A fast path if the segmentName is an LLC segment name: get the partition id from the name directly + LLCSegmentName llcSegmentName = LLCSegmentName.of(segmentName); + if (llcSegmentName != null) { + return llcSegmentName.getPartitionGroupId(); + } + // Otherwise, retrieve the partition id from the segment zk metadata. + return getRealtimeSegmentPartitionId(segmentZKMetadata, partitionColumn); + } + + @Nullable + public static Integer getRealtimeSegmentPartitionId(SegmentZKMetadata segmentZKMetadata, Review Comment: (minor) Make this private? We probably don't want user to directly use this method? ########## pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/BasePartitionUpsertMetadataManager.java: ########## @@ -102,6 +117,12 @@ public abstract class BasePartitionUpsertMetadataManager implements PartitionUps // Initialize with 1 pending operation to indicate the metadata manager can take more operations private int _numPendingOperations = 1; private boolean _closed; + // The lock and boolean flag below ensure only one thread can start preloading and preloading happens only once. + private final Lock _preloadLock = new ReentrantLock(); + private volatile boolean _isPreloaded = false; Review Comment: Why do we need 2 flags? Can we initialize `_isPreloading` based on whether preload is enabled? ########## pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java: ########## @@ -452,6 +460,12 @@ public void addSegment(String segmentName, IndexLoadingConfig indexLoadingConfig _serverMetrics.addValueToTableGauge(_tableNameWithType, ServerGauge.SEGMENT_COUNT, 1L); } + private boolean isReadyToPreload() { Review Comment: Rename this to `isUpsertPreloadEnabled()`. We don't need to check both upsert and this, checking only this should be enough. -- 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