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 2019/03/08 18:18:14 UTC

[calcite] 02/03: [CALCITE-2674] Column name with escape character should not be verified as a function when the column name is same with it (Danny Chan)

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

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

commit 870d5f48de16d44e44a09891018ba2c138980f68
Author: yuzhao.cyz <yu...@alibaba-inc.com>
AuthorDate: Thu Dec 6 17:33:55 2018 +0800

    [CALCITE-2674] Column name with escape character should not be verified as a function when the column name is same with it (Danny Chan)
    
    Fix-ups (Julian Hyde):
    * Make SqlParserPos immutable, and create a sub-class for quoted identifiers;
    * Add case to SqlParserTest;
    * Convert 'assert' to 'assertThat' in a test.
    
    Close apache/calcite#958
---
 core/src/main/codegen/templates/Parser.jj          | 110 ++++++++++-------
 .../java/org/apache/calcite/sql/SqlIdentifier.java |  12 ++
 .../main/java/org/apache/calcite/sql/SqlUtil.java  |   2 +-
 .../apache/calcite/sql/parser/SqlParserPos.java    |  31 +++++
 .../apache/calcite/sql/parser/SqlParserTest.java   |  30 +++++
 .../apache/calcite/sql/validate/LexEscapeTest.java | 133 +++++++++++++++++++++
 .../java/org/apache/calcite/test/CalciteSuite.java |   2 +
 7 files changed, 273 insertions(+), 47 deletions(-)

diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index 56a3a7f..1cfb764 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -1559,7 +1559,7 @@ List<SqlNode> SelectList() :
 SqlNode SelectItem() :
 {
     SqlNode e;
-    SqlIdentifier id;
+    final SqlIdentifier id;
 }
 {
     e = SelectExpression()
@@ -1812,8 +1812,7 @@ SqlNode TableRef2(boolean lateral) :
     SqlNode tableRef;
     SqlNode over;
     SqlNodeList extendList = null;
-    String alias;
-    final SqlIdentifier id;
+    final SqlIdentifier alias;
     final Span s, s2;
     SqlNodeList args;
     SqlNode sample;
@@ -1897,18 +1896,16 @@ SqlNode TableRef2(boolean lateral) :
         tableRef = ExtendedTableRef()
     )
     [
-        [ <AS> ] alias = Identifier() {
-            id = new SqlIdentifier(alias, getPos());
-        }
+        [ <AS> ] alias = SimpleIdentifier()
         [ columnAliasList = ParenthesizedSimpleIdentifierList() ]
         {
             if (columnAliasList == null) {
                 tableRef = SqlStdOperatorTable.AS.createCall(
-                    Span.of(tableRef).end(this), tableRef, id);
+                    Span.of(tableRef).end(this), tableRef, alias);
             } else {
                 List<SqlNode> idList = new ArrayList<SqlNode>();
                 idList.add(tableRef);
-                idList.add(id);
+                idList.add(alias);
                 idList.addAll(columnAliasList.getList());
                 tableRef = SqlStdOperatorTable.AS.createCall(
                     Span.of(tableRef).end(this), idList);
@@ -3007,7 +3004,7 @@ void Expression2b(ExprContext exprContext, List<Object> list) :
 {
     SqlNode e;
     SqlOperator op;
-    String p;
+    SqlIdentifier p;
 }
 {
     (
@@ -3021,11 +3018,11 @@ void Expression2b(ExprContext exprContext, List<Object> list) :
     }
     (
         <DOT>
-        p = Identifier() {
+        p = SimpleIdentifier() {
             list.add(
                 new SqlParserUtil.ToTreeListItem(
                     SqlStdOperatorTable.DOT, getPos()));
-            list.add(new SqlIdentifier(p, getPos()));
+            list.add(p);
         }
     )*
 }
@@ -3052,7 +3049,7 @@ List<Object> Expression2(ExprContext exprContext) :
     SqlNodeList nodeList;
     SqlNode e;
     SqlOperator op;
-    String p;
+    SqlIdentifier p;
     final Span s = span();
 }
 {
@@ -3177,11 +3174,11 @@ List<Object> Expression2(ExprContext exprContext) :
                 }
                 (
                     <DOT>
-                    p = Identifier() {
+                    p = SimpleIdentifier() {
                         list.add(
                             new SqlParserUtil.ToTreeListItem(
                                 SqlStdOperatorTable.DOT, getPos()));
-                        list.add(new SqlIdentifier(p, getPos()));
+                        list.add(p);
                     }
                 )*
             |
@@ -4157,41 +4154,50 @@ SqlDynamicParam DynamicParam() :
     }
 }
 
-
 /**
- * Parses a simple identifier as a string.
+ * Parses one segment of an identifier that may be composite.
+ *
+ * <p>Each time it reads an identifier it writes one element to each list;
+ * the entry in {@code positions} records its position and whether the
+ * segment was quoted.
  */
-String Identifier() :
+void IdentifierSegment(List<String> names, List<SqlParserPos> positions) :
 {
-    String id;
+    final String id;
     char unicodeEscapeChar = BACKSLASH;
+    final SqlParserPos pos;
+    final Span span;
 }
 {
     (
-        <IDENTIFIER>
-        {
+        <IDENTIFIER> {
             id = unquotedIdentifier();
+            pos = getPos();
         }
     |
         <QUOTED_IDENTIFIER> {
             id = SqlParserUtil.strip(getToken(0).image, DQ, DQ, DQDQ,
                 quotedCasing);
+            pos = getPos().withQuoting(true);
         }
     |
         <BACK_QUOTED_IDENTIFIER> {
             id = SqlParserUtil.strip(getToken(0).image, "`", "`", "``",
                 quotedCasing);
+            pos = getPos().withQuoting(true);
         }
     |
         <BRACKET_QUOTED_IDENTIFIER> {
             id = SqlParserUtil.strip(getToken(0).image, "[", "]", "]]",
                 quotedCasing);
+            pos = getPos().withQuoting(true);
         }
     |
         <UNICODE_QUOTED_IDENTIFIER> {
-            id = getToken(0).image;
-            id = id.substring(id.indexOf('"'));
-            id = SqlParserUtil.strip(id, DQ, DQ, DQDQ, quotedCasing);
+            span = span();
+            String image = getToken(0).image;
+            image = image.substring(image.indexOf('"'));
+            image = SqlParserUtil.strip(image, DQ, DQ, DQDQ, quotedCasing);
         }
         [
             <UESCAPE> <QUOTED_STRING> {
@@ -4200,19 +4206,38 @@ String Identifier() :
             }
         ]
         {
-            SqlLiteral lit = SqlLiteral.createCharString(id, "UTF16", getPos());
+            pos = span.end(this).withQuoting(true);
+            SqlLiteral lit = SqlLiteral.createCharString(image, "UTF16", pos);
             lit = lit.unescapeUnicode(unicodeEscapeChar);
-            return lit.toValue();
+            id = lit.toValue();
         }
     |
-        id = NonReservedKeyWord()
+        id = NonReservedKeyWord() {
+            pos = getPos();
+        }
     )
     {
         if (id.length() > this.identifierMaxLength) {
-            throw SqlUtil.newContextException(getPos(),
+            throw SqlUtil.newContextException(pos,
                 RESOURCE.identifierTooLong(id, this.identifierMaxLength));
         }
-        return id;
+        names.add(id);
+        if (positions != null) {
+            positions.add(pos);
+        }
+    }
+}
+
+/**
+ * Parses a simple identifier as a String.
+ */
+String Identifier() :
+{
+    final List<String> names = new ArrayList<String>();
+}
+{
+    IdentifierSegment(names, null) {
+        return names.get(0);
     }
 }
 
@@ -4221,11 +4246,12 @@ String Identifier() :
  */
 SqlIdentifier SimpleIdentifier() :
 {
-    final String p;
+    final List<String> names = new ArrayList<String>();
+    final List<SqlParserPos> positions = new ArrayList<SqlParserPos>();
 }
 {
-    p = Identifier() {
-        return new SqlIdentifier(p, getPos());
+    IdentifierSegment(names, positions) {
+        return new SqlIdentifier(names.get(0), positions.get(0));
     }
 }
 
@@ -4268,38 +4294,30 @@ SqlNodeList ParenthesizedSimpleIdentifierList() :
  */
 SqlIdentifier CompoundIdentifier() :
 {
-    List<String> list = new ArrayList<String>();
-    List<SqlParserPos> posList = new ArrayList<SqlParserPos>();
-    String p;
+    final List<String> nameList = new ArrayList<String>();
+    final List<SqlParserPos> posList = new ArrayList<SqlParserPos>();
     boolean star = false;
 }
 {
-    p = Identifier()
-    {
-        posList.add(getPos());
-        list.add(p);
-    }
+    IdentifierSegment(nameList, posList)
     (
         <DOT>
-        p = Identifier() {
-            list.add(p);
-            posList.add(getPos());
-        }
+        IdentifierSegment(nameList, posList)
     )*
     (
         <DOT>
         <STAR> {
             star = true;
-            list.add("");
+            nameList.add("");
             posList.add(getPos());
         }
     )?
     {
         SqlParserPos pos = SqlParserPos.sum(posList);
         if (star) {
-            return SqlIdentifier.star(list, pos, posList);
+            return SqlIdentifier.star(nameList, pos, posList);
         }
-        return new SqlIdentifier(list, null, pos, posList);
+        return new SqlIdentifier(nameList, null, pos, posList);
     }
 }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
index ba6fb0c..fa20d45 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
@@ -358,6 +358,18 @@ public class SqlIdentifier extends SqlNode {
     return names.size() == 1 && !isStar();
   }
 
+  /**
+   * Returns whether the {@code i}th component of a compound identifier is
+   * quoted.
+   *
+   * @param i Ordinal of component
+   * @return Whether i'th component is quoted
+   */
+  public boolean isComponentQuoted(int i) {
+    return componentPositions != null
+        && componentPositions.get(i).isQuoted();
+  }
+
   public SqlMonotonicity getMonotonicity(SqlValidatorScope scope) {
     // for "star" column, whether it's static or dynamic return not_monotonic directly.
     if (Util.last(names).equals("") || DynamicRecordType.isDynamicStarColName(Util.last(names))) {
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 461b7ed..8b26190 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -663,7 +663,7 @@ public abstract class SqlUtil {
   public static SqlCall makeCall(
       SqlOperatorTable opTab,
       SqlIdentifier id) {
-    if (id.names.size() == 1) {
+    if (id.names.size() == 1 && !id.isComponentQuoted(0)) {
       final List<SqlOperator> list = new ArrayList<>();
       opTab.lookupOperatorOverloads(id, null, SqlSyntax.FUNCTION, list);
       for (SqlOperator operator : list) {
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
index b744e64..582b1d1 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
@@ -125,6 +125,24 @@ public class SqlParserPos implements Serializable {
     return endColumnNumber;
   }
 
+  /** Returns a {@code SqlParserPos} the same as this but quoted. */
+  public SqlParserPos withQuoting(boolean quoted) {
+    if (isQuoted() == quoted) {
+      return this;
+    } else if (quoted) {
+      return new QuotedParserPos(lineNumber, columnNumber, endLineNumber,
+          endColumnNumber);
+    } else {
+      return new SqlParserPos(lineNumber, columnNumber, endLineNumber,
+          endColumnNumber);
+    }
+  }
+
+  /** @return true if this SqlParserPos is quoted. **/
+  public boolean isQuoted() {
+    return false;
+  }
+
   @Override public String toString() {
     return RESOURCE.parserContext(lineNumber, columnNumber).str();
   }
@@ -292,6 +310,19 @@ public class SqlParserPos implements Serializable {
     return lineNumber == pos.lineNumber
         && columnNumber == pos.columnNumber;
   }
+
+  /** Parser position for an identifier segment that is quoted. */
+  private static class QuotedParserPos extends SqlParserPos {
+    QuotedParserPos(int startLineNumber, int startColumnNumber,
+        int endLineNumber, int endColumnNumber) {
+      super(startLineNumber, startColumnNumber, endLineNumber,
+          endColumnNumber);
+    }
+
+    @Override public boolean isQuoted() {
+      return true;
+    }
+  }
 }
 
 // End SqlParserPos.java
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 62d7852..b53df4d 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -18,6 +18,8 @@ package org.apache.calcite.sql.parser;
 
 import org.apache.calcite.avatica.util.Casing;
 import org.apache.calcite.avatica.util.Quoting;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlSetOption;
@@ -40,6 +42,7 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSortedSet;
 
 import org.hamcrest.BaseMatcher;
+import org.hamcrest.CustomTypeSafeMatcher;
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;
 import org.junit.Ignore;
@@ -59,6 +62,7 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import javax.annotation.Nonnull;
 
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
@@ -653,6 +657,21 @@ public class SqlParserTest {
     };
   }
 
+  /** Returns a {@link Matcher} that succeeds if the given {@link SqlNode} is a
+   * VALUES that contains a ROW that contains an identifier whose {@code i}th
+   * element is quoted. */
+  @Nonnull private static Matcher<SqlNode> isQuoted(final int i,
+      final boolean quoted) {
+    return new CustomTypeSafeMatcher<SqlNode>("quoting") {
+      protected boolean matchesSafely(SqlNode item) {
+        final SqlCall valuesCall = (SqlCall) item;
+        final SqlCall rowCall = valuesCall.operand(0);
+        final SqlIdentifier id = rowCall.operand(0);
+        return id.isComponentQuoted(i) == quoted;
+      }
+    };
+  }
+
   protected SortedSet<String> getReservedKeywords() {
     return keywords("c");
   }
@@ -1824,6 +1843,11 @@ public class SqlParserTest {
 
     checkExp("myMap[field] + myArray[1 + 2]",
         "(`MYMAP`[`FIELD`] + `MYARRAY`[(1 + 2)])");
+
+    getTester().checkNode("VALUES a", isQuoted(0, false));
+    getTester().checkNode("VALUES \"a\"", isQuoted(0, true));
+    getTester().checkNode("VALUES \"a\".\"b\"", isQuoted(1, true));
+    getTester().checkNode("VALUES \"a\".b", isQuoted(1, false));
   }
 
   @Test public void testBackTickIdentifier() {
@@ -1837,6 +1861,9 @@ public class SqlParserTest {
 
     checkExp("myMap[field] + myArray[1 + 2]",
         "(`MYMAP`[`FIELD`] + `MYARRAY`[(1 + 2)])");
+
+    getTester().checkNode("VALUES a", isQuoted(0, false));
+    getTester().checkNode("VALUES `a`", isQuoted(0, true));
   }
 
   @Test public void testBracketIdentifier() {
@@ -1863,6 +1890,9 @@ public class SqlParserTest {
         "SELECT *\n"
             + "FROM `MYMAP` AS `field`,\n"
             + "`MYARRAY` AS `1 + 2`");
+
+    getTester().checkNode("VALUES a", isQuoted(0, false));
+    getTester().checkNode("VALUES [a]", isQuoted(0, true));
   }
 
   @Test public void testBackTickQuery() {
diff --git a/core/src/test/java/org/apache/calcite/sql/validate/LexEscapeTest.java b/core/src/test/java/org/apache/calcite/sql/validate/LexEscapeTest.java
new file mode 100644
index 0000000..3763a88
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/sql/validate/LexEscapeTest.java
@@ -0,0 +1,133 @@
+/*
+ * 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.sql.validate;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableProject;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParser.Config;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+import org.apache.calcite.tools.Program;
+import org.apache.calcite.tools.Programs;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+
+import com.google.common.collect.ImmutableList;
+
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Testing {@link SqlValidator} and {@link Lex} quoting.
+ */
+public class LexEscapeTest {
+
+  private static Planner getPlanner(List<RelTraitDef> traitDefs,
+      Config parserConfig, Program... programs) {
+    final SchemaPlus rootSchema = Frameworks.createRootSchema(true);
+    rootSchema.add("TMP", new AbstractTable() {
+      @Override public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+        return typeFactory.createStructType(
+            ImmutableList.of(typeFactory.createSqlType(SqlTypeName.VARCHAR),
+                typeFactory.createSqlType(SqlTypeName.INTEGER)),
+            ImmutableList.of("localtime", "current_timestamp"));
+      }
+    });
+    final FrameworkConfig config = Frameworks.newConfigBuilder()
+        .parserConfig(parserConfig)
+        .defaultSchema(rootSchema)
+        .traitDefs(traitDefs)
+        .programs(programs)
+        .operatorTable(SqlStdOperatorTable.instance())
+        .build();
+    return Frameworks.getPlanner(config);
+  }
+
+  private static void runProjectQueryWithLex(Lex lex, String sql)
+      throws SqlParseException, ValidationException, RelConversionException {
+    Config javaLex = SqlParser.configBuilder().setLex(lex).build();
+    Planner planner = getPlanner(null, javaLex, Programs.ofRules(Programs.RULE_SET));
+    SqlNode parse = planner.parse(sql);
+    SqlNode validate = planner.validate(parse);
+    RelNode convert = planner.rel(validate).rel;
+    RelTraitSet traitSet =
+        planner.getEmptyTraitSet().replace(EnumerableConvention.INSTANCE);
+    RelNode transform = planner.transform(0, traitSet, convert);
+    assertThat(transform, instanceOf(EnumerableProject.class));
+    List<RelDataTypeField> fields = transform.getRowType().getFieldList();
+    // Get field type from sql text and validate we parsed it after validation.
+    assertThat(fields.size(), is(4));
+    assertThat(fields.get(0).getType().getSqlTypeName(), is(SqlTypeName.VARCHAR));
+    assertThat(fields.get(1).getType().getSqlTypeName(), is(SqlTypeName.TIME));
+    assertThat(fields.get(2).getType().getSqlTypeName(), is(SqlTypeName.INTEGER));
+    assertThat(fields.get(3).getType().getSqlTypeName(), is(SqlTypeName.TIMESTAMP));
+  }
+
+  @Test public void testCalciteEscapeOracle()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select \"localtime\", localtime, "
+        + "\"current_timestamp\", current_timestamp from TMP";
+    runProjectQueryWithLex(Lex.ORACLE, sql);
+  }
+
+  @Test public void testCalciteEscapeMySql()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select `localtime`, localtime, `current_timestamp`, current_timestamp from TMP";
+    runProjectQueryWithLex(Lex.MYSQL, sql);
+  }
+
+  @Test public void testCalciteEscapeMySqlAnsi()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select \"localtime\", localtime, "
+        + "\"current_timestamp\", current_timestamp from TMP";
+    runProjectQueryWithLex(Lex.MYSQL_ANSI, sql);
+  }
+
+  @Test public void testCalciteEscapeSqlServer()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select [localtime], localtime, [current_timestamp], current_timestamp from TMP";
+    runProjectQueryWithLex(Lex.SQL_SERVER, sql);
+  }
+
+  @Test public void testCalciteEscapeJava()
+      throws SqlParseException, ValidationException, RelConversionException {
+    String sql = "select `localtime`, localtime, `current_timestamp`, current_timestamp from TMP";
+    runProjectQueryWithLex(Lex.JAVA, sql);
+  }
+}
+
+// End LexEscapeTest.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 7fa5699..1024e80 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -54,6 +54,7 @@ import org.apache.calcite.sql.test.SqlTypeNameTest;
 import org.apache.calcite.sql.type.SqlTypeFactoryTest;
 import org.apache.calcite.sql.type.SqlTypeUtilTest;
 import org.apache.calcite.sql.validate.LexCaseSensitiveTest;
+import org.apache.calcite.sql.validate.LexEscapeTest;
 import org.apache.calcite.sql.validate.SqlValidatorUtilTest;
 import org.apache.calcite.test.enumerable.EnumerableCorrelateTest;
 import org.apache.calcite.test.fuzzer.RexProgramFuzzyTest;
@@ -154,6 +155,7 @@ import org.junit.runners.Suite;
     EnumerableCorrelateTest.class,
     LookupOperatorOverloadsTest.class,
     LexCaseSensitiveTest.class,
+    LexEscapeTest.class,
     CollationConversionTest.class,
     TraitConversionTest.class,
     ComboRuleTest.class,