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


##########
pinot-core/src/test/java/org/apache/pinot/core/query/executor/LogicalTableExecutionInfoTest.java:
##########
@@ -0,0 +1,224 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.executor;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.core.query.config.SegmentPrunerConfig;
+import org.apache.pinot.core.query.pruner.SegmentPrunerService;
+import org.apache.pinot.core.query.request.context.QueryContext;
+import org.apache.pinot.core.query.request.context.TimerContext;
+import 
org.apache.pinot.core.query.request.context.utils.QueryContextConverterUtils;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.SegmentContext;
+import org.apache.pinot.segment.spi.SegmentMetadata;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.env.PinotConfiguration;
+import org.apache.pinot.spi.utils.CommonConstants.Server;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertSame;
+import static org.testng.Assert.assertTrue;
+
+
+public class LogicalTableExecutionInfoTest {
+
+  private ExecutorService _executorService;
+  private SegmentPrunerService _segmentPrunerService;
+  private TimerContext _timerContext;
+
+  @BeforeClass
+  public void setUp() {
+    _executorService = Executors.newFixedThreadPool(2);
+    PinotConfiguration prunerConf = new PinotConfiguration();
+    prunerConf.setProperty(Server.CLASS, 
Server.DEFAULT_QUERY_EXECUTOR_PRUNER_CLASS);
+    _segmentPrunerService = new SegmentPrunerService(new 
SegmentPrunerConfig(prunerConf));
+    ServerMetrics serverMetrics = mock(ServerMetrics.class);
+    _timerContext = new TimerContext("logicalTable_OFFLINE", serverMetrics, 
System.currentTimeMillis());
+  }
+
+  @AfterClass
+  public void tearDown() {
+    if (_executorService != null) {
+      _executorService.shutdown();
+    }
+  }
+
+  /**
+   * 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

Review 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



##########
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();

Review 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. 



##########
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;

Review Comment:
   Lets move selectSegments() in SingleTableExecutionInfo to TableExecutionInfo 
so that it can be reused here without duplicating the logic



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