This is an automated email from the ASF dual-hosted git repository. jark pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push: new 71570bc [FLINK-12954][table-api] Supports create(drop) view grammar for sql parser 71570bc is described below commit 71570bc9a5dc549a34b020981e5cf87c479658e9 Author: yuzhao.cyz <yuzhao....@alibaba-inc.com> AuthorDate: Mon Jun 24 13:52:39 2019 +0800 [FLINK-12954][table-api] Supports create(drop) view grammar for sql parser This closes #8850 --- .../src/main/codegen/data/Parser.tdd | 8 +- .../src/main/codegen/includes/parserImpls.ftl | 117 ++++++++++++-------- .../flink/sql/parser/ddl/SqlCreateTable.java | 3 +- .../apache/flink/sql/parser/ddl/SqlCreateView.java | 122 +++++++++++++++++++++ .../apache/flink/sql/parser/ddl/SqlDropView.java | 74 +++++++++++++ .../flink/sql/parser/FlinkSqlParserImplTest.java | 45 ++++++++ 6 files changed, 320 insertions(+), 49 deletions(-) diff --git a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd index a6eeaa6..d3a4c64 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd +++ b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd @@ -25,6 +25,8 @@ "org.apache.flink.sql.parser.ddl.SqlCreateTable", "org.apache.flink.sql.parser.ddl.SqlDropTable" "org.apache.flink.sql.parser.ddl.SqlCreateTable.TableCreationContext", + "org.apache.flink.sql.parser.ddl.SqlCreateView", + "org.apache.flink.sql.parser.ddl.SqlDropView", "org.apache.flink.sql.parser.ddl.SqlTableColumn", "org.apache.flink.sql.parser.dml.RichSqlInsert", "org.apache.flink.sql.parser.dml.RichSqlInsertKeyword", @@ -414,13 +416,15 @@ # List of methods for parsing extensions to "CREATE [OR REPLACE]" calls. # Each must accept arguments "(SqlParserPos pos, boolean replace)". createStatementParserMethods: [ - "SqlCreateTable" + "SqlCreateTable", + "SqlCreateView" ] # List of methods for parsing extensions to "DROP" calls. # Each must accept arguments "(Span s)". dropStatementParserMethods: [ - "SqlDropTable" + "SqlDropTable", + "SqlDropView" ] # List of files in @includes directory that have parser method diff --git a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl index f28e26a..e94e42f 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl +++ b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl @@ -122,6 +122,31 @@ SqlNode PropertyValue() : } } +/** Parse a table properties. */ +SqlNodeList TableProperties(): +{ + SqlNode property; + final List<SqlNode> proList = new ArrayList<SqlNode>(); + final Span span; +} +{ + <LPAREN> { span = span(); } + [ + property = PropertyValue() + { + proList.add(property); + } + ( + <COMMA> property = PropertyValue() + { + proList.add(property); + } + )* + ] + <RPAREN> + { return new SqlNodeList(proList, span.end(this)); } +} + SqlCreate SqlCreateTable(Span s, boolean replace) : { final SqlParserPos startPos = s.pos(); @@ -131,8 +156,8 @@ SqlCreate SqlCreateTable(Span s, boolean replace) : SqlNodeList columnList = SqlNodeList.EMPTY; SqlCharStringLiteral comment = null; - SqlNodeList propertyList = null; - SqlNodeList partitionColumns = null; + SqlNodeList propertyList = SqlNodeList.EMPTY; + SqlNodeList partitionColumns = SqlNodeList.EMPTY; SqlParserPos pos = startPos; } { @@ -159,54 +184,12 @@ SqlCreate SqlCreateTable(Span s, boolean replace) : }] [ <PARTITIONED> <BY> - { - SqlNode column; - List<SqlNode> partitionKey = new ArrayList<SqlNode>(); - pos = getPos(); - - } - <LPAREN> - [ - column = SimpleIdentifier() - { - partitionKey.add(column); - } - ( - <COMMA> column = SimpleIdentifier() - { - partitionKey.add(column); - } - )* - ] - <RPAREN> - { - partitionColumns = new SqlNodeList(partitionKey, pos.plus(getPos())); - } + partitionColumns = ParenthesizedSimpleIdentifierList() ] [ <WITH> - { - SqlNode property; - List<SqlNode> proList = new ArrayList<SqlNode>(); - pos = getPos(); - } - <LPAREN> - [ - property = PropertyValue() - { - proList.add(property); - } - ( - <COMMA> property = PropertyValue() - { - proList.add(property); - } - )* - ] - <RPAREN> - { propertyList = new SqlNodeList(proList, pos.plus(getPos())); } + propertyList = TableProperties() ] - { return new SqlCreateTable(startPos.plus(getPos()), tableName, @@ -334,6 +317,48 @@ void PartitionSpecCommaList(SqlNodeList list) : <RPAREN> } +/** +* Parses a create view or replace existing view statement. +* CREATE [OR REPLACE] VIEW view_name [ (field1, field2 ...) ] AS select_statement +*/ +SqlCreate SqlCreateView(Span s, boolean replace) : { + SqlIdentifier viewName; + SqlCharStringLiteral comment = null; + SqlNode query; + SqlNodeList fieldList = SqlNodeList.EMPTY; +} +{ + <VIEW> + viewName = CompoundIdentifier() + [ + fieldList = ParenthesizedSimpleIdentifierList() + ] + [ <COMMENT> <QUOTED_STRING> { + String p = SqlParserUtil.parseString(token.image); + comment = SqlLiteral.createCharString(p, getPos()); + } + ] + <AS> + query = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) + { + return new SqlCreateView(s.pos(), viewName, fieldList, query, replace, comment); + } +} + +SqlDrop SqlDropView(Span s, boolean replace) : +{ + SqlIdentifier viewName = null; + boolean ifExists = false; +} +{ + <VIEW> + [<IF> <EXISTS> { ifExists = true; } ] + viewName = CompoundIdentifier() + { + return new SqlDropView(s.pos(), viewName, ifExists); + } +} + SqlIdentifier SqlArrayType() : { SqlParserPos pos; diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java index e43007c..84e4ca2 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java @@ -272,6 +272,7 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { writer.endList(keyFrame); } } + writer.newlineAndIndent(); writer.endList(frame); @@ -281,7 +282,7 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { comment.unparse(writer, leftPrec, rightPrec); } - if (this.partitionKeyList != null) { + if (this.partitionKeyList != null && this.partitionKeyList.size() > 0) { writer.newlineAndIndent(); writer.keyword("PARTITIONED BY"); SqlWriter.Frame withFrame = writer.startList("(", ")"); diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateView.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateView.java new file mode 100644 index 0000000..566de41 --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateView.java @@ -0,0 +1,122 @@ +/* + * 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.flink.sql.parser.ddl; + +import org.apache.flink.sql.parser.ExtendedSqlNode; +import org.apache.flink.sql.parser.error.SqlParseException; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import org.apache.calcite.sql.SqlCharStringLiteral; +import org.apache.calcite.sql.SqlCreate; +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.SqlSpecialOperator; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; + +import java.util.List; + +/** + * CREATE VIEW DDL sql call. + */ +public class SqlCreateView extends SqlCreate implements ExtendedSqlNode { + public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE_VIEW", SqlKind.CREATE_VIEW); + + private final SqlIdentifier viewName; + private final SqlNodeList fieldList; + private final SqlNode query; + private final SqlCharStringLiteral comment; + + public SqlCreateView( + SqlParserPos pos, + SqlIdentifier viewName, + SqlNodeList fieldList, + SqlNode query, + boolean replace, + SqlCharStringLiteral comment) { + super(OPERATOR, pos, replace, false); + this.viewName = viewName; + this.fieldList = fieldList; + this.query = query; + this.comment = comment; + } + + @Override + public List<SqlNode> getOperandList() { + List<SqlNode> ops = Lists.newArrayList(); + ops.add(viewName); + ops.add(fieldList); + ops.add(query); + ops.add(SqlLiteral.createBoolean(getReplace(), SqlParserPos.ZERO)); + return ops; + } + + public SqlIdentifier getViewName() { + return viewName; + } + + public SqlNodeList getFieldList() { + return fieldList; + } + + public SqlNode getQuery() { + return query; + } + + public SqlCharStringLiteral getComment() { + return comment; + } + + @Override + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { + writer.keyword("CREATE"); + if (getReplace()) { + writer.keyword("OR REPLACE"); + } + writer.keyword("VIEW"); + viewName.unparse(writer, leftPrec, rightPrec); + if (fieldList.size() > 0) { + fieldList.unparse(writer, 1, rightPrec); + } + if (comment != null) { + writer.newlineAndIndent(); + writer.keyword("COMMENT"); + comment.unparse(writer, leftPrec, rightPrec); + } + writer.newlineAndIndent(); + writer.keyword("AS"); + writer.newlineAndIndent(); + query.unparse(writer, leftPrec, rightPrec); + } + + private void printIndent(SqlWriter writer) { + writer.sep(",", false); + writer.newlineAndIndent(); + writer.print(" "); + } + + @Override + public void validate() throws SqlParseException { + // no-op + } +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDropView.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDropView.java new file mode 100644 index 0000000..b859c4a --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDropView.java @@ -0,0 +1,74 @@ +/* + * 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.flink.sql.parser.ddl; + +import org.apache.flink.sql.parser.ExtendedSqlNode; + +import org.apache.calcite.sql.SqlDrop; +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.SqlSpecialOperator; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.util.ImmutableNullableList; + +import java.util.List; + +/** + * DROP VIEW DDL sql call. + */ +public class SqlDropView extends SqlDrop implements ExtendedSqlNode { + private static final SqlOperator OPERATOR = + new SqlSpecialOperator("DROP VIEW", SqlKind.DROP_VIEW); + + private final SqlIdentifier viewName; + + public SqlDropView(SqlParserPos pos, SqlIdentifier viewName, boolean ifExists) { + super(OPERATOR, pos, ifExists); + this.viewName = viewName; + } + + @Override + public List<SqlNode> getOperandList() { + return ImmutableNullableList.of(viewName); + } + + public SqlIdentifier getViewName() { + return viewName; + } + + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { + writer.keyword("DROP"); + writer.keyword("VIEW"); + if (ifExists) { + writer.keyword("IF EXISTS"); + } + viewName.unparse(writer, leftPrec, rightPrec); + } + + public void validate() { + // no-op + } + + public String[] fullViewName() { + return viewName.names.toArray(new String[0]); + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java index b0f4399..2088259 100644 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java @@ -583,6 +583,51 @@ public class FlinkSqlParserImplTest extends SqlParserTest { "OVERWRITE expression is only used with INSERT mode"); } + @Test + public void testCreateView() { + final String sql = "create view v as select col1 from tbl"; + final String expected = "CREATE VIEW `V`\n" + + "AS\n" + + "SELECT `COL1`\n" + + "FROM `TBL`"; + check(sql, expected); + } + + @Test + public void testCreateViewWithComment() { + final String sql = "create view v COMMENT 'this is a view' as select col1 from tbl"; + final String expected = "CREATE VIEW `V`\n" + + "COMMENT 'this is a view'\n" + + "AS\n" + + "SELECT `COL1`\n" + + "FROM `TBL`"; + check(sql, expected); + } + + @Test + public void testCreateViewWithFieldNames() { + final String sql = "create view v(col1, col2) as select col3, col4 from tbl"; + final String expected = "CREATE VIEW `V` (`COL1`, `COL2`)\n" + + "AS\n" + + "SELECT `COL3`, `COL4`\n" + + "FROM `TBL`"; + check(sql, expected); + } + + @Test + public void testCreateViewWithInvalidName() { + final String sql = "create view v^(^*) COMMENT 'this is a view' as select col1 from tbl"; + final String expected = "(?s).*Encountered \"\\( \\*\" at line 1, column 14.*"; + + checkFails(sql, expected); + } + + @Test + public void testDropView() { + final String sql = "DROP VIEW IF EXISTS view_name"; + check(sql, "DROP VIEW IF EXISTS `VIEW_NAME`"); + } + /** Matcher that invokes the #validate() of the produced SqlNode. **/ private static class ValidationMatcher extends BaseMatcher<SqlNode> { private String expectedColumnSql;