You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2016/07/18 20:41:24 UTC

phoenix git commit: PHOENIX-2229 Support CREATE VIEW in Phoenix-Calcite Integration

Repository: phoenix
Updated Branches:
  refs/heads/calcite fd579c1b8 -> 3d4a75feb


PHOENIX-2229 Support CREATE VIEW in Phoenix-Calcite Integration


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

Branch: refs/heads/calcite
Commit: 3d4a75feb9d195a91fb2307715cb13b0e2f95667
Parents: fd579c1
Author: maryannxue <ma...@gmail.com>
Authored: Mon Jul 18 16:41:16 2016 -0400
Committer: maryannxue <ma...@gmail.com>
Committed: Mon Jul 18 16:41:16 2016 -0400

----------------------------------------------------------------------
 .../apache/phoenix/calcite/CalciteDDLIT.java    |   4 +-
 .../src/main/codegen/includes/parserImpls.ftl   | 118 +++++++++++++++----
 .../sql/SqlColumnDefInPkConstraintNode.java     |   4 +-
 .../apache/calcite/sql/SqlColumnDefNode.java    |   4 +-
 .../apache/calcite/sql/SqlTableOptionNode.java  |   4 +-
 .../calcite/jdbc/PhoenixPrepareImpl.java        |  44 +++++--
 .../phoenix/calcite/parse/SqlCreateTable.java   |  57 ++++++++-
 .../phoenix/calcite/parse/SqlCreateView.java    |  64 ----------
 8 files changed, 181 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d4a75fe/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDDLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDDLIT.java
index 032de96..0d1df0d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDDLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDDLIT.java
@@ -8,7 +8,9 @@ public class CalciteDDLIT extends BaseCalciteIT {
     private static final Properties PROPS = new Properties();
     
     @Test public void testCreateView() throws Exception {
-        start(PROPS).sql("create view v as select * from (values (1, 'a'), (2, 'b')) as t(x, y)").execute();
+        start(PROPS).sql("create table t0(a varchar(20) not null primary key, b integer)").execute();
+        start(PROPS).sql("create view v1 as select * from t0").execute();
+        start(PROPS).sql("create view v2 as select * from t0 where a = 'x'").execute();
     }
 
     @Test public void testCreateTable() throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d4a75fe/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 d6e36d5..7ce1596 100644
--- a/phoenix-core/src/main/codegen/includes/parserImpls.ftl
+++ b/phoenix-core/src/main/codegen/includes/parserImpls.ftl
@@ -46,35 +46,76 @@ public String originalSql() {
 SqlNode SqlCreateView() :
 {
     SqlParserPos pos;
-    SqlIdentifier name;
-    SqlNode query;
+    SqlParserPos queryPos;
+    SqlIdentifier tableName;
+    boolean ifNotExists;
+    SqlNodeList columnDefs;
+    SqlIdentifier baseTableName;
+    SqlNode where;
+    String viewStatementString;
+    SqlNodeList tableOptions;
 }
 {
-    <CREATE> { pos = getPos(); } <VIEW> name = CompoundIdentifier()
-    <AS>
-    query = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY)
+    <CREATE> { pos = getPos(); } <VIEW>
+    (
+        <IF> <NOT> <EXISTS> { ifNotExists = true; }
+        |
+        {
+            ifNotExists = false;
+        }
+    )
+    tableName = DualIdentifier()
+    (
+        <LPAREN>
+        columnDefs = ColumnDefList()
+        <RPAREN>
+        |
+        {
+            columnDefs = SqlNodeList.EMPTY;
+        }
+    )
+    (
+        <AS> <SELECT> { queryPos = getPos(); } <STAR> <FROM>
+        baseTableName = DualIdentifier()
+        where = WhereOpt()
+        {
+            queryPos = queryPos.plus(getPos());
+            String sql = originalSql();
+            int start = SqlParserUtil.lineColToIndex(sql, queryPos.getLineNum(), queryPos.getColumnNum());
+            int end = SqlParserUtil.lineColToIndex(sql, queryPos.getEndLineNum(), queryPos.getEndColumnNum());
+            viewStatementString = sql.substring(start, end + 1);            
+        }
+        |
+        {
+            baseTableName = null;
+            where = null;
+            viewStatementString = null;
+        }
+    )
+    (
+        tableOptions = TableOptionList()
+        |
+        {
+            tableOptions = SqlNodeList.EMPTY;
+        }
+    )
     {
-        String sql = originalSql();
-        SqlParserPos pos2 = query.getParserPosition();
-        SqlParserPos pos3 = getPos();
-        int start = SqlParserUtil.lineColToIndex(sql, pos2.getLineNum(), pos2.getColumnNum());
-        int end = SqlParserUtil.lineColToIndex(sql, pos3.getEndLineNum(), pos3.getEndColumnNum());
-        String queryString = sql.substring(start, end + 1);
-        System.out.println("[" + queryString + "]");
-        return new SqlCreateView(pos.plus(pos3), name, query, queryString);
+        return new SqlCreateTable(pos.plus(getPos()), tableName,
+            SqlLiteral.createBoolean(ifNotExists, SqlParserPos.ZERO),
+            columnDefs, baseTableName, where, viewStatementString, tableOptions);
     }
 }
 
 SqlNode SqlCreateTable() :
 {
-      SqlParserPos pos;
-      SqlIdentifier tableName;
-	  boolean ifNotExists;
-      SqlNodeList columnDefs;
-      SqlIdentifier pkConstraint;
-      SqlNodeList pkConstraintColumnDefs;
-      SqlNodeList tableOptions;
-      SqlNodeList splitKeys;
+    SqlParserPos pos;
+    SqlIdentifier tableName;
+    boolean ifNotExists;
+    SqlNodeList columnDefs;
+    SqlIdentifier pkConstraint;
+    SqlNodeList pkConstraintColumnDefs;
+    SqlNodeList tableOptions;
+    SqlNodeList splitKeys;
 }
 {
     <CREATE> { pos = getPos(); } <TABLE>
@@ -85,7 +126,7 @@ SqlNode SqlCreateTable() :
             ifNotExists = false;
         }
     )
-    tableName = CompoundIdentifier()
+    tableName = DualIdentifier()
     <LPAREN>
     columnDefs = ColumnDefList()
     (
@@ -114,7 +155,7 @@ SqlNode SqlCreateTable() :
         }
     )
     {
-        return new SqlCreateTable(pos, tableName,
+        return new SqlCreateTable(pos.plus(getPos()), tableName,
             SqlLiteral.createBoolean(ifNotExists, SqlParserPos.ZERO),
             columnDefs, pkConstraint, pkConstraintColumnDefs,
             tableOptions, splitKeys);
@@ -205,7 +246,7 @@ SqlColumnDefNode ColumnDef() :
     SqlParserPos pos;
 }
 {
-    columnName = CompoundIdentifier()
+    columnName = DualIdentifier()
     dataType = PhoenixDataType()
     [
         <NOT> <NULL>
@@ -278,7 +319,7 @@ SqlColumnDefInPkConstraintNode ColumnDefInPkConstraint() :
     SqlParserPos pos;
 }
 {
-    columnName = CompoundIdentifier()
+    columnName = DualIdentifier()
     [
         <ASC>
         {sortOrder = SortOrder.ASC;}
@@ -303,7 +344,7 @@ SqlTableOptionNode TableOption() :
     SqlParserPos pos;
 }
 {
-    key = CompoundIdentifier()
+    key = DualIdentifier()
     <EQ>
     value = Literal()
     {
@@ -311,3 +352,28 @@ SqlTableOptionNode TableOption() :
         return new SqlTableOptionNode(pos, key, (SqlLiteral) value);
     }
 }
+
+SqlIdentifier DualIdentifier() :
+{
+    List<String> list = new ArrayList<String>();
+    List<SqlParserPos> posList = new ArrayList<SqlParserPos>();
+    String p;
+}
+{
+    p = Identifier()
+    {
+        posList.add(getPos());
+        list.add(p);
+    }
+    [
+        <DOT>
+            p = Identifier() {
+                list.add(p);
+                posList.add(getPos());
+            }
+    ]
+    {
+        SqlParserPos pos = SqlParserPos.sum(posList);
+        return new SqlIdentifier(list, null, pos, posList);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d4a75fe/phoenix-core/src/main/java/org/apache/calcite/sql/SqlColumnDefInPkConstraintNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/calcite/sql/SqlColumnDefInPkConstraintNode.java b/phoenix-core/src/main/java/org/apache/calcite/sql/SqlColumnDefInPkConstraintNode.java
index c3cd467..b36a0f6 100644
--- a/phoenix-core/src/main/java/org/apache/calcite/sql/SqlColumnDefInPkConstraintNode.java
+++ b/phoenix-core/src/main/java/org/apache/calcite/sql/SqlColumnDefInPkConstraintNode.java
@@ -40,10 +40,8 @@ public class SqlColumnDefInPkConstraintNode extends SqlNode{
         final ColumnName name;
         if (columnName.isSimple()) {
             name = new ColumnName(columnName.getSimple());
-        } else if (columnName.names.size() == 2) {
-            name = new ColumnName(columnName.names.get(0), columnName.names.get(1));
         } else {
-            throw new RuntimeException("Invalid column name: " + columnName);
+            name = new ColumnName(columnName.names.get(0), columnName.names.get(1));
         }
         this.pkConstraint = new ColumnDefInPkConstraint(name, sortOrder, isRowTimestamp);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d4a75fe/phoenix-core/src/main/java/org/apache/calcite/sql/SqlColumnDefNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/calcite/sql/SqlColumnDefNode.java b/phoenix-core/src/main/java/org/apache/calcite/sql/SqlColumnDefNode.java
index c5349ba..f0c607c 100644
--- a/phoenix-core/src/main/java/org/apache/calcite/sql/SqlColumnDefNode.java
+++ b/phoenix-core/src/main/java/org/apache/calcite/sql/SqlColumnDefNode.java
@@ -44,10 +44,8 @@ public class SqlColumnDefNode extends SqlNode{
         final ColumnName name;
         if (columnName.isSimple()) {
             name = new ColumnName(columnName.getSimple());
-        } else if (columnName.names.size() == 2) {
-            name = new ColumnName(columnName.names.get(0), columnName.names.get(1));
         } else {
-            throw new RuntimeException("Invalid column name: " + columnName);
+            name = new ColumnName(columnName.names.get(0), columnName.names.get(1));
         }
         this.columnDef = new ColumnDef(name, dataType.typeName,
                 dataType.isArray, dataType.arraySize, isNullable,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d4a75fe/phoenix-core/src/main/java/org/apache/calcite/sql/SqlTableOptionNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/calcite/sql/SqlTableOptionNode.java b/phoenix-core/src/main/java/org/apache/calcite/sql/SqlTableOptionNode.java
index 8f64757..5119565 100644
--- a/phoenix-core/src/main/java/org/apache/calcite/sql/SqlTableOptionNode.java
+++ b/phoenix-core/src/main/java/org/apache/calcite/sql/SqlTableOptionNode.java
@@ -34,11 +34,9 @@ public class SqlTableOptionNode extends SqlNode {
         if (key.isSimple()) {
             familyName = "";
             propertyName = key.getSimple();
-        } else if (key.names.size() == 2) {
+        } else {
             familyName = key.names.get(0);
             propertyName = key.names.get(1);
-        } else {
-            throw new RuntimeException("Invalid table property name: " + key);
         }
         final Object v = SqlLiteral.value(literal);
         if (v instanceof NlsString) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d4a75fe/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixPrepareImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixPrepareImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixPrepareImpl.java
index 98a4caa..c9d372d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixPrepareImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixPrepareImpl.java
@@ -22,7 +22,7 @@ import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlColumnDefInPkConstraintNode;
 import org.apache.calcite.sql.SqlColumnDefNode;
-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;
@@ -35,7 +35,6 @@ import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Pair;
 import org.apache.phoenix.calcite.PhoenixSchema;
 import org.apache.phoenix.calcite.parse.SqlCreateTable;
-import org.apache.phoenix.calcite.parse.SqlCreateView;
 import org.apache.phoenix.calcite.parser.PhoenixParserImpl;
 import org.apache.phoenix.calcite.rel.PhoenixRel;
 import org.apache.phoenix.calcite.rel.PhoenixServerProject;
@@ -60,6 +59,8 @@ import org.apache.phoenix.parse.CreateTableStatement;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.PrimaryKeyConstraint;
+import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.schema.PTableType;
 
 import com.google.common.base.Function;
@@ -170,15 +171,16 @@ public class PhoenixPrepareImpl extends CalcitePrepareImpl {
             final ParseNodeFactory nodeFactory = new ParseNodeFactory();
             final PhoenixConnection connection = getPhoenixConnection(context.getRootSchema().plus());
             switch (node.getKind()) {
-            case CREATE_VIEW:
-                final SqlCreateView cv = (SqlCreateView) node;
-                System.out.println("Create view: " + cv.name);
-                break;
             case CREATE_TABLE:
+            case CREATE_VIEW:
                 final SqlCreateTable table = (SqlCreateTable) node;
-                final SqlIdentifier tableIdentifier = table.tableName;
-                final String schemaName = tableIdentifier.isSimple() ? null : tableIdentifier.skipLast(1).toString();
-                final String tableName = tableIdentifier.names.get(tableIdentifier.names.size() - 1);
+                final PTableType tableType = table.getKind() == SqlKind.CREATE_TABLE ? PTableType.TABLE : PTableType.VIEW;
+                final TableName name;
+                if (table.tableName.isSimple()) {
+                    name = nodeFactory.table(null, table.tableName.getSimple());
+                } else {
+                    name = nodeFactory.table(table.tableName.names.get(0), table.tableName.names.get(1));
+                }
                 final ListMultimap<String, org.apache.hadoop.hbase.util.Pair<String, Object>> props;
                 if (SqlNodeList.isEmptyList(table.tableOptions)) {
                     props = null;
@@ -203,6 +205,23 @@ public class PhoenixPrepareImpl extends CalcitePrepareImpl {
                     }
                     pkConstraint = nodeFactory.primaryKey(table.pkConstraint.getSimple(), pkColumns);
                 }
+                final TableName baseTableName;
+                final ParseNode where;
+                if (table.baseTableName == null) {
+                    baseTableName = tableType == PTableType.TABLE ? null : name;
+                    where = null;
+                } else {
+                    if (table.baseTableName.isSimple()) {
+                        baseTableName = nodeFactory.table(null, table.baseTableName.getSimple());
+                    } else {
+                        baseTableName = nodeFactory.table(table.baseTableName.names.get(0), table.baseTableName.names.get(1));
+                    }
+                    if (table.whereNode == null) {
+                        where = null;
+                    } else {
+                        where = new SQLParser(table.viewStatementString).parseQuery().getWhere();
+                    }
+                }
                 final List<ParseNode> splitNodes;
                 if (SqlNodeList.isEmptyList(table.splitKeyList)) {
                     splitNodes = null;
@@ -214,10 +233,9 @@ public class PhoenixPrepareImpl extends CalcitePrepareImpl {
                     }
                 }
                 final CreateTableStatement create = nodeFactory.createTable(
-                        nodeFactory.table(schemaName, tableName),
-                        props, columnDefs, pkConstraint,
-                        splitNodes, PTableType.TABLE, table.ifNotExists.booleanValue(),
-                        null, null, 0);
+                        name, props, columnDefs, pkConstraint,
+                        splitNodes, tableType, table.ifNotExists.booleanValue(),
+                        baseTableName, where, 0);
                 try (final PhoenixStatement stmt = new PhoenixStatement(connection)) {
                     final CreateTableCompiler compiler = new CreateTableCompiler(stmt, Operation.UPSERT);
                     final MutationPlan plan = compiler.compile(create);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d4a75fe/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateTable.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateTable.java
index 9c12c21..7c153d8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateTable.java
@@ -34,40 +34,79 @@ import java.util.List;
  * Parse tree node for SQL {@code CREATE TABLE} command.
  */
 public class SqlCreateTable extends SqlCall {
-    public static final SqlOperator OPERATOR = new SqlDdlOperator("CREATE TABLE", SqlKind.CREATE_TABLE);
+    public final SqlOperator operator;
 
     public final SqlIdentifier tableName;
     public final SqlLiteral ifNotExists;
     public final SqlNodeList columnDefs;
     public final SqlIdentifier pkConstraint;
     public final SqlNodeList pkConstraintColumnDefs;
+    public final SqlIdentifier baseTableName;
+    public final SqlNode whereNode;
+    public final String viewStatementString;
     public final SqlNodeList tableOptions;
     public final SqlNodeList splitKeyList;
     
 
     /** Creates a CREATE TABLE. */
-    public SqlCreateTable(SqlParserPos pos, SqlIdentifier tableName, SqlLiteral ifNotExists, SqlNodeList columnDefs, SqlIdentifier pkConstraint, SqlNodeList pkConstraintColumnDefs, SqlNodeList tableOptions, SqlNodeList splitKeyList) {
+    public SqlCreateTable(
+            SqlParserPos pos,
+            SqlIdentifier tableName,
+            SqlLiteral ifNotExists,
+            SqlNodeList columnDefs,
+            SqlIdentifier pkConstraint,
+            SqlNodeList pkConstraintColumnDefs,
+            SqlNodeList tableOptions,
+            SqlNodeList splitKeyList) {
         super(pos);
+        this.operator = new SqlDdlOperator("CREATE TABLE", SqlKind.CREATE_TABLE);
         this.tableName = tableName;
         this.ifNotExists = ifNotExists;
         this.columnDefs = columnDefs;
         this.pkConstraint = pkConstraint;
         this.pkConstraintColumnDefs = pkConstraintColumnDefs;
+        this.baseTableName = null;
+        this.whereNode = null;
+        this.viewStatementString = null;
         this.tableOptions = tableOptions;
         this.splitKeyList = splitKeyList;
     }
+    
+    /** Creates a CREATE VIEW. */
+    public SqlCreateTable(
+            SqlParserPos pos,
+            SqlIdentifier tableName,
+            SqlLiteral ifNotExists,
+            SqlNodeList columnDefs,
+            SqlIdentifier baseTableName,
+            SqlNode whereNode,
+            String viewStatementString,
+            SqlNodeList tableOptions) {
+        super(pos);
+        this.operator = new SqlDdlOperator("CREATE VIEW", SqlKind.CREATE_VIEW);
+        this.tableName = tableName;
+        this.ifNotExists = ifNotExists;
+        this.columnDefs = columnDefs;
+        this.pkConstraint = null;
+        this.pkConstraintColumnDefs = SqlNodeList.EMPTY;
+        this.baseTableName = baseTableName;
+        this.whereNode = whereNode;
+        this.viewStatementString = viewStatementString;
+        this.tableOptions = tableOptions;
+        this.splitKeyList = SqlNodeList.EMPTY;
+    }
 
     public SqlOperator getOperator() {
-        return OPERATOR;
+        return operator;
     }
 
     public List<SqlNode> getOperandList() {
-        return ImmutableList.of(tableName, ifNotExists, columnDefs, pkConstraint, pkConstraintColumnDefs, tableOptions, splitKeyList);
+        return ImmutableList.of(tableName, ifNotExists, columnDefs, pkConstraint, pkConstraintColumnDefs, baseTableName, whereNode, tableOptions, splitKeyList);
     }
 
     @Override
     public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
-        writer.keyword("CREATE TABLE");
+        writer.keyword(operator.getName());
         if(SqlLiteral.value(ifNotExists).equals(Boolean.TRUE)) {
         	writer.keyword("IF NOT EXISTS");
         }
@@ -79,6 +118,14 @@ public class SqlCreateTable extends SqlCall {
         	writer.keyword("PRIMARY KEY");
         	((SqlDdlOperator)getOperator()).unparseListClause(writer, pkConstraintColumnDefs);
         }
+        if (baseTableName != null) {
+            writer.keyword("AS SELECT * FROM");
+            baseTableName.unparse(writer, 0, 0);
+            if (whereNode != null) {
+                writer.keyword("WHERE");
+                whereNode.unparse(writer, 0, 0);
+            }
+        }
         if (SqlNodeList.isEmptyList(tableOptions)) {
             ((SqlDdlOperator)getOperator()).unparseListClause(writer, tableOptions);            
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d4a75fe/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateView.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateView.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateView.java
deleted file mode 100644
index 8d5af1a..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateView.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.phoenix.calcite.parse;
-
-import com.google.common.collect.ImmutableList;
-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.SqlOperator;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.parser.SqlParserPos;
-
-import java.util.List;
-
-/**
- * Parse tree node for SQL {@code CREATE VIEW} command.
- */
-public class SqlCreateView extends SqlCall {
-    public static final SqlOperator OPERATOR = new SqlDdlOperator("CREATE VIEW", SqlKind.CREATE_VIEW);
-
-    public final SqlIdentifier name;
-    public final SqlNode query;
-    public final String queryString;
-
-    /** Creates a CREATE VIEW. */
-    public SqlCreateView(SqlParserPos pos, SqlIdentifier name, SqlNode query, String queryString) {
-        super(pos);
-        this.name = name;
-        this.query = query;
-        this.queryString = queryString;
-    }
-
-    public SqlOperator getOperator() {
-        return OPERATOR;
-    }
-
-    public List<SqlNode> getOperandList() {
-        return ImmutableList.of(name, query);
-    }
-
-    @Override
-    public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
-        writer.keyword("CREATE VIEW");
-        name.unparse(writer, 0, 0);
-        writer.keyword(" ");
-        query.unparse(writer, 0, 0);
-    }
-}