twalthr commented on code in PR #25200:
URL: https://github.com/apache/flink/pull/25200#discussion_r1722860053
##########
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<SqlShowFunctions> {
+public class SqlShowFunctionsConverter extends
AbstractSqlShowConverter<SqlShowFunctions> {
+
+ @Override
+ public Operation getOperationWithoutPrep(
+ String qualifiedCatalogName,
Review Comment:
rename back to `catalogName` and `databaseName` because `getOperation` also
just calls them this way.
##########
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<SqlShowFunctions> {
+public class SqlShowFunctionsConverter extends
AbstractSqlShowConverter<SqlShowFunctions> {
+
+ @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
Review Comment:
nit: typo
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]