dawidwys commented on code in PR #23612:
URL: https://github.com/apache/flink/pull/23612#discussion_r1444489301
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowDatabasesOperation.java:
##########
@@ -20,26 +20,108 @@
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.internal.TableResultInternal;
+import org.apache.flink.table.functions.SqlLikeUtils;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.requireNonNull;
import static
org.apache.flink.table.api.internal.TableResultUtils.buildStringArrayResult;
/** Operation to describe a SHOW DATABASES statement. */
@Internal
public class ShowDatabasesOperation implements ShowOperation {
+ private final String preposition;
+ 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.preposition = null;
+ this.catalogName = null;
+ this.likeType = null;
+ this.likePattern = null;
+ this.notLike = false;
+ }
+
+ public ShowDatabasesOperation(String likeType, String likePattern, boolean
notLike) {
+ this.preposition = null;
+ this.catalogName = 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;
+ }
+ }
+
+ public ShowDatabasesOperation(
+ String preposition,
+ String catalogName,
+ String likeType,
+ String likePattern,
+ boolean notLike) {
+ this.preposition = preposition;
+ 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;
+ }
+ }
+
@Override
public String asSummaryString() {
- return "SHOW DATABASES";
+ StringBuilder builder = new StringBuilder();
+ builder.append("SHOW DATABASES");
+ if (preposition != null) {
+ builder.append(String.format(" %s %s", preposition, 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();
}
@Override
public TableResultInternal execute(Context ctx) {
- String[] databases =
- ctx.getCatalogManager()
-
.getCatalogOrThrowException(ctx.getCatalogManager().getCurrentCatalog())
- .listDatabases().stream()
- .sorted()
- .toArray(String[]::new);
- return buildStringArrayResult("database name", databases);
+ String cName =
+ catalogName == null ?
ctx.getCatalogManager().getCurrentCatalog() : catalogName;
+ List<String> databases =
+
ctx.getCatalogManager().getCatalogOrThrowException(cName).listDatabases();
+
+ if (likeType != null) {
+ databases =
+ databases.stream()
+ .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;
+ })
+ .collect(Collectors.toList());
+ }
+
+ return buildStringArrayResult(
+ "database name",
databases.stream().sorted().toArray(String[]::new));
Review Comment:
If `databases` is a `Stream<String>` you wouldn't need to collect the
filtering results.
```
Stream<String> 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;
});
}
```
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowDatabasesOperation.java:
##########
@@ -20,26 +20,108 @@
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.internal.TableResultInternal;
+import org.apache.flink.table.functions.SqlLikeUtils;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.requireNonNull;
import static
org.apache.flink.table.api.internal.TableResultUtils.buildStringArrayResult;
/** Operation to describe a SHOW DATABASES statement. */
@Internal
public class ShowDatabasesOperation implements ShowOperation {
+ private final String preposition;
Review Comment:
Is there a difference whether it is `FROM` or `IN` ? If there is no
difference, could we parse both, but use one of the two when unparsing? We
wouldn't need to store the original preposition.
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlShowDatabasesConverter.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.SqlShowDatabases;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.operations.Operation;
+import org.apache.flink.table.operations.ShowDatabasesOperation;
+
+/** A converter for {@link SqlShowDatabases}. */
+public class SqlShowDatabasesConverter implements
SqlNodeConverter<SqlShowDatabases> {
+
+ @Override
+ public Operation convertSqlNode(SqlShowDatabases sqlShowDatabases,
ConvertContext context) {
+ if (sqlShowDatabases.getPreposition() == null) {
+ return new ShowDatabasesOperation(
+ sqlShowDatabases.getLikeType(),
+ sqlShowDatabases.getLikeSqlPattern(),
+ sqlShowDatabases.isNotLike());
+ } else {
+ CatalogManager catalogManager = context.getCatalogManager();
+ String[] fullCatalogName = sqlShowDatabases.getCatalog();
+ String catalogName =
+ fullCatalogName.length == 0
+ ? catalogManager.getCurrentCatalog()
+ : fullCatalogName[0];
Review Comment:
Do we need to get the current catalog at this point? You handle that during
the execution already, don't you?
##########
docs/content.zh/docs/dev/table/sql/show.md:
##########
@@ -507,10 +507,21 @@ SHOW CURRENT CATALOG
## SHOW DATABASES
```sql
-SHOW DATABASES
+SHOW DATABASES [ ( FROM | IN ) catalog_name] [ [NOT] (LIKE | ILIKE)
<sql_like_pattern> ]
Review Comment:
Out of curiosity. Do you know chinese?
##########
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowDatabases.java:
##########
@@ -46,11 +100,40 @@ public SqlOperator getOperator() {
@Override
public List<SqlNode> getOperandList() {
- return Collections.EMPTY_LIST;
+ return catalogName == null
+ ? Collections.emptyList()
+ : Collections.singletonList(catalogName);
}
@Override
public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
- writer.keyword("SHOW DATABASES");
+ String resKeyword = "SHOW DATABASES";
+ writer.keyword(resKeyword);
Review Comment:
nit: inline `resKeyword`?
--
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]