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<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.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"))); + } + return Optional.empty(); + } + + 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, entry.getKey(), entry.getValue())) Review comment: I don't understand what's going on here. Shouldn't this be this? ```suggestion "%s'%s' = '%s'", printIndent, EncodingUtils.escapeSingleQuotes(entry.getKey()), EncodingUtils.escapeSingleQuotes(entry.getValue()))) ``` ########## 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<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()) { Review comment: nit: we can invert this condition to exit early and reduce the indentation level: ``` if (table.getResolvedSchema().getWatermarkSpecs().isEmpty()) { return Optional.empty(); } // … ``` ########## 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<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.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"))); + } + return Optional.empty(); + } + + 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, entry.getKey(), 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)); + // append columns Review comment: nit: all of these comments are redundant and noisy now because the method names already make it clear what's happening ########## 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 { Review comment: There's no need to make this an enum, it just looks confusing. Please just turn this into a class and add a private constructor. -- 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]
