From fac9cfd128dc784e9a12bd47da0be13c6f3e4e1a Mon Sep 17 00:00:00 2001 From: snuyanzin Date: Mon, 19 Aug 2024 16:05:22 +0200 Subject: [PATCH] Address feedback --- .../operations/ShowFunctionsOperation.java | 6 ++-- .../converters/AbstractSqlShowConverter.java | 32 +++++++++++++------ .../converters/SqlShowFunctionsConverter.java | 7 ++++ 3 files changed, 33 insertions(+), 12 deletions(-) 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 dab3f12def0b0..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 @@ -55,7 +55,7 @@ public enum FunctionScope { } private final FunctionScope functionScope; - private final String databaseName; + private final @Nullable String databaseName; public ShowFunctionsOperation(String catalogName, String databaseName) { // "SHOW FUNCTIONS" default is ALL scope @@ -64,8 +64,8 @@ public ShowFunctionsOperation(String catalogName, String databaseName) { public ShowFunctionsOperation( FunctionScope functionScope, - String catalogName, - String databaseName, + @Nullable String catalogName, + @Nullable String databaseName, @Nullable ShowLikeOperator likeOp) { this(functionScope, null, catalogName, databaseName, likeOp); } 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 index 556750b082772..cc34fd3fb2ace 100644 --- 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 @@ -36,13 +36,17 @@ protected Operation convertShowOperation(T sqlShowCall, ConvertContext context) final ShowLikeOperator likeOp = getLikeOp(sqlShowCall); if (sqlShowCall.getPreposition() == null) { final CatalogManager catalogManager = context.getCatalogManager(); - final String catalogName = - catalogManager.qualifyCatalog(catalogManager.getCurrentCatalog()); - final String databaseName = - catalogManager.qualifyDatabase(catalogManager.getCurrentDatabase()); + final String currentCatalogName = catalogManager.getCurrentCatalog(); + final String currentDatabaseName = catalogManager.getCurrentDatabase(); + if (skipQualifyingCatalogAndDatabase()) { + getOperationWithoutPrep( + currentCatalogName, currentDatabaseName, sqlShowCall, likeOp); + } + final String catalogName = catalogManager.qualifyCatalog(currentCatalogName); + final String databaseName = catalogManager.qualifyDatabase(currentDatabaseName); return getOperationWithoutPrep(catalogName, databaseName, sqlShowCall, likeOp); } - List sqlIdentifierNameList = sqlShowCall.getSqlIdentifierNameList(); + final List sqlIdentifierNameList = sqlShowCall.getSqlIdentifierNameList(); if (sqlIdentifierNameList.size() > 2) { throw new ValidationException( String.format( @@ -50,18 +54,24 @@ protected Operation convertShowOperation(T sqlShowCall, ConvertContext context) sqlShowCall.getOperator().getName(), String.join(".", sqlIdentifierNameList))); } - CatalogManager catalogManager = context.getCatalogManager(); - String catalogName = + final CatalogManager catalogManager = context.getCatalogManager(); + final String catalogName = sqlIdentifierNameList.size() == 1 ? catalogManager.getCurrentCatalog() : sqlIdentifierNameList.get(0); - String databaseName = + 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, catalogName, databaseName, sqlShowCall.getPreposition(), likeOp); + sqlShowCall, + qualifiedCatalogName, + qualifiedDatabaseName, + sqlShowCall.getPreposition(), + likeOp); } public ShowLikeOperator getLikeOp(SqlShowCall sqlShowCall) { @@ -85,4 +95,8 @@ public abstract Operation getOperation( @Override public abstract Operation convertSqlNode(T node, ConvertContext context); + + protected boolean skipQualifyingCatalogAndDatabase() { + return false; + } } 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 e6195ce76bff6..f817f677bbe9f 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 @@ -57,4 +57,11 @@ public Operation convertSqlNode(SqlShowFunctions sqlShowFunctions, ConvertContex private static FunctionScope getFunctionScope(SqlShowFunctions sqlShowFunctions) { return sqlShowFunctions.requireUser() ? FunctionScope.USER : FunctionScope.ALL; } + + @Override + protected boolean skipQualifyingCatalogAndDatabase() { + // It should be supported to list functions with unset catalog + // for more info FLINK-33093 + return true; + } }