You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2015/08/01 00:43:22 UTC

[45/50] [abbrv] incubator-calcite git commit: [CALCITE-259] Using sub-queries in CASE statement against JDBC tables generates invalid Oracle SQL (Yeong Wei)

[CALCITE-259] Using sub-queries in CASE statement against JDBC tables generates invalid Oracle SQL (Yeong Wei)


Project: http://git-wip-us.apache.org/repos/asf/incubator-calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/01b2d057
Tree: http://git-wip-us.apache.org/repos/asf/incubator-calcite/tree/01b2d057
Diff: http://git-wip-us.apache.org/repos/asf/incubator-calcite/diff/01b2d057

Branch: refs/heads/branch-release
Commit: 01b2d057c329165f1180c53a8d8329b1b225447f
Parents: 3fd4fa5
Author: Julian Hyde <jh...@apache.org>
Authored: Sun May 17 21:49:06 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun May 17 21:49:06 2015 -0700

----------------------------------------------------------------------
 .../apache/calcite/adapter/jdbc/JdbcRules.java  | 99 ++++++++++++++++++--
 .../apache/calcite/test/JdbcAdapterTest.java    | 10 ++
 2 files changed, 101 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/01b2d057/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
index c75bcd4..fc5cab5 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
@@ -71,6 +71,7 @@ import org.apache.calcite.schema.ModifiableTable;
 import org.apache.calcite.sql.JoinConditionType;
 import org.apache.calcite.sql.JoinType;
 import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlBasicCall;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlFunction;
@@ -84,6 +85,8 @@ import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlSetOperator;
+import org.apache.calcite.sql.fun.SqlCase;
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.InferTypes;
@@ -684,11 +687,14 @@ public class JdbcRules {
         List<AggregateCall> aggCalls)
         throws InvalidRelException {
       super(cluster, traitSet, input, indicator, groupSet, groupSets, aggCalls);
+      Convention convention = getConvention();
+      assert convention instanceof JdbcConvention;
       assert getConvention() instanceof JdbcConvention;
       assert this.groupSets.size() == 1 : "Grouping sets not supported";
       assert !this.indicator;
       for (AggregateCall aggCall : aggCalls) {
-        if (!canImplement(aggCall.getAggregation())) {
+        if (!canImplement(aggCall.getAggregation(),
+            ((JdbcConvention) convention).dialect)) {
           throw new InvalidRelException("cannot implement aggregate function "
               + aggCall.getAggregation());
         }
@@ -697,12 +703,23 @@ public class JdbcRules {
 
     /** Returns whether this JDBC data source can implement a given aggregate
      * function. */
-    private boolean canImplement(SqlAggFunction aggregation) {
-      return Arrays.asList(SqlStdOperatorTable.COUNT,
-          SqlStdOperatorTable.SUM,
-          SqlStdOperatorTable.SUM0,
-          SqlStdOperatorTable.MIN,
-          SqlStdOperatorTable.MAX).contains(aggregation);
+    private boolean canImplement(SqlAggFunction aggregation,
+        SqlDialect sqlDialect) {
+      List<SqlAggFunction> stdAggFuncs = new ArrayList<SqlAggFunction>();
+      stdAggFuncs.add(SqlStdOperatorTable.COUNT);
+      stdAggFuncs.add(SqlStdOperatorTable.SUM);
+      stdAggFuncs.add(SqlStdOperatorTable.SUM0);
+      stdAggFuncs.add(SqlStdOperatorTable.MIN);
+      stdAggFuncs.add(SqlStdOperatorTable.MAX);
+
+      switch(sqlDialect.getDatabaseProduct()) {
+      case MYSQL:
+        stdAggFuncs.add(SqlStdOperatorTable.SINGLE_VALUE);
+        break;
+      default:
+        break;
+      }
+      return stdAggFuncs.contains(aggregation);
     }
 
     @Override public JdbcAggregate copy(RelTraitSet traitSet, RelNode input,
@@ -731,7 +748,13 @@ public class JdbcRules {
         groupByList.add(field);
       }
       for (AggregateCall aggCall : aggCalls) {
-        addSelect(selectList, builder.context.toSql(aggCall), getRowType());
+        SqlNode aggCallSqlNode = builder.context.toSql(aggCall);
+        if (aggCall.getAggregation() instanceof SqlSingleValueAggFunction) {
+          aggCallSqlNode =
+              rewriteSingleValueExpr(aggCallSqlNode,
+                  builder.result().node, implementor.dialect);
+        }
+        addSelect(selectList, aggCallSqlNode, getRowType());
       }
       builder.setSelect(new SqlNodeList(selectList, POS));
       if (!groupByList.isEmpty() || aggCalls.isEmpty()) {
@@ -741,6 +764,66 @@ public class JdbcRules {
       }
       return builder.result();
     }
+
+    /** Rewrite SINGLE_VALUE into expression based on database variants
+     *  E.g. HSQLDB, MYSQL, ORACLE, etc
+     */
+    private SqlNode rewriteSingleValueExpr(SqlNode aggCall,
+        SqlNode resultNode, SqlDialect sqlDialect) {
+
+      SqlNode aggregatee = ((SqlBasicCall) aggCall).operand(0);
+      SqlNode caseOperand = null;
+      SqlNode elseExpr = null;
+      SqlNode countAggregatee = SqlStdOperatorTable
+          .COUNT.createCall(POS, Arrays.asList(aggregatee));
+      SqlNode asterisk = new SqlIdentifier(Arrays.asList("*"), POS);
+      SqlNode derivedTable = null;
+
+      switch(sqlDialect.getDatabaseProduct()) {
+      case MYSQL:
+        caseOperand = countAggregatee;
+
+        SqlNode nullValue = SqlLiteral.createNull(POS);
+        SqlSelect selectNull = new SqlSelect(
+            POS, SqlNodeList.EMPTY,
+            new SqlNodeList(Arrays.asList(nullValue), POS), null, null,
+            null, null, SqlNodeList.EMPTY, null, null, null);
+
+        SqlCall unionAll = SqlStdOperatorTable.UNION_ALL
+            .createCall(POS, selectNull, selectNull);
+
+        SqlIdentifier derivedTableAlias = new SqlIdentifier("tbl", POS);
+        derivedTable =
+            SqlStdOperatorTable.AS.createCall(POS, Arrays.asList(
+                new SqlNodeList(Arrays.asList(unionAll), POS),
+                derivedTableAlias));
+
+        elseExpr = new SqlSelect(
+            POS, SqlNodeList.EMPTY,
+            new SqlNodeList(Arrays.asList(asterisk), POS), derivedTable, null,
+            null, null, SqlNodeList.EMPTY, null, null, null);
+        break;
+      default:
+        LOGGER.fine("SINGLE_VALUE rewrite not supported for "
+            + sqlDialect.getDatabaseProduct());
+        return aggCall;
+      }
+
+      SqlNodeList whenList = new SqlNodeList(Arrays.asList(
+          SqlLiteral.createExactNumeric("0", POS),
+          SqlLiteral.createExactNumeric("1", POS)), POS);
+
+      SqlNodeList thenList = new SqlNodeList(Arrays.asList(
+          SqlLiteral.createNull(POS),
+          aggregatee), POS);
+
+      SqlNode caseExpr =
+          new SqlCase(POS, caseOperand, whenList, thenList, elseExpr);
+
+      LOGGER.fine("SINGLE_VALUE rewritten into [" + caseExpr + "]");
+
+      return caseExpr;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/01b2d057/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
index e7faeef..326d6ba 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
@@ -331,6 +331,16 @@ public class JdbcAdapterTest {
     rs.close();
     calciteConnection.close();
   }
+
+  @Test(expected = RuntimeException.class)
+  public void testSubQueryWithSingleValue() {
+    String sql = "SELECT \"full_name\" FROM \"employee\" WHERE "
+        + "\"employee_id\" = (SELECT \"employee_id\" FROM \"salary\")";
+    CalciteAssert.model(JdbcTest.FOODMART_MODEL).query(sql)
+        .explainContains("SINGLE_VALUE")
+        .enable(CalciteAssert.DB == CalciteAssert.DatabaseInstance.MYSQL)
+        .runs();
+  }
 }
 
 // End JdbcAdapterTest.java