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:
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(
Review comment:
This method is largely a duplicate of `buildShowCreateTableRow`. It'd be
nice to refactor this a bit to extract some of the common logic, i.e. to format
the columns.
##########
File path:
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowCreate.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.SqlIdentifier;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+/** Abstract class for {@link SqlShowCreateTable} and {@link
SqlShowCreateView}. */
+public abstract class SqlShowCreate extends SqlCall {
+
+ protected SqlIdentifier sqlIdentifier;
Review comment:
```suggestion
protected final SqlIdentifier sqlIdentifier;
```
##########
File path:
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/catalog/CatalogViewITCase.scala
##########
@@ -307,6 +307,91 @@ class CatalogViewITCase(isStreamingMode: Boolean) extends
AbstractTestBase {
// now we only have permanent view T3
assertEquals(permanentViewData.sorted,
TestCollectionTableFactory.RESULT.sorted)
}
+
+ @Test
+ def testShowCreateView(): Unit = {
+ val sourceData = List(toRow(1, "1000", 2))
+ TestCollectionTableFactory.initData(sourceData)
Review comment:
We have new infrastructure for creating source/sink tables that don't
require these big runtime implementation sources, see `TableFactoryHarness`.
You could use that and create the table using `TableDescriptor` /
`tableEnv.createTable`.
##########
File path:
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/catalog/CatalogViewITCase.scala
##########
@@ -307,6 +307,91 @@ class CatalogViewITCase(isStreamingMode: Boolean) extends
AbstractTestBase {
// now we only have permanent view T3
assertEquals(permanentViewData.sorted,
TestCollectionTableFactory.RESULT.sorted)
}
+
+ @Test
+ def testShowCreateView(): Unit = {
+ val sourceData = List(toRow(1, "1000", 2))
+ TestCollectionTableFactory.initData(sourceData)
+ val tableDDL: String =
+ s"""CREATE TABLE T1 (
+ |a int,
+ |b varchar,
+ |c int
+ |) with (
+ |'connector' = 'COLLECTION'
+ |)""".stripMargin
+ val tView1DDL: String = "CREATE TEMPORARY VIEW t_v1 AS SELECT a, b, c FROM
T1"
+ val tView2DDL = "CREATE TEMPORARY VIEW t_v2(d, e, f) AS SELECT a, b, c
FROM T1"
+ val view1DDL = "CREATE VIEW v1 AS SELECT a, b, c FROM T1"
+ val view2DDL = "CREATE VIEW v2(x, y, z) AS SELECT a, b, c FROM T1"
+ tableEnv.executeSql(tableDDL)
+ tableEnv.executeSql(tView1DDL)
+ tableEnv.executeSql(tView2DDL)
+ tableEnv.executeSql(view1DDL)
+ tableEnv.executeSql(view2DDL)
+ testShowCreateTemporaryView(tableEnv)
+ testShowCreateView(tableEnv)
Review comment:
The split of methods here is really odd, because none of these methods
are actually parameterized. I think this would be nicer to read if we organize
the test like this:
```
// … setup code …
// Temporary views
// … code to set up first view …
// … assert first view …
// … code to set up second view …
// … assert second view …
// Catalog views
// … code to set up first view …
// … assert first view …
// … code to set up second view …
// … assert second view …
```
If anything, we could split it between temporary and catalog views (into two
tests), that'd be a logical way to split this up a bit.
##########
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)
+ .collect(Collectors.joining(",
"))),
+ System.lineSeparator(),
+ ((CatalogView)
view.getOrigin()).getOriginalQuery()));
+ }
+ // append comment
+ String comment = view.getComment();
+ if (StringUtils.isNotEmpty(comment)) {
+ stringBuilder.append(String.format(" COMMENT '%s'", comment));
Review comment:
```suggestion
stringBuilder.append(String.format(" COMMENT '%s'",
EncodingUtils.escapeSingleQuotes(comment)));
```
##########
File path:
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
##########
@@ -1176,6 +1178,31 @@ public TableResult executeInternal(Operation operation) {
.getTableIdentifier()
.asSerializableString()));
}
+ } else if (operation instanceof ShowCreateViewOperation) {
+ ShowCreateViewOperation showCreateViewOperation =
(ShowCreateViewOperation) operation;
+ Optional<CatalogManager.TableLookupResult> result =
+
catalogManager.getTable(showCreateViewOperation.getViewIdentifier());
+ if (result.isPresent()) {
Review comment:
We can move this check into the Optional, which will save us from one
level of indentation:
```
final CatalogManager.TableLookupResult result = catalogManager
.getTable(showCreateViewOperation.getViewIdentifier())
.orElseThrow(() -> new ValidationException(…));
return TableResultImpl.builder()…
```
--
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]