You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ra...@apache.org on 2016/11/10 07:15:16 UTC

phoenix git commit: PHOENIX-3409 Support ALTER TABLE in Phoenix-Calcite Integration(Rajeshbabu)

Repository: phoenix
Updated Branches:
  refs/heads/calcite 3fd7d965d -> d65122b88


PHOENIX-3409 Support ALTER TABLE in Phoenix-Calcite Integration(Rajeshbabu)


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

Branch: refs/heads/calcite
Commit: d65122b88b60adbe1f3062da93e32e7b2d914f71
Parents: 3fd7d96
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Thu Nov 10 12:55:22 2016 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Thu Nov 10 12:55:22 2016 +0530

----------------------------------------------------------------------
 phoenix-core/src/main/codegen/data/Parser.tdd   |  1 +
 .../src/main/codegen/includes/parserImpls.ftl   | 88 +++++++++++++++++++-
 .../calcite/jdbc/PhoenixCalciteFactory.java     | 16 +++-
 .../phoenix/calcite/PhoenixPrepareImpl.java     | 64 ++++++++++++++
 .../phoenix/calcite/parse/SqlAlterTable.java    | 61 ++++++++++++++
 5 files changed, 228 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d65122b8/phoenix-core/src/main/codegen/data/Parser.tdd
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/codegen/data/Parser.tdd b/phoenix-core/src/main/codegen/data/Parser.tdd
index 61781e3..6b6bd87 100644
--- a/phoenix-core/src/main/codegen/data/Parser.tdd
+++ b/phoenix-core/src/main/codegen/data/Parser.tdd
@@ -75,6 +75,7 @@
     "SqlDropFunction()"
     "SqlUploadJarsNode()"
     "SqlDeleteJarNode()"
+    "SqlAlterTable()"
   ]
 
   # List of methods for parsing custom literals.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d65122b8/phoenix-core/src/main/codegen/includes/parserImpls.ftl
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/codegen/includes/parserImpls.ftl b/phoenix-core/src/main/codegen/includes/parserImpls.ftl
index e86ef85..984009e 100644
--- a/phoenix-core/src/main/codegen/includes/parserImpls.ftl
+++ b/phoenix-core/src/main/codegen/includes/parserImpls.ftl
@@ -181,6 +181,75 @@ SqlNode SqlCreateTable() :
 
 /**
  * Parses statement
+ *   ALTER TABLE
+ */
+SqlNode SqlAlterTable() :
+{
+    SqlParserPos pos;
+    SqlIdentifier tableName;
+    boolean isView = false;
+    boolean ifExists = false;
+    SqlNodeList columnNames = null;
+    boolean ifNotExists = false;
+    SqlNodeList newColumnDefs = null;
+    SqlNodeList tableOptions = null;
+}
+{
+    <ALTER> { pos = getPos(); } 
+    (
+        <VIEW> { isView = true; }
+        |
+        <TABLE> { isView = false; }
+    )
+    tableName = DualIdentifier()
+    (
+        (
+            <ADD>
+            (
+                <IF> <NOT> <EXISTS> { ifNotExists = true; }
+                |
+                {
+                    ifNotExists = false;
+                }
+            )
+            newColumnDefs = ColumnDefList()
+            (
+                tableOptions = FamilyOptionList()
+                |
+                {
+                    tableOptions = SqlNodeList.EMPTY;
+                }
+            )
+        )
+        |
+        <DROP> <COLUMN>
+        (
+            <IF> <EXISTS> { ifExists = true; }
+            |
+            {
+                ifExists = false;
+            }
+        )
+        columnNames = ColumnNamesList()
+        |
+        <SET>
+        tableOptions = FamilyOptionList()
+        |
+        {
+            tableOptions = SqlNodeList.EMPTY;
+        }
+    )
+    {
+    return new SqlAlterTable(pos.plus(getPos()), tableName,
+            SqlLiteral.createBoolean(isView, SqlParserPos.ZERO), 
+            SqlLiteral.createBoolean(ifExists, SqlParserPos.ZERO),columnNames,
+            SqlLiteral.createBoolean(ifNotExists, SqlParserPos.ZERO), newColumnDefs,
+            tableOptions);
+     }
+}
+
+/**
+ * Parses statement
  *   CREATE INDEX
  */
 SqlNode SqlCreateIndex() :
@@ -546,6 +615,23 @@ SqlNodeList ColumnDefList() :
     }
 }
 
+SqlNodeList ColumnNamesList() :
+{
+    SqlParserPos pos;
+    SqlNode e;
+    List<SqlNode> columnNamesList;
+}
+{
+    { pos = getPos(); }
+    e = DualIdentifier() { columnNamesList = startList(e); }
+    (
+        <COMMA> e = DualIdentifier() { columnNamesList.add(e); }
+    ) *
+    {
+        return new SqlNodeList(columnNamesList, pos.plus(getPos()));
+    }
+}
+
 SqlNodeList PkConstraintColumnDefList() :
 {
     SqlParserPos pos;
@@ -876,4 +962,4 @@ SqlNodeList FunctionArguementsList() :
     {
         return new SqlNodeList(functionArguements, pos.plus(getPos()));
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d65122b8/phoenix-core/src/main/java/org/apache/calcite/jdbc/PhoenixCalciteFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/calcite/jdbc/PhoenixCalciteFactory.java b/phoenix-core/src/main/java/org/apache/calcite/jdbc/PhoenixCalciteFactory.java
index b0040e9..ef42737 100644
--- a/phoenix-core/src/main/java/org/apache/calcite/jdbc/PhoenixCalciteFactory.java
+++ b/phoenix-core/src/main/java/org/apache/calcite/jdbc/PhoenixCalciteFactory.java
@@ -3,6 +3,7 @@ package org.apache.calcite.jdbc;
 import java.io.File;
 import java.io.InputStream;
 import java.io.Reader;
+import java.sql.DatabaseMetaData;
 import java.sql.NClob;
 import java.sql.ResultSetMetaData;
 import java.sql.RowId;
@@ -290,7 +291,20 @@ public class PhoenixCalciteFactory extends CalciteFactory {
         private static interface PhoenixConnectionCallable {
             void call(PhoenixConnection conn) throws SQLException;
         }
-        
+
+        @Override
+        public DatabaseMetaData getMetaData() throws SQLException {
+            for (String subSchemaName : getRootSchema().getSubSchemaNames()) {
+                try {
+                    PhoenixSchema phoenixSchema =
+                            getRootSchema().getSubSchema(subSchemaName).unwrap(PhoenixSchema.class);
+                    return phoenixSchema.pc.getMetaData();
+                } catch (ClassCastException e) {
+                }
+            }
+            return super.getMetaData();
+        }
+
         @SuppressWarnings("unchecked")
         @Override
         public <T> T unwrap(Class<T> iface) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d65122b8/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java
index 1d9ca1e..dc4e29e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java
@@ -45,6 +45,7 @@ import org.apache.calcite.tools.Programs;
 import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.NlsString;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.calcite.parse.SqlAlterTable;
 import org.apache.phoenix.calcite.parse.SqlCreateFunction;
 import org.apache.phoenix.calcite.parse.SqlCreateIndex;
 import org.apache.phoenix.calcite.parse.SqlCreateSequence;
@@ -79,6 +80,7 @@ import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
+import org.apache.phoenix.parse.AddColumnStatement;
 import org.apache.phoenix.parse.ColumnDef;
 import org.apache.phoenix.parse.ColumnDefInPkConstraint;
 import org.apache.phoenix.parse.ColumnName;
@@ -86,6 +88,7 @@ import org.apache.phoenix.parse.CreateFunctionStatement;
 import org.apache.phoenix.parse.CreateIndexStatement;
 import org.apache.phoenix.parse.CreateSequenceStatement;
 import org.apache.phoenix.parse.CreateTableStatement;
+import org.apache.phoenix.parse.DropColumnStatement;
 import org.apache.phoenix.parse.DropFunctionStatement;
 import org.apache.phoenix.parse.DropIndexStatement;
 import org.apache.phoenix.parse.DropSequenceStatement;
@@ -103,6 +106,7 @@ import org.apache.phoenix.parse.SQLParser;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.parse.UDFParseNode;
 import org.apache.phoenix.parse.UpdateStatisticsStatement;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTableType;
@@ -438,6 +442,66 @@ public class PhoenixPrepareImpl extends CalcitePrepareImpl {
                 client.dropSequence(drop);
                 break;                
             }
+            case ALTER_TABLE: {
+                final SqlAlterTable alterTable = (SqlAlterTable) node;
+                final TableName name;
+                if (alterTable.tableName.isSimple()) {
+                    name = TableName.create(null, alterTable.tableName.getSimple());
+                } else {
+                    name = TableName.create(alterTable.tableName.names.get(0), alterTable.tableName.names.get(1));
+                }
+                final NamedTableNode namedTable = NamedTableNode.create(name);
+                if(alterTable.newColumnDefs != null || alterTable.tableOptions != null) {
+                    final List<ColumnDef> columnDefs = Lists.newArrayList();
+                    if(alterTable.newColumnDefs != null) {
+                        for (SqlNode columnDef : alterTable.newColumnDefs) {
+                            columnDefs.add(((SqlColumnDefNode) columnDef).columnDef);
+                        }
+                    }
+                    boolean ifNotExists = false;
+                    if(alterTable.ifNotExists != null) {
+                        ifNotExists = alterTable.ifNotExists.booleanValue();
+                    }
+                    final ListMultimap<String, Pair<String, Object>> props = convertOptions(alterTable.tableOptions);
+                    AddColumnStatement addColumn =
+                            nodeFactory
+                                    .addColumn(
+                                        namedTable,
+                                        alterTable.isView.booleanValue() ? PTableType.VIEW
+                                                : (QueryConstants.SYSTEM_SCHEMA_NAME.equals(name
+                                                        .getSchemaName()) ? PTableType.SYSTEM
+                                                        : PTableType.TABLE), columnDefs,
+                                        ifNotExists, props);
+                    MetaDataClient client = new MetaDataClient(connection);
+                    client.addColumn(addColumn);
+                } else {
+                    final List<ColumnName> columnNames = Lists.newArrayList();
+                    for (SqlNode e : alterTable.columnNames) {
+                        SqlIdentifier n = (SqlIdentifier) e;
+                        ColumnName columnName;
+                        if (n.isSimple()) {
+                            columnName = ColumnName.caseSensitiveColumnName(n.getSimple());
+                        } else {
+                            columnName = ColumnName.caseSensitiveColumnName(n.names.get(0), n.names.get(1));
+                        }
+                        columnNames.add(columnName);
+                    }
+                    boolean ifExists = false;
+                    if(alterTable.ifExists != null) {
+                        ifExists = alterTable.ifExists.booleanValue();
+                    }
+                    DropColumnStatement dropColumn =
+                            nodeFactory.dropColumn(
+                                namedTable,
+                                alterTable.isView.booleanValue() ? PTableType.VIEW
+                                        : (QueryConstants.SYSTEM_SCHEMA_NAME.equals(name
+                                                .getSchemaName()) ? PTableType.SYSTEM
+                                                : PTableType.TABLE), columnNames, ifExists);
+                    MetaDataClient client = new MetaDataClient(connection);
+                    client.dropColumn(dropColumn);
+                }
+                break;
+            }
             case OTHER_DDL: {
                 if (node instanceof SqlUpdateStatistics) {
                     SqlUpdateStatistics updateStatsNode = (SqlUpdateStatistics) node;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d65122b8/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlAlterTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlAlterTable.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlAlterTable.java
new file mode 100644
index 0000000..5dde16f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlAlterTable.java
@@ -0,0 +1,61 @@
+package org.apache.phoenix.calcite.parse;
+
+import java.util.List;
+
+import org.apache.calcite.sql.SqlCall;
+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.SqlOperator;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Parse tree node for SQL {@code ALTER TABLE} command.
+ */
+public class SqlAlterTable extends SqlCall {
+    public final SqlOperator operator;
+    public final SqlIdentifier tableName;
+    public final SqlLiteral isView;
+    public final SqlLiteral ifExists;
+    public final SqlNodeList columnNames;
+    public final SqlLiteral ifNotExists;
+    public final SqlNodeList newColumnDefs;
+    public final SqlNodeList tableOptions;
+
+    /** Creates a ALTER TABLE. */
+    public SqlAlterTable(
+            SqlParserPos pos,
+            SqlIdentifier tableName,
+            SqlLiteral isView,
+            SqlLiteral ifExists,
+            SqlNodeList columnNames,
+            SqlLiteral ifNotExists,
+            SqlNodeList newColumnDefs,
+            SqlNodeList tableOptions) {
+        super(pos);
+        this.operator = new SqlDdlOperator("ALTER TABLE", SqlKind.ALTER_TABLE);
+        this.tableName = tableName;
+        this.isView = isView;
+        this.ifExists = ifExists;
+        this.columnNames = columnNames;
+        this.ifNotExists = ifNotExists;
+        this.newColumnDefs = newColumnDefs;
+        this.tableOptions = tableOptions;
+    }
+
+
+    @Override
+    public SqlOperator getOperator() {
+        return this.operator;
+    }
+
+    @Override
+    public List<SqlNode> getOperandList() {
+        return ImmutableList.of(tableName, isView, ifExists, columnNames, ifNotExists, newColumnDefs, tableOptions);
+    }
+
+}