From 07f97e0913a540282715c191bf9237fa4e0e305c Mon Sep 17 00:00:00 2001 From: snuyanzin Date: Mon, 19 Aug 2024 16:09:18 +0200 Subject: [PATCH] Add missing return and fix test --- .../converters/AbstractSqlShowConverter.java | 6 +++--- .../converters/SqlShowFunctionsConverter.java | 4 ++-- .../SqlOtherOperationConverterTest.java | 15 +++++++++------ 3 files changed, 14 insertions(+), 11 deletions(-) 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 cc34fd3fb2ace..80b5d4d36136f 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 @@ -38,8 +38,8 @@ protected Operation convertShowOperation(T sqlShowCall, ConvertContext context) final CatalogManager catalogManager = context.getCatalogManager(); final String currentCatalogName = catalogManager.getCurrentCatalog(); final String currentDatabaseName = catalogManager.getCurrentDatabase(); - if (skipQualifyingCatalogAndDatabase()) { - getOperationWithoutPrep( + if (skipQualifyingDefaultCatalogAndDatabase()) { + return getOperationWithoutPrep( currentCatalogName, currentDatabaseName, sqlShowCall, likeOp); } final String catalogName = catalogManager.qualifyCatalog(currentCatalogName); @@ -96,7 +96,7 @@ public abstract Operation getOperation( @Override public abstract Operation convertSqlNode(T node, ConvertContext context); - protected boolean skipQualifyingCatalogAndDatabase() { + protected boolean skipQualifyingDefaultCatalogAndDatabase() { 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 f817f677bbe9f..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 @@ -59,9 +59,9 @@ private static FunctionScope getFunctionScope(SqlShowFunctions sqlShowFunctions) } @Override - protected boolean skipQualifyingCatalogAndDatabase() { + protected boolean skipQualifyingDefaultCatalogAndDatabase() { // It should be supported to list functions with unset catalog - // for more info FLINK-33093 + // for more info FLINhK-33093 return true; } } 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 235d8d2a79050..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 @@ -218,7 +218,7 @@ private static Stream inputForShowTablesTest() { "SHOW TABLES IN builtin.db2"), Arguments.of( "SHOW TABLES", - new ShowTablesOperation(null, null, null, null), + new ShowTablesOperation("builtin", "default", null, null), "SHOW TABLES")); } @@ -246,7 +246,7 @@ private static Stream inputForShowViewsTest() { "SHOW VIEWS IN builtin.db2"), Arguments.of( "SHOW VIEWS", - new ShowViewsOperation(null, null, null, null), + new ShowViewsOperation("builtin", "default", null, null), "SHOW VIEWS")); } @@ -280,11 +280,13 @@ private static Stream inputForShowFunctionsTest() { return Stream.of( Arguments.of( "show functions", - new ShowFunctionsOperation(FunctionScope.ALL, null, null, null, null), + new ShowFunctionsOperation( + FunctionScope.ALL, null, "builtin", "default", null), "SHOW FUNCTIONS"), Arguments.of( "show user functions", - new ShowFunctionsOperation(FunctionScope.USER, null, null, null, null), + new ShowFunctionsOperation( + FunctionScope.USER, null, "builtin", "default", null), "SHOW USER FUNCTIONS"), Arguments.of( "show functions from cat1.db1 not like 'f%'", @@ -330,7 +332,7 @@ private static Stream inputForShowDatabasesTest() { return Stream.of( Arguments.of( "SHOW DATABASES", - new ShowDatabasesOperation(null, null, null), + new ShowDatabasesOperation("builtin", null, null), "SHOW DATABASES"), Arguments.of( "show databases from cat1 not like 'f%'", @@ -382,7 +384,8 @@ private static Stream inputForShowProceduresTest() { "SHOW procedures in db1", new ShowProceduresOperation("builtin", "db1", "IN", null)), Arguments.of( - "SHOW procedures", new ShowProceduresOperation(null, null, null, null))); + "SHOW procedures", + new ShowProceduresOperation("builtin", "default", null, null))); } @ParameterizedTest