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 2020/06/24 22:32:40 UTC

[calcite] branch master updated: [CALCITE-4087] Hoist, a utility to replace literals in a SQL string with placeholders

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


The following commit(s) were added to refs/heads/master by this push:
     new 6f90aca  [CALCITE-4087] Hoist, a utility to replace literals in a SQL string with placeholders
6f90aca is described below

commit 6f90acaaac962f666741dae8fa20170e1d9a71e4
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Tue Jun 23 17:06:50 2020 -0700

    [CALCITE-4087] Hoist, a utility to replace literals in a SQL string with placeholders
---
 .../org/apache/calcite/test/BabelParserTest.java   |  39 +++++
 .../main/java/org/apache/calcite/tools/Hoist.java  | 191 +++++++++++++++++++++
 .../apache/calcite/sql/parser/SqlParserTest.java   |  51 ++++++
 3 files changed, 281 insertions(+)

diff --git a/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java b/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
index b5d185f..47d5500 100644
--- a/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
+++ b/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
@@ -16,11 +16,15 @@
  */
 package org.apache.calcite.test;
 
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.dialect.MysqlSqlDialect;
 import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
+import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.parser.SqlParserImplFactory;
 import org.apache.calcite.sql.parser.SqlParserTest;
 import org.apache.calcite.sql.parser.SqlParserUtil;
 import org.apache.calcite.sql.parser.babel.SqlBabelParserImpl;
+import org.apache.calcite.tools.Hoist;
 
 import com.google.common.base.Throwables;
 
@@ -281,4 +285,39 @@ class BabelParserTest extends SqlParserTest {
         + "(`BAR` INTEGER NOT NULL, `BAZ` VARCHAR(30))";
     sql(sql).ok(expected);
   }
+
+  /** Similar to {@link #testHoist()} but using custom parser. */
+  @Test void testHoistMySql() {
+    // SQL contains back-ticks, which require MySQL's quoting,
+    // and DATEADD, which requires Babel.
+    final String sql = "select 1 as x,\n"
+        + "  'ab' || 'c' as y\n"
+        + "from `my emp` /* comment with 'quoted string'? */ as e\n"
+        + "where deptno < 40\n"
+        + "and DATEADD(day, 1, hiredate) > date '2010-05-06'";
+    final SqlDialect dialect = MysqlSqlDialect.DEFAULT;
+    final Hoist.Hoisted hoisted =
+        Hoist.create(Hoist.config()
+            .withParserConfig(
+                dialect.configureParser(SqlParser.configBuilder())
+                    .setParserFactory(SqlBabelParserImpl::new)
+                    .build()))
+            .hoist(sql);
+
+    // Simple toString converts each variable to '?N'
+    final String expected = "select ?0 as x,\n"
+        + "  ?1 || ?2 as y\n"
+        + "from `my emp` /* comment with 'quoted string'? */ as e\n"
+        + "where deptno < ?3\n"
+        + "and DATEADD(day, ?4, hiredate) > ?5";
+    assertThat(hoisted.toString(), is(expected));
+
+    // Custom string converts variables to '[N:TYPE:VALUE]'
+    final String expected2 = "select [0:DECIMAL:1] as x,\n"
+        + "  [1:CHAR:ab] || [2:CHAR:c] as y\n"
+        + "from `my emp` /* comment with 'quoted string'? */ as e\n"
+        + "where deptno < [3:DECIMAL:40]\n"
+        + "and DATEADD(day, [4:DECIMAL:1], hiredate) > [5:DATE:2010-05-06]";
+    assertThat(hoisted.substitute(SqlParserTest::varToStr), is(expected2));
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/tools/Hoist.java b/core/src/main/java/org/apache/calcite/tools/Hoist.java
new file mode 100644
index 0000000..c8e7f21
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/tools/Hoist.java
@@ -0,0 +1,191 @@
+/*
+ * 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.tools;
+
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.SqlShuttle;
+import org.apache.calcite.util.ImmutableBeans;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import java.sql.PreparedStatement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Function;
+import javax.annotation.Nonnull;
+
+/**
+ * Utility that extracts constants from a SQL query.
+ *
+ * <p>Simple use:
+ *
+ * <blockquote><code>
+ * final String sql =<br>
+ *     "select 'x' from emp where deptno &lt; 10";<br>
+ * final Hoist.Hoisted hoisted =<br>
+ *     Hoist.create(Hoist.config()).hoist();<br>
+ * print(hoisted); // "select ?0 from emp where deptno &lt; ?1"
+ * </code></blockquote>
+ *
+ * <p>Calling {@link Hoisted#toString()} generates a string that is similar to
+ * SQL where a user has manually converted all constants to bind variables, and
+ * which could then be executed using {@link PreparedStatement#execute()}.
+ * That is not a goal of this utility, but see
+ * <a href="https://issues.apache.org/jira/browse/CALCITE-963">[CALCITE-963]
+ * Hoist literals</a>.
+ *
+ * <p>For more advanced formatting, use {@link Hoisted#substitute(Function)}.
+ *
+ * <p>Adjust {@link Config} to use a different parser or parsing options.
+ */
+public class Hoist {
+  private final Config config;
+
+  /** Creates a Config. */
+  public static Config config() {
+    return ImmutableBeans.create(Config.class)
+        .withParserConfig(SqlParser.configBuilder().build());
+  }
+
+  /** Creates a Hoist. */
+  public static Hoist create(Config config) {
+    return new Hoist(config);
+  }
+
+  private Hoist(Config config) {
+    this.config = Objects.requireNonNull(config);
+  }
+
+  /** Converts a {@link Variable} to a string "?N",
+   * where N is the {@link Variable#ordinal}. */
+  public static String ordinalString(Variable v) {
+    return "?" + v.ordinal;
+  }
+
+  /** Converts a {@link Variable} to a string "?N",
+   * where N is the {@link Variable#ordinal},
+   * if the fragment is a character literal. Other fragments are unchanged. */
+  public static String ordinalStringIfChar(Variable v) {
+    if (v.node instanceof SqlLiteral
+        && ((SqlLiteral) v.node).getTypeName() == SqlTypeName.CHAR) {
+      return "?" + v.ordinal;
+    } else {
+      return v.sql();
+    }
+  }
+
+  /** Hoists literals in a given SQL string, returning a {@link Hoisted}. */
+  public Hoisted hoist(String sql) {
+    final List<Variable> variables = new ArrayList<>();
+    final SqlParser parser = SqlParser.create(sql, config.parserConfig());
+    final SqlNode node;
+    try {
+      node = parser.parseQuery();
+    } catch (SqlParseException e) {
+      throw new RuntimeException(e);
+    }
+    node.accept(new SqlShuttle() {
+      @Override public SqlNode visit(SqlLiteral literal) {
+        variables.add(new Variable(sql, variables.size(), literal));
+        return super.visit(literal);
+      }
+    });
+    return new Hoisted(sql, variables);
+  }
+
+  /** Configuration. */
+  public interface Config {
+    /** Returns the configuration for the SQL parser. */
+    @ImmutableBeans.Property
+    @Nonnull
+    SqlParser.Config parserConfig();
+
+    /** Sets {@link #parserConfig()}. */
+    Config withParserConfig(SqlParser.Config parserConfig);
+  }
+
+  /** Variable. */
+  public static class Variable {
+    /** Original SQL of whole statement. */
+    public final String originalSql;
+    /** Zero-based ordinal in statement. */
+    public final int ordinal;
+    /** Parse tree node (typically a literal). */
+    public final SqlNode node;
+    /** Zero-based position within the SQL text of start of node. */
+    public final int start;
+    /** Zero-based position within the SQL text after end of node. */
+    public final int end;
+
+    private Variable(String originalSql, int ordinal, SqlNode node) {
+      this.originalSql = Objects.requireNonNull(originalSql);
+      this.ordinal = ordinal;
+      this.node = Objects.requireNonNull(node);
+      final SqlParserPos pos = node.getParserPosition();
+      start = SqlParserUtil.lineColToIndex(originalSql,
+          pos.getLineNum(), pos.getColumnNum());
+      end = SqlParserUtil.lineColToIndex(originalSql,
+          pos.getEndLineNum(), pos.getEndColumnNum()) + 1;
+
+      Preconditions.checkArgument(ordinal >= 0);
+      Preconditions.checkArgument(start >= 0);
+      Preconditions.checkArgument(start <= end);
+      Preconditions.checkArgument(end <= originalSql.length());
+    }
+
+    /** Returns SQL text of the region of the statement covered by this
+     * Variable. */
+    public String sql() {
+      return originalSql.substring(start, end);
+    }
+  }
+
+  /** Result of hoisting. */
+  public static class Hoisted {
+    public final String originalSql;
+    public final List<Variable> variables;
+
+    Hoisted(String originalSql, List<Variable> variables) {
+      this.originalSql = originalSql;
+      this.variables = ImmutableList.copyOf(variables);
+    }
+
+    @Override public String toString() {
+      return substitute(Hoist::ordinalString);
+    }
+
+    /** Returns the SQL string with variables replaced according to the
+     * given substitution function. */
+    public String substitute(Function<Variable, String> fn) {
+      final StringBuilder b = new StringBuilder(originalSql);
+      for (Variable variable : Lists.reverse(variables)) {
+        final String s = fn.apply(variable);
+        b.replace(variable.start, variable.end, s);
+      }
+      return b.toString();
+    }
+  }
+}
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 64157a9..e5d3cec 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
@@ -23,6 +23,7 @@ import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlExplain;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlSetOption;
@@ -35,6 +36,7 @@ import org.apache.calcite.sql.util.SqlShuttle;
 import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.test.DiffTestCase;
+import org.apache.calcite.tools.Hoist;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.ConversionUtil;
 import org.apache.calcite.util.Pair;
@@ -8798,6 +8800,55 @@ public class SqlParserTest {
     sql(sql4).fails("(?s).*Encountered \"a .\" at .*");
   }
 
+  /** Tests {@link Hoist}. */
+  @Test protected void testHoist() {
+    final String sql = "select 1 as x,\n"
+        + "  'ab' || 'c' as y\n"
+        + "from emp /* comment with 'quoted string'? */ as e\n"
+        + "where deptno < 40\n"
+        + "and hiredate > date '2010-05-06'";
+    final Hoist.Hoisted hoisted = Hoist.create(Hoist.config()).hoist(sql);
+
+    // Simple toString converts each variable to '?N'
+    final String expected = "select ?0 as x,\n"
+        + "  ?1 || ?2 as y\n"
+        + "from emp /* comment with 'quoted string'? */ as e\n"
+        + "where deptno < ?3\n"
+        + "and hiredate > ?4";
+    assertThat(hoisted.toString(), is(expected));
+
+    // As above, using the function explicitly.
+    assertThat(hoisted.substitute(Hoist::ordinalString), is(expected));
+
+    // Simple toString converts each variable to '?N'
+    final String expected1 = "select 1 as x,\n"
+        + "  ?1 || ?2 as y\n"
+        + "from emp /* comment with 'quoted string'? */ as e\n"
+        + "where deptno < 40\n"
+        + "and hiredate > date '2010-05-06'";
+    assertThat(hoisted.substitute(Hoist::ordinalStringIfChar), is(expected1));
+
+    // Custom function converts variables to '[N:TYPE:VALUE]'
+    final String expected2 = "select [0:DECIMAL:1] as x,\n"
+        + "  [1:CHAR:ab] || [2:CHAR:c] as y\n"
+        + "from emp /* comment with 'quoted string'? */ as e\n"
+        + "where deptno < [3:DECIMAL:40]\n"
+        + "and hiredate > [4:DATE:2010-05-06]";
+    assertThat(hoisted.substitute(SqlParserTest::varToStr), is(expected2));
+  }
+
+  protected static String varToStr(Hoist.Variable v) {
+    if (v.node instanceof SqlLiteral) {
+      SqlLiteral literal = (SqlLiteral) v.node;
+      return "[" + v.ordinal
+          + ":" + literal.getTypeName()
+          + ":" + literal.toValue()
+          + "]";
+    } else {
+      return "[" + v.ordinal + "]";
+    }
+  }
+
   //~ Inner Interfaces -------------------------------------------------------
 
   /**