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 2015/09/02 02:09:54 UTC

[18/18] incubator-calcite git commit: [CALCITE-852] DDL statements

[CALCITE-852] DDL statements


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

Branch: refs/heads/master
Commit: 3e4d2b5c7eb07cac9c3cc9799de2793d1521b93a
Parents: 3948751
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Aug 20 15:06:22 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Sep 1 16:17:17 2015 -0700

----------------------------------------------------------------------
 .../org/apache/calcite/jdbc/CalcitePrepare.java | 27 +++++++
 .../calcite/prepare/CalcitePrepareImpl.java     | 19 +++++
 .../java/org/apache/calcite/sql/SqlCall.java    |  4 +
 .../java/org/apache/calcite/sql/SqlKind.java    | 82 ++++++++++++++++++--
 .../java/org/apache/calcite/test/JdbcTest.java  | 69 ++++++++++++++++
 5 files changed, 195 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e4d2b5c/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java b/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
index ae5fba4..b2c1821 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalcitePrepare.java
@@ -39,6 +39,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.runtime.ArrayBindable;
 import org.apache.calcite.runtime.Bindable;
 import org.apache.calcite.schema.Table;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.util.ImmutableIntList;
@@ -76,6 +77,12 @@ public interface CalcitePrepare {
 
   ConvertResult convert(Context context, String sql);
 
+  /** Executes a DDL statement.
+   *
+   * <p>The statement identified itself as DDL in the
+   * {@link org.apache.calcite.jdbc.CalcitePrepare.ParseResult#kind} field. */
+  void executeDdl(Context context, SqlNode node);
+
   /** Analyzes a view.
    *
    * @param context Context
@@ -222,6 +229,26 @@ public interface CalcitePrepare {
       this.rowType = rowType;
       this.typeFactory = validator.getTypeFactory();
     }
+
+    /** Returns the kind of statement.
+     *
+     * <p>Possibilities include:
+     *
+     * <ul>
+     *   <li>Queries: usually {@link SqlKind#SELECT}, but
+     *   other query operators such as {@link SqlKind#UNION} and
+     *   {@link SqlKind#ORDER_BY} are possible
+     *   <li>DML statements: {@link SqlKind#INSERT}, {@link SqlKind#UPDATE} etc.
+     *   <li>Session control statements: {@link SqlKind#COMMIT}
+     *   <li>DDL statements: {@link SqlKind#CREATE_TABLE},
+     *   {@link SqlKind#DROP_INDEX}
+     * </ul>
+     *
+     * @return Kind of statement, never null
+     */
+    public SqlKind kind() {
+      return sqlNode.getKind();
+    }
   }
 
   /** The result of parsing and validating a SQL query and converting it to

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e4d2b5c/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index 3f9293c..b787a0f 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -412,6 +412,10 @@ public class CalcitePrepareImpl implements CalcitePrepare {
         constraint, ImmutableIntList.copyOf(columnMapping));
   }
 
+  @Override public void executeDdl(Context context, SqlNode node) {
+    throw new UnsupportedOperationException();
+  }
+
   /** Factory method for default SQL parser. */
   protected SqlParser createParser(String sql) {
     return createParser(sql, createParserConfig());
@@ -659,6 +663,21 @@ public class CalcitePrepareImpl implements CalcitePrepare {
 
       Hook.PARSE_TREE.run(new Object[] {sql, sqlNode});
 
+      if (sqlNode.getKind().belongsTo(SqlKind.DDL)) {
+        executeDdl(context, sqlNode);
+
+        // Return a dummy signature that contains no rows
+        final Bindable<T> bindable = new Bindable<T>() {
+          public Enumerable<T> bind(DataContext dataContext) {
+            return Linq4j.emptyEnumerable();
+          }
+        };
+        return new CalciteSignature<>(sql, ImmutableList.<AvaticaParameter>of(),
+            ImmutableMap.<String, Object>of(), null,
+            ImmutableList.<ColumnMetaData>of(), Meta.CursorFactory.OBJECT,
+            ImmutableList.<RelCollation>of(), -1, bindable);
+      }
+
       final CalciteSchema rootSchema = context.getRootSchema();
       final ChainedSqlOperatorTable opTab =
           new ChainedSqlOperatorTable(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e4d2b5c/core/src/main/java/org/apache/calcite/sql/SqlCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCall.java b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
index 41e9eb3..3461b2d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
@@ -63,6 +63,10 @@ public abstract class SqlCall extends SqlNode {
     throw new UnsupportedOperationException();
   }
 
+  @Override public SqlKind getKind() {
+    return getOperator().getKind();
+  }
+
   public abstract SqlOperator getOperator();
 
   public abstract List<SqlNode> getOperandList();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e4d2b5c/core/src/main/java/org/apache/calcite/sql/SqlKind.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 0bdb82c..fa821f8 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -613,7 +613,64 @@ public enum SqlKind {
   GROUPING_ID,
 
   /** The internal {@code GROUP_ID()} function. */
-  GROUP_ID;
+  GROUP_ID,
+
+  // DDL and session control statements follow. The list is not exhaustive: feel
+  // free to add more.
+
+  /** {@code COMMIT} session control statement. */
+  COMMIT,
+
+  /** {@code ROLLBACK} session control statement. */
+  ROLLBACK,
+
+  /** {@code ALTER SESSION} DDL statement. */
+  ALTER_SESSION,
+
+  /** {@code CREATE TABLE} DDL statement. */
+  CREATE_TABLE,
+
+  /** {@code ALTER TABLE} DDL statement. */
+  ALTER_TABLE,
+
+  /** {@code DROP TABLE} DDL statement. */
+  DROP_TABLE,
+
+  /** {@code CREATE VIEW} DDL statement. */
+  CREATE_VIEW,
+
+  /** {@code ALTER VIEW} DDL statement. */
+  ALTER_VIEW,
+
+  /** {@code DROP VIEW} DDL statement. */
+  DROP_VIEW,
+
+  /** {@code CREATE SEQUENCE} DDL statement. */
+  CREATE_SEQUENCE,
+
+  /** {@code ALTER SEQUENCE} DDL statement. */
+  ALTER_SEQUENCE,
+
+  /** {@code DROP SEQUENCE} DDL statement. */
+  DROP_SEQUENCE,
+
+  /** {@code CREATE INDEX} DDL statement. */
+  CREATE_INDEX,
+
+  /** {@code ALTER INDEX} DDL statement. */
+  ALTER_INDEX,
+
+  /** {@code DROP INDEX} DDL statement. */
+  DROP_INDEX,
+
+  /** DDL statement not handled above.
+   *
+   * <p><b>Note to other projects</b>: If you are extending Calcite's SQL parser
+   * and have your own object types you no doubt want to define CREATE and DROP
+   * commands for them. Use OTHER_DDL in the short term, but we are happy to add
+   * new enum values for your object types. Just ask!
+   */
+  OTHER_DDL;
 
   //~ Static fields/initializers ---------------------------------------------
 
@@ -679,6 +736,16 @@ public enum SqlKind {
       EnumSet.of(INSERT, DELETE, UPDATE, MERGE, PROCEDURE_CALL);
 
   /**
+   * Category consisting of all DDL operators.
+   */
+  public static final EnumSet<SqlKind> DDL =
+      EnumSet.of(COMMIT, ROLLBACK, ALTER_SESSION,
+          CREATE_TABLE, ALTER_TABLE, DROP_TABLE,
+          CREATE_VIEW, ALTER_VIEW, DROP_VIEW,
+          CREATE_SEQUENCE, ALTER_SEQUENCE, DROP_SEQUENCE,
+          CREATE_INDEX, ALTER_INDEX, DROP_INDEX);
+
+  /**
    * Category consisting of query node types.
    *
    * <p>Consists of:
@@ -697,9 +764,9 @@ public enum SqlKind {
   /**
    * Category of all SQL statement types.
    *
-   * <p>Consists of all types in {@link #QUERY} and {@link #DML}.
+   * <p>Consists of all types in {@link #QUERY}, {@link #DML} and {@link #DDL}.
    */
-  public static final Set<SqlKind> TOP_LEVEL = plus(QUERY, DML);
+  public static final EnumSet<SqlKind> TOP_LEVEL = concat(QUERY, DML, DDL);
 
   /**
    * Category consisting of regular and special functions.
@@ -762,10 +829,13 @@ public enum SqlKind {
     return category.contains(this);
   }
 
-  private static <E extends Enum<E>> EnumSet<E> plus(EnumSet<E> set0,
-      EnumSet<E> set1) {
+  @SafeVarargs
+  private static <E extends Enum<E>> EnumSet<E> concat(EnumSet<E> set0,
+      EnumSet<E>... sets) {
     EnumSet<E> set = set0.clone();
-    set.addAll(set1);
+    for (EnumSet<E> s : sets) {
+      set.addAll(s);
+    }
     return set;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/3e4d2b5c/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index e1ee242..a024686 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -31,6 +31,7 @@ import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.config.Lex;
 import org.apache.calcite.jdbc.CalciteConnection;
 import org.apache.calcite.jdbc.CalciteMetaImpl;
+import org.apache.calcite.jdbc.CalcitePrepare;
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.jdbc.Driver;
 import org.apache.calcite.linq4j.BaseQueryable;
@@ -40,6 +41,7 @@ import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.function.Function0;
 import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.function.Function2;
 import org.apache.calcite.linq4j.tree.Types;
@@ -74,10 +76,20 @@ import org.apache.calcite.schema.impl.AbstractTableQueryable;
 import org.apache.calcite.schema.impl.TableFunctionImpl;
 import org.apache.calcite.schema.impl.TableMacroImpl;
 import org.apache.calcite.schema.impl.ViewTable;
+import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlSpecialOperator;
 import org.apache.calcite.sql.advise.SqlAdvisorGetHintsFunction;
+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.SqlParserPos;
 import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.parser.impl.SqlParserImpl;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.JsonBuilder;
@@ -102,6 +114,7 @@ import java.io.File;
 import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.io.Reader;
 import java.lang.reflect.Method;
 import java.math.BigDecimal;
 import java.net.URL;
@@ -997,6 +1010,19 @@ public class JdbcTest {
     assertTrue(connection.isClosed());
   }
 
+  /** Tests that a driver can be extended with its own parser and can execute
+   * its own flavor of DDL. */
+  @Test public void testMockDdl() throws Exception {
+    final MockDdlDriver driver = new MockDdlDriver();
+    try (Connection connection =
+             driver.connect("jdbc:calcite:", new Properties());
+        Statement statement = connection.createStatement()) {
+      assertThat(driver.counter, is(0));
+      statement.executeQuery("COMMIT");
+      assertThat(driver.counter, is(1));
+    }
+  }
+
   /**
    * The example in the README.
    */
@@ -6811,6 +6837,49 @@ public class JdbcTest {
     }
   }
 
+  /** Mock driver that can execute a trivial DDL statement. */
+  public static class MockDdlDriver extends org.apache.calcite.jdbc.Driver {
+    public int counter;
+
+    public MockDdlDriver() {
+    }
+
+    @Override protected Function0<CalcitePrepare> createPrepareFactory() {
+      return new Function0<CalcitePrepare>() {
+        @Override public CalcitePrepare apply() {
+          return new CalcitePrepareImpl() {
+            @Override protected SqlParser.ConfigBuilder createParserConfig() {
+              return super.createParserConfig().setParserFactory(
+                  new SqlParserImplFactory() {
+                    @Override public SqlAbstractParserImpl
+                    getParser(Reader stream) {
+                      return new SqlParserImpl(stream) {
+                        @Override public SqlNode parseSqlStmtEof() {
+                          return new SqlCall(SqlParserPos.ZERO) {
+                            @Override public SqlOperator getOperator() {
+                              return new SqlSpecialOperator("COMMIT",
+                                  SqlKind.COMMIT);
+                            }
+
+                            @Override public List<SqlNode> getOperandList() {
+                              return ImmutableList.of();
+                            }
+                          };
+                        }
+                      };
+                    }
+                  });
+            }
+
+            @Override public void executeDdl(Context context, SqlNode node) {
+              ++counter;
+            }
+          };
+        }
+      };
+    }
+  }
+
   /** Dummy table. */
   public static class MyTable {
     public String mykey = "foo";