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 2016/05/04 02:31:00 UTC

[1/2] calcite git commit: [CALCITE-1219] Add method SqlOperatorBinding.isOperandLiteral() (Hsuan-Yi Chu)

Repository: calcite
Updated Branches:
  refs/heads/master 1be4056b4 -> 44a6ba67b


[CALCITE-1219] Add method SqlOperatorBinding.isOperandLiteral() (Hsuan-Yi Chu)

isOperandLiteral() determines whether an operand is a literal.

Close apache/calcite#222


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

Branch: refs/heads/master
Commit: 1b4209ec973b8b86749fb3cc14262cdd17df0999
Parents: 1be4056
Author: hsuanyi <hs...@apache.org>
Authored: Wed Apr 27 20:54:29 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue May 3 11:36:43 2016 -0700

----------------------------------------------------------------------
 .../org/apache/calcite/rex/RexCallBinding.java  |  4 +
 .../java/org/apache/calcite/rex/RexUtil.java    | 33 +++++++++
 .../org/apache/calcite/sql/SqlCallBinding.java  |  4 +
 .../apache/calcite/sql/SqlOperatorBinding.java  | 11 +++
 .../java/org/apache/calcite/sql/SqlUtil.java    | 36 ++++++++-
 .../java/org/apache/calcite/util/UtilTest.java  | 78 ++++++++++++++++++++
 6 files changed, 164 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/1b4209ec/core/src/main/java/org/apache/calcite/rex/RexCallBinding.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexCallBinding.java b/core/src/main/java/org/apache/calcite/rex/RexCallBinding.java
index c2828aa..96ce806 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCallBinding.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCallBinding.java
@@ -112,6 +112,10 @@ public class RexCallBinding extends SqlOperatorBinding {
     return RexUtil.isNullLiteral(operands.get(ordinal), allowCast);
   }
 
+  @Override public boolean isOperandLiteral(int ordinal, boolean allowCast) {
+    return RexUtil.isLiteral(operands.get(ordinal), allowCast);
+  }
+
   // implement SqlOperatorBinding
   public int getOperandCount() {
     return operands.size();

http://git-wip-us.apache.org/repos/asf/calcite/blob/1b4209ec/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index 86ffbfb..d8221c6 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -230,6 +230,39 @@ public class RexUtil {
     }
   }
 
+  /**
+   * Returns whether a node represents a literal.
+   *
+   * <p>Examples:
+   *
+   * <ul>
+   * <li>For <code>CAST(literal AS <i>type</i>)</code>, returns true if <code>
+   * allowCast</code> is true, false otherwise.
+   * <li>For <code>CAST(CAST(literal AS <i>type</i>) AS <i>type</i>))</code>,
+   * returns false.
+   * </ul>
+   *
+   * @param node The node, never null.
+   * @param allowCast whether to regard CAST(literal) as a literal
+   * @return Whether the node is a literal
+   */
+  public static boolean isLiteral(RexNode node, boolean allowCast) {
+    assert node != null;
+    if (node instanceof RexLiteral) {
+      return true;
+    }
+    if (allowCast) {
+      if (node.isA(SqlKind.CAST)) {
+        RexCall call = (RexCall) node;
+        if (isLiteral(call.operands.get(0), false)) {
+          // node is "CAST(literal as type)"
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
   /** Returns whether an expression is a cast just for the purposes of
    * nullability, not changing any other aspect of the type. */
   public static boolean isNullabilityCast(RelDataTypeFactory typeFactory,

http://git-wip-us.apache.org/repos/asf/calcite/blob/1b4209ec/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java b/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
index f90f7be..24c3f4c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
@@ -223,6 +223,10 @@ public class SqlCallBinding extends SqlOperatorBinding {
     return SqlUtil.isNullLiteral(call.operand(ordinal), allowCast);
   }
 
+  @Override public boolean isOperandLiteral(int ordinal, boolean allowCast) {
+    return SqlUtil.isLiteral(call.operand(ordinal), allowCast);
+  }
+
   @Override public int getOperandCount() {
     return call.getOperandList().size();
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1b4209ec/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java b/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
index 2ca13f9..5baeaf0 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
@@ -152,6 +152,17 @@ public abstract class SqlOperatorBinding {
   }
 
   /**
+   * Determines whether an operand is a literal.
+   *
+   * @param ordinal   zero-based ordinal of operand of interest
+   * @param allowCast whether to regard CAST(literal) as a literal
+   * @return whether operand is literal
+   */
+  public boolean isOperandLiteral(int ordinal, boolean allowCast) {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
    * @return the number of bound operands
    */
   public abstract int getOperandCount();

http://git-wip-us.apache.org/repos/asf/calcite/blob/1b4209ec/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
index 40e1255..a620f2a 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -188,6 +188,39 @@ public abstract class SqlUtil {
   /**
    * Returns whether a node is a literal.
    *
+   * <p>Examples:
+   *
+   * <ul>
+   * <li>For <code>CAST(literal AS <i>type</i>)</code>, returns true if <code>
+   * allowCast</code> is true, false otherwise.
+   * <li>For <code>CAST(CAST(literal AS <i>type</i>) AS <i>type</i>))</code>,
+   * returns false.
+   * </ul>
+   *
+   * @param node The node, never null.
+   * @param allowCast whether to regard CAST(literal) as a literal
+   * @return Whether the node is a literal
+   */
+  public static boolean isLiteral(SqlNode node, boolean allowCast) {
+    assert node != null;
+    if (node instanceof SqlLiteral) {
+      return true;
+    }
+    if (allowCast) {
+      if (node.getKind() == SqlKind.CAST) {
+        SqlCall call = (SqlCall) node;
+        if (isLiteral(call.operand(0), false)) {
+          // node is "CAST(literal as type)"
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Returns whether a node is a literal.
+   *
    * <p>Many constructs which require literals also accept <code>CAST(NULL AS
    * <i>type</i>)</code>. This method does not accept casts, so you should
    * call {@link #isNullLiteral} first.
@@ -196,8 +229,7 @@ public abstract class SqlUtil {
    * @return Whether the node is a literal
    */
   public static boolean isLiteral(SqlNode node) {
-    assert node != null;
-    return node instanceof SqlLiteral;
+    return isLiteral(node, false);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/1b4209ec/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index 548b538..c3a7670 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -22,11 +22,26 @@ import org.apache.calcite.examples.RelBuilderExample;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.function.Parameter;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.runtime.ConsList;
 import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.runtime.Resources;
 import org.apache.calcite.runtime.Utilities;
+import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.util.SqlBuilder;
 import org.apache.calcite.sql.util.SqlString;
 import org.apache.calcite.test.DiffTestCase;
@@ -81,6 +96,7 @@ import static org.hamcrest.CoreMatchers.startsWith;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -1752,6 +1768,68 @@ public class UtilTest {
     }
     assertThat(local2.get(), is("x"));
   }
+
+  /** Tests {@link org.apache.calcite.sql.SqlUtil#isLiteral(SqlNode, boolean)}.
+   *
+   * <p>The method {@link org.apache.calcite.sql.SqlUtil#isLiteral(SqlNode, boolean)}
+   * was added to enhance Calcite's public API
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1219">[CALCITE-1219]</a>
+   */
+  @Test public void testSqlNodeLiteral() {
+    final RelDataTypeFactory relDataTypeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+    final SqlDataTypeSpec integerType = SqlTypeUtil.convertTypeToSpec(
+        relDataTypeFactory.createSqlType(SqlTypeName.INTEGER));
+    final SqlNode literal = SqlLiteral.createExactNumeric(
+        "0",
+        SqlParserPos.ZERO);
+    final SqlNode castLiteral = SqlStdOperatorTable.CAST.createCall(
+        SqlParserPos.ZERO,
+        literal,
+        integerType);
+    final SqlNode castCastLiteral = SqlStdOperatorTable.CAST.createCall(
+        SqlParserPos.ZERO,
+        castLiteral,
+        integerType);
+
+    // SqlLiteral is considered as a Literal
+    assertSame(true, SqlUtil.isLiteral(literal, true));
+    // CAST(SqlLiteral as type) is considered as a Literal
+    assertSame(true, SqlUtil.isLiteral(castLiteral, true));
+    // CAST(CAST(SqlLiteral as type) as type) is NOT considered as a Literal
+    assertSame(false, SqlUtil.isLiteral(castCastLiteral, true));
+  }
+
+  /** Tests {@link org.apache.calcite.rex.RexUtil#isLiteral(RexNode, boolean)}.
+   *
+   * <p>The method {@link org.apache.calcite.rex.RexUtil#isLiteral(RexNode, boolean)}
+   * was added to enhance Calcite's public API
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1219">[CALCITE-1219]</a>
+   */
+  @Test public void testRexNodeLiteral() {
+    final RelDataTypeFactory relDataTypeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+    final RelDataType integerDataType = relDataTypeFactory.createSqlType(SqlTypeName.INTEGER);
+
+    final RexBuilder rexBuilder = new RexBuilder(relDataTypeFactory);
+    final RexNode literal = rexBuilder.makeZeroLiteral(
+        relDataTypeFactory.createSqlType(SqlTypeName.INTEGER));
+
+    final RexNode castLiteral = rexBuilder.makeCall(
+        integerDataType,
+        SqlStdOperatorTable.CAST,
+        Lists.newArrayList(literal));
+
+    final RexNode castCastLiteral = rexBuilder.makeCall(
+        integerDataType,
+        SqlStdOperatorTable.CAST,
+        Lists.newArrayList(castLiteral));
+
+    // RexLiteral is considered as a Literal
+    assertSame(true, RexUtil.isLiteral(literal, true));
+    // CAST(RexLiteral as type) is considered as a Literal
+    assertSame(true, RexUtil.isLiteral(castLiteral, true));
+    // CAST(CAST(RexLiteral as type) as type) is NOT considered as a Literal
+    assertSame(false, RexUtil.isLiteral(castCastLiteral, true));
+  }
 }
 
 // End UtilTest.java


[2/2] calcite git commit: Fix up [CALCITE-1219], moving tests to new class SqlOperatorBindingTest

Posted by jh...@apache.org.
Fix up [CALCITE-1219], moving tests to new class SqlOperatorBindingTest


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

Branch: refs/heads/master
Commit: 44a6ba67b92c193468f0de9dc3417c218624e69f
Parents: 1b4209e
Author: Julian Hyde <jh...@apache.org>
Authored: Tue May 3 11:56:16 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue May 3 11:56:16 2016 -0700

----------------------------------------------------------------------
 .../org/apache/calcite/test/CalciteSuite.java   |   1 +
 .../calcite/test/SqlOperatorBindingTest.java    | 126 +++++++++++++++++++
 .../java/org/apache/calcite/util/UtilTest.java  |  78 ------------
 3 files changed, 127 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/44a6ba67/core/src/test/java/org/apache/calcite/test/CalciteSuite.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 d9c9e4a..e864910 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -86,6 +86,7 @@ import org.junit.runners.Suite;
     TraitPropagationTest.class,
     RelWriterTest.class,
     RexProgramTest.class,
+    SqlOperatorBindingTest.class,
     RexTransformerTest.class,
     BinarySearchTest.class,
     EnumerablesTest.class,

http://git-wip-us.apache.org/repos/asf/calcite/blob/44a6ba67/core/src/test/java/org/apache/calcite/test/SqlOperatorBindingTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlOperatorBindingTest.java b/core/src/test/java/org/apache/calcite/test/SqlOperatorBindingTest.java
new file mode 100644
index 0000000..3762c45
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/test/SqlOperatorBindingTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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.test;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+
+import com.google.common.collect.Lists;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertSame;
+
+/**
+ * Unit tests for {@link RexProgram} and
+ * {@link RexProgramBuilder}.
+ */
+public class SqlOperatorBindingTest {
+  private RexBuilder rexBuilder;
+  private RelDataType integerDataType;
+  private SqlDataTypeSpec integerType;
+
+  //~ Methods ----------------------------------------------------------------
+
+  /**
+   * Creates a SqlOperatorBindingTest.
+   */
+  public SqlOperatorBindingTest() {
+    super();
+  }
+
+  @Before
+  public void setUp() {
+    JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+    integerDataType = typeFactory.createSqlType(SqlTypeName.INTEGER);
+    integerType = SqlTypeUtil.convertTypeToSpec(integerDataType);
+    rexBuilder = new RexBuilder(typeFactory);
+  }
+
+  /** Tests {@link org.apache.calcite.sql.SqlUtil#isLiteral(SqlNode, boolean)},
+   * which was added to enhance Calcite's public API
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1219">[CALCITE-1219]
+   * Add a method to SqlOperatorBinding to determine whether operand is a
+   * literal</a>.
+   */
+  @Test public void testSqlNodeLiteral() {
+    final SqlNode literal = SqlLiteral.createExactNumeric(
+        "0",
+        SqlParserPos.ZERO);
+    final SqlNode castLiteral = SqlStdOperatorTable.CAST.createCall(
+        SqlParserPos.ZERO,
+        literal,
+        integerType);
+    final SqlNode castCastLiteral = SqlStdOperatorTable.CAST.createCall(
+        SqlParserPos.ZERO,
+        castLiteral,
+        integerType);
+
+    // SqlLiteral is considered as a Literal
+    assertSame(true, SqlUtil.isLiteral(literal, true));
+    // CAST(SqlLiteral as type) is considered as a Literal
+    assertSame(true, SqlUtil.isLiteral(castLiteral, true));
+    // CAST(CAST(SqlLiteral as type) as type) is NOT considered as a Literal
+    assertSame(false, SqlUtil.isLiteral(castCastLiteral, true));
+  }
+
+  /** Tests {@link org.apache.calcite.rex.RexUtil#isLiteral(RexNode, boolean)},
+   * which was added to enhance Calcite's public API
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1219">[CALCITE-1219]
+   * Add a method to SqlOperatorBinding to determine whether operand is a
+   * literal</a>.
+   */
+  @Test public void testRexNodeLiteral() {
+    final RexNode literal = rexBuilder.makeZeroLiteral(
+        integerDataType);
+
+    final RexNode castLiteral = rexBuilder.makeCall(
+        integerDataType,
+        SqlStdOperatorTable.CAST,
+        Lists.newArrayList(literal));
+
+    final RexNode castCastLiteral = rexBuilder.makeCall(
+        integerDataType,
+        SqlStdOperatorTable.CAST,
+        Lists.newArrayList(castLiteral));
+
+    // RexLiteral is considered as a Literal
+    assertSame(true, RexUtil.isLiteral(literal, true));
+    // CAST(RexLiteral as type) is considered as a Literal
+    assertSame(true, RexUtil.isLiteral(castLiteral, true));
+    // CAST(CAST(RexLiteral as type) as type) is NOT considered as a Literal
+    assertSame(false, RexUtil.isLiteral(castCastLiteral, true));
+  }
+}
+
+// End SqlOperatorBindingTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/44a6ba67/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index c3a7670..548b538 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -22,26 +22,11 @@ import org.apache.calcite.examples.RelBuilderExample;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.function.Parameter;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.runtime.ConsList;
 import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.runtime.Resources;
 import org.apache.calcite.runtime.Utilities;
-import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDialect;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlUtil;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.util.SqlBuilder;
 import org.apache.calcite.sql.util.SqlString;
 import org.apache.calcite.test.DiffTestCase;
@@ -96,7 +81,6 @@ import static org.hamcrest.CoreMatchers.startsWith;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -1768,68 +1752,6 @@ public class UtilTest {
     }
     assertThat(local2.get(), is("x"));
   }
-
-  /** Tests {@link org.apache.calcite.sql.SqlUtil#isLiteral(SqlNode, boolean)}.
-   *
-   * <p>The method {@link org.apache.calcite.sql.SqlUtil#isLiteral(SqlNode, boolean)}
-   * was added to enhance Calcite's public API
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-1219">[CALCITE-1219]</a>
-   */
-  @Test public void testSqlNodeLiteral() {
-    final RelDataTypeFactory relDataTypeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
-    final SqlDataTypeSpec integerType = SqlTypeUtil.convertTypeToSpec(
-        relDataTypeFactory.createSqlType(SqlTypeName.INTEGER));
-    final SqlNode literal = SqlLiteral.createExactNumeric(
-        "0",
-        SqlParserPos.ZERO);
-    final SqlNode castLiteral = SqlStdOperatorTable.CAST.createCall(
-        SqlParserPos.ZERO,
-        literal,
-        integerType);
-    final SqlNode castCastLiteral = SqlStdOperatorTable.CAST.createCall(
-        SqlParserPos.ZERO,
-        castLiteral,
-        integerType);
-
-    // SqlLiteral is considered as a Literal
-    assertSame(true, SqlUtil.isLiteral(literal, true));
-    // CAST(SqlLiteral as type) is considered as a Literal
-    assertSame(true, SqlUtil.isLiteral(castLiteral, true));
-    // CAST(CAST(SqlLiteral as type) as type) is NOT considered as a Literal
-    assertSame(false, SqlUtil.isLiteral(castCastLiteral, true));
-  }
-
-  /** Tests {@link org.apache.calcite.rex.RexUtil#isLiteral(RexNode, boolean)}.
-   *
-   * <p>The method {@link org.apache.calcite.rex.RexUtil#isLiteral(RexNode, boolean)}
-   * was added to enhance Calcite's public API
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-1219">[CALCITE-1219]</a>
-   */
-  @Test public void testRexNodeLiteral() {
-    final RelDataTypeFactory relDataTypeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
-    final RelDataType integerDataType = relDataTypeFactory.createSqlType(SqlTypeName.INTEGER);
-
-    final RexBuilder rexBuilder = new RexBuilder(relDataTypeFactory);
-    final RexNode literal = rexBuilder.makeZeroLiteral(
-        relDataTypeFactory.createSqlType(SqlTypeName.INTEGER));
-
-    final RexNode castLiteral = rexBuilder.makeCall(
-        integerDataType,
-        SqlStdOperatorTable.CAST,
-        Lists.newArrayList(literal));
-
-    final RexNode castCastLiteral = rexBuilder.makeCall(
-        integerDataType,
-        SqlStdOperatorTable.CAST,
-        Lists.newArrayList(castLiteral));
-
-    // RexLiteral is considered as a Literal
-    assertSame(true, RexUtil.isLiteral(literal, true));
-    // CAST(RexLiteral as type) is considered as a Literal
-    assertSame(true, RexUtil.isLiteral(castLiteral, true));
-    // CAST(CAST(RexLiteral as type) as type) is NOT considered as a Literal
-    assertSame(false, RexUtil.isLiteral(castCastLiteral, true));
-  }
 }
 
 // End UtilTest.java