twalthr commented on code in PR #25200:
URL: https://github.com/apache/flink/pull/25200#discussion_r1721523945
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowDatabasesOperation.java:
##########
@@ -19,83 +19,54 @@
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:
+ *
+ * <pre>{@code
+ * SHOW DATABASES [ ( FROM | IN ) catalog_name] [ [NOT] (LIKE | ILIKE)
<sql_like_pattern> ]
+ * }</pre>
+ */
@Internal
-public class ShowDatabasesOperation implements ShowOperation {
-
- private final String catalogName;
- private final LikeType likeType;
- private final String likePattern;
- private final boolean notLike;
+public class ShowDatabasesOperation extends AbstractShowOperation {
- public ShowDatabasesOperation() {
- // "SHOW DATABASES" command with all options being default
- this(null, null, null, false);
+ public ShowDatabasesOperation(
+ @Nullable 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(ShowLikeOperator likeOp) {
+ this(null, 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() {
+ this(null, 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<String> retrieveDataForTableResult(Context ctx) {
+ final CatalogManager catalogManager = ctx.getCatalogManager();
+ final String catalogName =
catalogManager.qualifyCatalog(this.catalogName);
+ String cName = catalogName == null ?
catalogManager.getCurrentCatalog() : catalogName;
Review Comment:
`catalogName` can never be null at this location. that is the core
assumption of `qualifyCatalog`
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowProceduresOperation.java:
##########
@@ -20,130 +20,83 @@
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:
+ *
+ * <pre>{@code
+ * SHOW PROCEDURES [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT]
(LIKE | ILIKE)
+ * <sql_like_pattern> ] statement
+ * }</pre>
*/
@Internal
-public class ShowProceduresOperation implements ExecutableOperation {
-
- private final @Nullable String catalogName;
+public class ShowProceduresOperation extends AbstractShowOperation {
private final @Nullable String databaseName;
- private final @Nullable String preposition;
-
- private final boolean notLike;
-
- // 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);
- }
public ShowProceduresOperation(
- @Nullable String preposition,
@Nullable String catalogName,
@Nullable String databaseName,
- boolean notLike,
- @Nullable String likeType,
- @Nullable String sqlLikePattern) {
- this.preposition = preposition;
- this.catalogName = catalogName;
+ @Nullable String preposition,
+ @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(@Nullable ShowLikeOperator likeOp) {
+ this(null, null, null, likeOp);
}
@Override
- public TableResultInternal execute(Context ctx) {
- final List<String> procedures;
- CatalogManager catalogManager = ctx.getCatalogManager();
+ protected Collection<String> retrieveDataForTableResult(Context ctx) {
+ final CatalogManager catalogManager = ctx.getCatalogManager();
+ final String catalogName =
catalogManager.qualifyCatalog(this.catalogName);
+ final String dbName =
catalogManager.qualifyDatabase(this.databaseName);
try {
if (preposition == null) {
// it's to show current_catalog.current_database
- procedures =
- catalogManager
-
.getCatalogOrError(catalogManager.getCurrentCatalog())
-
.listProcedures(catalogManager.getCurrentDatabase());
+ return catalogManager
+ .getCatalogOrError(catalogManager.getCurrentCatalog())
Review Comment:
calls to `getCurrentCatalog()` and `getCurrentDatabase()` should not be
required anymore?
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowViewsConverter.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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<SqlShowViews> {
+ @Override
+ public Operation getOperationWithoutPrep(SqlShowViews sqlShowCall,
ShowLikeOperator likeOp) {
+ return new ShowViewsOperation(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,
+ sqlIdentifierNameList ->
+ String.format(
+ "show views from/in identifier [ %s ] format
error",
Review Comment:
can we also deduplicate this code? isn't it always parameterized like "show
XXXX from/in identifier [ %s ] format error"
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java:
##########
@@ -951,40 +951,29 @@ 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.getCatalogName().orElseGet(() ->
validateCatalog(getCurrentCatalog())),
Review Comment:
we can slightly simplify the logic here:
```
qualifyCatalog(identifier.getCatalogName().orElse(null)),
qualifyDatabase(identifier.getDatabaseName().orElse(null)),
```
And validateCatalog/Database is not required:
```
public String qualifyCatalog(@Nullable String catalogName) {
if (!StringUtils.isNullOrWhitespaceOnly(catalogName)) {
return catalogName;
}
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;
}
```
the reason why I'm suggesting this is because CatalogManager is complex
enough and it is not obivous what the difference is between qualify and
validate method.
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractSqlShowConverter.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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 java.util.List;
+import java.util.function.Function;
+
+public abstract class AbstractSqlShowConverter<T extends SqlShowCall>
+ implements SqlNodeConverter<T> {
+
+ protected Operation convertShowOperation(
+ T sqlShowCall, Function<List<String>, String> msg, ConvertContext
context) {
+ final ShowLikeOperator likeOp = getLikeOp(sqlShowCall);
+ if (sqlShowCall.getPreposition() == null) {
+ return getOperationWithoutPrep(sqlShowCall, likeOp);
+ }
+ List<String> fullDatabaseName = sqlShowCall.getSqlIdentifierNameList();
+ if (fullDatabaseName.size() > 2) {
+ throw new ValidationException(msg.apply(fullDatabaseName));
+ }
+ CatalogManager catalogManager = context.getCatalogManager();
+ String catalogName =
+ fullDatabaseName.size() == 1
+ ? catalogManager.getCurrentCatalog()
Review Comment:
here `null` could enter the flow again. and `getOperation` is not marked as
`@Nullable` in parameters
--
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]