[GitHub] [flink] Airblader commented on a change in pull request #17352: [FLINK-10230][table] Support 'SHOW CREATE VIEW' syntax to print the query of a view

2021-09-30 Thread GitBox


Airblader commented on a change in pull request #17352:
URL: https://github.com/apache/flink/pull/17352#discussion_r719099470



##
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/ShowCreateUtil.java
##
@@ -0,0 +1,221 @@
+/*
+ * 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.api.internal;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogView;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.QueryOperationCatalogView;
+import org.apache.flink.table.catalog.ResolvedCatalogBaseTable;
+import org.apache.flink.table.catalog.ResolvedCatalogTable;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.utils.EncodingUtils;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/** SHOW CREATE statement Util. */
+@Internal
+enum ShowCreateUtil {
+;
+
+static String buildCreateFormattedPrefix(
+String tableType, boolean isTemporary, ObjectIdentifier 
identifier) {
+return String.format(
+"CREATE %s%s %s (\n",
+isTemporary ? "TEMPORARY " : "", tableType, 
identifier.asSerializableString());
+}
+
+static Optional extractFormattedPrimaryKey(
+ResolvedCatalogBaseTable table, String printIndent) {
+Optional primaryKey = 
table.getResolvedSchema().getPrimaryKey();
+return primaryKey.map(
+uniqueConstraint -> String.format("%s%s", printIndent, 
uniqueConstraint));
+}
+
+static String getColumnString(Column column) {
+final StringBuilder sb = new StringBuilder();
+sb.append(EncodingUtils.escapeIdentifier(column.getName()));
+sb.append(" ");
+// skip data type for computed column
+if (column instanceof Column.ComputedColumn) {
+sb.append(
+column.explainExtras()
+.orElseThrow(
+() ->
+new TableException(
+String.format(
+"Column expression 
can not be null for computed column '%s'",
+
column.getName();
+} else {
+
sb.append(column.getDataType().getLogicalType().asSerializableString());
+column.explainExtras()
+.ifPresent(
+e -> {
+sb.append(" ");
+sb.append(e);
+});
+}
+// TODO: Print the column comment until FLINK-18958 is fixed
+return sb.toString();
+}
+
+static String extractFormattedColumns(ResolvedCatalogBaseTable table, 
String printIndent) {
+return table.getResolvedSchema().getColumns().stream()
+.map(column -> String.format("%s%s", printIndent, 
getColumnString(column)))
+.collect(Collectors.joining(",\n"));
+}
+
+static Optional extractFormattedWatermarkSpecs(
+ResolvedCatalogBaseTable table, String printIndent) {
+if (!table.getResolvedSchema().getWatermarkSpecs().isEmpty()) {
+return Optional.of(
+table.getResolvedSchema().getWatermarkSpecs().stream()
+.map(
+watermarkSpec ->
+String.format(
+"%sWATERMARK FOR %s AS %s",
+printIndent,
+
EncodingUtils.escapeIdentifier(
+
watermarkSpec.getRowtimeAttribute()),
+ 

[GitHub] [flink] Airblader commented on a change in pull request #17352: [FLINK-10230][table] Support 'SHOW CREATE VIEW' syntax to print the query of a view

2021-09-27 Thread GitBox


Airblader commented on a change in pull request #17352:
URL: https://github.com/apache/flink/pull/17352#discussion_r716428189



##
File path: docs/content/docs/dev/table/sql/show.md
##
@@ -26,7 +26,7 @@ under the License.
 
 # SHOW Statements
 
-SHOW statements are used to list all catalogs, or list all databases in the 
current catalog, or list all tables/views in the current catalog and the 
current database, or show current catalog and database, or show create 
statement for specified table, or list all functions including system functions 
and user-defined functions in the current catalog and current database, or list 
only user-defined functions in the current catalog and current database, or 
list enabled module names, or list all loaded modules with enabled status in 
the current session, or list the columns of the table or the view with the 
given name and the optional like clause.
+SHOW statements are used to list all catalogs, or list all databases in the 
current catalog, or list all tables/views in the current catalog and the 
current database, or show current catalog and database, or show create 
statement for specified table or specified view, or list all functions 
including system functions and user-defined functions in the current catalog 
and current database, or list only user-defined functions in the current 
catalog and current database, or list enabled module names, or list all loaded 
modules with enabled status in the current session, or list the columns of the 
table or the view with the given name and the optional like clause.

Review comment:
   This isn't entirely related to your PR, but the sentence here is very 
lengthy and pretty much unreadable at this point. Can we maybe rephrase this a 
bit?
   
   > `SHOW` statements are used to list objects within their corresponding 
parent, such as catalogs, databases, tables and views, columns, functions, and 
modules. See the individual commands for more details and additional options.
   > `SHOW CREATE` statements are used to print a DDL statement with which a 
given object can be created.

##
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
##
@@ -1403,44 +1429,120 @@ private String buildShowCreateTableRow(
 
.asSerializableString()))
 .collect(Collectors.joining("\n")));
 }
-// append constraint
-if (schema.getPrimaryKey().isPresent()) {
-sb.append(",\n");
-sb.append(String.format("%s%s", printIndent, 
schema.getPrimaryKey().get()));
-}
-sb.append("\n) ");
-// append comment
+return Optional.empty();
+}
+
+private Optional 
extractFormattedComment(ResolvedCatalogBaseTable table) {
 String comment = table.getComment();
 if (StringUtils.isNotEmpty(comment)) {
-sb.append(String.format("COMMENT '%s'\n", comment));
+return Optional.of(EncodingUtils.escapeSingleQuotes(comment));
 }
-// append partitions
-ResolvedCatalogTable catalogTable = (ResolvedCatalogTable) table;
-if (catalogTable.isPartitioned()) {
-sb.append("PARTITIONED BY (")
-.append(
-catalogTable.getPartitionKeys().stream()
-.map(EncodingUtils::escapeIdentifier)
-.collect(Collectors.joining(", ")))
-.append(")\n");
+return Optional.empty();
+}
+
+private Optional 
extractFormattedPartitionedInfo(ResolvedCatalogTable catalogTable) {
+if (!catalogTable.isPartitioned()) {
+return Optional.empty();
 }
+return Optional.of(
+catalogTable.getPartitionKeys().stream()
+.map(EncodingUtils::escapeIdentifier)
+.collect(Collectors.joining(", ")));
+}
+
+private Optional extractFormattedOptions(
+ResolvedCatalogBaseTable table, String printIndent) {
+if (Objects.isNull(table.getOptions()) || 
table.getOptions().isEmpty()) {
+return Optional.empty();
+}
+return Optional.of(
+table.getOptions().entrySet().stream()
+.map(
+entry ->
+String.format(
+"%s'%s' = '%s'",
+printIndent, entry.getKey(), 
entry.getValue()))
+.collect(Collectors.joining(",\n")));
+}
+
+private String extractFormattedColumnNames(ResolvedCatalogBaseTable 
baseTable) {
+return baseTable.getResolvedSchema().getColumns().stream()
+.map(Column::getName)
+

[GitHub] [flink] Airblader commented on a change in pull request #17352: [FLINK-10230][table] Support 'SHOW CREATE VIEW' syntax to print the query of a view

2021-09-24 Thread GitBox


Airblader commented on a change in pull request #17352:
URL: https://github.com/apache/flink/pull/17352#discussion_r715618204



##
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
##
@@ -1441,6 +1468,59 @@ private String buildShowCreateTableRow(
 return sb.toString();
 }
 
+/** Show create view statement only for views. */
+private String buildShowCreateViewRow(
+ResolvedCatalogBaseTable view,
+ObjectIdentifier viewIdentifier,
+boolean isTemporary) {
+final String printIndent = "  ";
+CatalogBaseTable.TableKind kind = view.getTableKind();
+if (kind != CatalogBaseTable.TableKind.VIEW) {
+throw new TableException(
+String.format(
+"SHOW CREATE VIEW does not support showing CREATE 
TABLE statement with identifier %s.",
+viewIdentifier.asSerializableString()));
+}
+StringBuilder stringBuilder = new StringBuilder();
+ResolvedSchema schema = view.getResolvedSchema();
+if (view.getOrigin() instanceof QueryOperationCatalogView) {
+stringBuilder.append(
+String.format(
+"CREATE %sVIEW %s (%s",
+isTemporary ? "TEMPORARY " : "",
+viewIdentifier.asSerializableString(),
+System.lineSeparator()));
+// append columns
+stringBuilder.append(
+schema.getColumns().stream()
+.map(
+column ->
+String.format(
+"%s%s", printIndent, 
getColumnString(column)))
+.collect(Collectors.joining("," + 
System.lineSeparator(;
+stringBuilder.append("\n)");
+} else {
+stringBuilder.append(
+String.format(
+"CREATE %sVIEW %s%s as%s%s",
+isTemporary ? "TEMPORARY " : "",
+viewIdentifier.asSerializableString(),
+String.format(
+"(%s)",
+schema.getColumns().stream()
+.map(Column::getName)

Review comment:
   ```suggestion
   .map(Column::getName)
   
.map(EncodingUtils::escapeIdentifier)
   ```

##
File path: 
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/catalog/CatalogViewITCase.scala
##
@@ -18,24 +18,24 @@
 
 package org.apache.flink.table.planner.catalog
 
-import org.apache.flink.table.api.config.{ExecutionConfigOptions, 
TableConfigOptions}
+import com.google.common.collect.Lists
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.internal.TableEnvironmentImpl
 import org.apache.flink.table.api.{EnvironmentSettings, TableEnvironment}
 import 
org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory
 import org.apache.flink.test.util.AbstractTestBase
 import org.apache.flink.types.Row
-
+import org.apache.flink.util.CollectionUtil
 import org.junit.Assert.assertEquals
 import org.junit.rules.ExpectedException
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
 import org.junit.{Before, Rule, Test}
 
 import java.util
-
 import scala.collection.JavaConversions._
 
-/** Test cases for view related DDLs. */
+/** Test cases for view related DDLs. TODO roc todo */

Review comment:
   TODO?

##
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
##
@@ -1441,6 +1468,59 @@ private String buildShowCreateTableRow(
 return sb.toString();
 }
 
+/** Show create view statement only for views. */
+private String buildShowCreateViewRow(
+ResolvedCatalogBaseTable view,
+ObjectIdentifier viewIdentifier,
+boolean isTemporary) {
+final String printIndent = "  ";
+CatalogBaseTable.TableKind kind = view.getTableKind();
+if (kind != CatalogBaseTable.TableKind.VIEW) {
+throw new TableException(
+String.format(
+"SHOW CREATE VIEW does not support showing CREATE 
TABLE statement with identifier %s.",

Review comment:
   I think we could make this more helpful for the user:
   
   ```suggestion
   "SHOW CREATE VIEW is only supported for views, 
but '%s' is a table. Please use SHOW CREATE TABLE instead.",
   ```

##
File path: