shauryachats commented on code in PR #17868:
URL: https://github.com/apache/pinot/pull/17868#discussion_r2934642771
##########
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) {
Review Comment:
You can use `segmentToContext.addAll(providedContexts)`.
--
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]