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

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


The following commit(s) were added to refs/heads/master by this push:
     new 16ac894  Handle queries like count(distinct ) (#5223)
16ac894 is described below

commit 16ac894344fcad449299c1cdb7241329f503d05e
Author: Xiang Fu <fx19880...@gmail.com>
AuthorDate: Thu Apr 9 10:48:35 2020 -0700

    Handle queries like count(distinct ) (#5223)
---
 .../apache/pinot/sql/parsers/CalciteSqlParser.java |  5 ++-
 .../pinot/sql/parsers/CalciteSqlCompilerTest.java  | 49 ++++++++++++++++++++++
 2 files changed, 53 insertions(+), 1 deletion(-)

diff --git 
a/pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java 
b/pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java
index 3596356..75b2eb8 100644
--- 
a/pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java
+++ 
b/pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java
@@ -56,7 +56,6 @@ import org.apache.pinot.pql.parsers.Pql2Compiler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 public class CalciteSqlParser {
 
   private static final Logger LOGGER = 
LoggerFactory.getLogger(CalciteSqlParser.class);
@@ -573,6 +572,10 @@ public class CalciteSqlParser {
         if (funcSqlNode.getOperator().getKind() == SqlKind.OTHER_FUNCTION) {
           funcName = funcSqlNode.getOperator().getName();
         }
+        if (funcName.equalsIgnoreCase(SqlKind.COUNT.toString()) && 
(funcSqlNode.getFunctionQuantifier() != null) && funcSqlNode
+            
.getFunctionQuantifier().toValue().equalsIgnoreCase(AggregationFunctionType.DISTINCT.getName()))
 {
+          funcName = AggregationFunctionType.DISTINCTCOUNT.getName();
+        }
         final Expression funcExpr = 
RequestUtils.getFunctionExpression(funcName);
         for (SqlNode child : funcSqlNode.getOperands()) {
           if (child instanceof SqlNodeList) {
diff --git 
a/pinot-common/src/test/java/org/apache/pinot/sql/parsers/CalciteSqlCompilerTest.java
 
b/pinot-common/src/test/java/org/apache/pinot/sql/parsers/CalciteSqlCompilerTest.java
index c252142..0992d92 100644
--- 
a/pinot-common/src/test/java/org/apache/pinot/sql/parsers/CalciteSqlCompilerTest.java
+++ 
b/pinot-common/src/test/java/org/apache/pinot/sql/parsers/CalciteSqlCompilerTest.java
@@ -1354,4 +1354,53 @@ public class CalciteSqlCompilerTest {
     Assert.assertEquals("CAST", 
pinotQuery.getSelectList().get(0).getFunctionCall().getOperands().get(0).getFunctionCall().getOperator());
     Assert.assertEquals("CAST", 
pinotQuery.getSelectList().get(0).getFunctionCall().getOperands().get(0).getFunctionCall().getOperands().get(0).getFunctionCall().getOperator());
   }
+
+  @Test
+  public void testDistinctCountRewrite() {
+    String query = "SELECT count(distinct bar) FROM foo";
+    PinotQuery pinotQuery = CalciteSqlParser.compileToPinotQuery(query);
+    Assert.assertEquals(pinotQuery.getSelectList().size(), 1);
+    
Assert.assertEquals(pinotQuery.getSelectList().get(0).getFunctionCall().getOperator(),
 "distinctCount");
+    Assert.assertEquals(
+        
pinotQuery.getSelectList().get(0).getFunctionCall().getOperands().get(0).getIdentifier().getName(),
 "bar");
+
+    query = "SELECT count(distinct bar) FROM foo GROUP BY city";
+    pinotQuery = CalciteSqlParser.compileToPinotQuery(query);
+    Assert.assertEquals(pinotQuery.getSelectList().size(), 1);
+    
Assert.assertEquals(pinotQuery.getSelectList().get(0).getFunctionCall().getOperator(),
 "distinctCount");
+    Assert.assertEquals(
+        
pinotQuery.getSelectList().get(0).getFunctionCall().getOperands().get(0).getIdentifier().getName(),
 "bar");
+
+    query = "SELECT count(distinct bar), distinctCount(bar) FROM foo GROUP BY 
city";
+    pinotQuery = CalciteSqlParser.compileToPinotQuery(query);
+    Assert.assertEquals(pinotQuery.getSelectList().size(), 2);
+    
Assert.assertEquals(pinotQuery.getSelectList().get(0).getFunctionCall().getOperator(),
 "distinctCount");
+    Assert.assertEquals(
+        
pinotQuery.getSelectList().get(0).getFunctionCall().getOperands().get(0).getIdentifier().getName(),
 "bar");
+
+    
Assert.assertEquals(pinotQuery.getSelectList().get(1).getFunctionCall().getOperator(),
 "distinctCount");
+    Assert.assertEquals(
+        
pinotQuery.getSelectList().get(1).getFunctionCall().getOperands().get(0).getIdentifier().getName(),
 "bar");
+
+    query = "SELECT count(distinct bar), count(*), sum(a),min(a),max(b) FROM 
foo GROUP BY city";
+    pinotQuery = CalciteSqlParser.compileToPinotQuery(query);
+    Assert.assertEquals(pinotQuery.getSelectList().size(), 5);
+    
Assert.assertEquals(pinotQuery.getSelectList().get(0).getFunctionCall().getOperator(),
 "distinctCount");
+    Assert.assertEquals(
+        
pinotQuery.getSelectList().get(0).getFunctionCall().getOperands().get(0).getIdentifier().getName(),
 "bar");
+
+    query = "SELECT count(distinct bar) AS distinct_bar, count(*), 
sum(a),min(a),max(b) FROM foo GROUP BY city";
+    pinotQuery = CalciteSqlParser.compileToPinotQuery(query);
+    Assert.assertEquals(pinotQuery.getSelectList().size(), 5);
+    
Assert.assertEquals(pinotQuery.getSelectList().get(0).getFunctionCall().getOperator(),
 "AS");
+    Assert.assertEquals(
+        
pinotQuery.getSelectList().get(0).getFunctionCall().getOperands().get(0).getFunctionCall().getOperator(),
+        "distinctCount");
+    Assert.assertEquals(
+        
pinotQuery.getSelectList().get(0).getFunctionCall().getOperands().get(0).getFunctionCall().getOperands().get(0)
+            .getIdentifier().getName(), "bar");
+    Assert.assertEquals(
+        
pinotQuery.getSelectList().get(0).getFunctionCall().getOperands().get(1).getIdentifier().getName(),
+        "distinct_bar");
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org

Reply via email to