Copilot commented on code in PR #17789:
URL: https://github.com/apache/pinot/pull/17789#discussion_r2868716583
##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/offline/OfflineTableDataManager.java:
##########
@@ -57,8 +101,122 @@ protected void doAddOnlineSegment(String segmentName)
}
}
+ @Override
+ public void addSegment(ImmutableSegment immutableSegment, @Nullable
SegmentZKMetadata zkMetadata) {
+ String segmentName = immutableSegment.getSegmentName();
+ Preconditions.checkState(!_shutDown,
+ "Table data manager is already shut down, cannot add segment: %s to
table: %s",
+ segmentName, _tableNameWithType);
+ if (isUpsertEnabled()) {
+ handleUpsert(immutableSegment, zkMetadata);
+ return;
+ }
+ super.addSegment(immutableSegment, zkMetadata);
+ }
+
+ @Override
+ public List<SegmentContext> getSegmentContexts(List<IndexSegment>
selectedSegments,
+ Map<String, String> queryOptions) {
+ List<SegmentContext> segmentContexts = new
ArrayList<>(selectedSegments.size());
+ selectedSegments.forEach(s -> segmentContexts.add(new SegmentContext(s)));
+ if (isUpsertEnabled() && !QueryOptionsUtils.isSkipUpsert(queryOptions)) {
+ _tableUpsertMetadataManager.setSegmentContexts(segmentContexts,
queryOptions);
+ }
+ return segmentContexts;
+ }
+
@Override
public void addConsumingSegment(String segmentName) {
throw new UnsupportedOperationException("Cannot add CONSUMING segment to
OFFLINE table");
}
+
+ public boolean isUpsertEnabled() {
+ return _tableUpsertMetadataManager != null;
+ }
+
+ @VisibleForTesting
+ public TableUpsertMetadataManager getTableUpsertMetadataManager() {
+ return _tableUpsertMetadataManager;
+ }
+
+ public Map<Integer, Long> getPartitionToPrimaryKeyCount() {
+ if (isUpsertEnabled()) {
+ return _tableUpsertMetadataManager.getPartitionToPrimaryKeyCount();
+ }
+ return Collections.emptyMap();
+ }
+
+ private void handleUpsert(ImmutableSegment immutableSegment, @Nullable
SegmentZKMetadata zkMetadata) {
+ String segmentName = immutableSegment.getSegmentName();
+ _logger.info("Adding immutable segment: {} with upsert enabled",
segmentName);
+
+ // Set the ZK creation time so that same creation time can be used to
break the comparison ties across replicas,
+ // to ensure data consistency of replica.
+ setZkCreationTimeIfAvailable(immutableSegment, zkMetadata);
+
+ Integer partitionId = SegmentUtils.getSegmentPartitionId(segmentName,
_tableNameWithType, _helixManager, null);
+ Preconditions.checkNotNull(partitionId, "Failed to get partition id for
segment: " + segmentName
+ + " (upsert-enabled table: " + _tableNameWithType + ")");
+ PartitionUpsertMetadataManager partitionUpsertMetadataManager =
+ _tableUpsertMetadataManager.getOrCreatePartitionManager(partitionId);
+
+ _serverMetrics.addValueToTableGauge(_tableNameWithType,
ServerGauge.DOCUMENT_COUNT,
+ immutableSegment.getSegmentMetadata().getTotalDocs());
+ _serverMetrics.addValueToTableGauge(_tableNameWithType,
ServerGauge.SEGMENT_COUNT, 1L);
+ ImmutableSegmentDataManager newSegmentManager = new
ImmutableSegmentDataManager(immutableSegment);
+ SegmentDataManager oldSegmentManager =
_segmentDataManagerMap.get(segmentName);
+ if (oldSegmentManager == null) {
+ // When adding a new segment, we should register it 'before' it is fully
initialized by
+ // partitionUpsertMetadataManager. Because when processing docs in the
new segment, the docs in the other
+ // segments may be invalidated, making the queries see less valid docs
than expected. We should let query
+ // access the new segment asap even though its validDocId bitmap is
still being filled by
+ // partitionUpsertMetadataManager.
+ registerSegment(segmentName, newSegmentManager,
partitionUpsertMetadataManager);
+ partitionUpsertMetadataManager.trackNewlyAddedSegment(segmentName);
+ partitionUpsertMetadataManager.addSegment(immutableSegment);
Review Comment:
Offline upsert path doesn’t handle the partition upsert metadata manager’s
preload mode. Unlike RealtimeTableDataManager, this code never calls
`preloadSegments(...)` and doesn’t branch on
`partitionUpsertMetadataManager.isPreloading()` to use `preloadSegment(...)`.
If preload is enabled in the upsert config, the preload flag may remain true
and segments will be processed via the slower add/replace path (and potentially
with different registration ordering than intended). Consider adding the
preload trigger (based on ZK metadata partition id) and a preload branch
mirroring the realtime implementation.
##########
pinot-core/src/main/java/org/apache/pinot/core/query/executor/SingleTableExecutionInfo.java:
##########
@@ -81,10 +82,9 @@ public static SingleTableExecutionInfo
create(InstanceDataManager instanceDataMa
indexSegments.add(segmentDataManager.getSegment());
}
} else {
- RealtimeTableDataManager rtdm = (RealtimeTableDataManager)
tableDataManager;
- TableUpsertMetadataManager tumm = rtdm.getTableUpsertMetadataManager();
+ TableUpsertMetadataManager tumm =
getTableUpsertMetadataManager(tableDataManager);
boolean isUsingConsistencyMode =
-
rtdm.getTableUpsertMetadataManager().getContext().getConsistencyMode() !=
UpsertConfig.ConsistencyMode.NONE;
+ tumm.getContext().getConsistencyMode() !=
UpsertConfig.ConsistencyMode.NONE;
if (isUsingConsistencyMode) {
Review Comment:
`getTableUpsertMetadataManager(...)` is annotated as nullable but its return
value is dereferenced unconditionally when `isUpsertTable(tableDataManager)` is
true. While current `isUpsertTable` checks make this safe, the nullable
contract is misleading and can mask future regressions (or trigger nullness
warnings). Consider enforcing non-null here (e.g.,
`Preconditions.checkState(tumm != null, ...)` / `Objects.requireNonNull`)
and/or making the helper return non-null for the supported manager types.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]