From 82c18ae2c894aa5e9936364bc64dc0e08762913e Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Fri, 16 Aug 2024 09:40:28 +0200 Subject: [PATCH] [FLINK-36085][table] Refactor SHOW operations with LIKE/ILIKE Signed-off-by: snuyanzin --- .../parse/HiveParserDDLSemanticAnalyzer.java | 15 +- .../flink/sql/parser/dql/SqlShowCall.java | 106 ++++++ .../flink/sql/parser/dql/SqlShowColumns.java | 73 +--- .../sql/parser/dql/SqlShowDatabases.java | 83 +---- .../sql/parser/dql/SqlShowFunctions.java | 90 +---- .../flink/sql/parser/dql/SqlShowModels.java | 88 +---- .../sql/parser/dql/SqlShowProcedures.java | 88 +---- .../flink/sql/parser/dql/SqlShowTables.java | 93 +---- .../flink/sql/parser/dql/SqlShowViews.java | 94 +---- .../flink/table/catalog/CatalogManager.java | 72 ++-- .../operations/AbstractShowOperation.java | 108 ++++++ .../operations/DescribeTableOperation.java | 3 +- .../flink/table/operations/LikeType.java | 34 -- .../operations/ShowColumnsOperation.java | 87 ++--- .../operations/ShowDatabasesOperation.java | 91 ++--- .../operations/ShowFunctionsOperation.java | 193 +++------- .../operations/ShowProceduresOperation.java | 125 ++----- .../table/operations/ShowTablesOperation.java | 133 +++---- .../table/operations/ShowViewsOperation.java | 128 +++---- .../table/operations/utils/LikeType.java | 78 ++++ .../operations/utils/ShowLikeOperator.java | 94 +++++ .../SqlNodeToOperationConversion.java | 83 +---- .../converters/AbstractSqlShowConverter.java | 102 ++++++ .../converters/SqlNodeConverters.java | 2 + .../converters/SqlShowDatabasesConverter.java | 20 +- .../converters/SqlShowFunctionsConverter.java | 71 ++-- .../converters/SqlShowProcedureConverter.java | 53 ++- .../converters/SqlShowTablesConverter.java | 50 +++ .../converters/SqlShowViewsConverter.java | 50 +++ .../planner/catalog/UnknownCatalogTest.java | 42 +++ .../SqlOtherOperationConverterTest.java | 333 ++++++++++-------- .../runtime/stream/sql/ProcedureITCase.java | 126 +++---- 32 files changed, 1321 insertions(+), 1487 deletions(-) create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowCall.java create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/AbstractShowOperation.java delete mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/LikeType.java create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/LikeType.java create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/ShowLikeOperator.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractSqlShowConverter.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowTablesConverter.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowViewsConverter.java diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveParserDDLSemanticAnalyzer.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveParserDDLSemanticAnalyzer.java index 8b72fece0a2aa..bd8ef491519da 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveParserDDLSemanticAnalyzer.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveParserDDLSemanticAnalyzer.java @@ -321,7 +321,7 @@ public Operation convertToOperation(HiveParserASTNode ast) throws SemanticExcept res = convertDescribeTable(ast); break; case HiveASTParser.TOK_SHOWDATABASES: - res = convertShowDatabases(); + res = convertShowDatabases(catalogRegistry.getCurrentCatalog()); break; case HiveASTParser.TOK_SHOWTABLES: res = convertShowTables(ast, false); @@ -1803,8 +1803,8 @@ private Operation convertShowPartitions(HiveParserASTNode ast) throws SemanticEx HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULTPARTITIONNAME)); } - private Operation convertShowDatabases() { - return new ShowDatabasesOperation(); + private Operation convertShowDatabases(String catalogName) { + return new ShowDatabasesOperation(catalogName); } private Operation convertShowTables(HiveParserASTNode ast, boolean expectView) { @@ -1843,7 +1843,11 @@ private Operation convertShowTables(HiveParserASTNode ast, boolean expectView) { if (pattern != null) { handleUnsupportedOperation("SHOW TABLES/VIEWS LIKE is not supported"); } - return expectView ? new ShowViewsOperation() : new ShowTablesOperation(); + return expectView + ? new ShowViewsOperation( + catalogRegistry.getCurrentCatalog(), catalogRegistry.getCurrentDatabase()) + : new ShowTablesOperation( + catalogRegistry.getCurrentCatalog(), catalogRegistry.getCurrentDatabase()); } /** @@ -1857,7 +1861,8 @@ private Operation convertShowFunctions(HiveParserASTNode ast) { assert (ast.getChild(0).getType() == HiveASTParser.KW_LIKE); throw new ValidationException("SHOW FUNCTIONS LIKE is not supported yet"); } - return new ShowFunctionsOperation(); + return new ShowFunctionsOperation( + catalogRegistry.getCurrentCatalog(), catalogRegistry.getCurrentDatabase()); } private Operation convertAlterTableRename( diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowCall.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowCall.java new file mode 100644 index 0000000000000..3b855812a798b --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowCall.java @@ -0,0 +1,106 @@ +/* + * 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.dql; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlCharStringLiteral; +import org.apache.calcite.sql.SqlIdentifier; +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.Collections; +import java.util.List; +import java.util.Objects; + +public abstract class SqlShowCall extends SqlCall { + private final String preposition; + private final SqlIdentifier sqlIdentifier; + // different like type such as like, ilike + private final String likeType; + private final SqlCharStringLiteral likeLiteral; + private final boolean notLike; + + public SqlShowCall( + SqlParserPos pos, + String preposition, + SqlIdentifier databaseName, + String likeType, + SqlCharStringLiteral likeLiteral, + boolean notLike) { + super(pos); + this.preposition = preposition; + this.sqlIdentifier = preposition != null ? databaseName : null; + this.likeType = likeType; + this.likeLiteral = likeType == null ? null : likeLiteral; + this.notLike = likeType != null && notLike; + } + + @Override + public abstract SqlOperator getOperator(); + + @Override + public List getOperandList() { + return Objects.isNull(sqlIdentifier) + ? Collections.emptyList() + : Collections.singletonList(sqlIdentifier); + } + + @Override + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { + String keyword = getOperationName(); + if (preposition == null) { + writer.keyword(keyword); + } else if (sqlIdentifier != null) { + writer.keyword(keyword + " " + preposition); + sqlIdentifier.unparse(writer, leftPrec, rightPrec); + } + if (isWithLike()) { + final String notPrefix = isNotLike() ? "NOT " : ""; + writer.keyword(String.format("%s%s '%s'", notPrefix, likeType, getLikeSqlPattern())); + } + } + + public String getPreposition() { + return preposition; + } + + public List getSqlIdentifierNameList() { + return Objects.isNull(this.sqlIdentifier) ? Collections.emptyList() : sqlIdentifier.names; + } + + public boolean isWithLike() { + return likeType != null; + } + + public String getLikeType() { + return likeType; + } + + public String getLikeSqlPattern() { + return Objects.isNull(likeLiteral) ? null : likeLiteral.getValueAs(String.class); + } + + public boolean isNotLike() { + return notLike; + } + + abstract String getOperationName(); +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowColumns.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowColumns.java index c21c06cbbc313..f22e802bb4fd5 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowColumns.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowColumns.java @@ -18,66 +18,37 @@ package org.apache.flink.sql.parser.dql; -import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCharStringLiteral; 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 java.util.Collections; -import java.util.List; -import java.util.Objects; - import static java.util.Objects.requireNonNull; /** Show [[catalog.] dataBasesName.]sqlIdentifier sql call. Here we add Rich in */ -public class SqlShowColumns extends SqlCall { +public class SqlShowColumns extends SqlShowCall { public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("SHOW COLUMNS", SqlKind.OTHER); - protected final SqlIdentifier tableName; - protected final String preposition; - protected final boolean notLike; - protected final SqlCharStringLiteral likeLiteral; - public SqlShowColumns( SqlParserPos pos, String preposition, SqlIdentifier tableName, boolean notLike, SqlCharStringLiteral likeLiteral) { - super(pos); - this.preposition = - requireNonNull( - preposition, "Preposition of 'SHOW COLUMNS' must be 'FROM' or 'IN'."); - this.tableName = requireNonNull(tableName, "tableName should not be null."); - this.notLike = notLike; - this.likeLiteral = likeLiteral; - } - - public String getLikeSqlPattern() { - return Objects.isNull(this.likeLiteral) ? null : likeLiteral.getValueAs(String.class); - } - - public boolean isNotLike() { - return notLike; - } - - public SqlCharStringLiteral getLikeLiteral() { - return likeLiteral; - } - - public boolean isWithLike() { - return Objects.nonNull(likeLiteral); - } - - public String getPreposition() { - return preposition; + // only LIKE currently supported for SHOW COLUMNS + super( + pos, + preposition, + tableName, + likeLiteral == null ? null : "LIKE", + likeLiteral, + notLike); + requireNonNull(preposition, "Preposition of 'SHOW COLUMNS' must be 'FROM' or 'IN'."); + requireNonNull(tableName, "tableName should not be null."); } @Override @@ -86,25 +57,7 @@ public SqlOperator getOperator() { } @Override - public List getOperandList() { - return Collections.singletonList(tableName); - } - - public String[] fullTableName() { - return tableName.names.toArray(new String[0]); - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - writer.keyword("SHOW COLUMNS " + this.preposition); - tableName.unparse(writer, leftPrec, rightPrec); - if (isWithLike()) { - if (isNotLike()) { - writer.keyword( - String.format("NOT LIKE '%s'", likeLiteral.getValueAs(String.class))); - } else { - writer.keyword(String.format("LIKE '%s'", likeLiteral.getValueAs(String.class))); - } - } + String getOperationName() { + return "SHOW COLUMNS"; } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowDatabases.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowDatabases.java index ad7af7f782c70..aaccf448f2075 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowDatabases.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowDatabases.java @@ -20,21 +20,13 @@ import org.apache.flink.sql.parser.impl.ParseException; -import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCharStringLiteral; 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 java.util.Collections; -import java.util.List; - -import static java.util.Objects.requireNonNull; - /** * SHOW Databases sql call. The full syntax for show databases is as followings: * @@ -43,23 +35,11 @@ * ] statement * } */ -public class SqlShowDatabases extends SqlCall { +public class SqlShowDatabases extends SqlShowCall { public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("SHOW DATABASES", SqlKind.OTHER); - private final String preposition; - private final SqlIdentifier catalogName; - private final String likeType; - private final SqlCharStringLiteral likeLiteral; - private final boolean notLike; - - public String[] getCatalog() { - return catalogName == null || catalogName.names.isEmpty() - ? new String[] {} - : catalogName.names.toArray(new String[0]); - } - public SqlShowDatabases( SqlParserPos pos, String preposition, @@ -68,29 +48,17 @@ public SqlShowDatabases( SqlCharStringLiteral likeLiteral, boolean notLike) throws ParseException { - super(pos); - this.preposition = preposition; - - this.catalogName = - preposition != null - ? requireNonNull(catalogName, "Catalog name must not be null.") - : null; - if (this.catalogName != null && this.catalogName.names.size() > 1) { + super(pos, preposition, catalogName, likeType, likeLiteral, notLike); + if (catalogName != null && catalogName.names.size() > 1) { throw new ParseException( String.format( "Show databases from/in identifier [ %s ] format error, catalog must be a single part identifier.", - String.join(".", this.catalogName.names))); + String.join(".", catalogName.names))); } + } - if (likeType != null) { - this.likeType = likeType; - this.likeLiteral = requireNonNull(likeLiteral, "Like pattern must not be null"); - this.notLike = notLike; - } else { - this.likeType = null; - this.likeLiteral = null; - this.notLike = false; - } + public String getCatalogName() { + return getSqlIdentifierNameList().isEmpty() ? null : getSqlIdentifierNameList().get(0); } @Override @@ -99,40 +67,7 @@ public SqlOperator getOperator() { } @Override - public List getOperandList() { - return catalogName == null - ? Collections.emptyList() - : Collections.singletonList(catalogName); - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - writer.keyword("SHOW DATABASES"); - if (preposition != null) { - writer.keyword(preposition); - catalogName.unparse(writer, leftPrec, rightPrec); - } - if (likeType != null) { - writer.keyword( - isNotLike() - ? String.format("NOT %s '%s'", likeType, getLikeSqlPattern()) - : String.format("%s '%s'", likeType, getLikeSqlPattern())); - } - } - - public String getLikeSqlPattern() { - return likeLiteral == null ? null : likeLiteral.getValueAs(String.class); - } - - public boolean isNotLike() { - return notLike; - } - - public String getPreposition() { - return preposition; - } - - public String getLikeType() { - return likeType; + String getOperationName() { + return "SHOW DATABASES"; } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowFunctions.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowFunctions.java index 02ce1d42eefb2..8a3b9f0100f90 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowFunctions.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowFunctions.java @@ -18,22 +18,13 @@ package org.apache.flink.sql.parser.dql; -import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCharStringLiteral; 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 java.util.Collections; -import java.util.List; -import java.util.Objects; - -import static java.util.Objects.requireNonNull; - /** * Show Functions sql call. The full syntax for show functions is as followings: * @@ -42,18 +33,12 @@ * ] statement * } */ -public class SqlShowFunctions extends SqlCall { +public class SqlShowFunctions extends SqlShowCall { public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("SHOW FUNCTIONS", SqlKind.OTHER); private final boolean requireUser; - private final String preposition; - private final SqlIdentifier databaseName; - // different like type such as like, ilike - private final String likeType; - private final SqlCharStringLiteral likeLiteral; - private final boolean notLike; public SqlShowFunctions( SqlParserPos pos, @@ -63,22 +48,8 @@ public SqlShowFunctions( String likeType, SqlCharStringLiteral likeLiteral, boolean notLike) { - super(pos); + super(pos, preposition, databaseName, likeType, likeLiteral, notLike); this.requireUser = requireUser; - this.preposition = preposition; - this.databaseName = - preposition != null - ? requireNonNull(databaseName, "Database name must not be null.") - : null; - if (likeType != null) { - this.likeType = likeType; - this.likeLiteral = requireNonNull(likeLiteral, "Like pattern must not be null"); - this.notLike = notLike; - } else { - this.likeType = null; - this.likeLiteral = null; - this.notLike = false; - } } @Override @@ -86,63 +57,12 @@ public SqlOperator getOperator() { return OPERATOR; } - @Override - public List getOperandList() { - return Objects.isNull(databaseName) - ? Collections.emptyList() - : Collections.singletonList(databaseName); - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - String keyword; - if (requireUser) { - keyword = "SHOW USER FUNCTIONS"; - } else { - keyword = "SHOW FUNCTIONS"; - } - if (preposition == null) { - writer.keyword(keyword); - } else if (databaseName != null) { - writer.keyword(keyword + " " + preposition); - databaseName.unparse(writer, leftPrec, rightPrec); - } - if (isWithLike()) { - if (isNotLike()) { - writer.keyword(String.format("NOT %s '%s'", likeType, getLikeSqlPattern())); - } else { - writer.keyword(String.format("%s '%s'", likeType, getLikeSqlPattern())); - } - } - } - public boolean requireUser() { return requireUser; } - public String getPreposition() { - return preposition; - } - - public String[] fullDatabaseName() { - return Objects.isNull(this.databaseName) - ? new String[] {} - : databaseName.names.toArray(new String[0]); - } - - public boolean isWithLike() { - return likeType != null; - } - - public String getLikeType() { - return likeType; - } - - public String getLikeSqlPattern() { - return Objects.isNull(likeLiteral) ? null : likeLiteral.getValueAs(String.class); - } - - public boolean isNotLike() { - return notLike; + @Override + String getOperationName() { + return requireUser ? "SHOW USER FUNCTIONS" : "SHOW FUNCTIONS"; } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowModels.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowModels.java index af11de261bf9a..e9e61342b4b61 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowModels.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowModels.java @@ -18,75 +18,34 @@ package org.apache.flink.sql.parser.dql; -import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCharStringLiteral; 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 java.util.Collections; -import java.util.List; -import java.util.Objects; - -import static java.util.Objects.requireNonNull; - /** {@link SqlNode} to describe the SHOW MODELS syntax. */ -public class SqlShowModels extends SqlCall { +public class SqlShowModels extends SqlShowCall { public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("SHOW MODELS", SqlKind.OTHER); - protected final SqlIdentifier databaseName; - protected final String preposition; - protected final boolean notLike; - protected final SqlCharStringLiteral likeLiteral; - - public SqlShowModels(SqlParserPos pos) { - super(pos); - this.preposition = null; - this.databaseName = null; - this.notLike = false; - this.likeLiteral = null; - } - public SqlShowModels( SqlParserPos pos, String preposition, SqlIdentifier databaseName, boolean notLike, SqlCharStringLiteral likeLiteral) { - super(pos); - this.preposition = preposition; - this.databaseName = - preposition != null - ? requireNonNull(databaseName, "Database name must not be null.") - : null; - this.notLike = notLike; - this.likeLiteral = likeLiteral; - } - - public String getLikeSqlPattern() { - return Objects.isNull(this.likeLiteral) ? null : likeLiteral.getValueAs(String.class); - } - - public boolean isNotLike() { - return notLike; - } - - public SqlCharStringLiteral getLikeLiteral() { - return likeLiteral; - } - - public boolean isWithLike() { - return Objects.nonNull(likeLiteral); - } - - public String getPreposition() { - return preposition; + // only LIKE currently supported for SHOW MODELS + super( + pos, + preposition, + databaseName, + likeLiteral == null ? null : "LIKE", + likeLiteral, + notLike); } @Override @@ -95,32 +54,7 @@ public SqlOperator getOperator() { } @Override - public List getOperandList() { - return Objects.isNull(this.databaseName) - ? Collections.emptyList() - : Collections.singletonList(databaseName); - } - - public String[] fullDatabaseName() { - return Objects.isNull(this.databaseName) - ? new String[] {} - : databaseName.names.toArray(new String[0]); - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - if (this.preposition == null) { - writer.keyword("SHOW MODELS"); - } else if (databaseName != null) { - writer.keyword("SHOW MODELS " + this.preposition); - databaseName.unparse(writer, leftPrec, rightPrec); - } - if (isWithLike()) { - if (isNotLike()) { - writer.keyword(String.format("NOT LIKE '%s'", getLikeSqlPattern())); - } else { - writer.keyword(String.format("LIKE '%s'", getLikeSqlPattern())); - } - } + String getOperationName() { + return "SHOW MODELS"; } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowProcedures.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowProcedures.java index ec259c8b0cde7..cbc04fd6669fc 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowProcedures.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowProcedures.java @@ -18,24 +18,15 @@ package org.apache.flink.sql.parser.dql; -import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCharStringLiteral; 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 javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - -import static java.util.Objects.requireNonNull; - /** * SHOW PROCEDURES sql call. The full syntax for show procedures is as followings: * @@ -44,18 +35,11 @@ * ] statement * } */ -public class SqlShowProcedures extends SqlCall { +public class SqlShowProcedures extends SqlShowCall { public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("SHOW PROCEDURES", SqlKind.OTHER); - @Nullable private final SqlIdentifier databaseName; - @Nullable private final String preposition; - private final boolean notLike; - // different like type such as like, ilike - @Nullable private final String likeType; - @Nullable private final SqlCharStringLiteral likeLiteral; - public SqlShowProcedures( SqlParserPos pos, @Nullable String preposition, @@ -63,42 +47,7 @@ public SqlShowProcedures( boolean notLike, @Nullable String likeType, @Nullable SqlCharStringLiteral likeLiteral) { - super(pos); - this.preposition = preposition; - this.databaseName = - preposition != null - ? requireNonNull(databaseName, "Database name must not be null.") - : null; - - if (likeType != null) { - this.likeType = likeType; - this.likeLiteral = requireNonNull(likeLiteral, "Like pattern must not be null"); - this.notLike = notLike; - } else { - this.likeType = null; - this.likeLiteral = null; - this.notLike = false; - } - } - - public String getLikeSqlPattern() { - return Objects.isNull(this.likeLiteral) ? null : likeLiteral.getValueAs(String.class); - } - - public boolean isNotLike() { - return notLike; - } - - public String getLikeType() { - return likeType; - } - - public SqlCharStringLiteral getLikeLiteral() { - return likeLiteral; - } - - public String getPreposition() { - return preposition; + super(pos, preposition, databaseName, likeType, likeLiteral, notLike); } @Override @@ -107,36 +56,7 @@ public SqlOperator getOperator() { } @Override - public List getOperandList() { - return Objects.isNull(this.databaseName) - ? Collections.emptyList() - : Collections.singletonList(databaseName); - } - - public String[] fullDatabaseName() { - return Objects.isNull(this.databaseName) - ? new String[] {} - : databaseName.names.toArray(new String[0]); - } - - public boolean isWithLike() { - return likeType != null; - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - if (this.preposition == null) { - writer.keyword("SHOW PROCEDURES"); - } else if (databaseName != null) { - writer.keyword("SHOW PROCEDURES " + this.preposition); - databaseName.unparse(writer, leftPrec, rightPrec); - } - if (isWithLike()) { - if (isNotLike()) { - writer.keyword(String.format("NOT %s '%s'", likeType, getLikeSqlPattern())); - } else { - writer.keyword(String.format("%s '%s'", likeType, getLikeSqlPattern())); - } - } + String getOperationName() { + return "SHOW PROCEDURES"; } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowTables.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowTables.java index 09b0965b49fc4..4c1d8592a353f 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowTables.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowTables.java @@ -18,75 +18,40 @@ package org.apache.flink.sql.parser.dql; -import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCharStringLiteral; 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 java.util.Collections; -import java.util.List; -import java.util.Objects; - -import static java.util.Objects.requireNonNull; - -/** SHOW Tables sql call. */ -public class SqlShowTables extends SqlCall { +/** + * SHOW TABLES sql call. The full syntax for show functions is as followings: + * + *
{@code
+ * SHOW TABLES [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] LIKE
+ *  ] statement
+ * }
+ */ +public class SqlShowTables extends SqlShowCall { public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("SHOW TABLES", SqlKind.OTHER); - protected final SqlIdentifier databaseName; - protected final String preposition; - protected final boolean notLike; - protected final SqlCharStringLiteral likeLiteral; - - public SqlShowTables(SqlParserPos pos) { - super(pos); - this.preposition = null; - this.databaseName = null; - this.notLike = false; - this.likeLiteral = null; - } - public SqlShowTables( SqlParserPos pos, String preposition, SqlIdentifier databaseName, boolean notLike, SqlCharStringLiteral likeLiteral) { - super(pos); - this.preposition = preposition; - this.databaseName = - preposition != null - ? requireNonNull(databaseName, "Database name must not be null.") - : null; - this.notLike = notLike; - this.likeLiteral = likeLiteral; - } - - public String getLikeSqlPattern() { - return Objects.isNull(this.likeLiteral) ? null : likeLiteral.getValueAs(String.class); - } - - public boolean isNotLike() { - return notLike; - } - - public SqlCharStringLiteral getLikeLiteral() { - return likeLiteral; - } - - public boolean isWithLike() { - return Objects.nonNull(likeLiteral); - } - - public String getPreposition() { - return preposition; + // only LIKE currently supported for SHOW TABLES + super( + pos, + preposition, + databaseName, + likeLiteral == null ? null : "LIKE", + likeLiteral, + notLike); } @Override @@ -95,27 +60,7 @@ public SqlOperator getOperator() { } @Override - public List getOperandList() { - return Objects.isNull(this.databaseName) - ? Collections.emptyList() - : Collections.singletonList(databaseName); - } - - public List fullDatabaseName() { - return Objects.isNull(this.databaseName) ? Collections.emptyList() : databaseName.names; - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - if (this.preposition == null) { - writer.keyword("SHOW TABLES"); - } else if (databaseName != null) { - writer.keyword("SHOW TABLES " + this.preposition); - databaseName.unparse(writer, leftPrec, rightPrec); - } - if (isWithLike()) { - final String prefix = isNotLike() ? "NOT " : ""; - writer.keyword(String.format("%sLIKE '%s'", prefix, getLikeSqlPattern())); - } + String getOperationName() { + return "SHOW TABLES"; } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowViews.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowViews.java index 22f9d2be15718..3fb593f8f0423 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowViews.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowViews.java @@ -18,75 +18,40 @@ package org.apache.flink.sql.parser.dql; -import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCharStringLiteral; 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 java.util.Collections; -import java.util.List; -import java.util.Objects; - -import static java.util.Objects.requireNonNull; - -/** SHOW VIEWS sql call. */ -public class SqlShowViews extends SqlCall { +/** + * SHOW VIEWS sql call. The full syntax for show functions is as followings: + * + *
{@code
+ * SHOW VIEWS [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] LIKE
+ *  ] statement
+ * }
+ */ +public class SqlShowViews extends SqlShowCall { public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("SHOW VIEWS", SqlKind.OTHER); - protected final String preposition; - protected final SqlIdentifier databaseName; - protected final boolean notLike; - protected final SqlCharStringLiteral likeLiteral; - - public SqlShowViews(SqlParserPos pos) { - super(pos); - this.preposition = null; - this.databaseName = null; - this.notLike = false; - this.likeLiteral = null; - } - public SqlShowViews( SqlParserPos pos, String preposition, SqlIdentifier databaseName, boolean notLike, SqlCharStringLiteral likeLiteral) { - super(pos); - this.preposition = preposition; - this.databaseName = - preposition != null - ? requireNonNull(databaseName, "Database must not be null") - : null; - this.notLike = notLike; - this.likeLiteral = likeLiteral; - } - - public String getLikeSqlPattern() { - return likeLiteral == null ? null : likeLiteral.getValueAs(String.class); - } - - public boolean isNotLike() { - return notLike; - } - - public SqlCharStringLiteral getLikeLiteral() { - return likeLiteral; - } - - public boolean isWithLike() { - return Objects.nonNull(likeLiteral); - } - - public String getPreposition() { - return preposition; + // only LIKE currently supported for SHOW VIEWS + super( + pos, + preposition, + databaseName, + likeLiteral == null ? null : "LIKE", + likeLiteral, + notLike); } @Override @@ -95,28 +60,7 @@ public SqlOperator getOperator() { } @Override - public List getOperandList() { - return databaseName == null - ? Collections.emptyList() - : Collections.singletonList(databaseName); - } - - public List fullDatabaseName() { - return databaseName == null ? Collections.emptyList() : databaseName.names; - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - if (preposition == null) { - writer.keyword("SHOW VIEWS"); - } else if (databaseName != null) { - writer.keyword("SHOW VIEWS " + preposition); - databaseName.unparse(writer, leftPrec, rightPrec); - } - - if (isWithLike()) { - final String prefix = isNotLike() ? "NOT " : ""; - writer.keyword(String.format("%sLIKE '%s'", prefix, getLikeSqlPattern())); - } + String getOperationName() { + return "SHOW VIEWS"; } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java index bba3673c43572..7f5508c1b60af 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java @@ -484,7 +484,7 @@ public Catalog getCatalogOrError(String catalogName) throws CatalogNotExistExcep * @return the current catalog * @see CatalogManager#qualifyIdentifier(UnresolvedIdentifier) */ - public String getCurrentCatalog() { + public @Nullable String getCurrentCatalog() { return currentCatalogName; } @@ -530,7 +530,7 @@ public void setCurrentCatalog(@Nullable String catalogName) throws CatalogNotExi * @return the current database * @see CatalogManager#qualifyIdentifier(UnresolvedIdentifier) */ - public String getCurrentDatabase() { + public @Nullable String getCurrentDatabase() { return currentDatabaseName; } @@ -951,40 +951,46 @@ private boolean permanentDatabaseExists(String catalogName, String databaseName) */ public ObjectIdentifier qualifyIdentifier(UnresolvedIdentifier identifier) { return ObjectIdentifier.of( - identifier - .getCatalogName() - .orElseGet( - () -> { - final String currentCatalog = getCurrentCatalog(); - if (StringUtils.isNullOrWhitespaceOnly(currentCatalog)) { - throw new ValidationException( - "A current catalog has not been set. Please use a" - + " fully qualified identifier (such as" - + " 'my_catalog.my_database.my_table') or" - + " set a current catalog using" - + " 'USE CATALOG my_catalog'."); - } - return currentCatalog; - }), - identifier - .getDatabaseName() - .orElseGet( - () -> { - final String currentDatabase = getCurrentDatabase(); - if (StringUtils.isNullOrWhitespaceOnly(currentDatabase)) { - throw new ValidationException( - "A current database has not been set. Please use a" - + " fully qualified identifier (such as" - + " 'my_database.my_table' or" - + " 'my_catalog.my_database.my_table') or" - + " set a current database using" - + " 'USE my_database'."); - } - return currentDatabase; - }), + identifier.getCatalogName().orElseGet(() -> qualifyCatalog(getCurrentCatalog())), + identifier.getDatabaseName().orElseGet(() -> qualifyDatabase(getCurrentDatabase())), identifier.getObjectName()); } + /** Qualifies catalog name. Throws {@link ValidationException} if not set. */ + public String qualifyCatalog(@Nullable String catalogName) { + if (!StringUtils.isNullOrWhitespaceOnly(catalogName)) { + return catalogName; + } + final String currentCatalogName = getCurrentCatalog(); + if (StringUtils.isNullOrWhitespaceOnly(currentCatalogName)) { + throw new ValidationException( + "A current catalog has not been set. Please use a" + + " fully qualified identifier (such as" + + " 'my_catalog.my_database.my_table') or" + + " set a current catalog using" + + " 'USE CATALOG my_catalog'."); + } + return currentCatalogName; + } + + /** Qualifies database name. Throws {@link ValidationException} if not set. */ + public String qualifyDatabase(@Nullable String databaseName) { + if (!StringUtils.isNullOrWhitespaceOnly(databaseName)) { + return databaseName; + } + final String currentDatabaseName = getCurrentDatabase(); + if (StringUtils.isNullOrWhitespaceOnly(currentDatabaseName)) { + throw new ValidationException( + "A current database has not been set. Please use a" + + " fully qualified identifier (such as" + + " 'my_database.my_table' or" + + " 'my_catalog.my_database.my_table') or" + + " set a current database using" + + " 'USE my_database'."); + } + return currentDatabaseName; + } + /** * Creates a table in a given fully qualified path. * diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/AbstractShowOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/AbstractShowOperation.java new file mode 100644 index 0000000000000..9a4984e317ebb --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/AbstractShowOperation.java @@ -0,0 +1,108 @@ +/* + * 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.table.operations; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.internal.TableResultInternal; +import org.apache.flink.table.operations.utils.ShowLikeOperator; + +import javax.annotation.Nullable; + +import java.util.Collection; +import java.util.Objects; + +import static org.apache.flink.table.api.internal.TableResultUtils.buildStringArrayResult; + +/** + * Base class for SHOW operations. It provides support of functionality: + * + *
+ * [ ( FROM | IN ) catalog_name] [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]
+ * 
+ */ +@Internal +public abstract class AbstractShowOperation implements ShowOperation { + protected final String catalogName; + protected final @Nullable String preposition; + protected final @Nullable ShowLikeOperator likeOp; + + public AbstractShowOperation( + String catalogName, @Nullable String preposition, @Nullable ShowLikeOperator likeOp) { + this.catalogName = catalogName; + this.preposition = preposition; + this.likeOp = likeOp; + } + + protected abstract Collection retrieveDataForTableResult(Context ctx); + + protected abstract String getOperationName(); + + protected abstract String getColumnName(); + + @Override + public TableResultInternal execute(Context ctx) { + final Collection views = retrieveDataForTableResult(ctx); + final String[] rows = + views.stream() + .filter(row -> ShowLikeOperator.likeFilter(row, likeOp)) + .sorted() + .toArray(String[]::new); + return buildStringArrayResult(getColumnName(), rows); + } + + @Override + public String asSummaryString() { + StringBuilder builder = new StringBuilder().append(getOperationName()); + if (preposition != null) { + builder.append(" ").append(getPrepositionSummaryString()); + } + if (likeOp != null) { + builder.append(" ").append(likeOp); + } + return builder.toString(); + } + + protected String getPrepositionSummaryString() { + return String.format("%s %s", preposition, catalogName); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AbstractShowOperation that = (AbstractShowOperation) o; + return Objects.equals(catalogName, that.catalogName) + && Objects.equals(preposition, that.preposition) + && Objects.equals(likeOp, that.likeOp); + } + + @Override + public int hashCode() { + return Objects.hash(catalogName, preposition, likeOp); + } + + @Override + public String toString() { + return asSummaryString(); + } +} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/DescribeTableOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/DescribeTableOperation.java index f706f8981ae03..3b8fb628c9c5c 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/DescribeTableOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/DescribeTableOperation.java @@ -61,8 +61,7 @@ public String asSummaryString() { @Override public TableResultInternal execute(Context ctx) { // DESCRIBE is a synonym for SHOW COLUMNS without LIKE pattern. - ShowColumnsOperation showColumns = - new ShowColumnsOperation(sqlIdentifier, null, false, false, "FROM"); + ShowColumnsOperation showColumns = new ShowColumnsOperation(sqlIdentifier, "FROM", null); return showColumns.execute(ctx); } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/LikeType.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/LikeType.java deleted file mode 100644 index 2c4a2a45b46f1..0000000000000 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/LikeType.java +++ /dev/null @@ -1,34 +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.flink.table.operations; - -import org.apache.flink.annotation.Internal; - -/** Like types enums. */ -@Internal -public enum LikeType { - /** case-sensitive to match a pattern. */ - LIKE, - /** case-insensitive to match a pattern. */ - ILIKE; - - public static LikeType of(String type) { - return LikeType.valueOf(type.toUpperCase()); - } -} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowColumnsOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowColumnsOperation.java index 0947f7618085d..979a2c00de3e1 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowColumnsOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowColumnsOperation.java @@ -27,7 +27,7 @@ import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.WatermarkSpec; -import org.apache.flink.table.functions.SqlLikeUtils; +import org.apache.flink.table.operations.utils.ShowLikeOperator; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalType; @@ -35,6 +35,8 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -43,64 +45,32 @@ import static org.apache.flink.table.api.internal.TableResultUtils.buildTableResult; -/** Show columns from [[catalog.]database.]table. */ +/** + * Operation to describe a SHOW COLUMNS statement. The full syntax for SHOW COLUMNS is as + * followings: + * + *
{@code
+ * SHOW COLUMNS [ ( FROM | IN ) [catalog_name.]database_name.]table [ [NOT] (LIKE | ILIKE)
+ * <sql_like_pattern> ] statement
+ * }
+ */ @Internal -public class ShowColumnsOperation implements ShowOperation { +public class ShowColumnsOperation extends AbstractShowOperation { private final ObjectIdentifier tableIdentifier; - private final boolean useLike; - private final boolean notLike; - private final @Nullable String likePattern; - private final String preposition; public ShowColumnsOperation( ObjectIdentifier tableIdentifier, - @Nullable String likePattern, - boolean useLike, - boolean notLike, - String preposition) { + @Nullable String preposition, + @Nullable ShowLikeOperator likeOp) { + super(tableIdentifier.getCatalogName(), preposition, likeOp); this.tableIdentifier = tableIdentifier; - this.likePattern = likePattern; - this.useLike = useLike; - this.notLike = notLike; - this.preposition = preposition; - } - - public String getLikePattern() { - return likePattern; - } - - public String getPreposition() { - return preposition; - } - - public boolean isUseLike() { - return useLike; - } - - public boolean isNotLike() { - return notLike; } public ObjectIdentifier getTableIdentifier() { return tableIdentifier; } - @Override - public String asSummaryString() { - if (useLike) { - if (notLike) { - return String.format( - "SHOW COLUMNS %s %s %s LIKE '%s'", - preposition, tableIdentifier.asSummaryString(), "NOT", likePattern); - } - return String.format( - "SHOW COLUMNS %s %s LIKE '%s'", - preposition, tableIdentifier.asSummaryString(), likePattern); - } - return String.format("SHOW COLUMNS %s %s", preposition, tableIdentifier.asSummaryString()); - } - @Override public TableResultInternal execute(Context ctx) { Optional result = ctx.getCatalogManager().getTable(tableIdentifier); @@ -113,14 +83,10 @@ public TableResultInternal execute(Context ctx) { ResolvedSchema schema = result.get().getResolvedSchema(); Object[][] rows = generateTableColumnsRows(schema); - if (useLike) { + if (likeOp != null) { rows = Arrays.stream(rows) - .filter( - row -> - notLike - != SqlLikeUtils.like( - row[0].toString(), likePattern, "\\")) + .filter(row -> ShowLikeOperator.likeFilter(row[0].toString(), likeOp)) .toArray(Object[][]::new); } boolean nonComments = isSchemaNonColumnComments(schema); @@ -204,4 +170,21 @@ private DataType[] generateTableColumnsDataTypes(boolean nonComments) { } return result.toArray(new DataType[0]); } + + @Override + protected String getOperationName() { + return "SHOW COLUMNS"; + } + + @Override + protected String getColumnName() { + // Dummy implementation since the main logic is overridden in execute method + return null; + } + + @Override + protected Collection retrieveDataForTableResult(Context ctx) { + // Dummy implementation since the main logic is overridden in execute method + return Collections.emptyList(); + } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowDatabasesOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowDatabasesOperation.java index 19fe9e7884311..80bbf6cb774be 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowDatabasesOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowDatabasesOperation.java @@ -19,83 +19,50 @@ package org.apache.flink.table.operations; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.api.internal.TableResultInternal; -import org.apache.flink.table.functions.SqlLikeUtils; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.operations.utils.ShowLikeOperator; -import java.util.stream.Stream; +import javax.annotation.Nullable; -import static java.util.Objects.requireNonNull; -import static org.apache.flink.table.api.internal.TableResultUtils.buildStringArrayResult; +import java.util.Collection; -/** Operation to describe a SHOW DATABASES statement. */ +/** + * Operation to describe a SHOW DATABASES statement. The full syntax for SHOW DATABASES is as + * followings: + * + *
{@code
+ * SHOW DATABASES [ ( FROM | IN ) catalog_name] [ [NOT] (LIKE | ILIKE) <sql_like_pattern> ]
+ * }
+ */ @Internal -public class ShowDatabasesOperation implements ShowOperation { +public class ShowDatabasesOperation extends AbstractShowOperation { - private final String catalogName; - private final LikeType likeType; - private final String likePattern; - private final boolean notLike; - - public ShowDatabasesOperation() { - // "SHOW DATABASES" command with all options being default - this(null, null, null, false); + public ShowDatabasesOperation( + String catalogName, @Nullable String preposition, @Nullable ShowLikeOperator likeOp) { + super(catalogName, preposition, likeOp); } - public ShowDatabasesOperation(String likeType, String likePattern, boolean notLike) { - this(null, likeType, likePattern, notLike); + public ShowDatabasesOperation(String catalogName, ShowLikeOperator likeOp) { + this(catalogName, null, likeOp); } - public ShowDatabasesOperation( - String catalogName, String likeType, String likePattern, boolean notLike) { - this.catalogName = catalogName; - if (likeType != null) { - this.likeType = LikeType.of(likeType); - this.likePattern = requireNonNull(likePattern, "Like pattern must not be null"); - this.notLike = notLike; - } else { - this.likeType = null; - this.likePattern = null; - this.notLike = false; - } + public ShowDatabasesOperation(String catalogName) { + this(catalogName, null, null); } @Override - public String asSummaryString() { - StringBuilder builder = new StringBuilder(); - builder.append("SHOW DATABASES"); - if (catalogName != null) { - builder.append(String.format(" FROM/IN %s", catalogName)); - } - if (likeType != null) { - if (notLike) { - builder.append(String.format(" NOT %s '%s'", likeType.name(), likePattern)); - } else { - builder.append(String.format(" %s '%s'", likeType.name(), likePattern)); - } - } - return builder.toString(); + protected Collection retrieveDataForTableResult(Context ctx) { + final CatalogManager catalogManager = ctx.getCatalogManager(); + return catalogManager.getCatalogOrThrowException(catalogName).listDatabases(); } @Override - public TableResultInternal execute(Context ctx) { - String cName = - catalogName == null ? ctx.getCatalogManager().getCurrentCatalog() : catalogName; - Stream databases = - ctx.getCatalogManager().getCatalogOrThrowException(cName).listDatabases().stream(); - - if (likeType != null) { - databases = - databases.filter( - row -> { - if (likeType == LikeType.ILIKE) { - return notLike != SqlLikeUtils.ilike(row, likePattern, "\\"); - } else if (likeType == LikeType.LIKE) { - return notLike != SqlLikeUtils.like(row, likePattern, "\\"); - } - return false; - }); - } + protected String getOperationName() { + return "SHOW DATABASES"; + } - return buildStringArrayResult("database name", databases.sorted().toArray(String[]::new)); + @Override + protected String getColumnName() { + return "database name"; } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowFunctionsOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowFunctionsOperation.java index f19cc31a05dc5..697432efc1c9d 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowFunctionsOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowFunctionsOperation.java @@ -19,21 +19,26 @@ package org.apache.flink.table.operations; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.api.internal.TableResultInternal; import org.apache.flink.table.functions.FunctionIdentifier; -import org.apache.flink.table.functions.SqlLikeUtils; +import org.apache.flink.table.operations.utils.ShowLikeOperator; -import java.util.Arrays; +import javax.annotation.Nullable; -import static java.util.Objects.requireNonNull; -import static org.apache.flink.table.api.internal.TableResultUtils.buildStringArrayResult; +import java.util.Arrays; +import java.util.Collection; +import java.util.stream.Collectors; /** - * Operation to describe a SHOW FUNCTIONS [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] - * (LIKE | ILIKE) <sql_like_pattern> ] statement. + * Operation to describe a SHOW FUNCTIONS statement. The full syntax for SHOW FUNCTIONS is as + * followings: + * + *
{@code
+ * SHOW [USER] FUNCTIONS [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] (LIKE | ILIKE)
+ * <sql_like_pattern> ] statement
+ * }
*/ @Internal -public class ShowFunctionsOperation implements ShowOperation { +public class ShowFunctionsOperation extends AbstractShowOperation { /** * Represent scope of function. @@ -50,161 +55,71 @@ public enum FunctionScope { } private final FunctionScope functionScope; - private final String preposition; - private final String catalogName; - private final String databaseName; - // different like type such as like, ilike - private final LikeType likeType; - private final String likePattern; - private final boolean notLike; + private final @Nullable String databaseName; - public ShowFunctionsOperation() { + public ShowFunctionsOperation(String catalogName, String databaseName) { // "SHOW FUNCTIONS" default is ALL scope - this.functionScope = FunctionScope.ALL; - this.preposition = null; - this.catalogName = null; - this.databaseName = null; - this.likeType = null; - this.likePattern = null; - this.notLike = false; + this(FunctionScope.ALL, catalogName, databaseName, null); } public ShowFunctionsOperation( - FunctionScope functionScope, String likeType, String likePattern, boolean notLike) { - this.functionScope = functionScope; - this.preposition = null; - this.catalogName = null; - this.databaseName = null; - if (likeType != null) { - this.likeType = LikeType.of(likeType); - this.likePattern = requireNonNull(likePattern, "Like pattern must not be null"); - this.notLike = notLike; - } else { - this.likeType = null; - this.likePattern = null; - this.notLike = false; - } + FunctionScope functionScope, + @Nullable String catalogName, + @Nullable String databaseName, + @Nullable ShowLikeOperator likeOp) { + this(functionScope, null, catalogName, databaseName, likeOp); } public ShowFunctionsOperation( FunctionScope functionScope, - String preposition, + @Nullable String preposition, String catalogName, String databaseName, - String likeType, - String likePattern, - boolean notLike) { + @Nullable ShowLikeOperator likeOp) { + super(catalogName, preposition, likeOp); this.functionScope = functionScope; - this.preposition = preposition; - this.catalogName = catalogName; this.databaseName = databaseName; - if (likeType != null) { - this.likeType = LikeType.of(likeType); - this.likePattern = requireNonNull(likePattern, "Like pattern must not be null"); - this.notLike = notLike; - } else { - this.likeType = null; - this.likePattern = null; - this.notLike = false; - } } @Override - public String asSummaryString() { - StringBuilder builder = new StringBuilder(); - if (functionScope == FunctionScope.ALL) { - builder.append("SHOW FUNCTIONS"); - } else { - builder.append(String.format("SHOW %s FUNCTIONS", functionScope)); - } - if (preposition != null) { - builder.append(String.format(" %s %s.%s", preposition, catalogName, databaseName)); - } - if (isWithLike()) { - if (isNotLike()) { - builder.append(String.format(" NOT %s '%s'", likeType.name(), likePattern)); - } else { - builder.append(String.format(" %s '%s'", likeType.name(), likePattern)); - } + protected Collection retrieveDataForTableResult(Context ctx) { + switch (functionScope) { + case USER: + if (preposition == null) { + return Arrays.asList(ctx.getFunctionCatalog().getUserDefinedFunctions()); + } + return ctx.getFunctionCatalog().getUserDefinedFunctions(catalogName, databaseName) + .stream() + .map(FunctionIdentifier::getFunctionName) + .collect(Collectors.toList()); + case ALL: + if (preposition == null) { + return Arrays.asList(ctx.getFunctionCatalog().getFunctions()); + } + return Arrays.asList( + ctx.getFunctionCatalog().getFunctions(catalogName, databaseName)); + default: + throw new UnsupportedOperationException( + String.format( + "SHOW FUNCTIONS with %s scope is not supported.", databaseName)); } - return builder.toString(); } - public FunctionScope getFunctionScope() { - return functionScope; - } - - public boolean isLike() { - return likeType == LikeType.LIKE; - } - - public boolean isWithLike() { - return likeType != null; + @Override + protected String getOperationName() { + return functionScope == FunctionScope.ALL ? "SHOW FUNCTIONS" : "SHOW USER FUNCTIONS"; } - public boolean isNotLike() { - return notLike; + @Override + protected String getColumnName() { + return "function name"; } @Override - public TableResultInternal execute(Context ctx) { - final String[] functionNames; - if (preposition == null) { - // it's to show current_catalog.current_database - switch (functionScope) { - case USER: - functionNames = ctx.getFunctionCatalog().getUserDefinedFunctions(); - break; - case ALL: - functionNames = ctx.getFunctionCatalog().getFunctions(); - break; - default: - throw new UnsupportedOperationException( - String.format( - "SHOW FUNCTIONS with %s scope is not supported.", - functionScope)); - } - } else { - switch (functionScope) { - case USER: - functionNames = - ctx.getFunctionCatalog() - .getUserDefinedFunctions(catalogName, databaseName).stream() - .map(FunctionIdentifier::getFunctionName) - .toArray(String[]::new); - break; - case ALL: - functionNames = - ctx.getFunctionCatalog().getFunctions(catalogName, databaseName); - break; - default: - throw new UnsupportedOperationException( - String.format( - "SHOW FUNCTIONS with %s scope is not supported.", - functionScope)); - } - } - - String[] rows; - if (isWithLike()) { - rows = - Arrays.stream(functionNames) - .filter( - row -> { - if (likeType == LikeType.ILIKE) { - return isNotLike() - != SqlLikeUtils.ilike(row, likePattern, "\\"); - } else { - return isNotLike() - != SqlLikeUtils.like(row, likePattern, "\\"); - } - }) - .sorted() - .toArray(String[]::new); - } else { - rows = Arrays.stream(functionNames).sorted().toArray(String[]::new); + public String getPrepositionSummaryString() { + if (databaseName == null) { + return super.getPrepositionSummaryString(); } - - return buildStringArrayResult("function name", rows); + return super.getPrepositionSummaryString() + "." + databaseName; } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowProceduresOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowProceduresOperation.java index 9d263e9e336ec..56d56d1f45b95 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowProceduresOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowProceduresOperation.java @@ -20,130 +20,77 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.internal.TableResultInternal; import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; -import org.apache.flink.table.functions.SqlLikeUtils; +import org.apache.flink.table.operations.utils.ShowLikeOperator; import javax.annotation.Nullable; -import java.util.List; - -import static java.util.Objects.requireNonNull; -import static org.apache.flink.table.api.internal.TableResultUtils.buildStringArrayResult; +import java.util.Collection; /** - * Operation to describe a SHOW PROCEDURES [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] - * (LIKE | ILIKE) <sql_like_pattern> ] statement. + * Operation to describe a SHOW PROCEDURES statement. The full syntax for SHOW PROCEDURES is as + * followings: + * + *
{@code
+ * SHOW PROCEDURES [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] (LIKE | ILIKE)
+ * <sql_like_pattern> ] statement
+ * }
*/ @Internal -public class ShowProceduresOperation implements ExecutableOperation { - - private final @Nullable String catalogName; - - private final @Nullable String databaseName; - private final @Nullable String preposition; - - private final boolean notLike; +public class ShowProceduresOperation extends AbstractShowOperation { - // different like type such as like, ilike - private final LikeType likeType; - - @Nullable private final String sqlLikePattern; - - public ShowProceduresOperation(boolean isNotLike, String likeType, String sqlLikePattern) { - this(null, null, null, isNotLike, likeType, sqlLikePattern); - } + private final String databaseName; public ShowProceduresOperation( + String catalogName, + String databaseName, @Nullable String preposition, - @Nullable String catalogName, - @Nullable String databaseName, - boolean notLike, - @Nullable String likeType, - @Nullable String sqlLikePattern) { - this.preposition = preposition; - this.catalogName = catalogName; + @Nullable ShowLikeOperator likeOp) { + super(catalogName, preposition, likeOp); this.databaseName = databaseName; - - if (likeType != null) { - this.likeType = LikeType.of(likeType); - this.sqlLikePattern = requireNonNull(sqlLikePattern, "Like pattern must not be null"); - this.notLike = notLike; - } else { - this.likeType = null; - this.sqlLikePattern = null; - this.notLike = false; - } } - public boolean isWithLike() { - return likeType != null; + public ShowProceduresOperation( + String catalogName, String databaseName, @Nullable ShowLikeOperator likeOp) { + this(catalogName, databaseName, null, likeOp); } @Override - public TableResultInternal execute(Context ctx) { - final List procedures; - CatalogManager catalogManager = ctx.getCatalogManager(); + protected Collection retrieveDataForTableResult(Context ctx) { + final CatalogManager catalogManager = ctx.getCatalogManager(); try { if (preposition == null) { // it's to show current_catalog.current_database - procedures = - catalogManager - .getCatalogOrError(catalogManager.getCurrentCatalog()) - .listProcedures(catalogManager.getCurrentDatabase()); + return catalogManager.getCatalogOrError(catalogName).listProcedures(databaseName); } else { Catalog catalog = catalogManager.getCatalogOrThrowException(catalogName); - procedures = catalog.listProcedures(databaseName); + return catalog.listProcedures(databaseName); } } catch (DatabaseNotExistException e) { throw new TableException( String.format( "Fail to show procedures because the Database `%s` to show from/in does not exist in Catalog `%s`.", - preposition == null - ? catalogManager.getCurrentDatabase() - : databaseName, - preposition == null - ? catalogManager.getCurrentCatalog() - : catalogName)); + databaseName, catalogName)); } + } - final String[] rows; - if (isWithLike()) { - rows = - procedures.stream() - .filter( - row -> { - boolean likeMatch = - likeType == LikeType.ILIKE - ? SqlLikeUtils.ilike( - row, sqlLikePattern, "\\") - : SqlLikeUtils.like( - row, sqlLikePattern, "\\"); - return notLike != likeMatch; - }) - .sorted() - .toArray(String[]::new); - } else { - rows = procedures.stream().sorted().toArray(String[]::new); - } - return buildStringArrayResult("procedure name", rows); + @Override + protected String getOperationName() { + return "SHOW PROCEDURES"; } @Override - public String asSummaryString() { - StringBuilder builder = new StringBuilder().append("SHOW PROCEDURES"); - if (this.preposition != null) { - builder.append(String.format(" %s %s.%s", preposition, catalogName, databaseName)); - } - if (isWithLike()) { - if (notLike) { - builder.append(String.format(" %s %s %s", "NOT", likeType.name(), sqlLikePattern)); - } else { - builder.append(String.format(" %s %s", likeType.name(), sqlLikePattern)); - } + protected String getColumnName() { + return "procedure name"; + } + + @Override + public String getPrepositionSummaryString() { + if (databaseName == null) { + return super.getPrepositionSummaryString(); } - return builder.toString(); + return super.getPrepositionSummaryString() + "." + databaseName; } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowTablesOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowTablesOperation.java index 5b36283701579..348c9df6c2112 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowTablesOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowTablesOperation.java @@ -20,124 +20,77 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.internal.TableResultInternal; import org.apache.flink.table.catalog.Catalog; -import org.apache.flink.table.functions.SqlLikeUtils; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.operations.utils.ShowLikeOperator; -import java.util.Set; +import javax.annotation.Nullable; -import static org.apache.flink.table.api.internal.TableResultUtils.buildStringArrayResult; -import static org.apache.flink.util.Preconditions.checkNotNull; +import java.util.Set; -/** Operation to describe a SHOW TABLES statement. */ +/** + * Operation to describe a SHOW TABLES statement. The full syntax for SHOW TABLES is as followings: + * + *
{@code
+ * SHOW TABLES [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] LIKE
+ * <sql_like_pattern> ] statement
+ * }
+ */ @Internal -public class ShowTablesOperation implements ShowOperation { +public class ShowTablesOperation extends AbstractShowOperation { - private final String catalogName; private final String databaseName; - private final boolean useLike; - private final boolean notLike; - private final String likePattern; - private final String preposition; - - public ShowTablesOperation() { - this.catalogName = null; - this.databaseName = null; - this.likePattern = null; - this.useLike = false; - this.notLike = false; - this.preposition = null; - } - - public ShowTablesOperation(String likePattern, boolean useLike, boolean notLike) { - this.catalogName = null; - this.databaseName = null; - this.likePattern = - useLike ? checkNotNull(likePattern, "Like pattern must not be null") : null; - this.useLike = useLike; - this.notLike = notLike; - this.preposition = null; - } public ShowTablesOperation( String catalogName, String databaseName, - String likePattern, - boolean useLike, - boolean notLike, - String preposition) { - this.catalogName = checkNotNull(catalogName, "Catalog name must not be null"); - this.databaseName = checkNotNull(databaseName, "Database name must not be null"); - this.likePattern = - useLike ? checkNotNull(likePattern, "Like pattern must not be null") : null; - this.useLike = useLike; - this.notLike = notLike; - this.preposition = checkNotNull(preposition, "Preposition must not be null"); - } - - public String getLikePattern() { - return likePattern; - } - - public String getPreposition() { - return preposition; - } - - public boolean isUseLike() { - return useLike; - } - - public boolean isNotLike() { - return notLike; + @Nullable String preposition, + @Nullable ShowLikeOperator likeOp) { + super(catalogName, preposition, likeOp); + this.databaseName = databaseName; } - public String getCatalogName() { - return catalogName; - } - - public String getDatabaseName() { - return databaseName; + public ShowTablesOperation( + String catalogName, String databaseName, @Nullable ShowLikeOperator likeOp) { + this(catalogName, databaseName, null, likeOp); } - @Override - public String asSummaryString() { - StringBuilder builder = new StringBuilder().append("SHOW TABLES"); - if (this.preposition != null) { - builder.append(String.format(" %s %s.%s", preposition, catalogName, databaseName)); - } - if (this.useLike) { - final String prefix = notLike ? "NOT " : ""; - builder.append(String.format(" %sLIKE '%s'", prefix, likePattern)); - } - return builder.toString(); + public ShowTablesOperation(String catalogName, String databaseName) { + this(catalogName, databaseName, null); } @Override - public TableResultInternal execute(Context ctx) { - final Set tables; + protected Set retrieveDataForTableResult(Context ctx) { + final CatalogManager catalogManager = ctx.getCatalogManager(); if (preposition == null) { - tables = ctx.getCatalogManager().listTables(); + return catalogManager.listTables(); } else { - Catalog catalog = ctx.getCatalogManager().getCatalogOrThrowException(catalogName); + Catalog catalog = catalogManager.getCatalogOrThrowException(catalogName); if (catalog.databaseExists(databaseName)) { - tables = ctx.getCatalogManager().listTables(catalogName, databaseName); + return catalogManager.listTables(catalogName, databaseName); } else { throw new ValidationException( String.format( "Database '%s'.'%s' doesn't exist.", catalogName, databaseName)); } } + } - final String[] rows; - if (useLike) { - rows = - tables.stream() - .filter(row -> notLike != SqlLikeUtils.like(row, likePattern, "\\")) - .sorted() - .toArray(String[]::new); - } else { - rows = tables.stream().sorted().toArray(String[]::new); + @Override + protected String getOperationName() { + return "SHOW TABLES"; + } + + @Override + protected String getColumnName() { + return "table name"; + } + + @Override + public String getPrepositionSummaryString() { + if (databaseName == null) { + return super.getPrepositionSummaryString(); } - return buildStringArrayResult("table name", rows); + return super.getPrepositionSummaryString() + "." + databaseName; } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowViewsOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowViewsOperation.java index d609e6e66947d..180cea511e7d9 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowViewsOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowViewsOperation.java @@ -20,124 +20,76 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.internal.TableResultInternal; import org.apache.flink.table.catalog.Catalog; -import org.apache.flink.table.functions.SqlLikeUtils; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.operations.utils.ShowLikeOperator; -import java.util.Set; +import javax.annotation.Nullable; -import static org.apache.flink.table.api.internal.TableResultUtils.buildStringArrayResult; -import static org.apache.flink.util.Preconditions.checkNotNull; +import java.util.Set; -/** Operation to describe a SHOW VIEWS statement. */ +/** + * Operation to describe a SHOW VIEWS statement. The full syntax for SHOW VIEWS is as followings: + * + *
{@code
+ * SHOW VIEWS [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] LIKE
+ * <sql_like_pattern> ] statement
+ * }
+ */ @Internal -public class ShowViewsOperation implements ShowOperation { +public class ShowViewsOperation extends AbstractShowOperation { - private final String catalogName; private final String databaseName; - private final boolean useLike; - private final boolean notLike; - private final String likePattern; - private final String preposition; - - public ShowViewsOperation() { - catalogName = null; - databaseName = null; - useLike = false; - notLike = false; - likePattern = null; - preposition = null; - } - - public ShowViewsOperation(String likePattern, boolean useLike, boolean notLike) { - this.catalogName = null; - this.databaseName = null; - this.useLike = useLike; - this.notLike = notLike; - this.likePattern = - useLike ? checkNotNull(likePattern, "Like pattern must not be null.") : null; - this.preposition = null; - } public ShowViewsOperation( String catalogName, String databaseName, - String likePattern, - boolean useLike, - boolean notLike, - String preposition) { - this.catalogName = checkNotNull(catalogName, "Catalog name must not be null."); - this.databaseName = checkNotNull(databaseName, "Database name must not be null"); - this.useLike = useLike; - this.notLike = notLike; - this.likePattern = - useLike ? checkNotNull(likePattern, "Like pattern must not be null.") : null; - this.preposition = checkNotNull(preposition, "Preposition must not be null"); - } - - public String getLikePattern() { - return likePattern; - } - - public String getPreposition() { - return preposition; - } - - public boolean isUseLike() { - return useLike; - } - - public boolean isNotLike() { - return notLike; + @Nullable String preposition, + @Nullable ShowLikeOperator likeOp) { + super(catalogName, preposition, likeOp); + this.databaseName = databaseName; } - public String getCatalogName() { - return catalogName; + public ShowViewsOperation( + String catalogName, String databaseName, @Nullable ShowLikeOperator likeOp) { + this(catalogName, databaseName, null, likeOp); } - public String getDatabaseName() { - return databaseName; + public ShowViewsOperation(String catalogName, String databaseName) { + this(catalogName, databaseName, null); } @Override - public String asSummaryString() { - StringBuilder builder = new StringBuilder().append("SHOW VIEWS"); - if (preposition != null) { - builder.append(String.format(" %s %s.%s", preposition, catalogName, databaseName)); - } - if (useLike) { - final String prefix = notLike ? "NOT " : ""; - builder.append(String.format(" %sLIKE '%s'", prefix, likePattern)); - } - return builder.toString(); + protected String getOperationName() { + return "SHOW VIEWS"; } - @Override - public TableResultInternal execute(Context ctx) { - final Set views; + protected Set retrieveDataForTableResult(Context ctx) { + final CatalogManager catalogManager = ctx.getCatalogManager(); if (preposition == null) { - views = ctx.getCatalogManager().listViews(); + return catalogManager.listViews(); } else { - Catalog catalog = ctx.getCatalogManager().getCatalogOrThrowException(catalogName); + Catalog catalog = catalogManager.getCatalogOrThrowException(catalogName); if (catalog.databaseExists(databaseName)) { - views = ctx.getCatalogManager().listViews(catalogName, databaseName); + return catalogManager.listViews(catalogName, databaseName); } else { throw new ValidationException( String.format( "Database '%s'.'%s' doesn't exist.", catalogName, databaseName)); } } + } - final String[] rows; - if (useLike) { - rows = - views.stream() - .filter(row -> notLike != SqlLikeUtils.like(row, likePattern, "\\")) - .sorted() - .toArray(String[]::new); - } else { - rows = views.stream().sorted().toArray(String[]::new); + @Override + protected String getColumnName() { + return "view name"; + } + + @Override + public String getPrepositionSummaryString() { + if (databaseName == null) { + return super.getPrepositionSummaryString(); } - return buildStringArrayResult("view name", rows); + return super.getPrepositionSummaryString() + "." + databaseName; } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/LikeType.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/LikeType.java new file mode 100644 index 0000000000000..161331339607c --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/LikeType.java @@ -0,0 +1,78 @@ +/* + * 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.table.operations.utils; + +import org.apache.flink.annotation.Internal; + +import java.util.Locale; + +/** Like types enums. */ +@Internal +public enum LikeType { + /** case-sensitive to match a pattern. */ + LIKE("LIKE"), + /** case-insensitive to match a pattern. */ + ILIKE("ILIKE"), + NOT_LIKE("NOT LIKE"), + NOT_ILIKE("NOT ILIKE"); + + private final String name; + + LikeType(String name) { + this.name = name; + } + + public static LikeType of(String type) { + return LikeType.valueOf(type.toUpperCase(Locale.ROOT)); + } + + public static LikeType of(String type, boolean notLike) { + if (type == null) { + return null; + } + LikeType likeType = LikeType.valueOf(type.toUpperCase(Locale.ROOT)); + if (notLike) { + if (likeType == LIKE) { + return NOT_LIKE; + } + return NOT_ILIKE; + } + return likeType; + } + + public boolean isNot() { + return this == NOT_LIKE || this == NOT_ILIKE; + } + + public boolean isILike() { + return this == ILIKE || this == NOT_ILIKE; + } + + public static LikeType of(boolean isILike, boolean notLike) { + if (isILike) { + return notLike ? NOT_ILIKE : ILIKE; + } else { + return notLike ? NOT_LIKE : LIKE; + } + } + + public String asSummaryString() { + return name; + } +} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/ShowLikeOperator.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/ShowLikeOperator.java new file mode 100644 index 0000000000000..24b6eb1cf2fe9 --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/ShowLikeOperator.java @@ -0,0 +1,94 @@ +/* + * 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.table.operations.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.functions.SqlLikeUtils; + +import javax.annotation.Nullable; + +import java.util.Objects; + +/** Like operator for SHOW operations. */ +@Internal +public class ShowLikeOperator { + private final LikeType likeType; + private final String likePattern; + + private ShowLikeOperator(LikeType likeType, String likePattern) { + this.likeType = likeType; + this.likePattern = likePattern; + } + + public static ShowLikeOperator of(LikeType likeType, String likePattern) { + return likeType == null ? null : new ShowLikeOperator(likeType, likePattern); + } + + public static ShowLikeOperator of(boolean withLike, boolean isNotLike, String likePattern) { + return of(withLike, false, isNotLike, likePattern); + } + + public static ShowLikeOperator of( + boolean withLike, boolean isILike, boolean isNotLike, String likePattern) { + if (!withLike) { + return null; + } + final LikeType likeType = LikeType.of(isILike, isNotLike); + return new ShowLikeOperator(likeType, likePattern); + } + + public static boolean likeFilter(String row, @Nullable ShowLikeOperator operator) { + if (operator == null) { + return true; + } + final boolean notLike = operator.likeType.isNot(); + final boolean isILike = operator.likeType.isILike(); + final boolean likeMatch = + isILike + ? SqlLikeUtils.ilike(row, operator.likePattern, "\\") + : SqlLikeUtils.like(row, operator.likePattern, "\\"); + return notLike != likeMatch; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ShowLikeOperator likeOp = (ShowLikeOperator) o; + return likeType == likeOp.likeType && Objects.equals(likePattern, likeOp.likePattern); + } + + @Override + public int hashCode() { + return Objects.hash(likeType, likePattern); + } + + @Override + public String toString() { + return asSummaryString(); + } + + public String asSummaryString() { + return likeType.asSummaryString() + " '" + likePattern + "'"; + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlNodeToOperationConversion.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlNodeToOperationConversion.java index a4937a67070b7..3e80342fcfe9c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlNodeToOperationConversion.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlNodeToOperationConversion.java @@ -73,8 +73,6 @@ import org.apache.flink.sql.parser.dql.SqlShowJars; import org.apache.flink.sql.parser.dql.SqlShowJobs; import org.apache.flink.sql.parser.dql.SqlShowModules; -import org.apache.flink.sql.parser.dql.SqlShowTables; -import org.apache.flink.sql.parser.dql.SqlShowViews; import org.apache.flink.sql.parser.dql.SqlUnloadModule; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableException; @@ -133,8 +131,6 @@ import org.apache.flink.table.operations.ShowCurrentCatalogOperation; import org.apache.flink.table.operations.ShowCurrentDatabaseOperation; import org.apache.flink.table.operations.ShowModulesOperation; -import org.apache.flink.table.operations.ShowTablesOperation; -import org.apache.flink.table.operations.ShowViewsOperation; import org.apache.flink.table.operations.SinkModifyOperation; import org.apache.flink.table.operations.SourceQueryOperation; import org.apache.flink.table.operations.StatementSetOperation; @@ -166,6 +162,8 @@ import org.apache.flink.table.operations.ddl.DropTableOperation; import org.apache.flink.table.operations.ddl.DropTempSystemFunctionOperation; import org.apache.flink.table.operations.ddl.DropViewOperation; +import org.apache.flink.table.operations.utils.LikeType; +import org.apache.flink.table.operations.utils.ShowLikeOperator; import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.hint.FlinkHints; import org.apache.flink.table.planner.operations.converters.SqlNodeConverters; @@ -317,14 +315,10 @@ private static Optional convertValidatedSqlNode( return Optional.of(converter.convertDropTable((SqlDropTable) validated)); } else if (validated instanceof SqlAlterTable) { return Optional.of(converter.convertAlterTable((SqlAlterTable) validated)); - } else if (validated instanceof SqlShowTables) { - return Optional.of(converter.convertShowTables((SqlShowTables) validated)); } else if (validated instanceof SqlShowColumns) { return Optional.of(converter.convertShowColumns((SqlShowColumns) validated)); } else if (validated instanceof SqlDropView) { return Optional.of(converter.convertDropView((SqlDropView) validated)); - } else if (validated instanceof SqlShowViews) { - return Optional.of(converter.convertShowViews((SqlShowViews) validated)); } else if (validated instanceof SqlCreateFunction) { return Optional.of(converter.convertCreateFunction((SqlCreateFunction) validated)); } else if (validated instanceof SqlDropFunction) { @@ -931,47 +925,16 @@ private Operation convertShowCurrentDatabase(SqlShowCurrentDatabase sqlShowCurre return new ShowCurrentDatabaseOperation(); } - /** Convert SHOW TABLES statement. */ - private Operation convertShowTables(SqlShowTables sqlShowTables) { - if (sqlShowTables.getPreposition() == null) { - return new ShowTablesOperation( - sqlShowTables.getLikeSqlPattern(), - sqlShowTables.isWithLike(), - sqlShowTables.isNotLike()); - } - List fullDatabaseName = sqlShowTables.fullDatabaseName(); - if (fullDatabaseName.size() > 2) { - throw new ValidationException( - String.format( - "show tables from/in identifier [ %s ] format error", - String.join(".", fullDatabaseName))); - } - String catalogName = - (fullDatabaseName.size() == 1) - ? catalogManager.getCurrentCatalog() - : fullDatabaseName.get(0); - String databaseName = - (fullDatabaseName.size() == 1) ? fullDatabaseName.get(0) : fullDatabaseName.get(1); - return new ShowTablesOperation( - catalogName, - databaseName, - sqlShowTables.getLikeSqlPattern(), - sqlShowTables.isWithLike(), - sqlShowTables.isNotLike(), - sqlShowTables.getPreposition()); - } - /** Convert SHOW COLUMNS statement. */ private Operation convertShowColumns(SqlShowColumns sqlShowColumns) { UnresolvedIdentifier unresolvedIdentifier = - UnresolvedIdentifier.of(sqlShowColumns.fullTableName()); + UnresolvedIdentifier.of(sqlShowColumns.getSqlIdentifierNameList()); ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); - return new ShowColumnsOperation( - identifier, - sqlShowColumns.getLikeSqlPattern(), - sqlShowColumns.isWithLike(), - sqlShowColumns.isNotLike(), - sqlShowColumns.getPreposition()); + ShowLikeOperator likeOp = + ShowLikeOperator.of( + LikeType.of(sqlShowColumns.getLikeType(), sqlShowColumns.isNotLike()), + sqlShowColumns.getLikeSqlPattern()); + return new ShowColumnsOperation(identifier, sqlShowColumns.getLikeSqlPattern(), likeOp); } /** Convert SHOW CREATE TABLE statement. */ @@ -1000,36 +963,6 @@ private Operation convertDropView(SqlDropView sqlDropView) { identifier, sqlDropView.getIfExists(), sqlDropView.isTemporary()); } - /** Convert SHOW VIEWS statement. */ - private Operation convertShowViews(SqlShowViews sqlShowViews) { - if (sqlShowViews.getPreposition() == null) { - return new ShowViewsOperation( - sqlShowViews.getLikeSqlPattern(), - sqlShowViews.isWithLike(), - sqlShowViews.isNotLike()); - } - List fullDatabaseName = sqlShowViews.fullDatabaseName(); - if (fullDatabaseName.size() > 2) { - throw new ValidationException( - String.format( - "show views from/in identifier [ %s ] format error", - String.join(".", fullDatabaseName))); - } - String catalogName = - (fullDatabaseName.size() == 1) - ? catalogManager.getCurrentCatalog() - : fullDatabaseName.get(0); - String databaseName = - (fullDatabaseName.size() == 1) ? fullDatabaseName.get(0) : fullDatabaseName.get(1); - return new ShowViewsOperation( - catalogName, - databaseName, - sqlShowViews.getLikeSqlPattern(), - sqlShowViews.isWithLike(), - sqlShowViews.isNotLike(), - sqlShowViews.getPreposition()); - } - /** Convert RICH EXPLAIN statement. */ private Operation convertRichExplain(SqlRichExplain sqlExplain) { SqlNode sqlNode = sqlExplain.getStatement(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractSqlShowConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractSqlShowConverter.java new file mode 100644 index 0000000000000..80b5d4d36136f --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractSqlShowConverter.java @@ -0,0 +1,102 @@ +/* + * 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.table.planner.operations.converters; + +import org.apache.flink.sql.parser.dql.SqlShowCall; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.utils.LikeType; +import org.apache.flink.table.operations.utils.ShowLikeOperator; + +import javax.annotation.Nullable; + +import java.util.List; + +public abstract class AbstractSqlShowConverter + implements SqlNodeConverter { + + protected Operation convertShowOperation(T sqlShowCall, ConvertContext context) { + final ShowLikeOperator likeOp = getLikeOp(sqlShowCall); + if (sqlShowCall.getPreposition() == null) { + final CatalogManager catalogManager = context.getCatalogManager(); + final String currentCatalogName = catalogManager.getCurrentCatalog(); + final String currentDatabaseName = catalogManager.getCurrentDatabase(); + if (skipQualifyingDefaultCatalogAndDatabase()) { + return getOperationWithoutPrep( + currentCatalogName, currentDatabaseName, sqlShowCall, likeOp); + } + final String catalogName = catalogManager.qualifyCatalog(currentCatalogName); + final String databaseName = catalogManager.qualifyDatabase(currentDatabaseName); + return getOperationWithoutPrep(catalogName, databaseName, sqlShowCall, likeOp); + } + final List sqlIdentifierNameList = sqlShowCall.getSqlIdentifierNameList(); + if (sqlIdentifierNameList.size() > 2) { + throw new ValidationException( + String.format( + "%s from/in identifier [ %s ] format error, it should be [catalog_name.]database_name.", + sqlShowCall.getOperator().getName(), + String.join(".", sqlIdentifierNameList))); + } + final CatalogManager catalogManager = context.getCatalogManager(); + final String catalogName = + sqlIdentifierNameList.size() == 1 + ? catalogManager.getCurrentCatalog() + : sqlIdentifierNameList.get(0); + + final String databaseName = + sqlIdentifierNameList.size() == 1 + ? sqlIdentifierNameList.get(0) + : sqlIdentifierNameList.get(1); + final String qualifiedCatalogName = catalogManager.qualifyCatalog(catalogName); + final String qualifiedDatabaseName = catalogManager.qualifyDatabase(databaseName); + return getOperation( + sqlShowCall, + qualifiedCatalogName, + qualifiedDatabaseName, + sqlShowCall.getPreposition(), + likeOp); + } + + public ShowLikeOperator getLikeOp(SqlShowCall sqlShowCall) { + return ShowLikeOperator.of( + LikeType.of(sqlShowCall.getLikeType(), sqlShowCall.isNotLike()), + sqlShowCall.getLikeSqlPattern()); + } + + public abstract Operation getOperationWithoutPrep( + String catalogName, + String databaseName, + T sqlShowCall, + @Nullable ShowLikeOperator likeOp); + + public abstract Operation getOperation( + T sqlShowCall, + String catalogName, + String databaseName, + @Nullable String prep, + @Nullable ShowLikeOperator likeOp); + + @Override + public abstract Operation convertSqlNode(T node, ConvertContext context); + + protected boolean skipQualifyingDefaultCatalogAndDatabase() { + return false; + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConverters.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConverters.java index 1aeddd6a1281f..5950f2e2c5d88 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConverters.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConverters.java @@ -64,6 +64,8 @@ public class SqlNodeConverters { register(new SqlAlterMaterializedTableSuspendConverter()); register(new SqlAlterMaterializedTableResumeConverter()); register(new SqlDropMaterializedTableConverter()); + register(new SqlShowTablesConverter()); + register(new SqlShowViewsConverter()); } /** diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowDatabasesConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowDatabasesConverter.java index b6a62eef41d3e..47c1ec9a781b3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowDatabasesConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowDatabasesConverter.java @@ -19,25 +19,29 @@ package org.apache.flink.table.planner.operations.converters; import org.apache.flink.sql.parser.dql.SqlShowDatabases; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ShowDatabasesOperation; +import org.apache.flink.table.operations.utils.LikeType; +import org.apache.flink.table.operations.utils.ShowLikeOperator; /** A converter for {@link SqlShowDatabases}. */ public class SqlShowDatabasesConverter implements SqlNodeConverter { @Override public Operation convertSqlNode(SqlShowDatabases sqlShowDatabases, ConvertContext context) { + final ShowLikeOperator likeOp = + ShowLikeOperator.of( + LikeType.of(sqlShowDatabases.getLikeType(), sqlShowDatabases.isNotLike()), + sqlShowDatabases.getLikeSqlPattern()); if (sqlShowDatabases.getPreposition() == null) { - return new ShowDatabasesOperation( - sqlShowDatabases.getLikeType(), - sqlShowDatabases.getLikeSqlPattern(), - sqlShowDatabases.isNotLike()); + final CatalogManager catalogManager = context.getCatalogManager(); + final String currentCatalogName = catalogManager.getCurrentCatalog(); + final String qualifiedCatalogName = catalogManager.qualifyCatalog(currentCatalogName); + return new ShowDatabasesOperation(qualifiedCatalogName, likeOp); } else { return new ShowDatabasesOperation( - sqlShowDatabases.getCatalog()[0], - sqlShowDatabases.getLikeType(), - sqlShowDatabases.getLikeSqlPattern(), - sqlShowDatabases.isNotLike()); + sqlShowDatabases.getCatalogName(), sqlShowDatabases.getPreposition(), likeOp); } } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowFunctionsConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowFunctionsConverter.java index 43faa5a26d40d..124bf7cfc2515 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowFunctionsConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowFunctionsConverter.java @@ -19,50 +19,49 @@ package org.apache.flink.table.planner.operations.converters; import org.apache.flink.sql.parser.dql.SqlShowFunctions; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ShowFunctionsOperation; +import org.apache.flink.table.operations.ShowFunctionsOperation.FunctionScope; +import org.apache.flink.table.operations.utils.ShowLikeOperator; /** A converter for {@link SqlShowFunctions}. */ -public class SqlShowFunctionsConverter implements SqlNodeConverter { +public class SqlShowFunctionsConverter extends AbstractSqlShowConverter { + + @Override + public Operation getOperationWithoutPrep( + String qualifiedCatalogName, + String qualifiedDatabaseName, + SqlShowFunctions sqlShowFunctions, + ShowLikeOperator likeOp) { + final FunctionScope functionScope = getFunctionScope(sqlShowFunctions); + return new ShowFunctionsOperation( + functionScope, qualifiedCatalogName, qualifiedDatabaseName, likeOp); + } + + @Override + public Operation getOperation( + SqlShowFunctions sqlShowFunctions, + String catalogName, + String databaseName, + String prep, + ShowLikeOperator likeOp) { + final FunctionScope functionScope = getFunctionScope(sqlShowFunctions); + return new ShowFunctionsOperation(functionScope, prep, catalogName, databaseName, likeOp); + } @Override public Operation convertSqlNode(SqlShowFunctions sqlShowFunctions, ConvertContext context) { - ShowFunctionsOperation.FunctionScope functionScope = - sqlShowFunctions.requireUser() - ? ShowFunctionsOperation.FunctionScope.USER - : ShowFunctionsOperation.FunctionScope.ALL; + return convertShowOperation(sqlShowFunctions, context); + } - if (sqlShowFunctions.getPreposition() == null) { - return new ShowFunctionsOperation( - functionScope, - sqlShowFunctions.getLikeType(), - sqlShowFunctions.getLikeSqlPattern(), - sqlShowFunctions.isNotLike()); - } + private static FunctionScope getFunctionScope(SqlShowFunctions sqlShowFunctions) { + return sqlShowFunctions.requireUser() ? FunctionScope.USER : FunctionScope.ALL; + } - String[] fullDatabaseName = sqlShowFunctions.fullDatabaseName(); - if (fullDatabaseName.length > 2) { - throw new ValidationException( - String.format( - "Show functions from/in identifier [ %s ] format error, it should be [catalog_name.]database_name.", - String.join(".", fullDatabaseName))); - } - CatalogManager catalogManager = context.getCatalogManager(); - String catalogName = - (fullDatabaseName.length == 1) - ? catalogManager.getCurrentCatalog() - : fullDatabaseName[0]; - String databaseName = - (fullDatabaseName.length == 1) ? fullDatabaseName[0] : fullDatabaseName[1]; - return new ShowFunctionsOperation( - functionScope, - sqlShowFunctions.getPreposition(), - catalogName, - databaseName, - sqlShowFunctions.getLikeType(), - sqlShowFunctions.getLikeSqlPattern(), - sqlShowFunctions.isNotLike()); + @Override + protected boolean skipQualifyingDefaultCatalogAndDatabase() { + // It should be supported to list functions with unset catalog + // for more info FLINhK-33093 + return true; } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowProcedureConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowProcedureConverter.java index 33b5cb73d3edf..a030ae9c3c11a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowProcedureConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowProcedureConverter.java @@ -19,43 +19,34 @@ package org.apache.flink.table.planner.operations.converters; import org.apache.flink.sql.parser.dql.SqlShowProcedures; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ShowProceduresOperation; +import org.apache.flink.table.operations.utils.ShowLikeOperator; /** A converter for {@link SqlShowProcedures}. */ -public class SqlShowProcedureConverter implements SqlNodeConverter { +public class SqlShowProcedureConverter extends AbstractSqlShowConverter { @Override - public Operation convertSqlNode(SqlShowProcedures sqlShowProcedures, ConvertContext context) { - if (sqlShowProcedures.getPreposition() == null) { - return new ShowProceduresOperation( - sqlShowProcedures.isNotLike(), - sqlShowProcedures.getLikeType(), - sqlShowProcedures.getLikeSqlPattern()); - } + public Operation getOperationWithoutPrep( + String qualifiedCatalogName, + String qualifiedDatabaseName, + SqlShowProcedures sqlShowCall, + ShowLikeOperator likeOp) { + return new ShowProceduresOperation(qualifiedCatalogName, qualifiedDatabaseName, likeOp); + } - String[] fullDatabaseName = sqlShowProcedures.fullDatabaseName(); - if (fullDatabaseName.length > 2) { - throw new ValidationException( - String.format( - "Show procedures from/in identifier [ %s ] format error, it should be [catalog_name.]database_name.", - String.join(".", fullDatabaseName))); - } - CatalogManager catalogManager = context.getCatalogManager(); - String catalogName = - (fullDatabaseName.length == 1) - ? catalogManager.getCurrentCatalog() - : fullDatabaseName[0]; - String databaseName = - (fullDatabaseName.length == 1) ? fullDatabaseName[0] : fullDatabaseName[1]; - return new ShowProceduresOperation( - sqlShowProcedures.getPreposition(), - catalogName, - databaseName, - sqlShowProcedures.isNotLike(), - sqlShowProcedures.getLikeType(), - sqlShowProcedures.getLikeSqlPattern()); + @Override + public Operation getOperation( + SqlShowProcedures sqlShowCall, + String catalogName, + String databaseName, + String prep, + ShowLikeOperator likeOp) { + return new ShowProceduresOperation(catalogName, databaseName, prep, likeOp); + } + + @Override + public Operation convertSqlNode(SqlShowProcedures sqlShowProcedures, ConvertContext context) { + return convertShowOperation(sqlShowProcedures, context); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowTablesConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowTablesConverter.java new file mode 100644 index 0000000000000..b0bf5ba69536a --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowTablesConverter.java @@ -0,0 +1,50 @@ +/* + * 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.table.planner.operations.converters; + +import org.apache.flink.sql.parser.dql.SqlShowTables; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.ShowTablesOperation; +import org.apache.flink.table.operations.utils.ShowLikeOperator; + +public class SqlShowTablesConverter extends AbstractSqlShowConverter { + @Override + public Operation getOperationWithoutPrep( + String qualifiedCatalogName, + String qualifiedDatabaseName, + SqlShowTables sqlShowCall, + ShowLikeOperator likeOp) { + return new ShowTablesOperation(qualifiedCatalogName, qualifiedDatabaseName, likeOp); + } + + @Override + public Operation getOperation( + SqlShowTables sqlShowCall, + String catalogName, + String databaseName, + String prep, + ShowLikeOperator likeOp) { + return new ShowTablesOperation(catalogName, databaseName, prep, likeOp); + } + + @Override + public Operation convertSqlNode(SqlShowTables sqlShowTables, ConvertContext context) { + return convertShowOperation(sqlShowTables, context); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowViewsConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowViewsConverter.java new file mode 100644 index 0000000000000..a214fbf4d3b45 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowViewsConverter.java @@ -0,0 +1,50 @@ +/* + * 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.table.planner.operations.converters; + +import org.apache.flink.sql.parser.dql.SqlShowViews; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.ShowViewsOperation; +import org.apache.flink.table.operations.utils.ShowLikeOperator; + +public class SqlShowViewsConverter extends AbstractSqlShowConverter { + @Override + public Operation getOperationWithoutPrep( + String qualifiedCatalogName, + String qualifiedDatabaseName, + SqlShowViews sqlShowCall, + ShowLikeOperator likeOp) { + return new ShowViewsOperation(qualifiedCatalogName, qualifiedDatabaseName, likeOp); + } + + @Override + public Operation getOperation( + SqlShowViews sqlShowCall, + String catalogName, + String databaseName, + String prep, + ShowLikeOperator likeOp) { + return new ShowViewsOperation(catalogName, databaseName, prep, likeOp); + } + + @Override + public Operation convertSqlNode(SqlShowViews sqlShowViews, ConvertContext context) { + return convertShowOperation(sqlShowViews, context); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/UnknownCatalogTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/UnknownCatalogTest.java index 36dd06d9fef2a..42df8dda6c97f 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/UnknownCatalogTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/UnknownCatalogTest.java @@ -34,6 +34,8 @@ import org.apache.flink.util.CollectionUtil; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.util.Arrays; import java.util.Collections; @@ -106,6 +108,46 @@ void testUnsetCatalogWithShowFunctions() { assertThat(functions).hasSizeGreaterThan(0); } + @ParameterizedTest(name = "{index}: {0}") + @ValueSource( + strings = { + "SHOW TABLES", + "SHOW TABLES IN db", + "SHOW VIEWS", + "SHOW VIEWS IN db", + "SHOW PROCEDURES", + "SHOW PROCEDURES IN db", + "SHOW COLUMNS IN db", + "SHOW DATABASES" + }) + void showForUnsetCatalog(String sql) { + TableEnvironment tEnv = TableEnvironment.create(ENVIRONMENT_SETTINGS); + + tEnv.useCatalog(null); + assertThatThrownBy(() -> tEnv.executeSql(sql)) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("A current catalog has not been set"); + } + + @ParameterizedTest(name = "{index}: {0}") + @ValueSource( + strings = { + "SHOW TABLES", + "SHOW VIEWS", + "SHOW PROCEDURES", + // Here `db` is considered as object name + "SHOW COLUMNS IN db" + }) + void showForUnsetDatabase(String sql) { + TableEnvironment tEnv = TableEnvironment.create(ENVIRONMENT_SETTINGS); + + tEnv.useCatalog("cat"); + tEnv.useDatabase(null); + assertThatThrownBy(() -> tEnv.executeSql(sql)) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("A current database has not been set"); + } + @Test void testUnsetCatalogWithFullyQualified() { TableEnvironment tEnv = TableEnvironment.create(ENVIRONMENT_SETTINGS); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlOtherOperationConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlOtherOperationConverterTest.java index 2b641c784967a..5cf2bcecff062 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlOtherOperationConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlOtherOperationConverterTest.java @@ -26,6 +26,7 @@ import org.apache.flink.table.operations.ShowCreateCatalogOperation; import org.apache.flink.table.operations.ShowDatabasesOperation; import org.apache.flink.table.operations.ShowFunctionsOperation; +import org.apache.flink.table.operations.ShowFunctionsOperation.FunctionScope; import org.apache.flink.table.operations.ShowModulesOperation; import org.apache.flink.table.operations.ShowPartitionsOperation; import org.apache.flink.table.operations.ShowProceduresOperation; @@ -43,12 +44,16 @@ import org.apache.flink.table.operations.command.ResetOperation; import org.apache.flink.table.operations.command.SetOperation; import org.apache.flink.table.operations.command.ShowJarsOperation; +import org.apache.flink.table.operations.utils.LikeType; +import org.apache.flink.table.operations.utils.ShowLikeOperator; import org.apache.flink.table.planner.parse.ExtendedParser; import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; import java.util.Arrays; @@ -56,6 +61,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Stream; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -188,68 +194,60 @@ void testShowModules() { assertThat(showModulesOperation.asSummaryString()).isEqualTo("SHOW MODULES"); } - @Test - void testShowTables() { - final String sql = "SHOW TABLES from cat1.db1 not like 't%'"; + @ParameterizedTest(name = "{index}: {0}") + @MethodSource("inputForShowTablesTest") + void testShowTables(String sql, ShowTablesOperation expected, String expectedSummary) { Operation operation = parse(sql); - assertThat(operation).isInstanceOf(ShowTablesOperation.class); - - ShowTablesOperation showTablesOperation = (ShowTablesOperation) operation; - assertThat(showTablesOperation.getCatalogName()).isEqualTo("cat1"); - assertThat(showTablesOperation.getDatabaseName()).isEqualTo("db1"); - assertThat(showTablesOperation.getPreposition()).isEqualTo("FROM"); - assertThat(showTablesOperation.isUseLike()).isTrue(); - assertThat(showTablesOperation.isNotLike()).isTrue(); - assertThat(showTablesOperation.asSummaryString()) - .isEqualTo("SHOW TABLES FROM cat1.db1 NOT LIKE 't%'"); - - final String sql2 = "SHOW TABLES in db2"; - showTablesOperation = (ShowTablesOperation) parse(sql2); - assertThat(showTablesOperation.getCatalogName()).isEqualTo("builtin"); - assertThat(showTablesOperation.getDatabaseName()).isEqualTo("db2"); - assertThat(showTablesOperation.getPreposition()).isEqualTo("IN"); - assertThat(showTablesOperation.isUseLike()).isFalse(); - assertThat(showTablesOperation.isNotLike()).isFalse(); - assertThat(showTablesOperation.asSummaryString()).isEqualTo("SHOW TABLES IN builtin.db2"); - - final String sql3 = "SHOW TABLES"; - showTablesOperation = (ShowTablesOperation) parse(sql3); - assertThat(showTablesOperation.getCatalogName()).isNull(); - assertThat(showTablesOperation.getDatabaseName()).isNull(); - assertThat(showTablesOperation.getPreposition()).isNull(); - assertThat(showTablesOperation.asSummaryString()).isEqualTo("SHOW TABLES"); + assertThat(operation).isInstanceOf(ShowTablesOperation.class).isEqualTo(expected); + assertThat(operation.asSummaryString()).isEqualTo(expectedSummary); } - @Test - void testShowViews() { - final String sql = "SHOW VIEWS from cat1.db1 not like 't%'"; + private static Stream inputForShowTablesTest() { + return Stream.of( + Arguments.of( + "SHOW TABLES from cat1.db1 not like 't%'", + new ShowTablesOperation( + "cat1", + "db1", + "FROM", + ShowLikeOperator.of(LikeType.NOT_LIKE, "t%")), + "SHOW TABLES FROM cat1.db1 NOT LIKE 't%'"), + Arguments.of( + "SHOW TABLES in db2", + new ShowTablesOperation("builtin", "db2", "IN", null), + "SHOW TABLES IN builtin.db2"), + Arguments.of( + "SHOW TABLES", + new ShowTablesOperation("builtin", "default", null, null), + "SHOW TABLES")); + } + + @ParameterizedTest(name = "{index}: {0}") + @MethodSource("inputForShowViewsTest") + void testShowViews(String sql, ShowViewsOperation expected, String expectedSummary) { Operation operation = parse(sql); - assertThat(operation).isInstanceOf(ShowViewsOperation.class); - - ShowViewsOperation showViewsOperation = (ShowViewsOperation) operation; - assertThat(showViewsOperation.getCatalogName()).isEqualTo("cat1"); - assertThat(showViewsOperation.getDatabaseName()).isEqualTo("db1"); - assertThat(showViewsOperation.getPreposition()).isEqualTo("FROM"); - assertThat(showViewsOperation.isUseLike()).isTrue(); - assertThat(showViewsOperation.isNotLike()).isTrue(); - assertThat(showViewsOperation.asSummaryString()) - .isEqualTo("SHOW VIEWS FROM cat1.db1 NOT LIKE 't%'"); - - final String sql2 = "SHOW VIEWS in db2"; - showViewsOperation = (ShowViewsOperation) parse(sql2); - assertThat(showViewsOperation.getCatalogName()).isEqualTo("builtin"); - assertThat(showViewsOperation.getDatabaseName()).isEqualTo("db2"); - assertThat(showViewsOperation.getPreposition()).isEqualTo("IN"); - assertThat(showViewsOperation.isUseLike()).isFalse(); - assertThat(showViewsOperation.isNotLike()).isFalse(); - assertThat(showViewsOperation.asSummaryString()).isEqualTo("SHOW VIEWS IN builtin.db2"); - - final String sql3 = "SHOW VIEWS"; - showViewsOperation = (ShowViewsOperation) parse(sql3); - assertThat(showViewsOperation.getCatalogName()).isNull(); - assertThat(showViewsOperation.getDatabaseName()).isNull(); - assertThat(showViewsOperation.getPreposition()).isNull(); - assertThat(showViewsOperation.asSummaryString()).isEqualTo("SHOW VIEWS"); + assertThat(operation).isInstanceOf(ShowViewsOperation.class).isEqualTo(expected); + assertThat(operation.asSummaryString()).isEqualTo(expectedSummary); + } + + private static Stream inputForShowViewsTest() { + return Stream.of( + Arguments.of( + "SHOW VIEWS from cat1.db1 not like 't%'", + new ShowViewsOperation( + "cat1", + "db1", + "FROM", + ShowLikeOperator.of(LikeType.NOT_LIKE, "t%")), + "SHOW VIEWS FROM cat1.db1 NOT LIKE 't%'"), + Arguments.of( + "SHOW VIEWS in db2", + new ShowViewsOperation("builtin", "db2", "IN", null), + "SHOW VIEWS IN builtin.db2"), + Arguments.of( + "SHOW VIEWS", + new ShowViewsOperation("builtin", "default", null, null), + "SHOW VIEWS")); } @Test @@ -270,82 +268,153 @@ void testShowFullModules() { assertThat(showModulesOperation.asSummaryString()).isEqualTo("SHOW FULL MODULES"); } - @Test - void testShowFunctions() { - final String sql1 = "SHOW FUNCTIONS"; - assertShowFunctions(sql1, sql1, ShowFunctionsOperation.FunctionScope.ALL); - - final String sql2 = "SHOW USER FUNCTIONS"; - assertShowFunctions(sql2, sql2, ShowFunctionsOperation.FunctionScope.USER); - - String sql = "show functions from cat1.db1 not like 'f%'"; - assertShowFunctions( - sql, - "SHOW FUNCTIONS FROM cat1.db1 NOT LIKE 'f%'", - ShowFunctionsOperation.FunctionScope.ALL); - - sql = "show user functions from cat1.db1 ilike 'f%'"; - assertShowFunctions( - sql, - "SHOW USER FUNCTIONS FROM cat1.db1 ILIKE 'f%'", - ShowFunctionsOperation.FunctionScope.USER); - - sql = "show functions in db1"; - assertShowFunctions( - sql, "SHOW FUNCTIONS IN builtin.db1", ShowFunctionsOperation.FunctionScope.ALL); + @ParameterizedTest(name = "{index}: {0}") + @MethodSource("inputForShowFunctionsTest") + void testShowFunctions(String sql, ShowFunctionsOperation expected, String expectedSummary) { + Operation operation = parse(sql); + assertThat(operation).isInstanceOf(ShowFunctionsOperation.class).isEqualTo(expected); + assertThat(operation.asSummaryString()).isEqualTo(expectedSummary); + } - // test fail case - assertThatThrownBy(() -> parse("show functions in cat.db.t")) - .isInstanceOf(ValidationException.class) - .hasMessage( - "Show functions from/in identifier [ cat.db.t ] format error, it should be [catalog_name.]database_name."); + private static Stream inputForShowFunctionsTest() { + return Stream.of( + Arguments.of( + "show functions", + new ShowFunctionsOperation( + FunctionScope.ALL, null, "builtin", "default", null), + "SHOW FUNCTIONS"), + Arguments.of( + "show user functions", + new ShowFunctionsOperation( + FunctionScope.USER, null, "builtin", "default", null), + "SHOW USER FUNCTIONS"), + Arguments.of( + "show functions from cat1.db1 not like 'f%'", + new ShowFunctionsOperation( + FunctionScope.ALL, + "FROM", + "cat1", + "db1", + ShowLikeOperator.of(LikeType.NOT_LIKE, "f%")), + "SHOW FUNCTIONS FROM cat1.db1 NOT LIKE 'f%'"), + Arguments.of( + "show user functions in cat1.db1 ilike 'f%'", + new ShowFunctionsOperation( + FunctionScope.USER, + "IN", + "cat1", + "db1", + ShowLikeOperator.of(LikeType.ILIKE, "f%")), + "SHOW USER FUNCTIONS IN cat1.db1 ILIKE 'f%'"), + Arguments.of( + "show functions in db1", + new ShowFunctionsOperation(FunctionScope.ALL, "IN", "builtin", "db1", null), + "SHOW FUNCTIONS IN builtin.db1")); } @Test - void testShowDatabases() { - final String sql1 = "SHOW DATABASES"; - assertShowDatabases(sql1, sql1); - - String sql = "show databases from db1 not like 'f%'"; - assertShowDatabases(sql, "SHOW DATABASES FROM/IN db1 NOT LIKE 'f%'"); - - sql = "show databases from db1 not ilike 'f%'"; - assertShowDatabases(sql, "SHOW DATABASES FROM/IN db1 NOT ILIKE 'f%'"); - - sql = "show databases from db1 like 'f%'"; - assertShowDatabases(sql, "SHOW DATABASES FROM/IN db1 LIKE 'f%'"); - - sql = "show databases from db1 ilike 'f%'"; - assertShowDatabases(sql, "SHOW DATABASES FROM/IN db1 ILIKE 'f%'"); - - sql = "show databases in db1"; - assertShowDatabases(sql, "SHOW DATABASES FROM/IN db1"); - + void testShowDatabasesFailCase() { assertThatThrownBy(() -> parse("show databases in db.t")) .isInstanceOf(SqlParserException.class) .hasMessage( "SQL parse failed. Show databases from/in identifier [ db.t ] format error, catalog must be a single part identifier."); } - @Test - void testShowProcedures() { - String sql = "SHOW procedures from cat1.db1 not like 't%'"; - assertShowProcedures(sql, "SHOW PROCEDURES FROM cat1.db1 NOT LIKE t%"); + @ParameterizedTest(name = "{index}: {0}") + @MethodSource("inputForShowDatabasesTest") + void testShowDatabases(String sql, ShowDatabasesOperation expected, String expectedSummary) { + Operation operation = parse(sql); + assertThat(operation).isInstanceOf(ShowDatabasesOperation.class).isEqualTo(expected); + assertThat(operation.asSummaryString()).isEqualTo(expectedSummary); + } - sql = "SHOW procedures from cat1.db1 ilike 't%'"; - assertShowProcedures(sql, "SHOW PROCEDURES FROM cat1.db1 ILIKE t%"); + private static Stream inputForShowDatabasesTest() { + return Stream.of( + Arguments.of( + "SHOW DATABASES", + new ShowDatabasesOperation("builtin", null, null), + "SHOW DATABASES"), + Arguments.of( + "show databases from cat1 not like 'f%'", + new ShowDatabasesOperation( + "cat1", "FROM", ShowLikeOperator.of(LikeType.NOT_LIKE, "f%")), + "SHOW DATABASES FROM cat1 NOT LIKE 'f%'"), + Arguments.of( + "show databases from cat1 not ilike 'f%'", + new ShowDatabasesOperation( + "cat1", "FROM", ShowLikeOperator.of(LikeType.NOT_ILIKE, "f%")), + "SHOW DATABASES FROM cat1 NOT ILIKE 'f%'"), + Arguments.of( + "show databases from cat1 like 'f%'", + new ShowDatabasesOperation( + "cat1", "FROM", ShowLikeOperator.of(LikeType.LIKE, "f%")), + "SHOW DATABASES FROM cat1 LIKE 'f%'"), + Arguments.of( + "show databases in cat1 ilike 'f%'", + new ShowDatabasesOperation( + "cat1", "IN", ShowLikeOperator.of(LikeType.ILIKE, "f%")), + "SHOW DATABASES IN cat1 ILIKE 'f%'"), + Arguments.of( + "show databases in cat1", + new ShowDatabasesOperation("cat1", "IN", null), + "SHOW DATABASES IN cat1")); + } - sql = "SHOW procedures in db1"; - assertShowProcedures(sql, "SHOW PROCEDURES IN builtin.db1"); + @ParameterizedTest(name = "{index}: {0}") + @MethodSource("inputForShowProceduresTest") + void testShowProcedures(String sql, ShowProceduresOperation expected) { + Operation operation = parse(sql); + assertThat(operation).isInstanceOf(ShowProceduresOperation.class).isEqualTo(expected); + } - sql = "SHOW procedures"; - assertShowProcedures(sql, "SHOW PROCEDURES"); + private static Stream inputForShowProceduresTest() { + return Stream.of( + Arguments.of( + "SHOW procedures from cat1.db1 not like 't%'", + new ShowProceduresOperation( + "cat1", + "db1", + "FROM", + ShowLikeOperator.of(LikeType.NOT_LIKE, "t%"))), + Arguments.of( + "SHOW procedures from cat1.db1 ilike 't%'", + new ShowProceduresOperation( + "cat1", "db1", "FROM", ShowLikeOperator.of(LikeType.ILIKE, "t%"))), + Arguments.of( + "SHOW procedures in db1", + new ShowProceduresOperation("builtin", "db1", "IN", null)), + Arguments.of( + "SHOW procedures", + new ShowProceduresOperation("builtin", "default", null, null))); + } + @ParameterizedTest + @MethodSource("argsForTestShowFailedCase") + void testShowProceduresFailCase(String sql, String expectedErrorMsg) { // test fail case - assertThatThrownBy(() -> parse("SHOW procedures in cat.db.t")) + assertThatThrownBy(() -> parse(sql)) .isInstanceOf(ValidationException.class) - .hasMessage( - "Show procedures from/in identifier [ cat.db.t ] format error, it should be [catalog_name.]database_name."); + .hasMessage(expectedErrorMsg); + } + + private static Stream argsForTestShowFailedCase() { + return Stream.of( + Arguments.of( + "SHOW procedures in cat.db.t", + "SHOW PROCEDURES from/in identifier [ cat.db.t ] format error," + + " it should be [catalog_name.]database_name."), + Arguments.of( + "SHOW Views in cat.db1.t2", + "SHOW VIEWS from/in identifier [ cat.db1.t2 ] format error," + + " it should be [catalog_name.]database_name."), + Arguments.of( + "SHOW functions in cat.db3.t5", + "SHOW FUNCTIONS from/in identifier [ cat.db3.t5 ] format error," + + " it should be [catalog_name.]database_name."), + Arguments.of( + "SHOW tables in cat1.db3.t2", + "SHOW TABLES from/in identifier [ cat1.db3.t2 ] format error," + + " it should be [catalog_name.]database_name.")); } @Test @@ -466,32 +535,4 @@ void testQuitCommands(String command) { ExtendedParser extendedParser = new ExtendedParser(); assertThat(extendedParser.parse(command)).get().isInstanceOf(QuitOperation.class); } - - private void assertShowFunctions( - String sql, - String expectedSummary, - ShowFunctionsOperation.FunctionScope expectedScope) { - Operation operation = parse(sql); - assertThat(operation).isInstanceOf(ShowFunctionsOperation.class); - - final ShowFunctionsOperation showFunctionsOperation = (ShowFunctionsOperation) operation; - - assertThat(showFunctionsOperation.getFunctionScope()).isEqualTo(expectedScope); - assertThat(showFunctionsOperation.asSummaryString()).isEqualTo(expectedSummary); - } - - private void assertShowDatabases(String sql, String expectedSummary) { - Operation operation = parse(sql); - assertThat(operation).isInstanceOf(ShowDatabasesOperation.class); - final ShowDatabasesOperation showDatabasesOperation = (ShowDatabasesOperation) operation; - assertThat(showDatabasesOperation.asSummaryString()).isEqualTo(expectedSummary); - } - - private void assertShowProcedures(String sql, String expectedSummary) { - Operation operation = parse(sql); - assertThat(operation).isInstanceOf(ShowProceduresOperation.class); - - final ShowProceduresOperation showProceduresOperation = (ShowProceduresOperation) operation; - assertThat(showProceduresOperation.asSummaryString()).isEqualTo(expectedSummary); - } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/ProcedureITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/ProcedureITCase.java index c1a54158721d3..64679cd43124d 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/ProcedureITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/ProcedureITCase.java @@ -39,10 +39,14 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.stream.Stream; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -62,74 +66,60 @@ public void before() throws Exception { tEnv().useCatalog("test_p"); } - @Test - void testShowProcedures() { - List rows = - CollectionUtil.iteratorToList(tEnv().executeSql("show procedures").collect()); - assertThat(rows).isEmpty(); - - // should throw exception since the database(`db1`) to show from doesn't - // exist - assertThatThrownBy(() -> tEnv().executeSql("show procedures in `db1`")) - .isInstanceOf(TableException.class) - .hasMessage( - "Fail to show procedures because the Database `db1` to show from/in does not exist in Catalog `test_p`."); - - // show procedure with specifying catalog & database, but the catalog haven't implemented - // the - // interface to list procedure - assertThatThrownBy( - () -> - tEnv().executeSql( - "show procedures in default_catalog.default_catalog")) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage( - "listProcedures is not implemented for class org.apache.flink.table.catalog.GenericInMemoryCatalog."); - - // show procedure in system database - rows = - CollectionUtil.iteratorToList( - tEnv().executeSql("show procedures in `system`").collect()); - assertThat(rows.toString()) - .isEqualTo( - "[+I[generate_n], +I[generate_user], +I[get_env_conf], +I[get_year], +I[named_args], +I[named_args_optional], +I[named_args_overload], +I[sum_n]]"); - - // show procedure with like - rows = - CollectionUtil.iteratorToList( - tEnv().executeSql("show procedures in `system` like 'generate%'") - .collect()); - assertThat(rows.toString()).isEqualTo("[+I[generate_n], +I[generate_user]]"); - rows = - CollectionUtil.iteratorToList( - tEnv().executeSql("show procedures in `system` like 'gEnerate%'") - .collect()); - assertThat(rows).isEmpty(); - - // show procedure with ilike - rows = - CollectionUtil.iteratorToList( - tEnv().executeSql("show procedures in `system` ilike 'gEnerate%'") - .collect()); - assertThat(rows.toString()).isEqualTo("[+I[generate_n], +I[generate_user]]"); - - // show procedure with not like - rows = - CollectionUtil.iteratorToList( - tEnv().executeSql("show procedures in `system` not like 'generate%'") - .collect()); - assertThat(rows.toString()) - .isEqualTo( - "[+I[get_env_conf], +I[get_year], +I[named_args], +I[named_args_optional], +I[named_args_overload], +I[sum_n]]"); - - // show procedure with not ilike - rows = - CollectionUtil.iteratorToList( - tEnv().executeSql("show procedures in `system` not ilike 'generaTe%'") - .collect()); - assertThat(rows.toString()) - .isEqualTo( - "[+I[get_env_conf], +I[get_year], +I[named_args], +I[named_args_optional], +I[named_args_overload], +I[sum_n]]"); + @ParameterizedTest(name = "{index}: {0}") + @MethodSource("argsForShowProcedures") + void testShowProcedures(String sql, String expected) { + List rows = CollectionUtil.iteratorToList(tEnv().executeSql(sql).collect()); + if (expected.isEmpty()) { + assertThat(rows).isEmpty(); + } else { + assertThat(rows.toString()).isEqualTo(expected); + } + } + + private static Stream argsForShowProcedures() { + return Stream.of( + Arguments.of("show procedures", ""), + Arguments.of( + "show procedures in `system`", + "[+I[generate_n], +I[generate_user], +I[get_env_conf], +I[get_year], +I[named_args], +I[named_args_optional], +I[named_args_overload], +I[sum_n]]"), + Arguments.of( + "show procedures in `system` like 'generate%'", + "[+I[generate_n], +I[generate_user]]"), + Arguments.of("show procedures in `system` like 'gEnerate%'", ""), + Arguments.of( + "show procedures in `system` ilike 'gEnerate%'", + "[+I[generate_n], +I[generate_user]]"), + Arguments.of( + "show procedures in `system` not like 'generate%'", + "[+I[get_env_conf], +I[get_year], +I[named_args], +I[named_args_optional], +I[named_args_overload], +I[sum_n]]"), + Arguments.of( + "show procedures in `system` not ilike 'generaTe%'", + "[+I[get_env_conf], +I[get_year], +I[named_args], +I[named_args_optional], +I[named_args_overload], +I[sum_n]]")); + } + + @ParameterizedTest(name = "{index}: {0}") + @MethodSource("argsForShowProceduresForFailedCases") + void testShowProceduresForFailedCase( + String sql, Class expectedExceptionClass, String expectedErrorMsg) { + assertThatThrownBy(() -> tEnv().executeSql(sql)) + .isInstanceOf(expectedExceptionClass) + .hasMessage(expectedErrorMsg); + } + + private static Stream argsForShowProceduresForFailedCases() { + return Stream.of( + // should throw exception since the database(`db1`) to show from doesn't exist + Arguments.of( + "show procedures in `db1`", + TableException.class, + "Fail to show procedures because the Database `db1` to show from/in does not exist in Catalog `test_p`."), + // show procedure with specifying catalog & database, but the catalog haven't + // implemented the interface to list procedure + Arguments.of( + "show procedures in default_catalog.default_catalog", + UnsupportedOperationException.class, + "listProcedures is not implemented for class org.apache.flink.table.catalog.GenericInMemoryCatalog.")); } @Test