You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2018/08/27 01:44:48 UTC

[GitHub] fjy closed pull request #6239: [Backport] SQL: Fix assumption that AND, OR have two arguments.

fjy closed pull request #6239: [Backport] SQL: Fix assumption that AND, OR have two arguments.
URL: https://github.com/apache/incubator-druid/pull/6239
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/sql/src/main/java/io/druid/sql/calcite/expression/BinaryOperatorConversion.java b/sql/src/main/java/io/druid/sql/calcite/expression/BinaryOperatorConversion.java
index bf4f069e832..deb50c3bf7e 100644
--- a/sql/src/main/java/io/druid/sql/calcite/expression/BinaryOperatorConversion.java
+++ b/sql/src/main/java/io/druid/sql/calcite/expression/BinaryOperatorConversion.java
@@ -19,6 +19,7 @@
 
 package io.druid.sql.calcite.expression;
 
+import com.google.common.base.Joiner;
 import io.druid.java.util.common.ISE;
 import io.druid.java.util.common.StringUtils;
 import io.druid.sql.calcite.planner.PlannerContext;
@@ -26,15 +27,17 @@
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlOperator;
 
+import java.util.stream.Collectors;
+
 public class BinaryOperatorConversion implements SqlOperatorConversion
 {
   private final SqlOperator operator;
-  private final String druidOperator;
+  private final Joiner joiner;
 
   public BinaryOperatorConversion(final SqlOperator operator, final String druidOperator)
   {
     this.operator = operator;
-    this.druidOperator = druidOperator;
+    this.joiner = Joiner.on(" " + druidOperator + " ");
   }
 
   @Override
@@ -55,16 +58,18 @@ public DruidExpression toDruidExpression(
         rowSignature,
         rexNode,
         operands -> {
-          if (operands.size() != 2) {
+          if (operands.size() < 2) {
             throw new ISE("WTF?! Got binary operator[%s] with %s args?", operator.getName(), operands.size());
           }
 
           return DruidExpression.fromExpression(
               StringUtils.format(
-                  "(%s %s %s)",
-                  operands.get(0).getExpression(),
-                  druidOperator,
-                  operands.get(1).getExpression()
+                  "(%s)",
+                  joiner.join(
+                      operands.stream()
+                              .map(DruidExpression::getExpression)
+                              .collect(Collectors.toList())
+                  )
               )
           );
         }
diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java
index dccdc5f3ef0..19f687fe975 100644
--- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java
@@ -1602,6 +1602,39 @@ public void testGroupByCaseWhen() throws Exception
     );
   }
 
+  @Test
+  public void testGroupByCaseWhenOfTripleAnd() throws Exception
+  {
+    testQuery(
+        "SELECT\n"
+        + "  CASE WHEN m1 > 1 AND m1 < 5 AND cnt = 1 THEN 'x' ELSE NULL END,"
+        + "  COUNT(*)\n"
+        + "FROM druid.foo\n"
+        + "GROUP BY 1",
+        ImmutableList.of(
+            GroupByQuery.builder()
+                        .setDataSource(CalciteTests.DATASOURCE1)
+                        .setInterval(QSS(Filtration.eternity()))
+                        .setGranularity(Granularities.ALL)
+                        .setVirtualColumns(
+                            EXPRESSION_VIRTUAL_COLUMN(
+                                "d0:v",
+                                "case_searched(((\"m1\" > 1) && (\"m1\" < 5) && (\"cnt\" == 1)),'x','')",
+                                ValueType.STRING
+                            )
+                        )
+                        .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0")))
+                        .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+                        .setContext(QUERY_CONTEXT_DEFAULT)
+                        .build()
+        ),
+        ImmutableList.of(
+            new Object[]{"", 3L},
+            new Object[]{"x", 3L}
+        )
+    );
+  }
+
   @Test
   public void testNullEmptyStringEquality() throws Exception
   {


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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