chucheng92 commented on code in PR #22888:
URL: https://github.com/apache/flink/pull/22888#discussion_r1246541234
##########
flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl:
##########
@@ -455,6 +455,46 @@ SqlShowFunctions SqlShowFunctions() :
}
}
+/**
+* Parses a show functions statement.
+* SHOW PROCEDURES [ ( FROM | IN ) [catalog_name.]database_name ] [ [NOT] (LIKE
| ILIKE) pattern;
+*/
+SqlShowProcedures SqlShowProcedures() :
+{
+ String prep = null;
+ SqlIdentifier databaseName = null;
+ boolean notLike = false;
+ boolean isIlike = false;
Review Comment:
we may support other LIKE form. how about not just use isIlike? we can offer
a LikeTypeEnum. WDYT?
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowProceduresOperation.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.operations;
+
+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 javax.annotation.Nullable;
+
+import java.util.List;
+
+import static
org.apache.flink.table.api.internal.TableResultUtils.buildStringArrayResult;
+
+/**
+ * Operation to describe a SHOW PROCEDURES [ ( FROM | IN )
[catalog_name.]database_name ] [ [NOT]
+ * (LIKE | ILIKE) <sql_like_pattern> ] statement.
+ */
+public class ShowProceduresOperation implements ExecutableOperation {
+
+ private final @Nullable String catalogName;
+
+ private final @Nullable String databaseName;
+ private final @Nullable String preposition;
+
+ protected final boolean useLike;
+
+ private final boolean notLike;
+
+ private final boolean isILike;
+
+ @Nullable private final String sqlLikePattern;
+
+ public ShowProceduresOperation(
+ boolean useLike, boolean isNotLike, boolean isILike, String
sqlLikePattern) {
+ this.catalogName = null;
+ this.databaseName = null;
+ this.preposition = null;
+ this.useLike = useLike;
+ this.notLike = isNotLike;
+ this.isILike = isILike;
+ this.sqlLikePattern = sqlLikePattern;
+ }
+
+ public ShowProceduresOperation(
+ @Nullable String catalogName,
+ @Nullable String databaseName,
+ String preposition,
+ boolean useLike,
+ boolean notLike,
+ boolean isILike,
+ @Nullable String sqlLikePattern) {
+ this.catalogName = catalogName;
+ this.databaseName = databaseName;
+ this.preposition = preposition;
+ this.useLike = useLike;
+ this.isILike = isILike;
+ this.notLike = notLike;
+ this.sqlLikePattern = sqlLikePattern;
+ }
+
+ @Override
+ public TableResultInternal execute(Context ctx) {
+ final List<String> procedures;
+ CatalogManager catalogManager = ctx.getCatalogManager();
+ try {
+ if (preposition == null) {
+ // it's to show current_catalog.current_database
+ procedures =
+ catalogManager
+
.getCatalogOrError(catalogManager.getCurrentCatalog())
+
.listProcedures(catalogManager.getCurrentDatabase());
+ } else {
+ Catalog catalog =
catalogManager.getCatalogOrThrowException(catalogName);
Review Comment:
here skipped the not existed catalog exception.
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ShowProceduresOperation.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.operations;
+
+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 javax.annotation.Nullable;
+
+import java.util.List;
+
+import static
org.apache.flink.table.api.internal.TableResultUtils.buildStringArrayResult;
+
+/**
+ * Operation to describe a SHOW PROCEDURES [ ( FROM | IN )
[catalog_name.]database_name ] [ [NOT]
+ * (LIKE | ILIKE) <sql_like_pattern> ] statement.
+ */
+public class ShowProceduresOperation implements ExecutableOperation {
+
+ private final @Nullable String catalogName;
+
+ private final @Nullable String databaseName;
+ private final @Nullable String preposition;
+
+ protected final boolean useLike;
+
+ private final boolean notLike;
+
+ private final boolean isILike;
+
+ @Nullable private final String sqlLikePattern;
+
+ public ShowProceduresOperation(
+ boolean useLike, boolean isNotLike, boolean isILike, String
sqlLikePattern) {
+ this.catalogName = null;
+ this.databaseName = null;
+ this.preposition = null;
+ this.useLike = useLike;
+ this.notLike = isNotLike;
+ this.isILike = isILike;
+ this.sqlLikePattern = sqlLikePattern;
+ }
+
+ public ShowProceduresOperation(
+ @Nullable String catalogName,
+ @Nullable String databaseName,
+ String preposition,
+ boolean useLike,
+ boolean notLike,
+ boolean isILike,
+ @Nullable String sqlLikePattern) {
+ this.catalogName = catalogName;
+ this.databaseName = databaseName;
+ this.preposition = preposition;
+ this.useLike = useLike;
+ this.isILike = isILike;
+ this.notLike = notLike;
+ this.sqlLikePattern = sqlLikePattern;
+ }
+
+ @Override
+ public TableResultInternal execute(Context ctx) {
+ final List<String> procedures;
+ CatalogManager catalogManager = ctx.getCatalogManager();
+ try {
+ if (preposition == null) {
+ // it's to show current_catalog.current_database
+ procedures =
+ catalogManager
+
.getCatalogOrError(catalogManager.getCurrentCatalog())
+
.listProcedures(catalogManager.getCurrentDatabase());
+ } else {
+ Catalog catalog =
catalogManager.getCatalogOrThrowException(catalogName);
Review Comment:
sorry. my mistake. getCatalogOrThrowException will throw validate exception
about not existed catalog.
##########
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowProcedures.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.sql.parser.dql;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * SHOW PROCEDURES sql call. 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>
+ */
+public class SqlShowProcedures extends SqlCall {
+
+ public static final SqlSpecialOperator OPERATOR =
+ new SqlSpecialOperator("SHOW PROCEDURES", SqlKind.OTHER);
+
+ private final SqlIdentifier databaseName;
+ private final String preposition;
+ private final boolean notLike;
+
+ /** ILike represents it's case insensitive when match procedures with
regex expression. */
+ private final boolean isILike;
+
+ private final SqlCharStringLiteral likeLiteral;
+
+ public SqlShowProcedures(
+ SqlParserPos pos,
+ String preposition,
+ SqlIdentifier databaseName,
+ boolean notLike,
+ boolean isILike,
Review Comment:
how about use a likeType to check like pattern?
--
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]