You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by vl...@apache.org on 2019/01/04 14:53:49 UTC

[calcite] branch master updated: [CALCITE-2767] Fix conversion of Case in RexSqlStandardConvertletTable (Benoit Hanotte)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new e67eb2d  [CALCITE-2767] Fix conversion of Case in RexSqlStandardConvertletTable (Benoit Hanotte)
e67eb2d is described below

commit e67eb2d74c0a1f32d74025048f30fd8a94d98f8f
Author: Benoit Hanotte <b....@criteo.com>
AuthorDate: Wed Jan 2 11:56:57 2019 +0100

    [CALCITE-2767] Fix conversion of Case in RexSqlStandardConvertletTable (Benoit Hanotte)
    
    `SqlCaseOperator.createCall()` requires 4 operands [1]:
     1. first value (i.e. CASE value WHEN...)
     2. when condifitions (as SqlNodeList)
     3. then values (as SqlNodeLists)
     4. else value
    
    `RexSqlStandardConvertletTable` always only pass 3 arguments [2] as it uses the generic form such as
    `CASE WHEN condition1 THEN val2 ... ELSE val3 END` and never the form
    `CASE val WHEN condition1 THEN val2 ELSE val3`.
    
    When calling `SqlCaseOperator.createCall()`, it needs to pass a null sql node for the first value in
    order to correctly provide the 4 expected arguments and not only the last 3.
    
    [1] https://github.com/apache/calcite/blob/59b76701d9c805f51b0929577ca604fdbfe00b0b/core/src/main/java/org/apache/calcite/sql/fun/SqlCaseOperator.java#L288
    [2] https://github.com/apache/calcite/blob/59b76701d9c805f51b0929577ca604fdbfe00b0b/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java#L239
    
    fixes #990
---
 .../calcite/rex/RexSqlStandardConvertletTable.java |   6 +-
 .../calcite/rex/RexToSqlNodeConverterImpl.java     |   5 +
 .../rex/RexSqlStandardConvertletTableTest.java     | 103 +++++++++++++++++++++
 .../java/org/apache/calcite/test/CalciteSuite.java |   2 +
 4 files changed, 111 insertions(+), 5 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java b/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java
index 159b281..9bad2f3 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java
@@ -232,11 +232,7 @@ public class RexSqlStandardConvertletTable
             ++i;
           }
           SqlNode elseExpr = operands[i];
-          SqlNode[] newOperands = new SqlNode[3];
-          newOperands[0] = whenList;
-          newOperands[1] = thenList;
-          newOperands[2] = elseExpr;
-          return op.createCall(null, SqlParserPos.ZERO, newOperands);
+          return op.createCall(null, SqlParserPos.ZERO, null, whenList, thenList, elseExpr);
         });
   }
 
diff --git a/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java b/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
index f9a8932..504fca2 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
@@ -122,6 +122,11 @@ public class RexToSqlNodeConverterImpl implements RexToSqlNodeConverter {
           SqlParserPos.ZERO);
     }
 
+    // Null
+    if (SqlTypeFamily.NULL == literal.getTypeName().getFamily()) {
+      return SqlLiteral.createNull(SqlParserPos.ZERO);
+    }
+
     return null;
   }
 
diff --git a/core/src/test/java/org/apache/calcite/rex/RexSqlStandardConvertletTableTest.java b/core/src/test/java/org/apache/calcite/rex/RexSqlStandardConvertletTableTest.java
new file mode 100644
index 0000000..2f6f132
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/rex/RexSqlStandardConvertletTableTest.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.rex;
+
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.test.SqlToRelTestBase;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.calcite.util.Closer;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Unit test for {@link org.apache.calcite.rex.RexSqlStandardConvertletTable}.
+ */
+public class RexSqlStandardConvertletTableTest extends SqlToRelTestBase {
+
+  @Test
+  public void testCoalesce() {
+    final Project project = (Project) convertSqlToRel(
+            "SELECT COALESCE(NULL, 'a')", false);
+    final RexNode rex = project.getChildExps().get(0);
+    final RexToSqlNodeConverter rexToSqlNodeConverter = rexToSqlNodeConverter();
+    final SqlNode convertedSql = rexToSqlNodeConverter.convertNode(rex);
+    assertEquals(
+            "CASE WHEN NULL IS NOT NULL THEN NULL ELSE 'a' END",
+            convertedSql.toString());
+  }
+
+  @Test
+  public void testCaseWithValue() {
+    final Project project =
+            (Project) convertSqlToRel(
+                    "SELECT CASE NULL WHEN NULL THEN NULL ELSE 'a' END", false);
+    final RexNode rex = project.getChildExps().get(0);
+    final RexToSqlNodeConverter rexToSqlNodeConverter = rexToSqlNodeConverter();
+    final SqlNode convertedSql = rexToSqlNodeConverter.convertNode(rex);
+    assertEquals(
+            "CASE WHEN NULL = NULL THEN NULL ELSE 'a' END",
+            convertedSql.toString());
+  }
+
+  @Test
+  public void testCaseNoValue() {
+    final Project project = (Project) convertSqlToRel(
+            "SELECT CASE WHEN NULL IS NULL THEN NULL ELSE 'a' END", false);
+    final RexNode rex = project.getChildExps().get(0);
+    final RexToSqlNodeConverter rexToSqlNodeConverter = rexToSqlNodeConverter();
+    final SqlNode convertedSql = rexToSqlNodeConverter.convertNode(rex);
+    assertEquals(
+            "CASE WHEN NULL IS NULL THEN NULL ELSE 'a' END",
+            convertedSql.toString());
+  }
+
+  private RelNode convertSqlToRel(String sql, boolean simplifyRex) {
+    final FrameworkConfig config = Frameworks.newConfigBuilder()
+            .defaultSchema(CalciteSchema.createRootSchema(false).plus())
+            .parserConfig(SqlParser.configBuilder().build())
+            .build();
+    final Planner planner = Frameworks.getPlanner(config);
+    try (Closer closer = new Closer()) {
+      closer.add(Hook.REL_BUILDER_SIMPLIFY.addThread(Hook.propertyJ(simplifyRex)));
+      final SqlNode parsed = planner.parse(sql);
+      final SqlNode validated = planner.validate(parsed);
+      return planner.rel(validated).rel;
+    } catch (SqlParseException | RelConversionException | ValidationException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static RexToSqlNodeConverter rexToSqlNodeConverter() {
+    final RexSqlStandardConvertletTable convertletTable = new RexSqlStandardConvertletTable();
+    return new RexToSqlNodeConverterImpl(convertletTable);
+  }
+
+}
+
+// End RexSqlStandardConvertletTableTest.java
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
index 4c6f620..aebc9b3 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -38,6 +38,7 @@ import org.apache.calcite.rel.rel2sql.RelToSqlConverterTest;
 import org.apache.calcite.rel.rules.DateRangeRulesTest;
 import org.apache.calcite.rex.RexBuilderTest;
 import org.apache.calcite.rex.RexExecutorTest;
+import org.apache.calcite.rex.RexSqlStandardConvertletTableTest;
 import org.apache.calcite.runtime.BinarySearchTest;
 import org.apache.calcite.runtime.EnumerablesTest;
 import org.apache.calcite.sql.SqlSetOptionOperatorTest;
@@ -122,6 +123,7 @@ import org.junit.runners.Suite;
     InduceGroupingTypeTest.class,
     RelOptPlanReaderTest.class,
     RexBuilderTest.class,
+    RexSqlStandardConvertletTableTest.class,
     SqlTypeFactoryTest.class,
     SqlTypeUtilTest.class,
     SqlValidatorUtilTest.class,