Copilot commented on code in PR #17868:
URL: https://github.com/apache/pinot/pull/17868#discussion_r2924519358


##########
pinot-core/src/main/java/org/apache/pinot/core/query/executor/LogicalTableExecutionInfo.java:
##########
@@ -79,19 +82,71 @@ public boolean hasRealtime() {
         .anyMatch(tableExecutionInfo -> 
tableExecutionInfo.getTableDataManager() instanceof RealtimeTableDataManager);
   }
 
+  /**
+   * Returns selected segments and contexts for the logical table. Unlike 
single-table execution, this collects
+   * all segments from every physical table, runs segment pruning once on the 
combined list (cross-table prune),
+   * then resolves segment contexts per table. This allows pruners such as 
SelectionQuerySegmentPruner (ORDER BY +
+   * LIMIT) to prune effectively across the logical table rather than per 
physical table.
+   */
   @Override
   public SelectedSegmentsInfo getSelectedSegmentsInfo(QueryContext 
queryContext, TimerContext timerContext,
       ExecutorService executorService, SegmentPrunerService 
segmentPrunerService) {
-    SelectedSegmentsInfo aggregatedSelectedSegmentsInfo = new 
SelectedSegmentsInfo();
-
+    // Collect all segments from all physical tables (no pruning yet)
+    List<IndexSegment> allSegments = new ArrayList<>();
+    Map<IndexSegment, SingleTableExecutionInfo> segmentToTable = new 
HashMap<>();
+    long numTotalDocs = 0;
     for (SingleTableExecutionInfo tableExecutionInfo : _tableExecutionInfos) {
-      SelectedSegmentsInfo selectedSegmentsInfo =
-          tableExecutionInfo.getSelectedSegmentsInfo(queryContext, 
timerContext, executorService, segmentPrunerService);
-      aggregatedSelectedSegmentsInfo.aggregate(selectedSegmentsInfo);
+      List<IndexSegment> indexSegments = tableExecutionInfo.getIndexSegments();
+      for (IndexSegment segment : indexSegments) {
+        allSegments.add(segment);
+        segmentToTable.put(segment, tableExecutionInfo);
+        numTotalDocs += segment.getSegmentMetadata().getTotalDocs();
+      }
+    }
+    int numTotalSegments = allSegments.size();
+
+    // Constant false shortcut: skip pruning
+    List<IndexSegment> selectedSegments;
+    SegmentPrunerStatistics prunerStats = new SegmentPrunerStatistics();
+    if ((queryContext.getFilter() != null && 
queryContext.getFilter().isConstantFalse())
+        || (queryContext.getHavingFilter() != null && 
queryContext.getHavingFilter().isConstantFalse())) {
+      selectedSegments = Collections.emptyList();
+    } else {
+      TimerContext.Timer segmentPruneTimer = 
timerContext.startNewPhaseTimer(ServerQueryPhase.SEGMENT_PRUNING);
+      selectedSegments = segmentPrunerService.prune(allSegments, queryContext, 
prunerStats, executorService);
+      segmentPruneTimer.stopAndRecord();

Review Comment:
   `segmentPruneTimer.stopAndRecord()` is not guaranteed to run if 
`segmentPrunerService.prune(...)` throws, which can skew phase timing metrics 
and leave timers unrecorded. Wrap the prune call in a try/finally so the timer 
is always stopped/recorded (or use a closeable/auto-close pattern if available 
in `TimerContext.Timer`).
   ```suggestion
         try {
           selectedSegments = segmentPrunerService.prune(allSegments, 
queryContext, prunerStats, executorService);
         } finally {
           segmentPruneTimer.stopAndRecord();
         }
   ```



##########
pinot-core/src/main/java/org/apache/pinot/core/query/executor/LogicalTableExecutionInfo.java:
##########
@@ -79,19 +82,71 @@ public boolean hasRealtime() {
         .anyMatch(tableExecutionInfo -> 
tableExecutionInfo.getTableDataManager() instanceof RealtimeTableDataManager);
   }
 
+  /**
+   * Returns selected segments and contexts for the logical table. Unlike 
single-table execution, this collects
+   * all segments from every physical table, runs segment pruning once on the 
combined list (cross-table prune),
+   * then resolves segment contexts per table. This allows pruners such as 
SelectionQuerySegmentPruner (ORDER BY +
+   * LIMIT) to prune effectively across the logical table rather than per 
physical table.
+   */
   @Override
   public SelectedSegmentsInfo getSelectedSegmentsInfo(QueryContext 
queryContext, TimerContext timerContext,
       ExecutorService executorService, SegmentPrunerService 
segmentPrunerService) {
-    SelectedSegmentsInfo aggregatedSelectedSegmentsInfo = new 
SelectedSegmentsInfo();
-
+    // Collect all segments from all physical tables (no pruning yet)
+    List<IndexSegment> allSegments = new ArrayList<>();
+    Map<IndexSegment, SingleTableExecutionInfo> segmentToTable = new 
HashMap<>();
+    long numTotalDocs = 0;
     for (SingleTableExecutionInfo tableExecutionInfo : _tableExecutionInfos) {
-      SelectedSegmentsInfo selectedSegmentsInfo =
-          tableExecutionInfo.getSelectedSegmentsInfo(queryContext, 
timerContext, executorService, segmentPrunerService);
-      aggregatedSelectedSegmentsInfo.aggregate(selectedSegmentsInfo);
+      List<IndexSegment> indexSegments = tableExecutionInfo.getIndexSegments();
+      for (IndexSegment segment : indexSegments) {
+        allSegments.add(segment);
+        segmentToTable.put(segment, tableExecutionInfo);
+        numTotalDocs += segment.getSegmentMetadata().getTotalDocs();
+      }
+    }
+    int numTotalSegments = allSegments.size();
+
+    // Constant false shortcut: skip pruning
+    List<IndexSegment> selectedSegments;
+    SegmentPrunerStatistics prunerStats = new SegmentPrunerStatistics();
+    if ((queryContext.getFilter() != null && 
queryContext.getFilter().isConstantFalse())
+        || (queryContext.getHavingFilter() != null && 
queryContext.getHavingFilter().isConstantFalse())) {
+      selectedSegments = Collections.emptyList();
+    } else {
+      TimerContext.Timer segmentPruneTimer = 
timerContext.startNewPhaseTimer(ServerQueryPhase.SEGMENT_PRUNING);
+      selectedSegments = segmentPrunerService.prune(allSegments, queryContext, 
prunerStats, executorService);
+      segmentPruneTimer.stopAndRecord();
+    }
+
+    // Build segment contexts for selected segments only, preserving prune 
order
+    List<SegmentContext> selectedSegmentContexts = new 
ArrayList<>(selectedSegments.size());
+    Map<SingleTableExecutionInfo, List<IndexSegment>> tableToSelected = new 
HashMap<>();
+    for (IndexSegment segment : selectedSegments) {
+      tableToSelected.computeIfAbsent(segmentToTable.get(segment), k -> new 
ArrayList<>()).add(segment);
+    }
+    Map<IndexSegment, SegmentContext> segmentToContext = new HashMap<>();
+    for (Map.Entry<SingleTableExecutionInfo, List<IndexSegment>> entry : 
tableToSelected.entrySet()) {
+      SingleTableExecutionInfo tableExecutionInfo = entry.getKey();
+      List<IndexSegment> segmentsForTable = entry.getValue();
+      Map<IndexSegment, SegmentContext> providedContexts = 
tableExecutionInfo.getProvidedSegmentContexts();
+      if (providedContexts != null) {
+        for (IndexSegment segment : segmentsForTable) {
+          segmentToContext.put(segment, providedContexts.get(segment));
+        }

Review Comment:
   If `providedContexts` is non-null but does not contain an entry for a 
selected segment, `providedContexts.get(segment)` will return null, and the 
code will later add a null `SegmentContext` into `selectedSegmentContexts` (or 
potentially trigger null-related failures downstream). Consider validating 
presence and either (a) falling back to `getSegmentContexts(...)` for missing 
segments, or (b) throwing an explicit exception with a clear message when a 
selected segment lacks a provided context.
   ```suggestion
           List<IndexSegment> missingSegments = null;
           for (IndexSegment segment : segmentsForTable) {
             SegmentContext context = providedContexts.get(segment);
             if (context != null) {
               segmentToContext.put(segment, context);
             } else {
               if (missingSegments == null) {
                 missingSegments = new ArrayList<>();
               }
               missingSegments.add(segment);
             }
           }
           if (missingSegments != null && !missingSegments.isEmpty()) {
             List<SegmentContext> contexts =
                 tableExecutionInfo.getSegmentContexts(missingSegments, 
queryContext.getQueryOptions());
             for (int i = 0; i < missingSegments.size(); i++) {
               segmentToContext.put(missingSegments.get(i), contexts.get(i));
             }
           }
   ```



-- 
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]

Reply via email to