You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/10/19 07:00:24 UTC

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

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



##########
File path: docs/content/docs/dev/table/sql/show.md
##########
@@ -341,6 +354,11 @@ table_env.execute_sql("SHOW VIEWS").print()
 # |   my_view |
 # +-----------+
 
+# show create view
+table_env.execute_sql("SHOW CREATE VIEW my_view").print()
+# CREATE VIEW `default_catalog`.`default_db`.`my_view` as
+# ...

Review comment:
       We can update the `my_view` to have a meaningful definition instead of a `...`.

##########
File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/ShowCreateUtil.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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
+class ShowCreateUtil {
+
+    private 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<String> extractFormattedPrimaryKey(
+            ResolvedCatalogBaseTable<?> table, String printIndent) {
+        Optional<UniqueConstraint> 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<String> extractFormattedWatermarkSpecs(
+            ResolvedCatalogBaseTable<?> table, String printIndent) {
+        if (table.getResolvedSchema().getWatermarkSpecs().isEmpty()) {
+            return Optional.empty();
+        }
+        return Optional.of(
+                table.getResolvedSchema().getWatermarkSpecs().stream()
+                        .map(
+                                watermarkSpec ->
+                                        String.format(
+                                                "%sWATERMARK FOR %s AS %s",
+                                                printIndent,
+                                                EncodingUtils.escapeIdentifier(
+                                                        watermarkSpec.getRowtimeAttribute()),
+                                                watermarkSpec
+                                                        .getWatermarkExpression()
+                                                        .asSerializableString()))
+                        .collect(Collectors.joining("\n")));
+    }
+
+    static Optional<String> extractFormattedComment(ResolvedCatalogBaseTable<?> table) {
+        String comment = table.getComment();
+        if (StringUtils.isNotEmpty(comment)) {
+            return Optional.of(EncodingUtils.escapeSingleQuotes(comment));
+        }
+        return Optional.empty();
+    }
+
+    static Optional<String> extractFormattedPartitionedInfo(ResolvedCatalogTable catalogTable) {
+        if (!catalogTable.isPartitioned()) {
+            return Optional.empty();
+        }
+        return Optional.of(
+                catalogTable.getPartitionKeys().stream()
+                        .map(EncodingUtils::escapeIdentifier)
+                        .collect(Collectors.joining(", ")));
+    }
+
+    static Optional<String> 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,
+                                                EncodingUtils.escapeSingleQuotes(entry.getKey()),
+                                                EncodingUtils.escapeSingleQuotes(entry.getValue())))
+                        .collect(Collectors.joining(",\n")));
+    }
+
+    static String extractFormattedColumnNames(ResolvedCatalogBaseTable<?> baseTable) {
+        return baseTable.getResolvedSchema().getColumns().stream()
+                .map(Column::getName)
+                .map(EncodingUtils::escapeIdentifier)
+                .collect(Collectors.joining(", "));
+    }
+
+    static String buildShowCreateTableRow(
+            ResolvedCatalogBaseTable<?> table,
+            ObjectIdentifier tableIdentifier,
+            boolean isTemporary) {
+        if (table.getTableKind() == CatalogBaseTable.TableKind.VIEW) {
+            throw new TableException(
+                    String.format(
+                            "SHOW CREATE TABLE is only supported for tables, but %s is a view. Please use SHOW CREATE VIEW instead.",
+                            tableIdentifier.asSerializableString()));
+        }
+        final String printIndent = "  ";
+        StringBuilder sb =
+                new StringBuilder()
+                        .append(buildCreateFormattedPrefix("TABLE", isTemporary, tableIdentifier));
+        sb.append(extractFormattedColumns(table, printIndent));
+        extractFormattedWatermarkSpecs(table, printIndent)
+                .ifPresent(watermarkSpecs -> sb.append(",\n").append(watermarkSpecs));
+        extractFormattedPrimaryKey(table, printIndent).ifPresent(pk -> sb.append(",\n").append(pk));
+        sb.append("\n) ");
+        extractFormattedComment(table)
+                .ifPresent(c -> sb.append(String.format("COMMENT '%s'\n", c)));
+        extractFormattedPartitionedInfo((ResolvedCatalogTable) table)
+                .ifPresent(
+                        partitionedInfoFormatted ->
+                                sb.append("PARTITIONED BY (")
+                                        .append(partitionedInfoFormatted)
+                                        .append(")\n"));
+        extractFormattedOptions(table, printIndent)
+                .ifPresent(v -> sb.append("WITH (\n").append(v).append("\n)\n"));
+        return sb.toString();
+    }
+
+    /** Show create view statement only for views. */
+    static String buildShowCreateViewRow(
+            ResolvedCatalogBaseTable<?> view,
+            ObjectIdentifier viewIdentifier,
+            boolean isTemporary) {
+        if (view.getTableKind() != CatalogBaseTable.TableKind.VIEW) {
+            throw new TableException(
+                    String.format(
+                            "SHOW CREATE VIEW is only supported for views, but %s is a table. Please use SHOW CREATE TABLE instead.",
+                            viewIdentifier.asSerializableString()));
+        }
+        final String printIndent = "  ";
+        StringBuilder stringBuilder = new StringBuilder();
+        if (view.getOrigin() instanceof QueryOperationCatalogView) {

Review comment:
       We can throw exception here, it's confusing to print show create view without the query definition. We can say we don't support show create view for views registered by Table API, ie. `TableEnvironment#createTemporaryView(String, Table)`.
   

##########
File path: flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/catalog/CatalogViewITCase.scala
##########
@@ -307,6 +308,95 @@ class CatalogViewITCase(isStreamingMode: Boolean) extends AbstractTestBase {
     // now we only have permanent view T3
     assertEquals(permanentViewData.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
+
+  private def buildTableDescriptor(): TableDescriptor = {
+    val tableDescriptor: TableDescriptor = TableFactoryHarness.newBuilder()
+      .boundedScanSource()
+      .schema(Schema.newBuilder()
+        .column("a", DataTypes.INT())
+        .column("b", DataTypes.STRING())
+        .column("c", DataTypes.INT())
+        .build())
+      .sink()
+      .build()
+    tableDescriptor
+  }
+
+  @Test
+  def testShowCreateTemporaryView(): Unit = {
+
+    tableEnv.createTable("T1", buildTableDescriptor())
+
+    val tView1DDL: String = "CREATE TEMPORARY VIEW t_v1 AS SELECT a, b, c FROM T1"
+    tableEnv.executeSql(tView1DDL)
+    val tView1ShowCreateResult: util.List[Row] = CollectionUtil.iteratorToList(
+      tableEnv
+        .executeSql("show create view t_v1")
+        .collect()
+    )
+    assertEquals(tView1ShowCreateResult, Lists.newArrayList(
+      Row.of(
+        s"""CREATE TEMPORARY VIEW `default_catalog`.`default_database`.`t_v1`(`a`, `b`, `c`) as
+           |SELECT `a`, `b`, `c`
+           |FROM `T1`"""
+          .stripMargin
+      )
+    ))
+
+    val tView2DDL: String = "CREATE TEMPORARY VIEW t_v2(d, e, f) AS SELECT a, b, c FROM T1"
+    tableEnv.executeSql(tView2DDL)
+    val tView2ShowCreateResult: util.List[Row] = CollectionUtil.iteratorToList(
+      tableEnv
+        .executeSql("show create view t_v2")
+        .collect()
+    )
+    assertEquals(tView2ShowCreateResult, Lists.newArrayList(
+      Row.of(
+        s"""CREATE TEMPORARY VIEW `default_catalog`.`default_database`.`t_v2`(`d`, `e`, `f`) as
+           |SELECT `a`, `b`, `c`
+           |FROM `T1`"""

Review comment:
       Could you add some more complex queries which includes UDF, joins, group by?

##########
File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/ShowCreateUtil.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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
+class ShowCreateUtil {
+
+    private 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<String> extractFormattedPrimaryKey(
+            ResolvedCatalogBaseTable<?> table, String printIndent) {
+        Optional<UniqueConstraint> 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<String> extractFormattedWatermarkSpecs(
+            ResolvedCatalogBaseTable<?> table, String printIndent) {
+        if (table.getResolvedSchema().getWatermarkSpecs().isEmpty()) {
+            return Optional.empty();
+        }
+        return Optional.of(
+                table.getResolvedSchema().getWatermarkSpecs().stream()
+                        .map(
+                                watermarkSpec ->
+                                        String.format(
+                                                "%sWATERMARK FOR %s AS %s",
+                                                printIndent,
+                                                EncodingUtils.escapeIdentifier(
+                                                        watermarkSpec.getRowtimeAttribute()),
+                                                watermarkSpec
+                                                        .getWatermarkExpression()
+                                                        .asSerializableString()))
+                        .collect(Collectors.joining("\n")));
+    }
+
+    static Optional<String> extractFormattedComment(ResolvedCatalogBaseTable<?> table) {
+        String comment = table.getComment();
+        if (StringUtils.isNotEmpty(comment)) {
+            return Optional.of(EncodingUtils.escapeSingleQuotes(comment));
+        }
+        return Optional.empty();
+    }
+
+    static Optional<String> extractFormattedPartitionedInfo(ResolvedCatalogTable catalogTable) {
+        if (!catalogTable.isPartitioned()) {
+            return Optional.empty();
+        }
+        return Optional.of(
+                catalogTable.getPartitionKeys().stream()
+                        .map(EncodingUtils::escapeIdentifier)
+                        .collect(Collectors.joining(", ")));
+    }
+
+    static Optional<String> 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,
+                                                EncodingUtils.escapeSingleQuotes(entry.getKey()),
+                                                EncodingUtils.escapeSingleQuotes(entry.getValue())))
+                        .collect(Collectors.joining(",\n")));
+    }
+
+    static String extractFormattedColumnNames(ResolvedCatalogBaseTable<?> baseTable) {
+        return baseTable.getResolvedSchema().getColumns().stream()
+                .map(Column::getName)
+                .map(EncodingUtils::escapeIdentifier)
+                .collect(Collectors.joining(", "));
+    }
+
+    static String buildShowCreateTableRow(
+            ResolvedCatalogBaseTable<?> table,
+            ObjectIdentifier tableIdentifier,
+            boolean isTemporary) {
+        if (table.getTableKind() == CatalogBaseTable.TableKind.VIEW) {
+            throw new TableException(
+                    String.format(
+                            "SHOW CREATE TABLE is only supported for tables, but %s is a view. Please use SHOW CREATE VIEW instead.",
+                            tableIdentifier.asSerializableString()));
+        }
+        final String printIndent = "  ";
+        StringBuilder sb =
+                new StringBuilder()
+                        .append(buildCreateFormattedPrefix("TABLE", isTemporary, tableIdentifier));
+        sb.append(extractFormattedColumns(table, printIndent));
+        extractFormattedWatermarkSpecs(table, printIndent)
+                .ifPresent(watermarkSpecs -> sb.append(",\n").append(watermarkSpecs));
+        extractFormattedPrimaryKey(table, printIndent).ifPresent(pk -> sb.append(",\n").append(pk));
+        sb.append("\n) ");
+        extractFormattedComment(table)
+                .ifPresent(c -> sb.append(String.format("COMMENT '%s'\n", c)));
+        extractFormattedPartitionedInfo((ResolvedCatalogTable) table)
+                .ifPresent(
+                        partitionedInfoFormatted ->
+                                sb.append("PARTITIONED BY (")
+                                        .append(partitionedInfoFormatted)
+                                        .append(")\n"));
+        extractFormattedOptions(table, printIndent)
+                .ifPresent(v -> sb.append("WITH (\n").append(v).append("\n)\n"));
+        return sb.toString();
+    }
+
+    /** Show create view statement only for views. */
+    static String buildShowCreateViewRow(
+            ResolvedCatalogBaseTable<?> view,
+            ObjectIdentifier viewIdentifier,
+            boolean isTemporary) {
+        if (view.getTableKind() != CatalogBaseTable.TableKind.VIEW) {
+            throw new TableException(
+                    String.format(
+                            "SHOW CREATE VIEW is only supported for views, but %s is a table. Please use SHOW CREATE TABLE instead.",
+                            viewIdentifier.asSerializableString()));
+        }
+        final String printIndent = "  ";
+        StringBuilder stringBuilder = new StringBuilder();
+        if (view.getOrigin() instanceof QueryOperationCatalogView) {
+            stringBuilder.append(buildCreateFormattedPrefix("VIEW", isTemporary, viewIdentifier));
+            stringBuilder.append(extractFormattedColumns(view, printIndent)).append("\n)");
+        } else {
+            stringBuilder.append(
+                    String.format(
+                            "CREATE %sVIEW %s%s as%s%s",
+                            isTemporary ? "TEMPORARY " : "",
+                            viewIdentifier.asSerializableString(),
+                            String.format("(%s)", extractFormattedColumnNames(view)),
+                            System.lineSeparator(),
+                            ((CatalogView) view.getOrigin()).getOriginalQuery()));

Review comment:
       I think we may need to use `#getExpandedQuery()` here which will expand identifiers. 




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org