You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by ab...@apache.org on 2022/09/28 15:31:47 UTC

[druid] branch master updated: Fix assertion error in sql planning for latest aggregators (#13151)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 61b34950e7 Fix assertion error in sql planning for latest aggregators (#13151)
61b34950e7 is described below

commit 61b34950e7dd80f35c8e3d1db71fcf24cfaf9238
Author: Abhishek Agarwal <14...@users.noreply.github.com>
AuthorDate: Wed Sep 28 21:01:32 2022 +0530

    Fix assertion error in sql planning for latest aggregators (#13151)
    
    * Fix sql planning bug for latest aggregators
    
    * change test name
    
    * Fix error messages
    
    * fix error message again
---
 .../builtin/EarliestLatestAnySqlAggregator.java    | 10 ++++-
 .../builtin/EarliestLatestBySqlAggregator.java     | 10 ++++-
 .../apache/druid/sql/calcite/CalciteQueryTest.java | 50 ++++++++++++++++++++++
 3 files changed, 68 insertions(+), 2 deletions(-)

diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java
index 58fd083dca..ba06148363 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java
@@ -202,12 +202,20 @@ public class EarliestLatestAnySqlAggregator implements SqlAggregator
         theAggFactory = aggregatorType.createAggregatorFactory(aggregatorName, fieldName, null, outputType, -1);
         break;
       case 2:
+        int maxStringBytes;
+        try {
+          maxStringBytes = RexLiteral.intValue(rexNodes.get(1));
+        }
+        catch (AssertionError ae) {
+          plannerContext.setPlanningError("The second argument '%s' to function '%s' is not a number", rexNodes.get(1), aggregateCall.getName());
+          return null;
+        }
         theAggFactory = aggregatorType.createAggregatorFactory(
             aggregatorName,
             fieldName,
             null,
             outputType,
-            RexLiteral.intValue(rexNodes.get(1))
+            maxStringBytes
         );
         break;
       default:
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestBySqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestBySqlAggregator.java
index f13a918e6f..46762306a4 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestBySqlAggregator.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestBySqlAggregator.java
@@ -122,12 +122,20 @@ public class EarliestLatestBySqlAggregator implements SqlAggregator
         );
         break;
       case 3:
+        int maxStringBytes;
+        try {
+          maxStringBytes = RexLiteral.intValue(rexNodes.get(2));
+        }
+        catch (AssertionError ae) {
+          plannerContext.setPlanningError("The third argument '%s' to function '%s' is not a number", rexNodes.get(2), aggregateCall.getName());
+          return null;
+        }
         theAggFactory = aggregatorType.createAggregatorFactory(
             aggregatorName,
             fieldName,
             EarliestLatestAnySqlAggregator.getColumnName(plannerContext, virtualColumnRegistry, args.get(1), rexNodes.get(1)),
             outputType,
-            RexLiteral.intValue(rexNodes.get(2))
+            maxStringBytes
         );
         break;
       default:
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
index d260126e52..bd21a57988 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
@@ -998,6 +998,56 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
     );
   }
 
+  @Test
+  public void testStringLatestGroupByWithAlwaysFalseCondition()
+  {
+    testQuery(
+        "SELECT LATEST(dim4, 10),dim2 FROM numfoo WHERE (dim1 = 'something' AND dim1 IN( 'something else') ) GROUP BY dim2",
+        ImmutableList.of(
+            Druids.newScanQueryBuilder()
+                .dataSource(InlineDataSource.fromIterable(
+                    ImmutableList.of(),
+                    RowSignature.builder()
+                        .add("EXPR$0", ColumnType.STRING)
+                        .add("dim2", ColumnType.STRING)
+                        .build()
+                ))
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("EXPR$0", "dim2")
+                .context(QUERY_CONTEXT_DEFAULT)
+                .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+                .legacy(false)
+                .build()
+        ),
+        ImmutableList.of()
+    );
+  }
+
+  @Test
+  public void testStringLatestByGroupByWithAlwaysFalseCondition()
+  {
+    testQuery(
+        "SELECT LATEST_BY(dim4, __time, 10),dim2 FROM numfoo WHERE (dim1 = 'something' AND dim1 IN( 'something else') ) GROUP BY dim2",
+        ImmutableList.of(
+            Druids.newScanQueryBuilder()
+                .dataSource(InlineDataSource.fromIterable(
+                    ImmutableList.of(),
+                    RowSignature.builder()
+                        .add("EXPR$0", ColumnType.STRING)
+                        .add("dim2", ColumnType.STRING)
+                        .build()
+                ))
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .columns("EXPR$0", "dim2")
+                .context(QUERY_CONTEXT_DEFAULT)
+                .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+                .legacy(false)
+                .build()
+        ),
+        ImmutableList.of()
+    );
+  }
+
   // This test the off-heap (buffer) version of the EarliestAggregator (Double/Float/Long)
   @Test
   public void testPrimitiveEarliestInSubquery()


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org