From 01263b82205acb238b1a7a54be9f7028cf947810 Mon Sep 17 00:00:00 2001 From: Jim Hughes Date: Fri, 19 Jan 2024 13:22:34 -0500 Subject: [PATCH] [FLINK-33495][FLINK-33496] Add DISTRIBUTED BY clause for CREATE TABLE - Adds distribution support to CatalogTable - Adds connector ability SupportsBucketing - Adds distribution to TableDescriptor. This closes #24155. --- .../src/main/codegen/data/Parser.tdd | 5 + .../src/main/codegen/includes/parserImpls.ftl | 66 +++++ .../flink/sql/parser/ddl/SqlCreateTable.java | 14 ++ .../sql/parser/ddl/SqlCreateTableAs.java | 7 + .../sql/parser/ddl/SqlCreateTableLike.java | 2 + .../flink/sql/parser/ddl/SqlDistribution.java | 110 ++++++++ .../sql/parser/ddl/SqlReplaceTableAs.java | 13 + .../flink/sql/parser/ddl/SqlTableLike.java | 5 + .../sql/parser/utils/ParserResource.java | 3 + .../flink/sql/parser/CreateTableLikeTest.java | 23 ++ .../sql/parser/FlinkSqlParserImplTest.java | 234 +++++++++++++++++- .../flink/table/api/TableDescriptor.java | 109 +++++++- .../table/api/internal/ShowCreateUtil.java | 6 + .../flink/table/catalog/CatalogManager.java | 36 ++- .../flink/table/api/TableDescriptorTest.java | 70 +++++- .../CatalogBaseTableResolutionTest.java | 66 +++++ .../table/test/program/SinkTestStep.java | 5 +- .../table/test/program/SourceTestStep.java | 7 +- .../table/test/program/TableTestStep.java | 20 +- .../flink/table/catalog/CatalogTable.java | 69 +++++- .../table/catalog/DefaultCatalogTable.java | 36 ++- .../table/catalog/ResolvedCatalogTable.java | 5 + .../table/catalog/TableDistribution.java | 145 +++++++++++ .../connector/sink/DynamicTableSink.java | 2 + .../sink/abilities/SupportsBucketing.java | 96 +++++++ .../sink/abilities/SupportsPartitioning.java | 9 + .../planner/connectors/DynamicSinkUtils.java | 58 ++++- .../operations/MergeTableLikeUtil.java | 28 +++ .../operations/SqlCreateTableConverter.java | 79 +++++- .../plan/abilities/sink/BucketingSpec.java | 57 +++++ .../plan/abilities/sink/SinkAbilitySpec.java | 1 + .../plan/nodes/exec/serde/JsonSerdeUtil.java | 3 + .../ResolvedCatalogTableJsonDeserializer.java | 28 ++- .../ResolvedCatalogTableJsonSerializer.java | 4 + .../TableDistributionJsonDeserializer.java | 80 ++++++ .../TableDistributionJsonSerializer.java | 65 +++++ .../factories/TableFactoryHarness.java | 1 + .../factories/TestValuesTableFactory.java | 36 ++- .../operations/MergeTableLikeUtilTest.java | 61 +++++ .../SqlDdlToOperationConverterTest.java | 154 ++++++++++++ .../serde/ContextResolvedTableSerdeTest.java | 77 ++++-- .../serde/ResolvedCatalogTableSerdeTest.java | 57 +++-- .../exec/stream/TableSinkRestoreTest.java | 4 + .../exec/stream/TableSinkTestPrograms.java | 38 +++ .../planner/utils/OperationMatchers.java | 29 +++ .../planner/plan/batch/sql/TableSinkTest.xml | 19 ++ .../planner/plan/stream/sql/TableSinkTest.xml | 28 +++ ...k-bucketing_hash-with-keys-with-count.json | 88 +++++++ .../savepoint/_metadata | Bin 0 -> 8379 bytes ...cketing_range_with_keys_without_count.json | 87 +++++++ .../savepoint/_metadata | Bin 0 -> 8379 bytes .../plan/sink-bucketing_with-count.json | 88 +++++++ .../savepoint/_metadata | Bin 0 -> 8379 bytes .../sink-bucketing_with-keys-and-count.json | 88 +++++++ .../savepoint/_metadata | Bin 0 -> 8379 bytes .../planner/catalog/CatalogTableITCase.scala | 39 +++ .../plan/batch/sql/TableSinkTest.scala | 17 ++ .../plan/stream/sql/TableSinkTest.scala | 59 +++++ 58 files changed, 2431 insertions(+), 105 deletions(-) create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDistribution.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/TableDistribution.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsBucketing.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/sink/BucketingSpec.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/TableDistributionJsonDeserializer.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/TableDistributionJsonSerializer.java create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_hash-with-keys-with-count/plan/sink-bucketing_hash-with-keys-with-count.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_hash-with-keys-with-count/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_range_with_keys_without_count/plan/sink-bucketing_range_with_keys_without_count.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_range_with_keys_without_count/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_with-count/plan/sink-bucketing_with-count.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_with-count/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_with-keys-and-count/plan/sink-bucketing_with-keys-and-count.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_with-keys-and-count/savepoint/_metadata diff --git a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd index 9153be2a08b00..8c39f41b4631c 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd +++ b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd @@ -61,6 +61,7 @@ "org.apache.flink.sql.parser.ddl.SqlCreateTableAs" "org.apache.flink.sql.parser.ddl.SqlCreateTableLike" "org.apache.flink.sql.parser.ddl.SqlCreateView" + "org.apache.flink.sql.parser.ddl.SqlDistribution" "org.apache.flink.sql.parser.ddl.SqlDropCatalog" "org.apache.flink.sql.parser.ddl.SqlDropDatabase" "org.apache.flink.sql.parser.ddl.SqlDropFunction" @@ -139,6 +140,7 @@ # keyword, please also add it to 'nonReservedKeywords' section. # Please keep the keyword in alphabetical order if new keyword is added. keywords: [ + "BUCKETS" "BYTES" "CATALOGS" "CHANGELOG_MODE" @@ -147,11 +149,14 @@ "COMPILE" "COLUMNS" "DATABASES" + "DISTRIBUTED" + "DISTRIBUTION" "DRAIN" "ENFORCED" "ESTIMATED_COST" "EXTENDED" "FUNCTIONS" + "HASH" "IF" "JSON_EXECUTION_PLAN" "PLAN_ADVICE" diff --git a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl index dfc7edaf9dfcf..58d1a2c6c390b 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl +++ b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl @@ -1349,6 +1349,48 @@ SqlNodeList TableProperties(): { return new SqlNodeList(proList, span.end(this)); } } +SqlNumericLiteral IntoBuckets(SqlParserPos startPos) : +{ + SqlNumericLiteral bucketCount; +} +{ + { bucketCount = UnsignedNumericLiteral(); + if (!bucketCount.isInteger()) { + throw SqlUtil.newContextException(getPos(), + ParserResource.RESOURCE.bucketCountMustBePositiveInteger()); + } + } + { + return bucketCount; + } +} + +SqlDistribution SqlDistribution(SqlParserPos startPos) : +{ + String distributionKind = null; + SqlNumericLiteral bucketCount = null; + SqlNodeList bucketColumns = SqlNodeList.EMPTY; + SqlDistribution distribution = null; +} +{ + ( + bucketCount = IntoBuckets(getPos()) + | + ( + ( + { distributionKind = "HASH"; } + | { distributionKind = "RANGE"; } + | { distributionKind = null; } + ) + { bucketColumns = ParenthesizedSimpleIdentifierList(); } + [ bucketCount = IntoBuckets(getPos()) ] + ) + ) + { + return new SqlDistribution(startPos, distributionKind, bucketColumns, bucketCount); + } +} + SqlCreate SqlCreateTable(Span s, boolean replace, boolean isTemporary) : { final SqlParserPos startPos = s.pos(); @@ -1362,6 +1404,10 @@ SqlCreate SqlCreateTable(Span s, boolean replace, boolean isTemporary) : SqlNode asQuery = null; SqlNodeList propertyList = SqlNodeList.EMPTY; + String distributionKind = null; + SqlNumericLiteral bucketCount = null; + SqlNodeList bucketColumns = SqlNodeList.EMPTY; + SqlDistribution distribution = null; SqlNodeList partitionColumns = SqlNodeList.EMPTY; SqlParserPos pos = startPos; } @@ -1389,6 +1435,11 @@ SqlCreate SqlCreateTable(Span s, boolean replace, boolean isTemporary) : String p = SqlParserUtil.parseString(token.image); comment = SqlLiteral.createCharString(p, getPos()); }] + [ + + distribution = SqlDistribution(getPos()) + ] + [ partitionColumns = ParenthesizedSimpleIdentifierList() @@ -1406,6 +1457,7 @@ SqlCreate SqlCreateTable(Span s, boolean replace, boolean isTemporary) : columnList, constraints, propertyList, + distribution, partitionColumns, watermark, comment, @@ -1423,6 +1475,7 @@ SqlCreate SqlCreateTable(Span s, boolean replace, boolean isTemporary) : columnList, constraints, propertyList, + distribution, partitionColumns, watermark, comment, @@ -1436,6 +1489,7 @@ SqlCreate SqlCreateTable(Span s, boolean replace, boolean isTemporary) : columnList, constraints, propertyList, + distribution, partitionColumns, watermark, comment, @@ -1451,6 +1505,7 @@ SqlCreate SqlCreateTable(Span s, boolean replace, boolean isTemporary) : columnList, constraints, propertyList, + distribution, partitionColumns, watermark, comment, @@ -1503,6 +1558,8 @@ SqlTableLikeOption SqlTableLikeOption(): { featureOption = FeatureOption.ALL;} | { featureOption = FeatureOption.CONSTRAINTS;} + | + { featureOption = FeatureOption.DISTRIBUTION;} | { featureOption = FeatureOption.GENERATED;} | @@ -1551,6 +1608,10 @@ SqlNode SqlReplaceTable() : List constraints = new ArrayList(); SqlWatermark watermark = null; SqlNodeList columnList = SqlNodeList.EMPTY; + String distributionKind = null; + SqlNumericLiteral bucketCount = null; + SqlNodeList bucketColumns = SqlNodeList.EMPTY; + SqlDistribution distribution = null; SqlNodeList partitionColumns = SqlNodeList.EMPTY; boolean ifNotExists = false; } @@ -1582,6 +1643,10 @@ SqlNode SqlReplaceTable() : String p = SqlParserUtil.parseString(token.image); comment = SqlLiteral.createCharString(p, getPos()); }] + [ + + distribution = SqlDistribution(getPos()) + ] [ partitionColumns = ParenthesizedSimpleIdentifierList() @@ -1598,6 +1663,7 @@ SqlNode SqlReplaceTable() : columnList, constraints, propertyList, + distribution, partitionColumns, watermark, comment, diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java index 4de2fa30a163d..da26891fcb8b8 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java @@ -63,6 +63,12 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { private final List tableConstraints; + public SqlDistribution getDistribution() { + return distribution; + } + + private final SqlDistribution distribution; + private final SqlNodeList partitionKeyList; private final SqlWatermark watermark; @@ -77,6 +83,7 @@ public SqlCreateTable( SqlNodeList columnList, List tableConstraints, SqlNodeList propertyList, + SqlDistribution distribution, SqlNodeList partitionKeyList, @Nullable SqlWatermark watermark, @Nullable SqlCharStringLiteral comment, @@ -89,6 +96,7 @@ public SqlCreateTable( columnList, tableConstraints, propertyList, + distribution, partitionKeyList, watermark, comment, @@ -103,6 +111,7 @@ protected SqlCreateTable( SqlNodeList columnList, List tableConstraints, SqlNodeList propertyList, + @Nullable SqlDistribution distribution, SqlNodeList partitionKeyList, @Nullable SqlWatermark watermark, @Nullable SqlCharStringLiteral comment, @@ -114,6 +123,7 @@ protected SqlCreateTable( this.tableConstraints = requireNonNull(tableConstraints, "table constraints should not be null"); this.propertyList = requireNonNull(propertyList, "propertyList should not be null"); + this.distribution = distribution; this.partitionKeyList = requireNonNull(partitionKeyList, "partitionKeyList should not be null"); this.watermark = watermark; @@ -256,6 +266,10 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { comment.unparse(writer, leftPrec, rightPrec); } + if (this.distribution != null) { + distribution.unparse(writer, leftPrec, rightPrec); + } + if (this.partitionKeyList.size() > 0) { writer.newlineAndIndent(); writer.keyword("PARTITIONED BY"); diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTableAs.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTableAs.java index 986d8dadab44b..ea66773a150a0 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTableAs.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTableAs.java @@ -79,6 +79,7 @@ public SqlCreateTableAs( SqlNodeList columnList, List tableConstraints, SqlNodeList propertyList, + SqlDistribution distribution, SqlNodeList partitionKeyList, @Nullable SqlWatermark watermark, @Nullable SqlCharStringLiteral comment, @@ -92,6 +93,7 @@ public SqlCreateTableAs( columnList, tableConstraints, propertyList, + distribution, partitionKeyList, watermark, comment, @@ -134,6 +136,11 @@ public void validate() throws SqlValidateException { getParserPosition(), "CREATE TABLE AS SELECT syntax does not support to specify explicit watermark yet."); } + if (getDistribution() != null) { + throw new SqlValidateException( + getParserPosition(), + "CREATE TABLE AS SELECT syntax does not support creating distributed tables yet."); + } // TODO flink dialect supports dynamic partition if (getPartitionKeyList().size() > 0) { throw new SqlValidateException( diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTableLike.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTableLike.java index 879701135e2c0..599a93829d473 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTableLike.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTableLike.java @@ -81,6 +81,7 @@ public SqlCreateTableLike( SqlNodeList columnList, List tableConstraints, SqlNodeList propertyList, + SqlDistribution distribution, SqlNodeList partitionKeyList, @Nullable SqlWatermark watermark, @Nullable SqlCharStringLiteral comment, @@ -94,6 +95,7 @@ public SqlCreateTableLike( columnList, tableConstraints, propertyList, + distribution, partitionKeyList, watermark, comment, diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDistribution.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDistribution.java new file mode 100644 index 0000000000000..fa026e6f001ee --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDistribution.java @@ -0,0 +1,110 @@ +/* + * 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.ddl; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlNumericLiteral; +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 org.apache.calcite.util.ImmutableNullableList; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Optional; + +/** + * Distribution statement in CREATE TABLE DDL, e.g. {@code DISTRIBUTED BY HASH(column1, column2) + * INTO BUCKETS 10}. + */ +public class SqlDistribution extends SqlCall { + + private static final SqlSpecialOperator OPERATOR = + new SqlSpecialOperator("DISTRIBUTED BY", SqlKind.OTHER); + + private final String distributionKind; + private final SqlNodeList bucketColumns; + private final SqlNumericLiteral bucketCount; + + public SqlDistribution( + SqlParserPos pos, + @Nullable String distributionKind, + @Nullable SqlNodeList bucketColumns, + @Nullable SqlNumericLiteral bucketCount) { + super(pos); + this.distributionKind = distributionKind; + this.bucketColumns = bucketColumns; + this.bucketCount = bucketCount; + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public List getOperandList() { + return ImmutableNullableList.of(bucketCount, bucketColumns); + } + + @Override + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { + writer.newlineAndIndent(); + + if (bucketColumns.size() == 0 && bucketCount != null) { + writer.keyword("DISTRIBUTED INTO"); + bucketCount.unparse(writer, leftPrec, rightPrec); + writer.keyword("BUCKETS"); + writer.newlineAndIndent(); + return; + } + + writer.keyword("DISTRIBUTED BY"); + if (distributionKind != null) { + writer.print(distributionKind); + } + SqlWriter.Frame bucketFrame = writer.startList("(", ")"); + bucketColumns.unparse(writer, leftPrec, rightPrec); + writer.endList(bucketFrame); + + if (bucketCount != null) { + writer.keyword("INTO"); + bucketCount.unparse(writer, leftPrec, rightPrec); + writer.keyword("BUCKETS"); + } + writer.newlineAndIndent(); + } + + public Optional getDistributionKind() { + return Optional.ofNullable(distributionKind); + } + + public SqlNumericLiteral getBucketCount() { + return bucketCount; + } + + public SqlNodeList getBucketColumns() { + return bucketColumns; + } +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlReplaceTableAs.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlReplaceTableAs.java index 487aeeed80b5e..5d343e8c2c6e9 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlReplaceTableAs.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlReplaceTableAs.java @@ -89,6 +89,12 @@ public class SqlReplaceTableAs extends SqlCreate implements ExtendedSqlNode { private final List tableConstraints; + public SqlDistribution getDistribution() { + return distribution; + } + + private final SqlDistribution distribution; + private final SqlNodeList partitionKeyList; private final SqlWatermark watermark; @@ -107,6 +113,7 @@ public SqlReplaceTableAs( SqlNodeList columnList, List tableConstraints, SqlNodeList propertyList, + SqlDistribution distribution, SqlNodeList partitionKeyList, @Nullable SqlWatermark watermark, @Nullable SqlCharStringLiteral comment, @@ -125,6 +132,7 @@ public SqlReplaceTableAs( this.tableConstraints = requireNonNull(tableConstraints, "table constraints should not be null"); this.propertyList = requireNonNull(propertyList, "propertyList should not be null"); + this.distribution = distribution; this.partitionKeyList = requireNonNull(partitionKeyList, "partitionKeyList should not be null"); this.watermark = watermark; @@ -179,6 +187,11 @@ public void validate() throws SqlValidateException { getParserPosition(), errorMsg + " syntax does not support to specify explicit watermark yet."); } + if (getDistribution() != null) { + throw new SqlValidateException( + getParserPosition(), + errorMsg + " syntax does not support creating distributed tables yet."); + } if (getPartitionKeyList().size() > 0) { throw new SqlValidateException( getParserPosition(), diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlTableLike.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlTableLike.java index eb1651cc2e033..15f883a05e9e9 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlTableLike.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlTableLike.java @@ -115,6 +115,7 @@ public enum MergingStrategy { *
    *
  • ALL - a shortcut to change the default merging strategy if none provided *
  • CONSTRAINTS - constraints such as primary and unique keys + *
  • DISTRIBUTION - distribution of the table *
  • GENERATED - computed columns *
  • METADATA - metadata columns *
  • WATERMARKS - watermark declarations @@ -138,6 +139,7 @@ public enum MergingStrategy { * LIKE `sourceTable` ( * INCLUDING GENERATED * INCLUDING CONSTRAINTS + * INCLUDING DISTRIBUTION * OVERWRITING OPTIONS * EXCLUDING PARTITIONS * ) @@ -146,6 +148,7 @@ public enum MergingStrategy { public enum FeatureOption { ALL, CONSTRAINTS, + DISTRIBUTION, GENERATED, METADATA, OPTIONS, @@ -191,6 +194,8 @@ public List getOptions() { static { invalidCombinations.put(FeatureOption.ALL, singletonList(MergingStrategy.OVERWRITING)); + invalidCombinations.put( + FeatureOption.DISTRIBUTION, singletonList(MergingStrategy.OVERWRITING)); invalidCombinations.put( FeatureOption.PARTITIONS, singletonList(MergingStrategy.OVERWRITING)); invalidCombinations.put( diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/utils/ParserResource.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/utils/ParserResource.java index 8a63777f543e8..0758b0d5887f6 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/utils/ParserResource.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/utils/ParserResource.java @@ -46,4 +46,7 @@ public interface ParserResource { @Resources.BaseMessage("WITH DRAIN could only be used after WITH SAVEPOINT.") Resources.ExInst withDrainOnlyUsedWithSavepoint(); + + @Resources.BaseMessage("Bucket count must be a positive integer.") + Resources.ExInst bucketCountMustBePositiveInteger(); } diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/CreateTableLikeTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/CreateTableLikeTest.java index 877ebafcd97c7..6004f5f9a6880 100644 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/CreateTableLikeTest.java +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/CreateTableLikeTest.java @@ -67,6 +67,7 @@ void testCreateTableLike() throws Exception { + "LIKE b (\n" + " EXCLUDING PARTITIONS\n" + " EXCLUDING CONSTRAINTS\n" + + " EXCLUDING DISTRIBUTION\n" + " EXCLUDING WATERMARKS\n" + " OVERWRITING GENERATED\n" + " OVERWRITING OPTIONS\n" @@ -86,6 +87,9 @@ void testCreateTableLike() throws Exception { option( MergingStrategy.EXCLUDING, FeatureOption.CONSTRAINTS), + option( + MergingStrategy.EXCLUDING, + FeatureOption.DISTRIBUTION), option( MergingStrategy.EXCLUDING, FeatureOption.WATERMARKS), @@ -152,6 +156,25 @@ void testInvalidOverwritingForAll() throws Exception { .hasMessage("Illegal merging strategy 'OVERWRITING' for 'ALL' option."); } + @Test + void testInvalidOverwritingForDistribution() throws Exception { + ExtendedSqlNode extendedSqlNode = + (ExtendedSqlNode) + createFlinkParser( + "CREATE TABLE t (\n" + + " a STRING\n" + + ")\n" + + "LIKE b (\n" + + " OVERWRITING DISTRIBUTION" + + ")") + .parseStmt(); + + assertThatThrownBy(extendedSqlNode::validate) + .isInstanceOf(SqlValidateException.class) + .hasMessageContaining( + "Illegal merging strategy 'OVERWRITING' for 'DISTRIBUTION' option."); + } + @Test void testInvalidOverwritingForConstraints() throws Exception { ExtendedSqlNode extendedSqlNode = diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java index ef1750a89fb9c..d12b9b0d3faed 100644 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java @@ -911,16 +911,154 @@ void testCreateTable() { sql(sql).ok(expected); } + String buildDistributionInput(final String distributionClause) { + return "CREATE TABLE tbl1 (\n" + + " a bigint,\n" + + " h varchar, \n" + + " b varchar,\n" + + " PRIMARY KEY (a, b)\n" + + ")\n" + + distributionClause + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + } + + String buildDistributionOutput(final String distributionClause) { + return "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT,\n" + + " `H` VARCHAR,\n" + + " `B` VARCHAR,\n" + + " PRIMARY KEY (`A`, `B`)\n" + + ")\n" + + distributionClause + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + } + + @Test + void testCreateTableWithDistribution() { + final String sql = buildDistributionInput("DISTRIBUTED BY HASH(a, h) INTO 6 BUCKETS\n"); + final String expected = + buildDistributionOutput("DISTRIBUTED BY HASH(`A`, `H`) INTO 6 BUCKETS\n"); + sql(sql).ok(expected); + } + @Test - void testCreateTableIfNotExists() { + void testCreateTableWithRangeDistribution() { + final String sql = buildDistributionInput("DISTRIBUTED BY RANGE(a, h) INTO 6 BUCKETS\n"); + final String expected = + buildDistributionOutput("DISTRIBUTED BY RANGE(`A`, `H`) INTO 6 BUCKETS\n"); + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithRandomDistribution() { + final String sql = buildDistributionInput("DISTRIBUTED BY ^RANDOM^(a, h) INTO 6 BUCKETS\n"); + sql(sql).fails("(?s).*Encountered \"RANDOM\" at line 7, column 16.*"); + } + + @Test + void testCreateTableWithDistributionNoAlgorithm() { + final String sql = buildDistributionInput("DISTRIBUTED BY (a, h) INTO 6 BUCKETS\n"); + final String expected = + buildDistributionOutput("DISTRIBUTED BY (`A`, `H`) INTO 6 BUCKETS\n"); + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithDistributionAlgorithmWithoutBuckets() { + final String sql = buildDistributionInput("DISTRIBUTED BY RANGE(a, h)\n"); + final String expected = buildDistributionOutput("DISTRIBUTED BY RANGE(`A`, `H`)\n"); + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithDistributionNoAlgorithmWithoutBuckets() { + final String sql = buildDistributionInput("DISTRIBUTED BY (a, h)\n"); + final String expected = buildDistributionOutput("DISTRIBUTED BY (`A`, `H`)\n"); + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithDistributionIntoBuckets() { + final String sql = buildDistributionInput("DISTRIBUTED INTO 3 BUCKETS\n"); + final String expected = buildDistributionOutput("DISTRIBUTED INTO 3 BUCKETS\n"); + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithDistributionIntoNegativeBuckets() { + final String sql = buildDistributionInput("DISTRIBUTED INTO ^-^3 BUCKETS\n"); + sql(sql).fails("(?s).*Encountered \"-\" at line 7, column 18.*"); + } + + @Test + void testCreateTableWithDistributionIntoDecimalBuckets() { + final String sql = buildDistributionInput("DISTRIBUTED INTO ^3.2^ BUCKETS\n"); + sql(sql).fails("(?s).*Bucket count must be a positive integer.*"); + } + + @Test + void testCreateTableWithBadDistribution() { final String sql = - "CREATE TABLE IF NOT EXISTS tbl1 (\n" + "CREATE TABLE tbl1 (\n" + " a bigint,\n" + " h varchar, \n" + " g as 2 * (a + 1), \n" + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'), \n" + " b varchar,\n" + " proc as PROCTIME(), \n" + + " meta STRING METADATA, \n" + + " my_meta STRING METADATA FROM 'meta', \n" + + " my_meta STRING METADATA FROM 'meta' VIRTUAL, \n" + + " meta STRING METADATA VIRTUAL, \n" + + " PRIMARY KEY (a, b)\n" + + ")\n" + + "DISTRIBUTED \n" + + " ^with^ (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + sql(sql).fails("(?s).*Encountered \"with\" at line 15, column 3.*"); + } + + @Test + void testCreateTableWithDistributionIfNotExists() { + final String sql = + "CREATE TABLE if not exists tbl1 (\n" + + " a bigint,\n" + + " h varchar, \n" + + " PRIMARY KEY (a, b)\n" + + ")\n" + + "DISTRIBUTED BY HASH(a, h) INTO 6 BUCKETS\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected = + "CREATE TABLE IF NOT EXISTS `TBL1` (\n" + + " `A` BIGINT,\n" + + " `H` VARCHAR,\n" + + " PRIMARY KEY (`A`, `B`)\n" + + ")\n" + + "DISTRIBUTED BY HASH(`A`, `H`) INTO 6 BUCKETS\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableIfNotExists() { + final String sql = + "CREATE TABLE IF NOT EXISTS tbl1 (\n" + + " a bigint,\n" + + " h varchar, \n" + " PRIMARY KEY (a, b)\n" + ")\n" + "PARTITIONED BY (a, h)\n" @@ -932,10 +1070,6 @@ void testCreateTableIfNotExists() { "CREATE TABLE IF NOT EXISTS `TBL1` (\n" + " `A` BIGINT,\n" + " `H` VARCHAR,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR,\n" - + " `PROC` AS `PROCTIME`(),\n" + " PRIMARY KEY (`A`, `B`)\n" + ")\n" + "PARTITIONED BY (`A`, `H`)\n" @@ -1692,6 +1826,72 @@ void testCreateTableWithLikeClause() { sql(sql).ok(expected); } + @Test + void testCreateTableWithLikeClauseIncludingDistribution() { + final String sql = + "create table source_table(\n" + + " a int,\n" + + " b bigint,\n" + + " c string\n" + + ")\n" + + "LIKE parent_table (\n" + + " INCLUDING ALL\n" + + " OVERWRITING OPTIONS\n" + + " INCLUDING DISTRIBUTION\n" + + " EXCLUDING PARTITIONS\n" + + " INCLUDING GENERATED\n" + + " INCLUDING METADATA\n" + + ")"; + final String expected = + "CREATE TABLE `SOURCE_TABLE` (\n" + + " `A` INTEGER,\n" + + " `B` BIGINT,\n" + + " `C` STRING\n" + + ")\n" + + "LIKE `PARENT_TABLE` (\n" + + " INCLUDING ALL\n" + + " OVERWRITING OPTIONS\n" + + " INCLUDING DISTRIBUTION\n" + + " EXCLUDING PARTITIONS\n" + + " INCLUDING GENERATED\n" + + " INCLUDING METADATA\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithLikeClauseExcludingDistribution() { + final String sql = + "create table source_table(\n" + + " a int,\n" + + " b bigint,\n" + + " c string\n" + + ")\n" + + "LIKE parent_table (\n" + + " INCLUDING ALL\n" + + " OVERWRITING OPTIONS\n" + + " EXCLUDING DISTRIBUTION\n" + + " INCLUDING PARTITIONS\n" + + " INCLUDING GENERATED\n" + + " INCLUDING METADATA\n" + + ")"; + final String expected = + "CREATE TABLE `SOURCE_TABLE` (\n" + + " `A` INTEGER,\n" + + " `B` BIGINT,\n" + + " `C` STRING\n" + + ")\n" + + "LIKE `PARENT_TABLE` (\n" + + " INCLUDING ALL\n" + + " OVERWRITING OPTIONS\n" + + " EXCLUDING DISTRIBUTION\n" + + " INCLUDING PARTITIONS\n" + + " INCLUDING GENERATED\n" + + " INCLUDING METADATA\n" + + ")"; + sql(sql).ok(expected); + } + @Test void testCreateTableWithLikeClauseWithoutColumns() { final String sql = @@ -2589,6 +2789,15 @@ void testCreateTableAsSelectWithConstraints() { .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); } + @Test + void testCreateTableAsSelectWithDistribution() { + sql("CREATE TABLE t DISTRIBUTED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node( + new ValidationMatcher() + .fails( + "CREATE TABLE AS SELECT syntax does not support creating distributed tables yet.")); + } + @Test void testCreateTableAsSelectWithPartitionKey() { sql("CREATE TABLE t PARTITIONED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") @@ -2649,6 +2858,13 @@ void testReplaceTableAsSelect() { new ValidationMatcher() .fails( "REPLACE TABLE AS SELECT syntax does not support to create partitioned table yet.")); + + // test replace table as select with distribution + sql("REPLACE TABLE t DISTRIBUTED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node( + new ValidationMatcher() + .fails( + "REPLACE TABLE AS SELECT syntax does not support creating distributed tables yet.")); } @Test @@ -2707,6 +2923,12 @@ void testCreateOrReplaceTableAsSelect() { new ValidationMatcher() .fails( "CREATE OR REPLACE TABLE AS SELECT syntax does not support to create partitioned table yet.")); + + sql("CREATE OR REPLACE TABLE t DISTRIBUTED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node( + new ValidationMatcher() + .fails( + "CREATE OR REPLACE TABLE AS SELECT syntax does not support creating distributed tables yet.")); } @Test diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableDescriptor.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableDescriptor.java index f2d17d07da3dd..906cdafdf0d3b 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableDescriptor.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableDescriptor.java @@ -22,6 +22,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.connector.format.Format; import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.utils.EncodingUtils; @@ -55,16 +56,19 @@ public class TableDescriptor { private final @Nullable Schema schema; private final Map options; + private final @Nullable TableDistribution distribution; private final List partitionKeys; private final @Nullable String comment; protected TableDescriptor( @Nullable Schema schema, Map options, + @Nullable TableDistribution distribution, List partitionKeys, @Nullable String comment) { this.schema = schema; this.options = Collections.unmodifiableMap(options); + this.distribution = distribution; this.partitionKeys = Collections.unmodifiableList(partitionKeys); this.comment = comment; } @@ -101,6 +105,10 @@ public Map getOptions() { return options; } + public Optional getDistribution() { + return Optional.ofNullable(distribution); + } + public List getPartitionKeys() { return partitionKeys; } @@ -123,7 +131,13 @@ public CatalogTable toCatalogTable() { + "It can only be omitted at certain " + "documented locations.")); - return CatalogTable.of(schema, getComment().orElse(null), getPartitionKeys(), getOptions()); + return CatalogTable.newBuilder() + .schema(schema) + .options(getOptions()) + .distribution(distribution) + .partitionKeys(partitionKeys) + .comment(getComment().orElse(null)) + .build(); } /** Converts this immutable instance into a mutable {@link Builder}. */ @@ -140,6 +154,9 @@ public String toString() { .map(EncodingUtils::escapeIdentifier) .collect(Collectors.joining(", ")); + final String distributedBy = + distribution == null ? "" : distribution.asSerializableString(); + final String partitionedBy = !partitionKeys.isEmpty() ? String.format("PARTITIONED BY (%s)", escapedPartitionKeys) @@ -156,9 +173,10 @@ public String toString() { .collect(Collectors.joining(String.format(",%n"))); return String.format( - "%s%nCOMMENT '%s'%n%s%nWITH (%n%s%n)", + "%s%nCOMMENT '%s'%n%s%s%nWITH (%n%s%n)", schema != null ? schema : "", comment != null ? comment : "", + distributedBy, partitionedBy, serializedOptions); } @@ -176,13 +194,14 @@ public boolean equals(Object obj) { TableDescriptor that = (TableDescriptor) obj; return Objects.equals(schema, that.schema) && options.equals(that.options) + && Objects.equals(distribution, that.distribution) && partitionKeys.equals(that.partitionKeys) && Objects.equals(comment, that.comment); } @Override public int hashCode() { - return Objects.hash(schema, options, partitionKeys, comment); + return Objects.hash(schema, options, distribution, partitionKeys, comment); } // --------------------------------------------------------------------------------------------- @@ -193,6 +212,7 @@ public static class Builder { private @Nullable Schema schema; private final Map options; + private @Nullable TableDistribution distribution; private final List partitionKeys; private @Nullable String comment; @@ -204,6 +224,7 @@ protected Builder() { protected Builder(TableDescriptor descriptor) { this.schema = descriptor.getSchema().orElse(null); this.options = new HashMap<>(descriptor.getOptions()); + this.distribution = descriptor.getDistribution().orElse(null); this.partitionKeys = new ArrayList<>(descriptor.getPartitionKeys()); this.comment = descriptor.getComment().orElse(null); } @@ -334,6 +355,86 @@ public Builder format( return this; } + /** + * Defines that the table should be distributed into buckets using a hash algorithm over the + * given columns. The number of buckets is connector-defined. + */ + public Builder distributedByHash(String... bucketKeys) { + validateBucketKeys(bucketKeys); + this.distribution = TableDistribution.ofHash(Arrays.asList(bucketKeys), null); + return this; + } + + /** + * Defines that the table should be distributed into the given number of buckets using a + * hash algorithm over the given columns. + */ + public Builder distributedByHash(int numberOfBuckets, String... bucketKeys) { + validateBucketKeys(bucketKeys); + this.distribution = + TableDistribution.ofHash(Arrays.asList(bucketKeys), numberOfBuckets); + return this; + } + + /** + * Defines that the table should be distributed into buckets using a range algorithm over + * the given columns. The number of buckets is connector-defined. + */ + public Builder distributedByRange(String... bucketKeys) { + validateBucketKeys(bucketKeys); + this.distribution = TableDistribution.ofRange(Arrays.asList(bucketKeys), null); + return this; + } + + /** + * Defines that the table should be distributed into the given number of buckets using a + * range algorithm over the given columns. + */ + public Builder distributedByRange(int numberOfBuckets, String... bucketKeys) { + validateBucketKeys(bucketKeys); + this.distribution = + TableDistribution.ofRange(Arrays.asList(bucketKeys), numberOfBuckets); + return this; + } + + /** + * Defines that the table should be distributed into buckets over the given columns. The + * number of buckets and used algorithm are connector-defined. + */ + public Builder distributedBy(String... bucketKeys) { + validateBucketKeys(bucketKeys); + this.distribution = TableDistribution.ofUnknown(Arrays.asList(bucketKeys), null); + return this; + } + + /** + * Defines that the table should be distributed into the given number of buckets by the + * given columns. The used algorithm is connector-defined. + */ + public Builder distributedBy(int numberOfBuckets, String... bucketKeys) { + validateBucketKeys(bucketKeys); + this.distribution = + TableDistribution.ofUnknown(Arrays.asList(bucketKeys), numberOfBuckets); + return this; + } + + /** + * Defines that the table should be distributed into the given number of buckets. The + * algorithm is connector-defined. + */ + public Builder distributedInto(int numberOfBuckets) { + this.distribution = TableDistribution.ofUnknown(numberOfBuckets); + return this; + } + + private static void validateBucketKeys(String[] bucketKeys) { + Preconditions.checkNotNull(bucketKeys, "Bucket keys must not be null."); + if (bucketKeys.length == 0) { + throw new ValidationException( + "At least one bucket key must be defined for a distribution."); + } + } + /** Define which columns this table is partitioned by. */ public Builder partitionedBy(String... partitionKeys) { this.partitionKeys.addAll(Arrays.asList(partitionKeys)); @@ -348,7 +449,7 @@ public Builder comment(@Nullable String comment) { /** Returns an immutable instance of {@link TableDescriptor}. */ public TableDescriptor build() { - return new TableDescriptor(schema, options, partitionKeys, comment); + return new TableDescriptor(schema, options, distribution, partitionKeys, comment); } } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/ShowCreateUtil.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/ShowCreateUtil.java index c40c57295ea42..36ad89176b26b 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/ShowCreateUtil.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/ShowCreateUtil.java @@ -27,6 +27,7 @@ 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.TableDistribution; import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.utils.EncodingUtils; @@ -64,6 +65,7 @@ public static String buildShowCreateTableRow( extractFormattedComment(table) .ifPresent( c -> sb.append(String.format("COMMENT '%s'%s", c, System.lineSeparator()))); + extractFormattedDistributedInfo((ResolvedCatalogTable) table).ifPresent(sb::append); extractFormattedPartitionedInfo((ResolvedCatalogTable) table) .ifPresent( partitionedInfoFormatted -> @@ -197,6 +199,10 @@ static Optional extractFormattedComment(ResolvedCatalogBaseTable tabl return Optional.empty(); } + static Optional extractFormattedDistributedInfo(ResolvedCatalogTable catalogTable) { + return catalogTable.getDistribution().map(TableDistribution::asSerializableString); + } + static Optional extractFormattedPartitionedInfo(ResolvedCatalogTable catalogTable) { if (!catalogTable.isPartitioned()) { return Optional.empty(); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java index d8e55b1437722..de6fdb8e8db21 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java @@ -63,6 +63,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.function.Consumer; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -1330,12 +1331,45 @@ public ResolvedCatalogTable resolveCatalogTable(CatalogTable table) { final ResolvedSchema resolvedSchema = table.getUnresolvedSchema().resolve(schemaResolver); - // Validate partition keys are included in physical columns + // Validate distribution keys are included in physical columns final List physicalColumns = resolvedSchema.getColumns().stream() .filter(Column::isPhysical) .map(Column::getName) .collect(Collectors.toList()); + + final Consumer distributionValidation = + distribution -> { + distribution + .getBucketKeys() + .forEach( + bucketKey -> { + if (!physicalColumns.contains(bucketKey)) { + throw new ValidationException( + String.format( + "Invalid bucket key '%s'. A bucket key for a distribution must " + + "reference a physical column in the schema. " + + "Available columns are: %s", + bucketKey, physicalColumns)); + } + }); + + distribution + .getBucketCount() + .ifPresent( + c -> { + if (c <= 0) { + throw new ValidationException( + String.format( + "Invalid bucket count '%s'. The number of " + + "buckets for a distributed table must be at least 1.", + c)); + } + }); + }; + + table.getDistribution().ifPresent(distributionValidation); + table.getPartitionKeys() .forEach( partitionKey -> { diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TableDescriptorTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TableDescriptorTest.java index d04eed8b63705..fef48494bc4a0 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TableDescriptorTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TableDescriptorTest.java @@ -20,9 +20,12 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.table.catalog.TableDistribution; import org.junit.jupiter.api.Test; +import java.util.Collections; + import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -50,6 +53,7 @@ void testBasic() { final TableDescriptor descriptor = TableDescriptor.forConnector("test-connector") .schema(schema) + .distributedByHash(1, "f0") .partitionedBy("f0") .comment("Test Comment") .build(); @@ -57,6 +61,9 @@ void testBasic() { assertThat(descriptor.getSchema()).isPresent(); assertThat(descriptor.getSchema().get()).isEqualTo(schema); + assertThat(descriptor.getDistribution()) + .contains(TableDistribution.ofHash(Collections.singletonList("f0"), 1)); + assertThat(descriptor.getPartitionKeys()).hasSize(1); assertThat(descriptor.getPartitionKeys().get(0)).isEqualTo("f0"); @@ -134,6 +141,7 @@ void testToString() { final TableDescriptor tableDescriptor = TableDescriptor.forConnector("test-connector") .schema(schema) + .distributedByRange(3, "f0") .partitionedBy("f0") .option(OPTION_A, true) .format(formatDescriptor) @@ -147,6 +155,7 @@ void testToString() { + " `f0` STRING\n" + ")\n" + "COMMENT 'Test Comment'\n" + + "DISTRIBUTED BY RANGE(`f0`) INTO 3 BUCKETS\n" + "PARTITIONED BY (`f0`)\n" + "WITH (\n" + " 'a' = 'true',\n" @@ -159,17 +168,60 @@ void testToString() { @Test void testFormatDescriptorWithPrefix() { assertThatThrownBy( - () -> { - TableDescriptor.forConnector("test-connector") - .schema(Schema.newBuilder().build()) - .format( - FormatDescriptor.forFormat("test-format") - .option("test-format.a", "A") - .build()) - .build(); - }) + () -> + TableDescriptor.forConnector("test-connector") + .schema(Schema.newBuilder().build()) + .format( + FormatDescriptor.forFormat("test-format") + .option("test-format.a", "A") + .build()) + .build()) .as( "Format options set using #format(FormatDescriptor) should not contain the prefix 'test-format.', but found 'test-format.a'.") .isInstanceOf(ValidationException.class); } + + @Test + void testDistributedBy() { + assertThat(getTableDescriptorBuilder().distributedByHash(3, "f0").build().toString()) + .contains("DISTRIBUTED BY HASH(`f0`) INTO 3 BUCKETS\n"); + assertThat(getTableDescriptorBuilder().distributedByHash("f0").build().toString()) + .contains("DISTRIBUTED BY HASH(`f0`)\n"); + assertThat(getTableDescriptorBuilder().distributedByRange(3, "f0").build().toString()) + .contains("DISTRIBUTED BY RANGE(`f0`) INTO 3 BUCKETS\n"); + assertThat(getTableDescriptorBuilder().distributedByRange("f0").build().toString()) + .contains("DISTRIBUTED BY RANGE(`f0`)\n"); + assertThat(getTableDescriptorBuilder().distributedBy(3, "f0").build().toString()) + .contains("DISTRIBUTED BY (`f0`) INTO 3 BUCKETS\n"); + assertThat(getTableDescriptorBuilder().distributedBy("f0").build().toString()) + .contains("DISTRIBUTED BY (`f0`)\n"); + assertThat(getTableDescriptorBuilder().distributedInto(3).build().toString()) + .contains("DISTRIBUTED INTO 3 BUCKETS\n"); + } + + @Test + void testDistributedByExceptions() { + assertThatThrownBy(() -> getTableDescriptorBuilder().distributedByHash(3)) + .as("At least one bucket key must be defined for a distribution.") + .isInstanceOf(ValidationException.class); + assertThatThrownBy(() -> getTableDescriptorBuilder().distributedBy()) + .as("At least one bucket key must be defined for a distribution.") + .isInstanceOf(ValidationException.class); + assertThatThrownBy(() -> getTableDescriptorBuilder().distributedByRange(3)) + .as("At least one bucket key must be defined for a distribution.") + .isInstanceOf(ValidationException.class); + } + + private static TableDescriptor.Builder getTableDescriptorBuilder() { + final Schema schema = Schema.newBuilder().column("f0", DataTypes.STRING()).build(); + final FormatDescriptor formatDescriptor = + FormatDescriptor.forFormat("test-format").option(OPTION_A, false).build(); + + return TableDescriptor.forConnector("test-connector") + .schema(schema) + .partitionedBy("f0") + .option(OPTION_A, true) + .format(formatDescriptor) + .comment("Test Comment"); + } } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/CatalogBaseTableResolutionTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/CatalogBaseTableResolutionTest.java index 488ae0f7243cc..fe0d84bd4934c 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/CatalogBaseTableResolutionTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/CatalogBaseTableResolutionTest.java @@ -201,6 +201,72 @@ void testInvalidPartitionKeys() { } } + @Test + void testValidDistributionKeys() { + final CatalogTable catalogTable = + new DefaultCatalogTable( + TABLE_SCHEMA, + null, + Collections.emptyList(), + Collections.emptyMap(), + null, + TableDistribution.ofHash(Collections.singletonList("county"), 6)); + final ResolvedCatalogTable resolvedTable = + resolveCatalogBaseTable(ResolvedCatalogTable.class, catalogTable); + assertThat(resolvedTable.getDistribution().get().getBucketKeys()) + .isEqualTo(Collections.singletonList("county")); + assertThat(resolvedTable.getDistribution().get().getKind()) + .isEqualTo(TableDistribution.Kind.HASH); + } + + @Test + void testInvalidDistributionKeys() { + final CatalogTable catalogTable = + new DefaultCatalogTable( + TABLE_SCHEMA, + null, + Collections.emptyList(), + Collections.emptyMap(), + null, + TableDistribution.ofHash(Collections.singletonList("countyINVALID"), 6)); + try { + resolveCatalogBaseTable(ResolvedCatalogTable.class, catalogTable); + fail("Invalid bucket keys expected."); + } catch (Exception e) { + assertThat(e) + .satisfies( + matching( + containsMessage( + "Invalid bucket key 'countyINVALID'. A bucket key for a distribution must " + + "reference a physical column in the schema. " + + "Available columns are: [id, region, county]"))); + } + } + + @Test + void testInvalidDistributionBucketCount() { + final CatalogTable catalogTable = + new DefaultCatalogTable( + TABLE_SCHEMA, + null, + Collections.emptyList(), + Collections.emptyMap(), + null, + TableDistribution.ofHash(Collections.singletonList("id"), 0)); + + try { + resolveCatalogBaseTable(ResolvedCatalogTable.class, catalogTable); + fail("Invalid bucket keys expected."); + } catch (Exception e) { + assertThat(e) + .satisfies( + matching( + containsMessage( + "Invalid bucket count '0'. The number of buckets for a " + + "distributed table must be at least 1."))); + } + } + // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java index 4b37b3efc602d..5d57e6095b5d5 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java @@ -18,6 +18,7 @@ package org.apache.flink.table.test.program; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.types.Row; import javax.annotation.Nullable; @@ -41,6 +42,7 @@ public final class SinkTestStep extends TableTestStep { SinkTestStep( String name, List schemaComponents, + @Nullable TableDistribution distribution, List partitionKeys, Map options, @Nullable List expectedBeforeRestore, @@ -48,7 +50,7 @@ public final class SinkTestStep extends TableTestStep { @Nullable List expectedBeforeRestoreStrings, @Nullable List expectedAfterRestoreStrings, boolean testChangelogData) { - super(name, schemaComponents, partitionKeys, options); + super(name, schemaComponents, distribution, partitionKeys, options); if (expectedBeforeRestore != null && expectedAfterRestoreStrings != null) { throw new IllegalArgumentException( "You can not mix Row/String representation in before/after restore data."); @@ -169,6 +171,7 @@ public SinkTestStep build() { return new SinkTestStep( name, schemaComponents, + distribution, partitionKeys, options, expectedBeforeRestore, diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java index 6653eb174c5ec..95ebcfc6dd99f 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java @@ -18,8 +18,11 @@ package org.apache.flink.table.test.program; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.types.Row; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -34,11 +37,12 @@ public final class SourceTestStep extends TableTestStep { SourceTestStep( String name, List schemaComponents, + @Nullable TableDistribution distribution, List partitionKeys, Map options, List dataBeforeRestore, List dataAfterRestore) { - super(name, schemaComponents, partitionKeys, options); + super(name, schemaComponents, distribution, partitionKeys, options); this.dataBeforeRestore = dataBeforeRestore; this.dataAfterRestore = dataAfterRestore; } @@ -85,6 +89,7 @@ public SourceTestStep build() { return new SourceTestStep( name, schemaComponents, + distribution, partitionKeys, options, dataBeforeRestore, diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java index d1ff6b8d018e7..5d05bcec9f6e2 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java @@ -22,6 +22,9 @@ import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.catalog.TableDistribution; + +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; @@ -29,6 +32,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; /** Abstract class for {@link SourceTestStep} and {@link SinkTestStep}. */ @@ -36,16 +40,19 @@ public abstract class TableTestStep implements TestStep { public final String name; public final List schemaComponents; + public final @Nullable TableDistribution distribution; public final List partitionKeys; public final Map options; TableTestStep( String name, List schemaComponents, + @Nullable TableDistribution distribution, List partitionKeys, Map options) { this.name = name; this.schemaComponents = schemaComponents; + this.distribution = distribution; this.partitionKeys = partitionKeys; this.options = options; } @@ -58,15 +65,20 @@ public TableResult apply(TableEnvironment env, Map extraOptions) final Map allOptions = new HashMap<>(options); allOptions.putAll(extraOptions); + final String distributedBy = + Optional.ofNullable(distribution) + .map(TableDistribution::asSerializableString) + .orElse(""); final String partitionedBy = partitionKeys.isEmpty() ? "" : "PARTITIONED BY (" + String.join(", ", partitionKeys) + ")\n"; final String createTable = String.format( - "CREATE TABLE %s (\n%s)\n%sWITH (\n%s)", + "CREATE TABLE %s (\n%s)\n%s%sWITH (\n%s)", name, String.join(",\n", schemaComponents), + distributedBy, partitionedBy, allOptions.entrySet().stream() .map(e -> String.format("'%s'='%s'", e.getKey(), e.getValue())) @@ -83,6 +95,7 @@ protected abstract static class AbstractBuilder< protected final String name; protected final List schemaComponents = new ArrayList<>(); + protected @Nullable TableDistribution distribution; protected final List partitionKeys = new ArrayList<>(); protected final Map options = new HashMap<>(); @@ -131,6 +144,11 @@ public SpecificBuilder addOption(ConfigOption option, String value) { return (SpecificBuilder) this; } + public SpecificBuilder addDistribution(@Nullable TableDistribution distribution) { + this.distribution = distribution; + return (SpecificBuilder) this; + } + public SpecificBuilder addPartitionKeys(String... partitionKeys) { this.partitionKeys.addAll(Arrays.asList(partitionKeys)); return (SpecificBuilder) this; diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogTable.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogTable.java index a2e287310050a..f3a80342d3905 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogTable.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogTable.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.table.api.Schema; import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; @@ -49,6 +50,12 @@ @PublicEvolving public interface CatalogTable extends CatalogBaseTable { + /** Builder for configuring and creating instances of {@link CatalogTable}. */ + @PublicEvolving + static CatalogTable.Builder newBuilder() { + return new CatalogTable.Builder(); + } + /** * Creates a basic implementation of this interface. * @@ -58,7 +65,9 @@ public interface CatalogTable extends CatalogBaseTable { * @param comment optional comment * @param partitionKeys list of partition keys or an empty list if not partitioned * @param options options to configure the connector + * @deprecated Use the builder {@link CatalogTable#newBuilder()} instead. */ + @Deprecated static CatalogTable of( Schema schema, @Nullable String comment, @@ -75,14 +84,16 @@ static CatalogTable of( * @param partitionKeys list of partition keys or an empty list if not partitioned * @param options options to configure the connector * @param snapshot table snapshot of the table + * @deprecated Use the builder {@link CatalogTable#newBuilder()} instead. */ + @Deprecated static CatalogTable of( Schema schema, @Nullable String comment, List partitionKeys, Map options, @Nullable Long snapshot) { - return new DefaultCatalogTable(schema, comment, partitionKeys, options, snapshot); + return new DefaultCatalogTable(schema, comment, partitionKeys, options, snapshot, null); } /** @@ -144,4 +155,60 @@ default Map toProperties() { default Optional getSnapshot() { return Optional.empty(); } + + /** Returns the distribution of the table if the {@code DISTRIBUTED} clause is defined. */ + default Optional getDistribution() { + return Optional.empty(); + } + + // -------------------------------------------------------------------------------------------- + + /** Builder for configuring and creating instances of {@link CatalogTable}. */ + @PublicEvolving + class Builder { + private @Nullable Schema schema; + private @Nullable String comment; + private List partitionKeys = Collections.emptyList(); + private Map options = Collections.emptyMap(); + private @Nullable Long snapshot; + private @Nullable TableDistribution distribution; + + private Builder() {} + + public Builder schema(Schema schema) { + this.schema = Preconditions.checkNotNull(schema, "Schema must not be null."); + return this; + } + + public Builder comment(@Nullable String comment) { + this.comment = comment; + return this; + } + + public Builder partitionKeys(List partitionKeys) { + this.partitionKeys = + Preconditions.checkNotNull(partitionKeys, "Partition keys must not be null."); + return this; + } + + public Builder options(Map options) { + this.options = Preconditions.checkNotNull(options, "Options must not be null."); + return this; + } + + public Builder snapshot(@Nullable Long snapshot) { + this.snapshot = snapshot; + return this; + } + + public Builder distribution(@Nullable TableDistribution distribution) { + this.distribution = distribution; + return this; + } + + public CatalogTable build() { + return new DefaultCatalogTable( + schema, comment, partitionKeys, options, snapshot, distribution); + } + } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/DefaultCatalogTable.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/DefaultCatalogTable.java index ebf87d03ef7a9..d2952c4803053 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/DefaultCatalogTable.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/DefaultCatalogTable.java @@ -37,9 +37,9 @@ public class DefaultCatalogTable implements CatalogTable { private final Schema schema; private final @Nullable String comment; + private final @Nullable TableDistribution distribution; private final List partitionKeys; private final Map options; - private final @Nullable Long snapshot; protected DefaultCatalogTable( @@ -47,7 +47,7 @@ protected DefaultCatalogTable( @Nullable String comment, List partitionKeys, Map options) { - this(schema, comment, partitionKeys, options, null); + this(schema, comment, partitionKeys, options, null, null); } protected DefaultCatalogTable( @@ -55,12 +55,14 @@ protected DefaultCatalogTable( @Nullable String comment, List partitionKeys, Map options, - @Nullable Long snapshot) { + @Nullable Long snapshot, + @Nullable TableDistribution distribution) { this.schema = checkNotNull(schema, "Schema must not be null."); this.comment = comment; this.partitionKeys = checkNotNull(partitionKeys, "Partition keys must not be null."); this.options = checkNotNull(options, "Options must not be null."); this.snapshot = snapshot; + this.distribution = distribution; checkArgument( options.entrySet().stream() @@ -88,19 +90,31 @@ public List getPartitionKeys() { return partitionKeys; } + @Override + public Optional getDistribution() { + return Optional.ofNullable(distribution); + } + @Override public Map getOptions() { return options; } + @Override + public Optional getSnapshot() { + return Optional.ofNullable(snapshot); + } + @Override public CatalogBaseTable copy() { - return new DefaultCatalogTable(schema, comment, partitionKeys, options, snapshot); + return new DefaultCatalogTable( + schema, comment, partitionKeys, options, snapshot, distribution); } @Override public CatalogTable copy(Map options) { - return new DefaultCatalogTable(schema, comment, partitionKeys, options, snapshot); + return new DefaultCatalogTable( + schema, comment, partitionKeys, options, snapshot, distribution); } @Override @@ -130,6 +144,7 @@ public boolean equals(Object o) { DefaultCatalogTable that = (DefaultCatalogTable) o; return schema.equals(that.schema) && Objects.equals(comment, that.comment) + && Objects.equals(distribution, that.distribution) && partitionKeys.equals(that.partitionKeys) && options.equals(that.options) && Objects.equals(snapshot, that.snapshot); @@ -137,7 +152,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(schema, comment, partitionKeys, options, snapshot); + return Objects.hash(schema, comment, distribution, partitionKeys, options, snapshot); } @Override @@ -147,7 +162,9 @@ public String toString() { + schema + ", comment='" + comment - + '\'' + + "'" + + ", distribution=" + + distribution + ", partitionKeys=" + partitionKeys + ", options=" @@ -156,9 +173,4 @@ public String toString() { + snapshot + '}'; } - - @Override - public Optional getSnapshot() { - return Optional.ofNullable(snapshot); - } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ResolvedCatalogTable.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ResolvedCatalogTable.java index a8c46f57d4e68..e52f6aa5a1383 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ResolvedCatalogTable.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ResolvedCatalogTable.java @@ -125,6 +125,11 @@ public Optional getSnapshot() { return origin.getSnapshot(); } + @Override + public Optional getDistribution() { + return origin.getDistribution(); + } + @Override public ResolvedCatalogTable copy(Map options) { return new ResolvedCatalogTable(origin.copy(options), resolvedSchema); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/TableDistribution.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/TableDistribution.java new file mode 100644 index 0000000000000..8c0d44023961e --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/TableDistribution.java @@ -0,0 +1,145 @@ +/* + * 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.catalog; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.connector.sink.abilities.SupportsBucketing; +import org.apache.flink.table.utils.EncodingUtils; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * Defines whether the given table is distributed across buckets using a specified algorithm and + * given columns. It represents the DISTRIBUTED BY clause in SQL. + * + * @see SupportsBucketing + */ +@PublicEvolving +public class TableDistribution { + + /** Distribution kind. */ + @PublicEvolving + public enum Kind { + UNKNOWN, + HASH, + RANGE + } + + private final Kind kind; + private final @Nullable Integer bucketCount; + private final List bucketKeys; + + private TableDistribution(Kind kind, @Nullable Integer bucketCount, List bucketKeys) { + this.kind = Preconditions.checkNotNull(kind, "Distribution kind must not be null."); + this.bucketCount = bucketCount; + this.bucketKeys = Preconditions.checkNotNull(bucketKeys, "Bucket keys must not be null."); + } + + /** Distribution of the given kind over the given keys with a declared number of buckets. */ + public static TableDistribution of( + Kind kind, @Nullable Integer bucketCount, List bucketKeys) { + return new TableDistribution(kind, bucketCount, bucketKeys); + } + + /** Connector-dependent distribution with a declared number of buckets. */ + public static TableDistribution ofUnknown(int bucketCount) { + return new TableDistribution(Kind.UNKNOWN, bucketCount, Collections.emptyList()); + } + + /** Connector-dependent distribution with a declared number of buckets. */ + public static TableDistribution ofUnknown( + List bucketKeys, @Nullable Integer bucketCount) { + return new TableDistribution(Kind.UNKNOWN, bucketCount, bucketKeys); + } + + /** Hash distribution over the given keys among the declared number of buckets. */ + public static TableDistribution ofHash(List bucketKeys, @Nullable Integer bucketCount) { + return new TableDistribution(Kind.HASH, bucketCount, bucketKeys); + } + + /** Range distribution over the given keys among the declared number of buckets. */ + public static TableDistribution ofRange( + List bucketKeys, @Nullable Integer bucketCount) { + return new TableDistribution(Kind.RANGE, bucketCount, bucketKeys); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TableDistribution that = (TableDistribution) o; + return kind == that.kind + && Objects.equals(bucketCount, that.bucketCount) + && Objects.equals(bucketKeys, that.bucketKeys); + } + + @Override + public int hashCode() { + return Objects.hash(kind, bucketCount, bucketKeys); + } + + public Kind getKind() { + return kind; + } + + public List getBucketKeys() { + return bucketKeys; + } + + public Optional getBucketCount() { + return Optional.ofNullable(bucketCount); + } + + public String asSerializableString() { + if (getBucketKeys().isEmpty() + && getBucketCount().isPresent() + && getBucketCount().get() != 0) { + return "DISTRIBUTED INTO " + getBucketCount().get() + " BUCKETS\n"; + } + + StringBuilder sb = new StringBuilder(); + sb.append("DISTRIBUTED BY "); + if (getKind() != null && getKind() != Kind.UNKNOWN) { + sb.append(getKind()); + } + sb.append("("); + sb.append( + getBucketKeys().stream() + .map(EncodingUtils::escapeIdentifier) + .collect(Collectors.joining(", "))); + sb.append(")"); + if (getBucketCount().isPresent() && getBucketCount().get() != 0) { + sb.append(" INTO "); + sb.append(getBucketCount().get()); + sb.append(" BUCKETS"); + } + sb.append("\n"); + return sb.toString(); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/DynamicTableSink.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/DynamicTableSink.java index 1812c1f1397ae..73abf45b01a44 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/DynamicTableSink.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/DynamicTableSink.java @@ -23,6 +23,7 @@ import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.RuntimeConverter; +import org.apache.flink.table.connector.sink.abilities.SupportsBucketing; import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; import org.apache.flink.table.connector.sink.abilities.SupportsWritingMetadata; @@ -66,6 +67,7 @@ *

    A {@link DynamicTableSink} can implement the following abilities: * *

      + *
    • {@link SupportsBucketing} *
    • {@link SupportsPartitioning} *
    • {@link SupportsOverwrite} *
    • {@link SupportsWritingMetadata} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsBucketing.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsBucketing.java new file mode 100644 index 0000000000000..9a05817fc5e65 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsBucketing.java @@ -0,0 +1,96 @@ +/* + * 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.connector.sink.abilities; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.catalog.TableDistribution; +import org.apache.flink.table.connector.sink.DynamicTableSink; + +import java.util.Set; + +/** + * Enables to write bucketed data into a {@link DynamicTableSink}. + * + *

      Buckets enable load balancing in an external storage system by splitting data into disjoint + * subsets. These subsets group rows with potentially "infinite" keyspace into smaller and more + * manageable chunks that allow for efficient parallel processing. + * + *

      Bucketing depends heavily on the semantics of the underlying connector. However, a user can + * influence the bucketing behavior by specifying the number of buckets, the bucketing algorithm, + * and (if the algorithm allows it) the columns which are used for target bucket calculation. + * + *

      All bucketing components (i.e. bucket number, distribution algorithm, bucket key columns) are + * optional from a SQL syntax perspective. This ability interface defines which algorithms ({@link + * #listAlgorithms()}) are effectively supported and whether a bucket count is mandatory ({@link + * #requiresBucketCount()}). The planner will perform necessary validation checks. + * + *

      Given the following SQL statements: + * + *

      {@code
      + * -- Example 1
      + * CREATE TABLE MyTable (uid BIGINT, name STRING) DISTRIBUTED BY HASH(uid) INTO 4 BUCKETS;
      + *
      + * -- Example 2
      + * CREATE TABLE MyTable (uid BIGINT, name STRING) DISTRIBUTED BY (uid) INTO 4 BUCKETS;
      + *
      + * -- Example 3
      + * CREATE TABLE MyTable (uid BIGINT, name STRING) DISTRIBUTED BY (uid);
      + *
      + * -- Example 4
      + * CREATE TABLE MyTable (uid BIGINT, name STRING) DISTRIBUTED INTO 4 BUCKETS;
      + * }
      + * + *

      Example 1 declares a hash function on a fixed number of 4 buckets (i.e. HASH(uid) % 4 = target + * bucket). Example 2 leaves the selection of an algorithm up to the connector, represented as + * {@link TableDistribution.Kind#UNKNOWN}. Additionally, Example 3 leaves the number of buckets up + * to the connector. In contrast, Example 4 only defines the number of buckets. + * + *

      A sink can implement both {@link SupportsPartitioning} and {@link SupportsBucketing}. + * Conceptually, a partition can be seen as kind of "directory" whereas buckets correspond to + * "files" per directory. Partitioning splits the data on a small, human-readable keyspace (e.g. by + * year or by geographical region). This enables efficient selection via equality, inequality, or + * ranges due to knowledge about existing partitions. Bucketing operates within partitions on a + * potentially large and infinite keyspace. + * + * @see SupportsPartitioning + */ +@PublicEvolving +public interface SupportsBucketing { + + /** + * Returns the set of supported bucketing algorithms. + * + *

      The set must be non-empty. Otherwise, the planner will throw an error during validation. + * + *

      If specifying an algorithm is optional, this set must include {@link + * TableDistribution.Kind#UNKNOWN}. + */ + Set listAlgorithms(); + + /** + * Returns whether the {@link DynamicTableSink} requires a bucket count. + * + *

      If this method returns {@code true}, the {@link DynamicTableSink} will require a bucket + * count. + * + *

      If this method return {@code false}, the {@link DynamicTableSink} may or may not consider + * the provided bucket count. + */ + boolean requiresBucketCount(); +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsPartitioning.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsPartitioning.java index e986eb1b8cb30..efec1794a1633 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsPartitioning.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsPartitioning.java @@ -83,6 +83,15 @@ *

      If the {@code PARTITION} clause contains no static assignments or is omitted entirely, all * values for partition keys are either derived from static parts of the query or obtained * dynamically. + * + *

      A sink can implement both {@link SupportsPartitioning} and {@link SupportsBucketing}. + * Conceptually, a partition can be seen as kind of "directory" whereas buckets correspond to + * "files" per directory. Partitioning splits the data on a small, human-readable keyspace (e.g. by + * year or by geographical region). This enables efficient selection via equality, inequality, or + * ranges due to knowledge about existing partitions. Bucketing operates within partitions on a + * potentially large and infinite keyspace. + * + * @see SupportsBucketing */ @PublicEvolving public interface SupportsPartitioning { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java index 39effac989e43..222e9361f391d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java @@ -37,9 +37,11 @@ import org.apache.flink.table.catalog.ExternalCatalogTable; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.connector.RowLevelModificationScanContext; import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.abilities.SupportsBucketing; import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; import org.apache.flink.table.connector.sink.abilities.SupportsRowLevelDelete; @@ -53,6 +55,7 @@ import org.apache.flink.table.planner.calcite.FlinkRelBuilder; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable; +import org.apache.flink.table.planner.plan.abilities.sink.BucketingSpec; import org.apache.flink.table.planner.plan.abilities.sink.OverwriteSpec; import org.apache.flink.table.planner.plan.abilities.sink.RowLevelDeleteSpec; import org.apache.flink.table.planner.plan.abilities.sink.RowLevelUpdateSpec; @@ -945,8 +948,8 @@ private static void pushMetadataProjection( } /** - * Prepares the given {@link DynamicTableSink}. It check whether the sink is compatible with the - * INSERT INTO clause and applies initial parameters. + * Prepares the given {@link DynamicTableSink}. It checks whether the sink is compatible with + * the INSERT INTO clause and applies initial parameters. */ private static void prepareDynamicSink( String tableDebugName, @@ -955,6 +958,12 @@ private static void prepareDynamicSink( DynamicTableSink sink, ResolvedCatalogTable table, List sinkAbilitySpecs) { + table.getDistribution() + .ifPresent( + distribution -> + validateBucketing( + tableDebugName, sink, distribution, sinkAbilitySpecs)); + validatePartitioning(tableDebugName, staticPartitions, sink, table.getPartitionKeys()); validateAndApplyOverwrite(tableDebugName, isOverwrite, sink, sinkAbilitySpecs); @@ -1030,6 +1039,51 @@ private static DataType fixSinkDataType( TypeTransformations.toNullable()); } + private static void validateBucketing( + String tableDebugName, + DynamicTableSink sink, + TableDistribution distribution, + List sinkAbilitySpecs) { + if (!(sink instanceof SupportsBucketing)) { + throw new TableException( + String.format( + "Table '%s' is distributed into buckets, but the underlying %s doesn't " + + "implement the %s interface.", + tableDebugName, + DynamicTableSink.class.getSimpleName(), + SupportsBucketing.class.getSimpleName())); + } + SupportsBucketing sinkWithBucketing = (SupportsBucketing) sink; + if (sinkWithBucketing.requiresBucketCount() && !distribution.getBucketCount().isPresent()) { + throw new ValidationException( + String.format( + "Table '%s' is a bucketed table, but the underlying %s requires the number of buckets to be set.", + tableDebugName, DynamicTableSink.class.getSimpleName())); + } + if (!sinkWithBucketing.listAlgorithms().contains(distribution.getKind())) { + if (distribution.getKind() == TableDistribution.Kind.UNKNOWN) { + throw new ValidationException( + String.format( + "Bucketed table '%s' must specify an algorithm. Supported algorithms: %s", + tableDebugName, + sinkWithBucketing.listAlgorithms().stream() + .map(TableDistribution.Kind::toString) + .sorted() + .collect(Collectors.toList()))); + } + throw new ValidationException( + String.format( + "Table '%s' is a bucketed table and it supports %s, but algorithm %s was requested.", + tableDebugName, + sinkWithBucketing.listAlgorithms().stream() + .map(TableDistribution.Kind::toString) + .sorted() + .collect(Collectors.toList()), + distribution.getKind())); + } + sinkAbilitySpecs.add(new BucketingSpec()); + } + private static void validatePartitioning( String tableDebugName, Map staticPartitions, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/MergeTableLikeUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/MergeTableLikeUtil.java index 185a7201e4ab0..70014686ef7b0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/MergeTableLikeUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/MergeTableLikeUtil.java @@ -37,6 +37,7 @@ import org.apache.flink.table.api.Schema.UnresolvedWatermarkSpec; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.DataTypeFactory; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.expressions.SqlCallExpression; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.types.logical.LogicalType; @@ -76,6 +77,7 @@ class MergeTableLikeUtil { defaultMergingStrategies.put(FeatureOption.GENERATED, MergingStrategy.INCLUDING); defaultMergingStrategies.put(FeatureOption.METADATA, MergingStrategy.INCLUDING); defaultMergingStrategies.put(FeatureOption.CONSTRAINTS, MergingStrategy.INCLUDING); + defaultMergingStrategies.put(FeatureOption.DISTRIBUTION, MergingStrategy.INCLUDING); defaultMergingStrategies.put(FeatureOption.PARTITIONS, MergingStrategy.INCLUDING); } @@ -162,6 +164,32 @@ public Schema mergeTables( return schemaBuilder.build(); } + /** + * Merges the distribution part of {@code CREATE TABLE} statement. + * + *

      Distribution is a single property of a Table, thus there can be at most a single instance + * of it. Therefore, it is not possible to use {@link MergingStrategy#INCLUDING} with a + * distribution defined in both source and derived table. + */ + public Optional mergeDistribution( + MergingStrategy mergingStrategy, + Optional sourceTableDistribution, + Optional derivedTabledDistribution) { + + if (derivedTabledDistribution.isPresent() + && sourceTableDistribution.isPresent() + && mergingStrategy != MergingStrategy.EXCLUDING) { + throw new ValidationException( + "The base table already has a distribution defined. You might want to specify " + + "EXCLUDING DISTRIBUTION."); + } + + if (derivedTabledDistribution.isPresent()) { + return derivedTabledDistribution; + } + return sourceTableDistribution; + } + /** * Merges the partitions part of {@code CREATE TABLE} statement. * diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlCreateTableConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlCreateTableConverter.java index 172048e6cc00b..c68758d21f6cc 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlCreateTableConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlCreateTableConverter.java @@ -31,6 +31,7 @@ import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ContextResolvedTable; import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.operations.CreateTableASOperation; import org.apache.flink.table.operations.Operation; @@ -42,7 +43,9 @@ import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlNumericLiteral; +import java.math.BigDecimal; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashSet; @@ -104,17 +107,22 @@ Operation convertCreateTableAS( .getClass() .getSimpleName())); CatalogTable catalogTable = createCatalogTable(sqlCreateTableAs); + CatalogTable tableWithResolvedSchema = + CatalogTable.newBuilder() + .schema( + Schema.newBuilder() + .fromResolvedSchema(query.getResolvedSchema()) + .build()) + .comment(catalogTable.getComment()) + .distribution(catalogTable.getDistribution().orElse(null)) + .options(catalogTable.getOptions()) + .partitionKeys(catalogTable.getPartitionKeys()) + .build(); CreateTableOperation createTableOperation = new CreateTableOperation( identifier, - CatalogTable.of( - Schema.newBuilder() - .fromResolvedSchema(query.getResolvedSchema()) - .build(), - catalogTable.getComment(), - catalogTable.getPartitionKeys(), - catalogTable.getOptions()), + tableWithResolvedSchema, sqlCreateTableAs.isIfNotExists(), sqlCreateTableAs.isTemporary()); @@ -125,6 +133,7 @@ Operation convertCreateTableAS( private CatalogTable createCatalogTable(SqlCreateTable sqlCreateTable) { final Schema sourceTableSchema; + final Optional sourceTableDistribution; final List sourcePartitionKeys; final List likeOptions; final Map sourceProperties; @@ -132,11 +141,13 @@ private CatalogTable createCatalogTable(SqlCreateTable sqlCreateTable) { SqlTableLike sqlTableLike = ((SqlCreateTableLike) sqlCreateTable).getTableLike(); CatalogTable table = lookupLikeSourceTable(sqlTableLike); sourceTableSchema = table.getUnresolvedSchema(); + sourceTableDistribution = table.getDistribution(); sourcePartitionKeys = table.getPartitionKeys(); likeOptions = sqlTableLike.getOptions(); sourceProperties = table.getOptions(); } else { sourceTableSchema = Schema.newBuilder().build(); + sourceTableDistribution = Optional.empty(); sourcePartitionKeys = Collections.emptyList(); likeOptions = Collections.emptyList(); sourceProperties = Collections.emptyMap(); @@ -164,6 +175,9 @@ private CatalogTable createCatalogTable(SqlCreateTable sqlCreateTable) { .orElseGet(Collections::emptyList), primaryKey.orElse(null)); + Optional mergedTableDistribution = + mergeDistribution(sourceTableDistribution, sqlCreateTable, mergingStrategies); + List partitionKeys = mergePartitions( sourcePartitionKeys, @@ -173,9 +187,16 @@ private CatalogTable createCatalogTable(SqlCreateTable sqlCreateTable) { String tableComment = OperationConverterUtils.getTableComment(sqlCreateTable.getComment()); - return catalogManager.resolveCatalogTable( - CatalogTable.of( - mergedSchema, tableComment, partitionKeys, new HashMap<>(mergedOptions))); + CatalogTable catalogTable = + CatalogTable.newBuilder() + .schema(mergedSchema) + .comment(tableComment) + .distribution(mergedTableDistribution.orElse(null)) + .options(new HashMap<>(mergedOptions)) + .partitionKeys(partitionKeys) + .build(); + + return catalogManager.resolveCatalogTable(catalogTable); } private CatalogTable lookupLikeSourceTable(SqlTableLike sqlTableLike) { @@ -220,6 +241,44 @@ private void verifyPartitioningColumnsExist(Schema mergedSchema, List pa } } + private Optional mergeDistribution( + Optional sourceTableDistribution, + SqlCreateTable sqlCreateTable, + Map mergingStrategies) { + + Optional derivedTabledDistribution = Optional.empty(); + if (sqlCreateTable.getDistribution() != null) { + TableDistribution.Kind kind = + TableDistribution.Kind.valueOf( + sqlCreateTable + .getDistribution() + .getDistributionKind() + .orElse(TableDistribution.Kind.UNKNOWN.toString())); + Integer bucketCount = null; + SqlNumericLiteral count = sqlCreateTable.getDistribution().getBucketCount(); + if (count != null && count.isInteger()) { + bucketCount = ((BigDecimal) (count).getValue()).intValue(); + } + + List bucketColumns = Collections.emptyList(); + + SqlNodeList columns = sqlCreateTable.getDistribution().getBucketColumns(); + if (columns != null) { + bucketColumns = + columns.getList().stream() + .map(p -> ((SqlIdentifier) p).getSimple()) + .collect(Collectors.toList()); + } + derivedTabledDistribution = + Optional.of(TableDistribution.of(kind, bucketCount, bucketColumns)); + } + + return mergeTableLikeUtil.mergeDistribution( + mergingStrategies.get(SqlTableLike.FeatureOption.DISTRIBUTION), + sourceTableDistribution, + derivedTabledDistribution); + } + private List mergePartitions( List sourcePartitionKeys, SqlNodeList derivedPartitionKeys, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/sink/BucketingSpec.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/sink/BucketingSpec.java new file mode 100644 index 0000000000000..c27b282772b62 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/sink/BucketingSpec.java @@ -0,0 +1,57 @@ +/* + * 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.planner.plan.abilities.sink; + +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.abilities.SupportsBucketing; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * No properties. This only checks whether the interface is implemented again during deserialization + */ +@JsonIgnoreProperties(ignoreUnknown = true) +@JsonTypeName("Bucketing") +public final class BucketingSpec implements SinkAbilitySpec { + + @Override + public void apply(DynamicTableSink tableSink) { + if (!(tableSink instanceof SupportsBucketing)) { + throw new TableException( + String.format( + "%s does not support SupportsBucketing.", + tableSink.getClass().getName())); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + return o != null && getClass() == o.getClass(); + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/sink/SinkAbilitySpec.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/sink/SinkAbilitySpec.java index ce1a494c493d9..471145b2a60bd 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/sink/SinkAbilitySpec.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/sink/SinkAbilitySpec.java @@ -32,6 +32,7 @@ @JsonIgnoreProperties(ignoreUnknown = true) @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") @JsonSubTypes({ + @JsonSubTypes.Type(value = BucketingSpec.class), @JsonSubTypes.Type(value = OverwriteSpec.class), @JsonSubTypes.Type(value = PartitioningSpec.class), @JsonSubTypes.Type(value = WritingMetadataSpec.class), diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java index b55fccbff2865..df1b9b78936c0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java @@ -27,6 +27,7 @@ import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.catalog.WatermarkSpec; import org.apache.flink.table.connector.ChangelogMode; @@ -160,6 +161,7 @@ private static void registerSerializers(SimpleModule module) { module.addSerializer(new ResolvedExpressionJsonSerializer()); module.addSerializer(new ResolvedSchemaJsonSerializer()); module.addSerializer(new RequiredDistributionJsonSerializer()); + module.addSerializer(new TableDistributionJsonSerializer()); } @SuppressWarnings({"unchecked", "rawtypes"}) @@ -188,6 +190,7 @@ private static void registerDeserializers(SimpleModule module) { module.addDeserializer(ResolvedSchema.class, new ResolvedSchemaJsonDeserializer()); module.addDeserializer( RequiredDistribution.class, new RequiredDistributionJsonDeserializer()); + module.addDeserializer(TableDistribution.class, new TableDistributionJsonDeserializer()); } private static void registerMixins(SimpleModule module) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableJsonDeserializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableJsonDeserializer.java index 86ce77e64cd84..a41df15f897f5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableJsonDeserializer.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableJsonDeserializer.java @@ -23,6 +23,7 @@ import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext; @@ -33,10 +34,12 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import static org.apache.flink.table.planner.plan.nodes.exec.serde.JsonSerdeUtil.deserializeOptionalField; import static org.apache.flink.table.planner.plan.nodes.exec.serde.JsonSerdeUtil.traverse; import static org.apache.flink.table.planner.plan.nodes.exec.serde.ResolvedCatalogTableJsonSerializer.COMMENT; +import static org.apache.flink.table.planner.plan.nodes.exec.serde.ResolvedCatalogTableJsonSerializer.DISTRIBUTION; import static org.apache.flink.table.planner.plan.nodes.exec.serde.ResolvedCatalogTableJsonSerializer.OPTIONS; import static org.apache.flink.table.planner.plan.nodes.exec.serde.ResolvedCatalogTableJsonSerializer.PARTITION_KEYS; import static org.apache.flink.table.planner.plan.nodes.exec.serde.ResolvedCatalogTableJsonSerializer.RESOLVED_SCHEMA; @@ -63,6 +66,15 @@ public ResolvedCatalogTable deserialize(JsonParser jsonParser, DeserializationCo ctx.readValue( traverse(jsonNode.required(RESOLVED_SCHEMA), jsonParser.getCodec()), ResolvedSchema.class); + + Optional distribution = + deserializeOptionalField( + jsonNode, + DISTRIBUTION, + TableDistribution.class, + jsonParser.getCodec(), + ctx); + List partitionKeys = ctx.readValue( traverse(jsonNode.required(PARTITION_KEYS), jsonParser.getCodec()), @@ -85,13 +97,15 @@ public ResolvedCatalogTable deserialize(JsonParser jsonParser, DeserializationCo .orElse(Collections.emptyMap()); return new ResolvedCatalogTable( - CatalogTable.of( - // Create the unresolved schema from the resolved one. We do this for safety - // reason, in case one tries to access the unresolved schema. - Schema.newBuilder().fromResolvedSchema(resolvedSchema).build(), - comment, - partitionKeys, - options), + // Create the unresolved schema from the resolved one. We do this for safety + // reason, in case one tries to access the unresolved schema. + CatalogTable.newBuilder() + .schema(Schema.newBuilder().fromResolvedSchema(resolvedSchema).build()) + .comment(comment) + .distribution(distribution.orElse(null)) + .partitionKeys(partitionKeys) + .options(options) + .build(), resolvedSchema); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableJsonSerializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableJsonSerializer.java index 5cab5cb5c0d7f..5bf7e1002f53d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableJsonSerializer.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableJsonSerializer.java @@ -44,6 +44,7 @@ final class ResolvedCatalogTableJsonSerializer extends StdSerializer { + private static final long serialVersionUID = 1L; + + TableDistributionJsonDeserializer() { + super(TableDistribution.class); + } + + @Override + public TableDistribution deserialize(JsonParser jsonParser, DeserializationContext ctx) + throws IOException { + JsonNode jsonNode = jsonParser.readValueAsTree(); + + if (!(jsonNode instanceof ObjectNode)) { + return null; + } + + TableDistribution.Kind kind = + ctx.readValue( + traverse(jsonNode.required(KIND), jsonParser.getCodec()), + TableDistribution.Kind.class); + Optional bucketCount = + deserializeOptionalField( + (ObjectNode) jsonNode, + BUCKET_COUNT, + Integer.class, + jsonParser.getCodec(), + ctx); + List bucketKeys = + ctx.readValue( + traverse(jsonNode.required(BUCKET_KEYS), jsonParser.getCodec()), + ctx.getTypeFactory().constructCollectionType(List.class, String.class)); + + return TableDistribution.of(kind, bucketCount.orElseGet(() -> null), bucketKeys); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/TableDistributionJsonSerializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/TableDistributionJsonSerializer.java new file mode 100644 index 0000000000000..d2acd8d1c7ab1 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/TableDistributionJsonSerializer.java @@ -0,0 +1,65 @@ +/* + * 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.planner.plan.nodes.exec.serde; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.catalog.TableDistribution; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer; + +import java.io.IOException; + +import static org.apache.flink.table.planner.plan.nodes.exec.serde.JsonSerdeUtil.serializeOptionalField; + +/** + * JSON serializer for {@link TableDistribution}. + * + * @see TableDistributionJsonDeserializer for the reverse operation + */ +@Internal +final class TableDistributionJsonSerializer extends StdSerializer { + private static final long serialVersionUID = 1L; + + static final String KIND = "kind"; + static final String BUCKET_COUNT = "bucketCount"; + static final String BUCKET_KEYS = "bucketKeys"; + + TableDistributionJsonSerializer() { + super(TableDistribution.class); + } + + @Override + public void serialize( + TableDistribution distribution, + JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) + throws IOException { + jsonGenerator.writeStartObject(); + + serializerProvider.defaultSerializeField(KIND, distribution.getKind(), jsonGenerator); + serializeOptionalField( + jsonGenerator, BUCKET_COUNT, distribution.getBucketCount(), serializerProvider); + serializerProvider.defaultSerializeField( + BUCKET_KEYS, distribution.getBucketKeys(), jsonGenerator); + + jsonGenerator.writeEndObject(); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TableFactoryHarness.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TableFactoryHarness.java index 4f1bd0fbd55ef..1df879d2dd0a8 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TableFactoryHarness.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TableFactoryHarness.java @@ -189,6 +189,7 @@ private HarnessTableDescriptor( super( schema, Collections.singletonMap(FactoryUtil.CONNECTOR.key(), IDENTIFIER), + null, Collections.emptyList(), null); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java index 4ed16ba684aae..3ebaa4a3a5cbf 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java @@ -44,6 +44,7 @@ import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.WatermarkSpec; import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.ProviderContext; import org.apache.flink.table.connector.RuntimeConverter; @@ -51,6 +52,7 @@ import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.sink.OutputFormatProvider; import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.connector.sink.abilities.SupportsBucketing; import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; import org.apache.flink.table.connector.sink.abilities.SupportsWritingMetadata; @@ -434,6 +436,14 @@ private static RowKind parseRowKind(String rowKindShortString) { .booleanType() .defaultValue(false) .withDescription("Option to determine whether to discard the late event."); + + private static final ConfigOption SINK_BUCKET_COUNT_REQUIRED = + ConfigOptions.key("sink.bucket-count-required") + .booleanType() + .defaultValue(false) + .withDescription( + "Option to determine whether or not to require the distribution bucket count"); + private static final ConfigOption SOURCE_NUM_ELEMENT_TO_SKIP = ConfigOptions.key("source.num-element-to-skip") .intType() @@ -683,6 +693,7 @@ public DynamicTableSink createDynamicTableSink(Context context) { TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema()); + boolean requireBucketCount = helper.getOptions().get(SINK_BUCKET_COUNT_REQUIRED); if (sinkClass.equals("DEFAULT")) { int rowTimeIndex = validateAndExtractRowtimeIndex( @@ -698,7 +709,8 @@ public DynamicTableSink createDynamicTableSink(Context context) { parallelism, changelogMode, rowTimeIndex, - tableSchema); + tableSchema, + requireBucketCount); } else { try { return InstantiationUtil.instantiate( @@ -747,6 +759,7 @@ public Set> optionalOptions() { WRITABLE_METADATA, ENABLE_WATERMARK_PUSH_DOWN, SINK_DROP_LATE_EVENT, + SINK_BUCKET_COUNT_REQUIRED, SOURCE_NUM_ELEMENT_TO_SKIP, SOURCE_SLEEP_AFTER_ELEMENTS, SOURCE_SLEEP_TIME, @@ -1958,7 +1971,8 @@ private static class TestValuesTableSink implements DynamicTableSink, SupportsWritingMetadata, SupportsPartitioning, - SupportsOverwrite { + SupportsOverwrite, + SupportsBucketing { private DataType consumedDataType; private int[] primaryKeyIndices; @@ -1971,6 +1985,7 @@ private static class TestValuesTableSink private final ChangelogMode changelogModeEnforced; private final int rowtimeIndex; private final TableSchema tableSchema; + private final boolean requireBucketCount; private TestValuesTableSink( DataType consumedDataType, @@ -1983,7 +1998,8 @@ private TestValuesTableSink( @Nullable Integer parallelism, @Nullable ChangelogMode changelogModeEnforced, int rowtimeIndex, - TableSchema tableSchema) { + TableSchema tableSchema, + boolean requireBucketCount) { this.consumedDataType = consumedDataType; this.primaryKeyIndices = primaryKeyIndices; this.tableName = tableName; @@ -1995,6 +2011,7 @@ private TestValuesTableSink( this.changelogModeEnforced = changelogModeEnforced; this.rowtimeIndex = rowtimeIndex; this.tableSchema = tableSchema; + this.requireBucketCount = requireBucketCount; } @Override @@ -2131,7 +2148,8 @@ public DynamicTableSink copy() { parallelism, changelogModeEnforced, rowtimeIndex, - tableSchema); + tableSchema, + requireBucketCount); } @Override @@ -2159,6 +2177,16 @@ public boolean requiresPartitionGrouping(boolean supportsGrouping) { @Override public void applyOverwrite(boolean overwrite) {} + + public Set listAlgorithms() { + return new HashSet<>( + Arrays.asList(TableDistribution.Kind.UNKNOWN, TableDistribution.Kind.HASH)); + } + + @Override + public boolean requiresBucketCount() { + return requireBucketCount; + } } /** A TableSink used for testing the implementation of {@link SinkFunction.Context}. */ diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/MergeTableLikeUtilTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/MergeTableLikeUtilTest.java index ee8cb485dab30..785b08ca18ce4 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/MergeTableLikeUtilTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/MergeTableLikeUtilTest.java @@ -32,6 +32,7 @@ import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.DataTypeFactory; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.calcite.FlinkTypeSystem; import org.apache.flink.table.planner.utils.PlannerMocks; @@ -57,6 +58,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; @@ -752,6 +754,65 @@ void mergeExcludingConstraintsOnDuplicate() { assertThat(mergedSchema).isEqualTo(expectedSchema); } + @Test + void mergeDistributionFromBaseTable() { + Optional sourceDistribution = + Optional.of(TableDistribution.ofHash(Collections.singletonList("a"), 3)); + Optional mergePartitions = + util.mergeDistribution( + getDefaultMergingStrategies().get(FeatureOption.DISTRIBUTION), + sourceDistribution, + Optional.empty()); + + assertThat(mergePartitions).isEqualTo(sourceDistribution); + } + + @Test + void mergeDistributionFromDerivedTable() { + Optional derivedDistribution = + Optional.of(TableDistribution.ofHash(Collections.singletonList("a"), 3)); + Optional mergePartitions = + util.mergeDistribution( + getDefaultMergingStrategies().get(FeatureOption.DISTRIBUTION), + Optional.empty(), + derivedDistribution); + + assertThat(mergePartitions).isEqualTo(derivedDistribution); + } + + @Test + void mergeIncludingDistributionFailsOnDuplicate() { + Optional sourceDistribution = + Optional.of(TableDistribution.ofHash(Collections.singletonList("a"), 3)); + Optional derivedDistribution = + Optional.of(TableDistribution.ofHash(Collections.singletonList("b"), 3)); + + assertThatThrownBy( + () -> + util.mergeDistribution( + MergingStrategy.INCLUDING, + sourceDistribution, + derivedDistribution)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "The base table already has a distribution defined. You might want " + + "to specify EXCLUDING DISTRIBUTION."); + } + + @Test + void mergeExcludingDistributionOnDuplicate() { + Optional sourceDistribution = + Optional.of(TableDistribution.ofHash(Collections.singletonList("a"), 3)); + Optional derivedDistribution = + Optional.of(TableDistribution.ofHash(Collections.singletonList("b"), 3)); + + Optional mergedPartitions = + util.mergeDistribution( + MergingStrategy.EXCLUDING, sourceDistribution, derivedDistribution); + + assertThat(mergedPartitions).isEqualTo(derivedDistribution); + } + @Test void mergePartitionsFromBaseTable() { List sourcePartitions = Arrays.asList("col1", "col2"); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlDdlToOperationConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlDdlToOperationConverterTest.java index c88140e5780b8..d3e272bb944a2 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlDdlToOperationConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlDdlToOperationConverterTest.java @@ -41,10 +41,12 @@ import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.TableChange; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; import org.apache.flink.table.expressions.SqlCallExpression; import org.apache.flink.table.factories.TestManagedTableFactory; +import org.apache.flink.table.operations.CreateTableASOperation; import org.apache.flink.table.operations.NopOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.SinkModifyOperation; @@ -90,6 +92,8 @@ import static org.apache.flink.table.planner.utils.OperationMatchers.entry; import static org.apache.flink.table.planner.utils.OperationMatchers.isCreateTableOperation; import static org.apache.flink.table.planner.utils.OperationMatchers.partitionedBy; +import static org.apache.flink.table.planner.utils.OperationMatchers.withDistribution; +import static org.apache.flink.table.planner.utils.OperationMatchers.withNoDistribution; import static org.apache.flink.table.planner.utils.OperationMatchers.withOptions; import static org.apache.flink.table.planner.utils.OperationMatchers.withSchema; import static org.assertj.core.api.Assertions.assertThat; @@ -510,6 +514,156 @@ public void testMergingCreateTableLike() { partitionedBy("a", "f0")))); } + @Test + public void testMergingCreateTableLikeExcludingDistribution() { + Map sourceProperties = new HashMap<>(); + sourceProperties.put("format.type", "json"); + CatalogTable catalogTable = + CatalogTable.newBuilder() + .schema( + Schema.newBuilder() + .column("f0", DataTypes.INT().notNull()) + .column("f1", DataTypes.TIMESTAMP(3)) + .columnByExpression("f2", "`f0` + 12345") + .watermark("f1", "`f1` - interval '1' second") + .build()) + .distribution(TableDistribution.ofHash(Collections.singletonList("f0"), 3)) + .partitionKeys(Arrays.asList("f0", "f1")) + .options(sourceProperties) + .build(); + + catalogManager.createTable( + catalogTable, ObjectIdentifier.of("builtin", "default", "sourceTable"), false); + + final String sql = + "create table derivedTable(\n" + + " a int,\n" + + " watermark for f1 as `f1` - interval '5' second\n" + + ")\n" + + "DISTRIBUTED BY (a, f0)\n" + + "with (\n" + + " 'connector.type' = 'kafka'" + + ")\n" + + "like sourceTable (\n" + + " EXCLUDING GENERATED\n" + + " EXCLUDING DISTRIBUTION\n" + + " EXCLUDING PARTITIONS\n" + + " OVERWRITING OPTIONS\n" + + " OVERWRITING WATERMARKS" + + ")"; + Operation operation = parseAndConvert(sql); + + assertThat(operation) + .is( + new HamcrestCondition<>( + isCreateTableOperation( + withDistribution( + TableDistribution.ofUnknown( + Arrays.asList("a", "f0"), null)), + withSchema( + Schema.newBuilder() + .column("f0", DataTypes.INT().notNull()) + .column("f1", DataTypes.TIMESTAMP(3)) + .column("a", DataTypes.INT()) + .watermark( + "f1", "`f1` - INTERVAL '5' SECOND") + .build()), + withOptions( + entry("connector.type", "kafka"), + entry("format.type", "json"))))); + } + + @Test + public void testCreateTableValidDistribution() { + final String sql = + "create table derivedTable(\n" + " a int\n" + ")\n" + "DISTRIBUTED BY (a)"; + Operation operation = parseAndConvert(sql); + assertThat(operation) + .is( + new HamcrestCondition<>( + isCreateTableOperation( + withDistribution( + TableDistribution.ofUnknown( + Collections.singletonList("a"), null))))); + } + + @Test + public void testCreateTableInvalidDistribution() { + final String sql = + "create table derivedTable(\n" + " a int\n" + ")\n" + "DISTRIBUTED BY (f3)"; + + assertThatThrownBy(() -> parseAndConvert(sql)) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Invalid bucket key 'f3'. A bucket key for a distribution must reference a physical column in the schema. Available columns are: [a]"); + } + + @Test + public void testMergingCreateTableAsWitDistribution() { + Map sourceProperties = new HashMap<>(); + sourceProperties.put("format.type", "json"); + CatalogTable catalogTable = + CatalogTable.newBuilder() + .schema( + Schema.newBuilder() + .column("f0", DataTypes.INT().notNull()) + .column("f1", DataTypes.TIMESTAMP(3)) + .columnByExpression("f2", "`f0` + 12345") + .watermark("f1", "`f1` - interval '1' second") + .build()) + .distribution(TableDistribution.ofHash(Collections.singletonList("f0"), 3)) + .partitionKeys(Arrays.asList("f0", "f1")) + .options(sourceProperties) + .build(); + + catalogManager.createTable( + catalogTable, ObjectIdentifier.of("builtin", "default", "sourceTable"), false); + + final String sql = + "create table derivedTable DISTRIBUTED BY (f0) AS SELECT * FROM sourceTable"; + assertThatThrownBy(() -> parseAndConvert(sql)) + .isInstanceOf(SqlValidateException.class) + .hasMessageContaining( + "CREATE TABLE AS SELECT syntax does not support creating distributed tables yet."); + } + + @Test + public void testMergingCreateTableAsWitEmptyDistribution() { + Map sourceProperties = new HashMap<>(); + sourceProperties.put("format.type", "json"); + CatalogTable catalogTable = + CatalogTable.newBuilder() + .schema( + Schema.newBuilder() + .column("f0", DataTypes.INT().notNull()) + .column("f1", DataTypes.TIMESTAMP(3)) + .columnByExpression("f2", "`f0` + 12345") + .watermark("f1", "`f1` - interval '1' second") + .build()) + .distribution(TableDistribution.ofHash(Collections.singletonList("f0"), 3)) + .partitionKeys(Arrays.asList("f0", "f1")) + .options(sourceProperties) + .build(); + + catalogManager.createTable( + catalogTable, ObjectIdentifier.of("builtin", "default", "sourceTable"), false); + + final String sql = "create table derivedTable AS SELECT * FROM sourceTable"; + Operation ctas = parseAndConvert(sql); + Operation operation = ((CreateTableASOperation) ctas).getCreateTableOperation(); + assertThat(operation) + .is( + new HamcrestCondition<>( + isCreateTableOperation( + withNoDistribution(), + withSchema( + Schema.newBuilder() + .column("f0", DataTypes.INT().notNull()) + .column("f1", DataTypes.TIMESTAMP(3)) + .column("f2", DataTypes.INT().notNull()) + .build())))); + } + @Test public void testCreateTableInvalidPartition() { final String sql = diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ContextResolvedTableSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ContextResolvedTableSerdeTest.java index 7c8f339615c34..a8c8ecbacbd5e 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ContextResolvedTableSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ContextResolvedTableSerdeTest.java @@ -33,6 +33,7 @@ import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.utils.CatalogManagerMocks; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; @@ -99,12 +100,20 @@ public class ContextResolvedTableSerdeTest { PLAN_OPTIONS.put("d", "4"); } + private static final TableDistribution DISTRIBUTION = + TableDistribution.ofHash(Collections.singletonList("a"), 6); + private static final List PARTITION_KEYS = Collections.singletonList("a"); private static final ResolvedCatalogTable RESOLVED_CATALOG_TABLE = new ResolvedCatalogTable( - CatalogTable.of( - CATALOG_TABLE_SCHEMA, "my comment", PARTITION_KEYS, CATALOG_OPTIONS), + CatalogTable.newBuilder() + .schema(CATALOG_TABLE_SCHEMA) + .comment("my comment") + .distribution(DISTRIBUTION) + .partitionKeys(PARTITION_KEYS) + .options(CATALOG_OPTIONS) + .build(), CATALOG_TABLE_RESOLVED_SCHEMA); // Mock catalog @@ -125,11 +134,13 @@ public class ContextResolvedTableSerdeTest { private static final ContextResolvedTable ANONYMOUS_CONTEXT_RESOLVED_TABLE = ContextResolvedTable.anonymous( new ResolvedCatalogTable( - CatalogTable.of( - CATALOG_TABLE_SCHEMA, - "my comment", - PARTITION_KEYS, - PLAN_OPTIONS), + CatalogTable.newBuilder() + .schema(CATALOG_TABLE_SCHEMA) + .comment("my comment") + .distribution(DISTRIBUTION) + .partitionKeys(PARTITION_KEYS) + .options(PLAN_OPTIONS) + .build(), CATALOG_TABLE_RESOLVED_SCHEMA)); private static final ContextResolvedTable TEMPORARY_CATALOG_CONTEXT_RESOLVED_TABLE = @@ -138,11 +149,13 @@ public class ContextResolvedTableSerdeTest { ContextResolvedTable.temporary( TEMPORARY_TABLE_IDENTIFIER, new ResolvedCatalogTable( - CatalogTable.of( - CATALOG_TABLE_SCHEMA, - "my comment", - PARTITION_KEYS, - PLAN_OPTIONS), + CatalogTable.newBuilder() + .schema(CATALOG_TABLE_SCHEMA) + .comment("my comment") + .distribution(DISTRIBUTION) + .partitionKeys(PARTITION_KEYS) + .options(PLAN_OPTIONS) + .build(), CATALOG_TABLE_RESOLVED_SCHEMA)); private static final ContextResolvedTable PERMANENT_CATALOG_CONTEXT_RESOLVED_TABLE = @@ -152,11 +165,13 @@ public class ContextResolvedTableSerdeTest { PERMANENT_TABLE_IDENTIFIER, CATALOG_MANAGER.getCatalog(DEFAULT_CATALOG).orElseThrow(AssertionError::new), new ResolvedCatalogTable( - CatalogTable.of( - CATALOG_TABLE_SCHEMA, - "my comment", - PARTITION_KEYS, - PLAN_OPTIONS), + CatalogTable.newBuilder() + .schema(CATALOG_TABLE_SCHEMA) + .comment("my comment") + .distribution(DISTRIBUTION) + .partitionKeys(PARTITION_KEYS) + .options(PLAN_OPTIONS) + .build(), CATALOG_TABLE_RESOLVED_SCHEMA)); @Test @@ -204,11 +219,12 @@ void temporaryTableAndMissingIdentifierInCatalog() throws Exception { ContextResolvedTable.temporary( objectIdentifier, new ResolvedCatalogTable( - CatalogTable.of( - CATALOG_TABLE_SCHEMA, - "my amazing table", - Collections.emptyList(), - PLAN_OPTIONS), + CatalogTable.newBuilder() + .schema(CATALOG_TABLE_SCHEMA) + .comment("my amazing table") + .partitionKeys(Collections.emptyList()) + .options(PLAN_OPTIONS) + .build(), CATALOG_TABLE_RESOLVED_SCHEMA)); final byte[] actualSerialized = createObjectWriter(ctx).writeValueAsBytes(spec); @@ -456,6 +472,12 @@ void withPermanentTable() throws Exception { assertThat(result.f1.isPermanent()).isTrue(); assertThat(result.f1.getIdentifier()).isEqualTo(PERMANENT_TABLE_IDENTIFIER); assertThat(result.f1.getResolvedSchema()).isEqualTo(CATALOG_TABLE_RESOLVED_SCHEMA); + assertThat( + result.f1 + .getResolvedTable() + .getDistribution() + .get()) + .isEqualTo(DISTRIBUTION); assertThat(result.f1.getResolvedTable().getPartitionKeys()) .isEqualTo(PARTITION_KEYS); assertThat(result.f1.getResolvedTable().getOptions()) @@ -578,11 +600,12 @@ void withPermanentTableAndMissingIdentifierInCatalog() throws Exception { objectIdentifier, CATALOG, new ResolvedCatalogTable( - CatalogTable.of( - CATALOG_TABLE_SCHEMA, - "my amazing table", - Collections.emptyList(), - PLAN_OPTIONS), + CatalogTable.newBuilder() + .schema(CATALOG_TABLE_SCHEMA) + .comment("my amazing table") + .partitionKeys(Collections.emptyList()) + .options(PLAN_OPTIONS) + .build(), CATALOG_TABLE_RESOLVED_SCHEMA)); final Tuple2 result = serDe(ctx, spec); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableSerdeTest.java index 1ba46d4136aac..a77ff7036ed52 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedCatalogTableSerdeTest.java @@ -27,6 +27,7 @@ import org.apache.flink.table.catalog.ExternalCatalogTable; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.catalog.WatermarkSpec; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; @@ -96,11 +97,17 @@ class ResolvedCatalogTableSerdeTest { private static final ResolvedCatalogTable FULL_RESOLVED_CATALOG_TABLE = new ResolvedCatalogTable( - CatalogTable.of( - Schema.newBuilder().fromResolvedSchema(FULL_RESOLVED_SCHEMA).build(), - "my table", - Collections.singletonList("c"), - OPTIONS), + CatalogTable.newBuilder() + .schema( + Schema.newBuilder() + .fromResolvedSchema(FULL_RESOLVED_SCHEMA) + .build()) + .comment("my table") + .distribution( + TableDistribution.ofHash(Collections.singletonList("a"), 1)) + .partitionKeys(Collections.singletonList("c")) + .options(OPTIONS) + .build(), FULL_RESOLVED_SCHEMA); static Stream testResolvedCatalogTableSerde() { @@ -119,11 +126,17 @@ static Stream testResolvedCatalogTableSerde() { return Stream.of( FULL_RESOLVED_CATALOG_TABLE, new ResolvedCatalogTable( - CatalogTable.of( - Schema.newBuilder().fromResolvedSchema(withoutPartitionKey).build(), - null, - Collections.singletonList("c"), - OPTIONS), + CatalogTable.newBuilder() + .schema( + Schema.newBuilder() + .fromResolvedSchema(withoutPartitionKey) + .build()) + .comment(null) + .distribution( + TableDistribution.ofHash(Collections.singletonList("a"), 1)) + .partitionKeys(Collections.singletonList("c")) + .options(OPTIONS) + .build(), withoutPartitionKey)); } @@ -146,23 +159,29 @@ void testDontSerializeOptions() throws IOException { JsonNode actualJson = objectReader.readTree(actualSerialized); assertThatJsonContains(actualJson, ResolvedCatalogTableJsonSerializer.RESOLVED_SCHEMA); assertThatJsonContains(actualJson, ResolvedCatalogTableJsonSerializer.PARTITION_KEYS); + assertThatJsonContains(actualJson, ResolvedCatalogTableJsonSerializer.DISTRIBUTION); assertThatJsonDoesNotContain(actualJson, ResolvedCatalogTableJsonSerializer.OPTIONS); assertThatJsonDoesNotContain(actualJson, ResolvedCatalogTableJsonSerializer.COMMENT); ResolvedCatalogTable actual = objectReader.readValue(actualSerialized, ResolvedCatalogTable.class); - assertThat(actual) - .isEqualTo( - new ResolvedCatalogTable( - CatalogTable.of( + ResolvedCatalogTable expected = + new ResolvedCatalogTable( + CatalogTable.newBuilder() + .schema( Schema.newBuilder() .fromResolvedSchema(FULL_RESOLVED_SCHEMA) - .build(), - null, - Collections.singletonList("c"), - Collections.emptyMap()), - FULL_RESOLVED_SCHEMA)); + .build()) + .comment(null) + .distribution( + TableDistribution.ofHash(Collections.singletonList("a"), 1)) + .partitionKeys(Collections.singletonList("c")) + .options(Collections.emptyMap()) + .build(), + FULL_RESOLVED_SCHEMA); + + assertThat(actual).isEqualTo(expected); } @Test diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkRestoreTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkRestoreTest.java index 1ab3651cf099f..4d888d3e2838b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkRestoreTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkRestoreTest.java @@ -34,6 +34,10 @@ public TableSinkRestoreTest() { @Override public List programs() { return Arrays.asList( + TableSinkTestPrograms.SINK_BUCKETING_WITH_COUNT, + TableSinkTestPrograms.SINK_BUCKETING_WITH_KEYS_AND_COUNT, + TableSinkTestPrograms.SINK_BUCKETING_HASH_WITH_KEYS_AND_COUNT, + TableSinkTestPrograms.SINK_BUCKETING_HASH_WITH_KEYS_AND_WITHOUT_COUNT, TableSinkTestPrograms.SINK_PARTITION, TableSinkTestPrograms.SINK_OVERWRITE, TableSinkTestPrograms.SINK_WRITING_METADATA, diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkTestPrograms.java index 3f5a51922fe80..0a42e821e986a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkTestPrograms.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkTestPrograms.java @@ -18,12 +18,15 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions; import org.apache.flink.table.test.program.SinkTestStep; import org.apache.flink.table.test.program.SourceTestStep; import org.apache.flink.table.test.program.TableTestProgram; import org.apache.flink.types.Row; +import java.util.Arrays; + /** {@link TableTestProgram} definitions for testing {@link StreamExecDeduplicate}. */ public class TableSinkTestPrograms { @@ -35,6 +38,41 @@ public class TableSinkTestPrograms { static final String[] SOURCE_SCHEMA = {"a INT", "b BIGINT", "c VARCHAR"}; + static final TableTestProgram SINK_BUCKETING_WITH_COUNT = + buildBucketingTest("with-count", TableDistribution.ofUnknown(3)); + static final TableTestProgram SINK_BUCKETING_WITH_KEYS_AND_COUNT = + buildBucketingTest( + "with-keys-and-count", TableDistribution.ofUnknown(Arrays.asList("a"), 3)); + static final TableTestProgram SINK_BUCKETING_HASH_WITH_KEYS_AND_COUNT = + buildBucketingTest( + "hash-with-keys-with-count", TableDistribution.ofHash(Arrays.asList("a"), 3)); + static final TableTestProgram SINK_BUCKETING_HASH_WITH_KEYS_AND_WITHOUT_COUNT = + buildBucketingTest( + "range_with_keys_without_count", + TableDistribution.ofHash(Arrays.asList("a"), null)); + + private static TableTestProgram buildBucketingTest( + final String suffix, final TableDistribution distribution) { + return TableTestProgram.of("sink-bucketing_" + suffix, "validates sink bucketing") + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema(SOURCE_SCHEMA) + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a INT", "b BIGINT", "c VARCHAR") + .addDistribution(distribution) + .addPartitionKeys("b") + .consumedBeforeRestore( + "+I[1, 1, hi]", "+I[2, 2, hello]", "+I[3, 2, hello world]") + .consumedAfterRestore("+I[4, 4, foo]", "+I[5, 2, foo bar]") + .build()) + .runSql("INSERT INTO sink_t SELECT * FROM source_t") + .build(); + } + static final TableTestProgram SINK_PARTITION = TableTestProgram.of("sink-partition", "validates sink partition") .setupTableSource( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/OperationMatchers.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/OperationMatchers.java index f2ee9d2f79f95..aae1e8b3640c3 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/OperationMatchers.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/OperationMatchers.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.utils; import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ddl.CreateTableOperation; @@ -31,6 +32,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import static org.hamcrest.CoreMatchers.equalTo; @@ -114,6 +116,33 @@ protected List featureValueOf(CreateTableOperation actual) { }; } + /** + * Checks that {@link CreateTableOperation} is equal to the given {@link TableDistribution}. + * + * @param distribution TableDistribution that the {@link CreateTableOperation} should have + * @see #isCreateTableOperation(Matcher[]) + */ + public static Matcher withDistribution(TableDistribution distribution) { + return new FeatureMatcher>( + equalTo(Optional.ofNullable(distribution)), + "distribution of the derived table", + "distribution") { + @Override + protected Optional featureValueOf(CreateTableOperation actual) { + return actual.getCatalogTable().getDistribution(); + } + }; + } + + /** + * Checks that {@link CreateTableOperation} has no {@link TableDistribution}. + * + * @see #isCreateTableOperation(Matcher[]) + */ + public static Matcher withNoDistribution() { + return withDistribution(null); + } + /** * Checks that the schema of {@link CreateTableOperation} is equal to the given {@link Schema}. * diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.xml index a2ee20136869b..dcc55d73b48a5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.xml @@ -44,6 +44,25 @@ Sink(table=[default_catalog.default_database.sink], fields=[a, b]) ]]> + + + + + + + + + + + + + + + + + + + ", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t]], fields=[a, b, c])", + "inputProperties" : [ ] + }, { + "id" : 6, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "INT" + }, { + "name" : "b", + "dataType" : "BIGINT" + }, { + "name" : "c", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "distribution" : { + "kind" : "HASH", + "bucketCount" : 3, + "bucketKeys" : [ "a" ] + }, + "partitionKeys" : [ "b" ] + } + }, + "abilities" : [ { + "type" : "Bucketing" + } ] + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[a, b, c])" + } ], + "edges" : [ { + "source" : 5, + "target" : 6, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_hash-with-keys-with-count/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_hash-with-keys-with-count/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..d184ba9026ac88871265a180102c0578e1b4d43c GIT binary patch literal 8379 zcmeGhO^g#&__ez%|Lg)AA}f$hSTHD^X{SHivIo;G1*)aHX$zP|U0-Kj+hM0OW#)C+ zB@sa*m|#E?J$O(#a52HeL`l4PQV;4y6R#R$Vq!Eg(F9{O_`TPecDn5@B@ixj($1TC zKi~iR-hAJedZFY)2=&9CU7mV_ypQepQoNuoMY};3tiT zMGMc9s-f!C7b=E=(W0;Dk4A$2U>FyI<6=OTWSPVYKCBcYzK}Am_+kOS;={pEG>oNC zI2Mo)8hH`u8(4{X#)t=vPE9AX@%&VFHkXg*ld}g?xqLP?c`Tor%FG^}I+#R5x#@H& z-yBHtbwkJF>0`;+Tr!)Ar&BY@EcANOal=m6 zs+uJv7br1xtmTNQVog0u%(3>aoQ|tj*`NTs4{iK~DbsV~ceKhf(G$yHVi+vpkV-Vg zI*2LG!~0(!-PrNrh4*K@$dg8!ifqgovM3r#pThH4WMqkH)uIy%Cfc(ibZ7AwSN|M- z&-+Tni@a$xC>#1bs9rVngoZ5uytxio!+J^NT@z`vUNcInj5S6UMN%5@h;@01TwWrV zm&myg?YUyQxn^RaI&Y$pcJeseS>p_dh+TbN)DPG=uak=zTmfu@E=G^y>cmMjXpc`D zmP&zeing9~5j|BnMPv$8_s^-iLeXdkav5F`@IDPQVCJT(4az~f-Gwv_20@e~*eWyh z3sq3#{&r5d!PM1g3qSz)Okj1^kcaDWfYWZBM(2wqdQ`@ z02%J(lbHPq2}9FBWuQuH?3i9L6t$>wrogWM9o!SR`P-EtF1>yYdK}s9zcYX);O&#> z{8!K4+Qq?mWQw-AG~Aq4SMSy)4BA6%N87RiYPD|ETU<7sqo%5tm`&@jDVPKvg=eO5_T0Zv z1-W`~*Or;mX{%+T1qNrM=NxN>C|aG$i83}@=?9$*Xhbj3@=`{Q$Fqqe@$5^tZ{Oxa+d0lhT2!~PSB_JoqRNi< z9C4%yMF7!r6`zN3ay&QT8+TI7;4VT)G7sH zUcB-4&h>Boc$opKa>90b7|>DvECOiPR!SRS0hfP%lBiT^+_^&>XFjI!GgMX+Ti9oL!I*3)t%!hW%j z!|wcO2W#rqSRRFi2ddiSy&A5H<k&|M$W1#A%kHnz)Yw7j)AjOn|8$-JZsXzObW)R{J@%YF6No#zrC*Qv7w{?b^i z7rC&n?q1}gOfL2!mtN!|tQh(rauMmhteW8GY zeb_HazM?Ek{%Fwe4@eOp|5x}!S ztYIixDp1vG(#Epo9Se6h8)9>+=!k=@VKpIkc@X611KkKA=Tw2&@@tpATXX%H;ZJ#W zV;#^~!MyN5QMkQCB^xFY$pVofFfw$p0q9U;nN$CusAFwp7^--*sf0MMC{c8z9aSf; z-E@yZfZhfPWC2OuV4Dya-9m$Ip#$AqgmCMoI<__u&rD@fiFg`Hv9^Xs8!&+Y5XX{Y dbq7;yEMQvb$c>vO7D26rqEGHvZaSGc_&>uD8`b~- literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_range_with_keys_without_count/plan/sink-bucketing_range_with_keys_without_count.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_range_with_keys_without_count/plan/sink-bucketing_range_with_keys_without_count.json new file mode 100644 index 0000000000000..3a8efb39fd177 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_range_with_keys_without_count/plan/sink-bucketing_range_with_keys_without_count.json @@ -0,0 +1,87 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 7, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "INT" + }, { + "name" : "b", + "dataType" : "BIGINT" + }, { + "name" : "c", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t]], fields=[a, b, c])", + "inputProperties" : [ ] + }, { + "id" : 8, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "INT" + }, { + "name" : "b", + "dataType" : "BIGINT" + }, { + "name" : "c", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "distribution" : { + "kind" : "HASH", + "bucketKeys" : [ "a" ] + }, + "partitionKeys" : [ "b" ] + } + }, + "abilities" : [ { + "type" : "Bucketing" + } ] + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[a, b, c])" + } ], + "edges" : [ { + "source" : 7, + "target" : 8, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_range_with_keys_without_count/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_range_with_keys_without_count/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..73f77300b020b934e90f136dcb05d16979526f12 GIT binary patch literal 8379 zcmeGhU5^{p^^SM5B;;c^VO4dPf>vS^1jI9AukE$F50%&J5L3syvYmhysmt}uwRf`P z8E58tvm1#tY1L9GQlV1&fYj1Fga=TNS|KF(1MmPy2p)L=AtWS3E2LJUty-n$T+i6! zwRhtPloxy@&z-p+=lh=Pb58D=T9^9hxW`gm*UbWS3_ zs~RTzv9d`WrD~fxb6lk!QPvXCjaX_eqF%ZXPi$197v>sPtv0V>t6fma%K4^c@JKQq zSEI4{`9vb7s>!)jd|q+sCTrM+#lu=e)oCnTiQ%@0npEc#N-{;Om4wE^v8Y}N$Eq-& z)Z zc4m1gOD0My`CPd(krn$U&ZYBbvumYnF_+HgRE_?aKGz|S z?Sct0xbi83nYz0~IW5EYKmT&`N3UJ|^IC{Vd2*zRbN-ae$wdA<-J}YVrQ{8lFKjvF z_@2=H&2K;dkI9!pzpRHyC{M;U+u8)x8@81(sSALQwg8*dswrYy2#B`-zDOA+!?1Q&|8R>^nvENnG49WvcZo?yFZUjz|h8wrt7z$SQI+%C{MU>grG zdWJR@o+jho{EF=w90=#+zNZ63FIUbpjRV!A8-}HGGSi1##H#?_pTY`Qd8FZi@{vvj zkmkW4n0}hN>xh1<0ct$i%ZV_UmKr?)$N(P&)?$rFLXTsDcJ~6gOeD~lIuM|MBs;)D ztK?q8Ei_FNQRm5iL|(LQ&Pk#V$GB*B1>rtmja+s}>`qC-y&hTQ7$Lyb5j0I1ElAE< z&3bUM!O2K*`ALxB39$(6SI*d`2`U3sy0d4kx~&^kLofw){onE9kvG2j?Szots0ky! z>`vYuKnL*DBDwPJ&%Sw3fRQv#eh|=bXIV?V_jMqr9fxk?`P#br;R7$eL;o~Fq#PM_ z;pl_7z+x}r@g4zw;seRevx!;gPvMF7WCO-A0*0?MS96R;%cze9Y`Vl9!>XZ8TW}~) z0{6o^Y8>B3w;CWlBgu29ZCo-GPPokrjl0Im$g;-XQz zE&Q>?+~a_v16~90gB^x*)XA(G#MG3)`NJ;N z?d_G*+^!p%?>(n~Q+nh-mmZ3Sz|v7zA#iL*JN6Eovl?@beRloEH&>rag1P0$zD;U2 z8JvrW&bmUgUSZC!Z~ozdgTMdu8Ukww!XA_m=!|$*0CeCechmgr9{|AC;U9sg|l-+wNf-R$T z0(WB1NY|AJ=f%4Y2lJzStZ7hVxoe(Nx*ML;Kb4;JzRhv`ehScYJD&W$8-Mftl;k}D zzZ!5GBc#ydHfY0;HM?H7t-v4Y3U!$b2Rq<2y1u#`#>ySQL@yZt+WpxDb+rfTZvXmQ z&U1&4>o?d1e`hQ|wl7M%b^BG1iAUHV?jozsMhmni$X#O?uEhIUN zTy7h=U{Z1hAJ{N#!RGhthWPviJZHokJRqMWn%}Q8)3gb}`VG@9ap^2mjCclMadp&{4^-k47CkW_LMX2*A>TZFozpV zlxqQ2C`9(Jk_p{#HR|XYYH4snn7&F5dqbxv6l39g-2XN#!(9)Kl$sT{S;r9{OsVNM zn5E;;Sk(q&f|mnLOEW1HEp@1Bb!fwEdEdgF$A;2bs`%o-Jy;XMCK1TbM+Ol>$*%(A z_Uq4mwExCalW&UZ#=Sseor=N(C-L4Am1a9kVOvatz{s|gHlS1OZGQcOlfJc)NvPuS zjuO(MqQuGRUR3?K4$?gf0eTN4kOd@p=X!*IbjxkJezuyy0BcxWzu;l p#d;c^X~P5pKpZ=Y)qPB{y#ck*mzy9>7(pF?qEG2;X(d}&`ai*gAJzZ> literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_with-count/plan/sink-bucketing_with-count.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_with-count/plan/sink-bucketing_with-count.json new file mode 100644 index 0000000000000..98c56bec986a6 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_with-count/plan/sink-bucketing_with-count.json @@ -0,0 +1,88 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "INT" + }, { + "name" : "b", + "dataType" : "BIGINT" + }, { + "name" : "c", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t]], fields=[a, b, c])", + "inputProperties" : [ ] + }, { + "id" : 2, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "INT" + }, { + "name" : "b", + "dataType" : "BIGINT" + }, { + "name" : "c", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "distribution" : { + "kind" : "UNKNOWN", + "bucketCount" : 3, + "bucketKeys" : [ ] + }, + "partitionKeys" : [ "b" ] + } + }, + "abilities" : [ { + "type" : "Bucketing" + } ] + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[a, b, c])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_with-count/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_with-count/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..c00b8a3103218ecf6f2e482c1fc9b8d1e608f27a GIT binary patch literal 8379 zcmeHMU5Fb+9G~Q_*Y9519^##CIqbEdHoN({T;hR{yQXb4%^gXuEm!cI>`rpq+w7*h zll1PudaY=Mf^9(`d{B$@MXVr*;EO(1L4C20K2%f?L_t(2h+6+M*-bW=+%-o_Uz&y8 z*`1Hy|8xGc|Cz)K1usIV8*X;Fss(c2{A1S#H~#CrRr&JsD<{8u{5`8}$%O{xU_RiN za83&6i7zkbN5WDhN231WoSX{;Z~(e)aX>Ld8`j0>xX|{DUN$6>CRip$hGI%sCuq2m zS5)$}Siy#v*HvgV#iC!F2?ypv(HXyQ@$_(bE+04@ENO+ph>u-`(_&UURn`<53`fI} zkUts?g`tdu!V$k{;tDD0iblN=pFbQ3c|*P&AvhPp!5|j>!#Ed~gTyQO1K1nN;}LH( z7Y=%TBs`oC=kqw4$H3MvaQCdpHWLk$GBFuX$FdXYnM^j8jn5oRWU}eR*i<$#k(xO= zaWIZNnaN}#TW^T-ah_wb_oYg7N?Si}E zn(!>0Tl5Z}Kk?(x<>WJ;ZG}cA?Go$MU{N>fGl5I5L8yTx3Z+?Asmc0Tp#86~4}|*P zx5{tO7Hi1zt^*eIz`knNfy9IM3&|IiM)0uD8;M3E-cVp9=fz>!=Os~jBpeN4IVfR- z23`dIz%N!7o(XclUE^4jYfzk1iC~m9suYRCJL2<{7#da`j?i4yVE(N?^XYm0H?+<) z&{Hc=V*Y0ehZLg9=0Qwx7QX-b==!b?&%Hb2My@2$c?wss$edP8DkdGB zH_*OS(e1%EF8$f}j{D`J8@ZFHSJJf#gak#`##L+r@b)UOinW5st45P(v#J*q39F1P ziULWXBUY!9$fuIX^(AsCM60iutdAL;FI5aQ(99keJFTCodL`;|qi&$aWgS~c;UZA$ zbqIPCmqt&bUaNglH(_e(8bv!#I+&iwogxwiuDjn$n88tmG9g6!`T&z5D#1{QA0wYp+{{ z8e4YZ`iBvXzIG*N?I>5kl04}yA;+Rsn zD}K*d;xLe?!%G0)-efq2jqzD*H1ZEN=~0PRptH*bITlNgABm-3x^w3aZ|dhXA8jz* z>QOmH^`au#(R1Ln%oBev?hUxX)0wi|5ZJcY{T(DH1!C;@;p(;9=PpFR-4bYB1*>HO z^TJc_mns)?#CY}ko4Yr^@%=>xR^p89aWT+A{w@NvV=5(e@PLayUH*( zb$2osd2FGRxpXoYVb#>9o|1Gj7q&HG=hQ9!1=;^^Cv&kkHLGhF%M4ASlet(sy(=y3 zO?e8-%Ny7Eh9$j|xmcTkYu)qdWGUX>Aikn({<@3k;4Dr8#~SDvZ}&Ck?$s45|=*R0l9_H`*|bJhEa+*pSDu zCV_T{JdL(mRfjMf(871G{jF$", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t]], fields=[a, b, c])", + "inputProperties" : [ ] + }, { + "id" : 4, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "INT" + }, { + "name" : "b", + "dataType" : "BIGINT" + }, { + "name" : "c", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "distribution" : { + "kind" : "UNKNOWN", + "bucketCount" : 3, + "bucketKeys" : [ "a" ] + }, + "partitionKeys" : [ "b" ] + } + }, + "abilities" : [ { + "type" : "Bucketing" + } ] + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[a, b, c])" + } ], + "edges" : [ { + "source" : 3, + "target" : 4, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_with-keys-and-count/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-sink_1/sink-bucketing_with-keys-and-count/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..2afd1f29a7ec004e80729cc387ca63366cfa2da4 GIT binary patch literal 8379 zcmeHM+lw1j7(Yq2+sn4yZMzn`wq>E)g4)dN=AQ0TcAK`*G`l3}R<`1LGINqlcQTXC zoNRXs_F~Zr6|0~RMbtjj2ce=MqJMxWczICY6jX{Jh=O3TC`$d#$xJfYWVcyb`qB){ zoSAd^eb;l&eCJC%UGO4=y5MG)t6CuUzOC>5_4VcLH$Hm(`?o7UJdm{7&a!s9lZ%aV zqkdD-X1xZX29_w4W<{kY>*rw)gtk3rmG4;PS7^O8WO3I43wmI8wd+8_gV8uIx{kS>O?R!oA;gYm$gD6Jj|}b2{9`kuV@M#@kb)QsDC&T z2!;aTP&gV1izc2UWnIyzH%drI4tgU&f7B1HBpQy2;ebCP`~5yI2?S+tARGvLF&U0} z{ZWa8{Qj^M4M_+MJOli7FDE<`#DxYYCgbT?b|O8K$;Ptrnf-}OHk}wdl1)scW)4s6 zkE6cKWHOPhH^lk4zNuL9NPH#}PbXr@#B@9jwQe+}8wCNEu~Z~NUInEK6z5bT7!{2w zB_hy;GO18SHHG**C5DF83^5d}DyN7s)LfO(aM>*C6tF#L!!OL3u4})eRi=R+U4jy$ zU38+5e{de94X6Nz^Op+8mf((Y0|En*hA23anzSAo8lwB&)4j~jIXH7@JeLJF6FTCYRU!?-+p z4E0*=le(!;V4R{Y#~e&g33r&P87&0}fX@_G6%Dz#9lJU2<_UBfxxiw>Z2>tnSqF1Zqjidz zs;DX>okVLG@wBc}ib5?UW}TKc5E}=gq0gG-kacLVTh+W<>c}y}zzb}C&7b>b$K}uP8y@6Jpe_@pKBx;k z_9UC$9*`&4mfkF%7=rQuo6(kTz&SkN_%bmiLn&9Cdb7i)Gt^MD0`qANCI!>LL-5Wl z&fdG`%b-^mZrd{x8g2AUG|zB0xK6X8kD}FyR5pGvp5~4^z`%0=F197&m{Pbce)m}7 zAdslTO90>AWSGLnco7?o{DV!pRiYJWaj_s%vGn+%So+MZTeo=AHcs=w2GcFSD#xf^ zQY3rz9C#t~@IR+^`P|^?Oj&MN*f!Pu9V90OVr;&C$f?l}#_IGQ;Y0Rv^&r%L^gmEcb&Jg|VeIBj5qz0x$FV0i)ii90 zFkh_gu)TG(g*Ub9Eca~Xblr=U)1H-<_U>dl-k%DzmK{rfcW1r1KQ+19v93Cn4G&5+ zEgQJ32$EhZ>6+t5I)_anz{GYe8Vz3!3&!LMFxe{U0B!tiK%H(vUFxsDx_B;H<62cV z;4hBFI*|+e>+VD@^4LNra_K}a!m^=HJtXNwF6?N;j;UMx3$p*;PUO;fa>XJ;Q|Lr4 zw>i#hic?rz-nh>6uR_askB&j%akH3hYEKd_}XjfjiJ-Cvv%Mw5s*(frdf2dEdd4x=}j} zt>fA>?eZSbw%LX9^Wyo@PhR})@57NNE!7p0|p4@YxO&PsnNF+@#WZp6q(`ugJUZ>^qA(`MX33)>} z?DGbK(ug;QeK{`)`vO?TBLv!j{a@jRwA>PGGtusLPR?kG&X+z%!0eD-7+auZ(ojl} z6PhFZ+}?2xuw3J!EuLG;epp9EktS~JV^GK$O0zueDU8=%Ck?$!45|>Gt_3W&8?Bi} zeX?Ro*pSDuCcz9L@-*6DRUN>PjD_!B`&-czv*@hJRB~pe#AM2!H`W13oZEODuhQn*|D|bvD8E=F&;}oD%P^_U=0%( i0Cg-%R=04)+5l#SmeM$RViD9@Ncv=sWG3UO{r>}IuOZ0* literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala index fc6416b605e87..200d151596e72 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala @@ -1020,6 +1020,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends TableITCaseBase { | watermark for ts1 as cast(timestampadd(hour, 8, ts1) as timestamp(3)), | constraint test_constraint primary key (a, b) not enforced |) comment 'test show create table statement' + |distributed by (a) |partitioned by (b,h) |with ( | 'connector' = 'kafka', @@ -1043,6 +1044,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends TableITCaseBase { | WATERMARK FOR `ts1` AS CAST(TIMESTAMPADD(HOUR, 8, `ts1`) AS TIMESTAMP(3)), | CONSTRAINT `test_constraint` PRIMARY KEY (`a`, `b`) NOT ENFORCED |) COMMENT 'test show create table statement' + |DISTRIBUTED BY (`a`) |PARTITIONED BY (`b`, `h`) |WITH ( | 'connector' = 'kafka', @@ -1079,6 +1081,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends TableITCaseBase { | watermark for ts1 as cast(timestampadd(hour, 8, ts1) as timestamp(3)), | constraint test_constraint primary key (pk1, pk2) not enforced |) comment 'test show create table statement' + |distributed into 5 buckets |partitioned by (h) |with ( | 'connector' = 'kafka', @@ -1103,6 +1106,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends TableITCaseBase { | WATERMARK FOR `ts1` AS CAST(TIMESTAMPADD(HOUR, 8, `ts1`) AS TIMESTAMP(3)), | CONSTRAINT `test_constraint` PRIMARY KEY (`pk1`, `pk2`) NOT ENFORCED |) COMMENT 'test show create table statement' + |DISTRIBUTED INTO 5 BUCKETS |PARTITIONED BY (`h`) |WITH ( | 'connector' = 'kafka', @@ -1114,6 +1118,41 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends TableITCaseBase { assertEquals(expectedDDL, row.getField(0).toString) } + @TestTemplate + def testCreateTableAndShowCreateTableWithDistributionAlgorithm(): Unit = { + val executedDDL = + """ + |create temporary table TBL1 ( + | a bigint not null, + | h string, + | b string not null + |) comment 'test show create table statement' + |distributed by range(a) into 7 buckets + |partitioned by (b,h) + |with ( + | 'connector' = 'kafka', + | 'kafka.topic' = 'log.test' + |) + |""".stripMargin + + val expectedDDL = + """ |CREATE TEMPORARY TABLE `default_catalog`.`default_database`.`TBL1` ( + | `a` BIGINT NOT NULL, + | `h` VARCHAR(2147483647), + | `b` VARCHAR(2147483647) NOT NULL + |) COMMENT 'test show create table statement' + |DISTRIBUTED BY RANGE(`a`) INTO 7 BUCKETS + |PARTITIONED BY (`b`, `h`) + |WITH ( + | 'connector' = 'kafka', + | 'kafka.topic' = 'log.test' + |) + |""".stripMargin + tableEnv.executeSql(executedDDL) + val row = tableEnv.executeSql("SHOW CREATE TABLE `TBL1`").collect().next() + assertEquals(expectedDDL, row.getField(0)) + } + @TestTemplate def testCreateViewAndShowCreateTable(): Unit = { val createTableDDL = diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala index 3a950be62bfd5..8a00d9be21967 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala @@ -93,6 +93,23 @@ class TableSinkTest extends TableTestBase { util.verifyExecPlan(stmtSet) } + @Test + def testDistribution(): Unit = { + util.addTable(s""" + |CREATE TABLE sink ( + | `a` INT, + | `b` BIGINT + |) DISTRIBUTED BY ( + | `b` + |) WITH ( + | 'connector' = 'values' + |) + |""".stripMargin) + val stmtSet = util.tableEnv.createStatementSet() + stmtSet.addInsertSql("INSERT INTO sink SELECT a,b FROM MyTable ORDER BY a") + util.verifyExecPlan(stmtSet) + } + @Test def testTableHints(): Unit = { util.tableEnv.executeSql(s""" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala index e230b39a3850b..a6efbc9856e81 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala @@ -799,6 +799,65 @@ class TableSinkTest extends TableTestBase { util.verifyRelPlan(stmtSet, ExplainDetail.CHANGELOG_MODE) } + @Test + def testDistribution(): Unit = { + util.addTable(s""" + |CREATE TABLE sink ( + | `a` INT, + | `b` BIGINT + |) DISTRIBUTED BY ( + | `b` + |) WITH ( + | 'connector' = 'values' + |) + |""".stripMargin) + val stmtSet = util.tableEnv.createStatementSet() + stmtSet.addInsertSql("INSERT INTO sink SELECT a,b FROM MyTable ORDER BY a") + util.verifyExecPlan(stmtSet) + } + + @Test + def testDistributionWithRequiredBucketCount(): Unit = { + util.addTable(s""" + |CREATE TABLE sink ( + | `a` INT, + | `b` BIGINT + |) DISTRIBUTED BY ( + | `b` + |) WITH ( + | 'connector' = 'values', + | 'sink.bucket-count-required' = 'true' + |) + |""".stripMargin) + val stmtSet = util.tableEnv.createStatementSet() + stmtSet.addInsertSql("INSERT INTO sink SELECT a,b FROM MyTable ORDER BY a") + + Assertions + .assertThatThrownBy(() => util.verifyExecPlan(stmtSet)) + .hasMessageContaining( + "Table 'default_catalog.default_database.sink' is a bucketed table, but the underlying DynamicTableSink requires the number of buckets to be set.") + } + + @Test + def testDistributionWithUnsupportedDistributionAlgorithm(): Unit = { + util.addTable(s""" + |CREATE TABLE sink ( + | `a` INT, + | `b` BIGINT + |) DISTRIBUTED BY RANGE ( + | `b` + |) WITH ( + | 'connector' = 'values' + |) + |""".stripMargin) + val stmtSet = util.tableEnv.createStatementSet() + stmtSet.addInsertSql("INSERT INTO sink SELECT a,b FROM MyTable ORDER BY a") + Assertions + .assertThatThrownBy(() => util.verifyExecPlan(stmtSet)) + .hasMessageContaining( + "Table 'default_catalog.default_database.sink' is a bucketed table and it supports [HASH, UNKNOWN], but algorithm RANGE was requested.") + } + @Test def testCreateTableAsSelect(): Unit = { // TODO: support explain CreateTableASOperation