Implement cross-table segment pruning for logical table#17868
Implement cross-table segment pruning for logical table#17868yashmayya merged 3 commits intoapache:masterfrom
Conversation
There was a problem hiding this comment.
Pull request overview
Adds cross-table segment pruning for logical tables by pruning the combined segment set once, and then resolving segment contexts per physical table while preserving prune order.
Changes:
- Collect all segments across physical tables and run
SegmentPrunerService.prune(...)once (cross-table) inLogicalTableExecutionInfo. - Add constant-false filter/having short-circuit to skip pruning and return no selected segments.
- Add new unit tests covering cross-table pruning behavior, constant-false shortcut, provided segment contexts, and order preservation.
Reviewed changes
Copilot reviewed 2 out of 2 changed files in this pull request and generated 2 comments.
| File | Description |
|---|---|
| pinot-core/src/main/java/org/apache/pinot/core/query/executor/LogicalTableExecutionInfo.java | Implements cross-table pruning and builds per-table SegmentContext mapping while preserving prune order. |
| pinot-core/src/test/java/org/apache/pinot/core/query/executor/LogicalTableExecutionInfoTest.java | Adds tests validating cross-table pruning correctness and key edge cases. |
You can also share your feedback on Copilot code review. Take the survey.
| for (IndexSegment segment : segmentsForTable) { | ||
| segmentToContext.put(segment, providedContexts.get(segment)); | ||
| } |
There was a problem hiding this 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.
| for (IndexSegment segment : segmentsForTable) { | |
| segmentToContext.put(segment, providedContexts.get(segment)); | |
| } | |
| 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)); | |
| } | |
| } |
There was a problem hiding this comment.
The providedContexts is created for the same segments at the beginning from the same indexSegments and possible a subset of them are processed here after pruning. So it should never be null.
| selectedSegments = segmentPrunerService.prune(allSegments, queryContext, prunerStats, executorService); | ||
| segmentPruneTimer.stopAndRecord(); |
There was a problem hiding this 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).
| selectedSegments = segmentPrunerService.prune(allSegments, queryContext, prunerStats, executorService); | |
| segmentPruneTimer.stopAndRecord(); | |
| try { | |
| selectedSegments = segmentPrunerService.prune(allSegments, queryContext, prunerStats, executorService); | |
| } finally { | |
| segmentPruneTimer.stopAndRecord(); | |
| } |
There was a problem hiding this comment.
This uses the same pattern fromSingleTableExecutionInfo - no try/finally. We can add it but I think it is not required.
Codecov Report❌ Patch coverage is
Additional details and impacted files@@ Coverage Diff @@
## master #17868 +/- ##
============================================
+ Coverage 63.23% 63.31% +0.08%
- Complexity 1480 1481 +1
============================================
Files 3190 3190
Lines 192283 192314 +31
Branches 29470 29477 +7
============================================
+ Hits 121589 121773 +184
+ Misses 61158 60991 -167
- Partials 9536 9550 +14
Flags with carried forward coverage won't be shown. Click here to find out more. ☔ View full report in Codecov by Sentry. 🚀 New features to boost your workflow:
|
|
LGTM. @abhishekbafna could you please address the Copilot suggestions? I can approve it then. |
@shauryachats I think, we do not need further changes. The code changes is line with the existing approach and code structure from the SingleTableExecutionInfo code. Let me know if you have any further thoughts. Thanks for the review. |
| .anyMatch(tableExecutionInfo -> tableExecutionInfo.getTableDataManager() instanceof RealtimeTableDataManager); | ||
| } | ||
|
|
||
| /** |
There was a problem hiding this comment.
Can we write an end-to-end integration test that verifies via the query starts the number of segments queried for an order and limit query?
There was a problem hiding this comment.
I could not get a integration test working. Added an unit test.
| int numTotalSegments = allSegments.size(); | ||
|
|
||
| // Constant false shortcut: skip pruning | ||
| List<IndexSegment> selectedSegments; |
There was a problem hiding this comment.
Lets move selectSegments() in SingleTableExecutionInfo to TableExecutionInfo so that it can be reused here without duplicating the logic
There was a problem hiding this comment.
I think, we should leave that.
The selectSegments() is currently a private method, to make accessible we would need to make it static and public/default access modifier. This seems more hack than code refactoring. We do not access to the SingleTableExecutionInfo object and may have to create it.
There was a problem hiding this comment.
I will try to push the selectSegments() to the top level class such that it is accessible in both.
| } | ||
| Map<IndexSegment, SegmentContext> segmentToContext = new HashMap<>(); | ||
| for (Map.Entry<SingleTableExecutionInfo, List<IndexSegment>> entry : tableToSelected.entrySet()) { | ||
| SingleTableExecutionInfo tableExecutionInfo = entry.getKey(); |
There was a problem hiding this comment.
Lets add a new API in SingleTableExecutionInfo
getSelectedSegmentsInfo(List<IndexSegment> selectedSegments, QueryContext queryContext, TimerContext timerContext,
ExecutorService executorService, SegmentPrunerService segmentPrunerService)
which is called here and from current SingleTableExecutionInfo.getSelectedSegmentsInfo() after it prunes segments.
That will avoid duplicate code of what happens after pruning.
There was a problem hiding this comment.
The new API does not make sense as it does not have functional use case.
Also, should this be added to the SingleTableExecutionInfo or to super class TableExecutionInfo? Accordingly the implementation would flow. Also, we would need instance of the SingleTableExecutionInfo to access it.
There was a problem hiding this comment.
This API belongs to SingleTableExecutionInfo because its per table.
You have access to SingleTableExecutionInfo here right ?
There was a problem hiding this comment.
I have move the method to TableExecutionInfo as default method.
|
|
||
| /** | ||
| * Verifies that for a logical table, all segments from all physical tables are collected and | ||
| * prune is invoked once (cross-table). With LIMIT 5 and segments of 10 docs each, only 1 segment |
There was a problem hiding this comment.
I think we should have a more realistic test (here or in integration tests) where we prune based on min/max statistics across tables. So the query should have both order and limit by clause.
Even if limit is 5 and number of docs per segment is 10, setup segment metadata such that the pruning returns multiple segments
That will be a more comprehensive test for the use case
There was a problem hiding this comment.
Added a test.
| assertEquals(selectedSegmentsInfo.getNumSelectedSegments(), 2, | ||
| "ORDER BY DESC LIMIT 10 with overlapping ranges should select exactly 2 segments"); | ||
| List<SegmentContext> contexts = selectedSegmentsInfo.getSelectedSegmentContexts(); |
| // Seg1: [100, 100] 10 docs - first in DESC order, covers LIMIT 10 | ||
| // Seg2: [90, 101] 10 docs - overlaps (max 101 > 100), kept | ||
| // Seg3, Seg4, Seg5: [1, 50] 10 docs each - max 50 < 100, pruned |
Implement cross-table segment pruning for logical table
Summary
For logical tables, segment pruning is now performed once across all segments from every physical table (cross-table prune) instead of once per physical table. This allows pruners such as
SelectionQuerySegmentPruner(ORDER BY + LIMIT) to prune effectively across the logical table, reducing the number of segments processed and aligning behavior with a single physical table holding the same data.Problem
LogicalTableExecutionInfo.getSelectedSegmentsInfopreviously calledSingleTableExecutionInfo.getSelectedSegmentsInfo(and thussegmentPrunerService.prune) per physical table.Solution
SingleTableExecutionInfoinstances into one list and maintain asegmentToTablemap.segmentPrunerService.prune(allSegments, queryContext, prunerStats, executorService)on the combined list.Single-table execution is unchanged; only the logical-table path in
LogicalTableExecutionInfois modified.Backward compatibility
SingleTableExecutionInfobehavior are unchanged; only the logical-table branch inLogicalTableExecutionInfo.getSelectedSegmentsInfois new.