This is an automated email from the ASF dual-hosted git repository.

jackie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 7406fac039 Optimize QueryContext.isUnsafeTrim (#16312)
7406fac039 is described below

commit 7406fac039125fbf709e42c4c1175d8160850054
Author: Song Fu <[email protected]>
AuthorDate: Fri Jul 11 12:24:50 2025 -0700

    Optimize QueryContext.isUnsafeTrim (#16312)
---
 .../core/query/request/context/QueryContext.java   | 41 +++++-----------------
 .../tests/GroupByTrimmingIntegrationTest.java      | 33 +++++++++++++++++
 2 files changed, 42 insertions(+), 32 deletions(-)

diff --git 
a/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java
 
b/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java
index 6c8076091e..9066a25739 100644
--- 
a/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java
+++ 
b/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java
@@ -20,7 +20,6 @@ package org.apache.pinot.core.query.request.context;
 
 import com.google.common.base.Preconditions;
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -140,6 +139,7 @@ public class QueryContext {
   // Whether server returns the final result with unpartitioned group key
   private boolean _serverReturnFinalResultKeyUnpartitioned;
   private boolean _accurateGroupByWithoutOrderBy;
+  private boolean _isUnsafeTrim;
   // Collection of index types to skip per column
   private Map<String, Set<FieldConfig.IndexType>> _skipIndexes;
 
@@ -175,36 +175,10 @@ public class QueryContext {
       return false;
     }
 
-    BitSet orderByKeysMatched = new BitSet(orderByKeys.size());
-
-    OUTER_GROUP:
-    for (ExpressionContext groupExp : groupByKeys) {
-      for (int i = 0; i < orderByKeys.size(); i++) {
-        OrderByExpressionContext orderExp = orderByKeys.get(i);
-        if (groupExp.equals(orderExp.getExpression())) {
-          orderByKeysMatched.set(i);
-          continue OUTER_GROUP;
-        }
-      }
-
-      return false;
-    }
-
-    OUTER_ORDER:
-    for (int i = 0, n = orderByKeys.size(); i < n; i++) {
-      if (orderByKeysMatched.get(i)) {
-        continue;
-      }
-
-      for (ExpressionContext groupExp : groupByKeys) {
-        if (groupExp.equals(orderByKeys.get(i).getExpression())) {
-          continue OUTER_ORDER;
-        }
-      }
-      return false;
-    }
-
-    return true;
+    Set<ExpressionContext> groupByKeyIdentSet = new HashSet<>(groupByKeys);
+    Set<ExpressionContext> orderByKeyIdentSet = new HashSet<>();
+    orderByKeys.forEach(key -> orderByKeyIdentSet.add(key.getExpression()));
+    return groupByKeyIdentSet.equals(orderByKeyIdentSet);
   }
 
   /**
@@ -564,7 +538,7 @@ public class QueryContext {
   }
 
   public boolean isUnsafeTrim() {
-    return !isSameOrderAndGroupByColumns(this) || getHavingFilter() != null;
+    return _isUnsafeTrim;
   }
 
   public static class Builder {
@@ -681,6 +655,9 @@ public class QueryContext {
       generateAggregationFunctions(queryContext);
       extractColumns(queryContext);
 
+      queryContext._isUnsafeTrim =
+          !queryContext.isSameOrderAndGroupByColumns(queryContext) || 
queryContext.getHavingFilter() != null;
+
       return queryContext;
     }
 
diff --git 
a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/GroupByTrimmingIntegrationTest.java
 
b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/GroupByTrimmingIntegrationTest.java
index b922add077..eaede8389d 100644
--- 
a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/GroupByTrimmingIntegrationTest.java
+++ 
b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/GroupByTrimmingIntegrationTest.java
@@ -497,6 +497,39 @@ public class GroupByTrimmingIntegrationTest extends 
BaseClusterIntegrationTestSe
             + "FILTER_MATCH_ENTIRE_SEGMENT(docs:1000),\t6,\t5\n");
   }
 
+  @Test
+  public void 
testSSQEGroupsTrimmedAtSegmentLevelWithOrderByAllGroupByKeysDuplicateKeyIsSafe()
+      throws Exception {
+    setUseMultiStageQueryEngine(false);
+
+    // trimming is safe on rows ordered by all group by keys (regardless of 
key order, direction or duplications)
+    String query = "SELECT i, j, COUNT(*) FROM mytable GROUP BY i, i, j ORDER 
BY j ASC, i DESC, j ASC LIMIT 5";
+
+    Connection conn = getPinotConnection();
+    assertTrimFlagNotSet(conn.execute(query));
+
+    ResultSetGroup result = conn.execute("SET minSegmentGroupTrimSize=5; " + 
query);
+    assertTrimFlagNotSet(result);
+
+    assertEquals(toResultStr(result),
+        "\"i\"[\"INT\"],\t\"j\"[\"LONG\"],\t\"count(*)\"[\"LONG\"]\n"
+            + "0,\t0,\t4\n"
+            + "1,\t1,\t4\n"
+            + "2,\t2,\t4\n"
+            + "3,\t3,\t4\n"
+            + "4,\t4,\t4");
+
+    assertEquals(toExplainStr(postQuery("EXPLAIN PLAN FOR " + query), false),
+        "BROKER_REDUCE(sort:[j ASC, i DESC],limit:5),\t1,\t0\n"
+            + "COMBINE_GROUP_BY,\t2,\t1\n"
+            + "PLAN_START(numSegmentsForThisPlan:4),\t-1,\t-1\n"
+            + "GROUP_BY(groupKeys:i, i, j, aggregations:count(*)),\t3,\t2\n"
+            + "PROJECT(i, j),\t4,\t3\n"
+            + "DOC_ID_SET,\t5,\t4\n"
+            + "FILTER_MATCH_ENTIRE_SEGMENT(docs:1000),\t6,\t5\n");
+  }
+
+
   @Test
   public void 
testSSQEGroupsTrimmedAtSegmentLevelWithOrderByAggregateIsNotSafe()
       throws Exception {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to