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/01/13 20:24:39 UTC

[calcite] 03/03: Allow "CREATE TABLE ... AS SELECT ..." in Quidem tests

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 7fafef95272e91f97db2a40189fde352720c16e4
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Sun Oct 6 16:03:54 2019 -0700

    Allow "CREATE TABLE ... AS SELECT ..." in Quidem tests
---
 core/src/test/codegen/includes/parserImpls.ftl     |  14 +-
 .../parserextensiontesting/SqlCreateTable.java     | 149 +++++++++++++++++----
 core/src/test/resources/sql/blank.iq               |  16 +++
 3 files changed, 150 insertions(+), 29 deletions(-)

diff --git a/core/src/test/codegen/includes/parserImpls.ftl b/core/src/test/codegen/includes/parserImpls.ftl
index 6863c52..be718ab 100644
--- a/core/src/test/codegen/includes/parserImpls.ftl
+++ b/core/src/test/codegen/includes/parserImpls.ftl
@@ -40,10 +40,20 @@ SqlCreate SqlCreateTable(Span s, boolean replace) :
 {
     final SqlIdentifier id;
     final SqlNodeList columnList;
+    final SqlNode query;
 }
 {
-    <TABLE> id = CompoundIdentifier() columnList = ExtendList() {
-        return new SqlCreateTable(s.end(columnList), id, columnList);
+    <TABLE> id = CompoundIdentifier()
+    (
+        columnList = ExtendList()
+    |   { columnList = null; }
+    )
+    (
+        <AS> query = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY)
+    |   { query = null; }
+    )
+    {
+        return new SqlCreateTable(s.end(this), id, columnList, query);
     }
 }
 
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java b/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java
index c2c101f..9717af0 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/parserextensiontesting/SqlCreateTable.java
@@ -25,13 +25,17 @@ import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.linq4j.Queryable;
 import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeImpl;
 import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.schema.impl.ViewTable;
+import org.apache.calcite.schema.impl.ViewTableMacro;
 import org.apache.calcite.sql.SqlCreate;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlExecutableStatement;
@@ -41,19 +45,35 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
+import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.pretty.SqlPrettyWriter;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.test.JdbcTest;
+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.ImmutableNullableList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
 import java.lang.reflect.Type;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.Objects;
+import java.util.function.BiConsumer;
+
+import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
  * Simple test example of a CREATE TABLE statement.
@@ -62,64 +82,139 @@ public class SqlCreateTable extends SqlCreate
     implements SqlExecutableStatement {
   private final SqlIdentifier name;
   private final SqlNodeList columnList;
+  private final SqlNode query;
 
   private static final SqlOperator OPERATOR =
       new SqlSpecialOperator("CREATE TABLE", SqlKind.CREATE_TABLE);
 
   /** Creates a SqlCreateTable. */
   public SqlCreateTable(SqlParserPos pos, SqlIdentifier name,
-      SqlNodeList columnList) {
+      SqlNodeList columnList, SqlNode query) {
     super(OPERATOR, pos, false, false);
-    this.name = name;
-    this.columnList = columnList;
+    this.name = Objects.requireNonNull(name);
+    this.columnList = columnList; // may be null
+    this.query = query; // for "CREATE TABLE ... AS query"; may be null
   }
 
   @Override public List<SqlNode> getOperandList() {
-    return ImmutableList.of(name, columnList);
+    return ImmutableNullableList.of(name, columnList, query);
   }
 
   @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
     writer.keyword("CREATE");
     writer.keyword("TABLE");
     name.unparse(writer, leftPrec, rightPrec);
-    SqlWriter.Frame frame = writer.startList("(", ")");
-    for (Pair<SqlIdentifier, SqlDataTypeSpec> pair : nameTypes()) {
-      writer.sep(",");
-      pair.left.unparse(writer, leftPrec, rightPrec); // name
-      pair.right.unparse(writer, leftPrec, rightPrec); // type
-      if (Boolean.FALSE.equals(pair.right.getNullable())) {
-        writer.keyword("NOT NULL");
-      }
+    if (columnList != null) {
+      SqlWriter.Frame frame = writer.startList("(", ")");
+      nameTypes((name, typeSpec) -> {
+        writer.sep(",");
+        name.unparse(writer, leftPrec, rightPrec);
+        typeSpec.unparse(writer, leftPrec, rightPrec);
+        if (Boolean.FALSE.equals(typeSpec.getNullable())) {
+          writer.keyword("NOT NULL");
+        }
+      });
+      writer.endList(frame);
+    }
+    if (query != null) {
+      writer.keyword("AS");
+      writer.newlineAndIndent();
+      query.unparse(writer, 0, 0);
     }
-    writer.endList(frame);
   }
 
-  /** Creates a list of (name, type) pairs from {@link #columnList}, in which
+  /** Calls an action for each (name, type) pair from {@code columnList}, in which
    * they alternate. */
-  private List<Pair<SqlIdentifier, SqlDataTypeSpec>> nameTypes() {
+  @SuppressWarnings({"unchecked"})
+  private void nameTypes(BiConsumer<SqlIdentifier, SqlDataTypeSpec> consumer) {
     final List list = columnList.getList();
-    //noinspection unchecked
-    return Pair.zip((List<SqlIdentifier>) Util.quotientList(list, 2, 0),
-        Util.quotientList((List<SqlDataTypeSpec>) list, 2, 1));
+    Pair.forEach((List<SqlIdentifier>) Util.quotientList(list, 2, 0),
+        Util.quotientList((List<SqlDataTypeSpec>) list, 2, 1), consumer);
   }
 
   public void execute(CalcitePrepare.Context context) {
-    final List<String> path = context.getDefaultSchemaPath();
-    CalciteSchema schema = context.getRootSchema();
-    for (String p : path) {
-      schema = schema.getSubSchema(p, true);
-    }
+    final CalciteSchema schema =
+        Schemas.subSchema(context.getRootSchema(),
+            context.getDefaultSchemaPath());
     final JavaTypeFactory typeFactory = context.getTypeFactory();
+    final RelDataType queryRowType;
+    if (query != null) {
+      // A bit of a hack: pretend it's a view, to get its row type
+      final String sql = query.toSqlString(CalciteSqlDialect.DEFAULT).getSql();
+      final ViewTableMacro viewTableMacro =
+          ViewTable.viewMacro(schema.plus(), sql, schema.path(null),
+              context.getObjectPath(), false);
+      final TranslatableTable x = viewTableMacro.apply(ImmutableList.of());
+      queryRowType = x.getRowType(typeFactory);
+
+      if (columnList != null
+          && queryRowType.getFieldCount() != columnList.size()) {
+        throw SqlUtil.newContextException(columnList.getParserPosition(),
+            RESOURCE.columnCountMismatch());
+      }
+    } else {
+      queryRowType = null;
+    }
     final RelDataTypeFactory.Builder builder = typeFactory.builder();
-    final SqlValidator validator = new ContextSqlValidator(context, false);
-    for (Pair<SqlIdentifier, SqlDataTypeSpec> pair : nameTypes()) {
-      builder.add(pair.left.getSimple(),
-          pair.right.deriveType(validator, true));
+    if (columnList != null) {
+      final SqlValidator validator = new ContextSqlValidator(context, false);
+      nameTypes((name, typeSpec) ->
+          builder.add(name.getSimple(), typeSpec.deriveType(validator, true)));
+    } else {
+      if (queryRowType == null) {
+        // "CREATE TABLE t" is invalid; because there is no "AS query" we need
+        // a list of column names and types, "CREATE TABLE t (INT c)".
+        throw SqlUtil.newContextException(name.getParserPosition(),
+            RESOURCE.createTableRequiresColumnList());
+      }
+      builder.addAll(queryRowType.getFieldList());
     }
     final RelDataType rowType = builder.build();
     schema.add(name.getSimple(),
         new MutableArrayTable(name.getSimple(),
             RelDataTypeImpl.proto(rowType)));
+    if (query != null) {
+      populate(name, query, context);
+    }
+  }
+
+  /** Populates the table called {@code name} by executing {@code query}. */
+  protected static void populate(SqlIdentifier name, SqlNode query,
+      CalcitePrepare.Context context) {
+    // Generate, prepare and execute an "INSERT INTO table query" statement.
+    // (It's a bit inefficient that we convert from SqlNode to SQL and back
+    // again.)
+    final FrameworkConfig config = Frameworks.newConfigBuilder()
+        .defaultSchema(
+            Objects.requireNonNull(
+                Schemas.subSchema(context.getRootSchema(),
+                    context.getDefaultSchemaPath())).plus())
+        .build();
+    final Planner planner = Frameworks.getPlanner(config);
+    try {
+      final StringBuilder buf = new StringBuilder();
+      final SqlPrettyWriter w =
+          new SqlPrettyWriter(
+              SqlPrettyWriter.config()
+                  .withDialect(CalciteSqlDialect.DEFAULT)
+                  .withAlwaysUseParentheses(false),
+              buf);
+      buf.append("INSERT INTO ");
+      name.unparse(w, 0, 0);
+      buf.append(" ");
+      query.unparse(w, 0, 0);
+      final String sql = buf.toString();
+      final SqlNode query1 = planner.parse(sql);
+      final SqlNode query2 = planner.validate(query1);
+      final RelRoot r = planner.rel(query2);
+      final PreparedStatement prepare = context.getRelRunner().prepare(r.rel);
+      int rowCount = prepare.executeUpdate();
+      Util.discard(rowCount);
+      prepare.close();
+    } catch (SqlParseException | ValidationException
+        | RelConversionException | SQLException e) {
+      throw new RuntimeException(e);
+    }
   }
 
   /** Table backed by a Java list. */
diff --git a/core/src/test/resources/sql/blank.iq b/core/src/test/resources/sql/blank.iq
index 763d58b..2d69aaf 100644
--- a/core/src/test/resources/sql/blank.iq
+++ b/core/src/test/resources/sql/blank.iq
@@ -41,6 +41,22 @@ select * from foo;
 
 !ok
 
+create table bar as select i, i + j as k from foo;
+(0 rows modified)
+
+!update
+
+select * from bar;
++---+---+
+| I | K |
++---+---+
+| 0 | 2 |
+| 1 | 1 |
++---+---+
+(2 rows)
+
+!ok
+
 # Correlated non-equi IN
 select * from foo as f where i in (
   select j from foo where i > f.i);