diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-sql/src/test/java/org/apache/flink/table/sql/CreateTableAsITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-sql/src/test/java/org/apache/flink/table/sql/CreateTableAsITCase.java index 3d86251def15a..06fbe73dd6639 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-sql/src/test/java/org/apache/flink/table/sql/CreateTableAsITCase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-sql/src/test/java/org/apache/flink/table/sql/CreateTableAsITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.DefaultIndex; +import org.apache.flink.table.catalog.ImmutableColumnsConstraint; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.test.util.SQLJobSubmission; @@ -46,7 +47,9 @@ public class CreateTableAsITCase extends SqlITCaseBase { Collections.emptyList(), UniqueConstraint.primaryKey("pk", Collections.singletonList("user_name")), Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("user_name")))); + DefaultIndex.newIndex("idx", Collections.singletonList("user_name"))), + ImmutableColumnsConstraint.immutableColumns( + "imt", Collections.singletonList("user_name"))); private static final DebeziumJsonDeserializationSchema DESERIALIZATION_SCHEMA = createDebeziumDeserializationSchema(SINK_TABLE_SCHEMA); diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-sql/src/test/java/org/apache/flink/table/sql/PlannerScalaFreeITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-sql/src/test/java/org/apache/flink/table/sql/PlannerScalaFreeITCase.java index 58d13ac589587..dbdbae5b551a9 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-sql/src/test/java/org/apache/flink/table/sql/PlannerScalaFreeITCase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-sql/src/test/java/org/apache/flink/table/sql/PlannerScalaFreeITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.DefaultIndex; +import org.apache.flink.table.catalog.ImmutableColumnsConstraint; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.test.util.SQLJobSubmission; @@ -53,7 +54,9 @@ public class PlannerScalaFreeITCase extends SqlITCaseBase { Collections.emptyList(), UniqueConstraint.primaryKey("pk", Collections.singletonList("user_name")), Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("user_name")))); + DefaultIndex.newIndex("idx", Collections.singletonList("user_name"))), + ImmutableColumnsConstraint.immutableColumns( + "imt", Collections.singletonList("user_name"))); private static final DebeziumJsonDeserializationSchema DESERIALIZATION_SCHEMA = createDebeziumDeserializationSchema(SINK_TABLE_SCHEMA); diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-sql/src/test/java/org/apache/flink/table/sql/UsingRemoteJarITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-sql/src/test/java/org/apache/flink/table/sql/UsingRemoteJarITCase.java index c49b045252c2c..0432aa76ecc84 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-sql/src/test/java/org/apache/flink/table/sql/UsingRemoteJarITCase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-sql/src/test/java/org/apache/flink/table/sql/UsingRemoteJarITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.DefaultIndex; +import org.apache.flink.table.catalog.ImmutableColumnsConstraint; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; @@ -47,7 +48,9 @@ public class UsingRemoteJarITCase extends HdfsITCaseBase { Collections.emptyList(), UniqueConstraint.primaryKey("pk", Collections.singletonList("user_name")), Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("user_name")))); + DefaultIndex.newIndex("idx", Collections.singletonList("user_name"))), + ImmutableColumnsConstraint.immutableColumns( + "imt", Collections.singletonList("user_name"))); private static final DebeziumJsonDeserializationSchema USER_ORDER_DESERIALIZATION_SCHEMA = createDebeziumDeserializationSchema(USER_ORDER_SCHEMA); diff --git a/flink-python/pyflink/table/tests/test_catalog_completeness.py b/flink-python/pyflink/table/tests/test_catalog_completeness.py index a3e171fc38437..c1ab64c96aa01 100644 --- a/flink-python/pyflink/table/tests/test_catalog_completeness.py +++ b/flink-python/pyflink/table/tests/test_catalog_completeness.py @@ -253,7 +253,8 @@ def java_class(cls): @classmethod def excluded_methods(cls): # getIndexes are not needed in Python API as they are used internally - return {'getIndexes'} + # TODO FLINK-39319 add immutable column constraint in Python API + return {'getIndexes', 'getImmutableColumns', 'getImmutableColumnIndexes'} if __name__ == '__main__': 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 bc8708578073c..3bba4c05f4ec1 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 @@ -1984,7 +1984,8 @@ public ResolvedCatalogView resolveCatalogView(CatalogView view) { .collect(Collectors.toList()), resolvedSchema.getWatermarkSpecs(), resolvedSchema.getPrimaryKey().orElse(null), - resolvedSchema.getIndexes()); + resolvedSchema.getIndexes(), + resolvedSchema.getImmutableColumns().orElse(null)); return new ResolvedCatalogView( // pass a view that has the query parsed and // validated already diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java index b44e843742084..4ce5177832dc0 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.Schema.UnresolvedComputedColumn; +import org.apache.flink.table.api.Schema.UnresolvedImmutableColumns; import org.apache.flink.table.api.Schema.UnresolvedIndex; import org.apache.flink.table.api.Schema.UnresolvedMetadataColumn; import org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn; @@ -92,7 +93,12 @@ public ResolvedSchema resolve(Schema schema) { final List indexes = resolveIndexes(schema.getIndexes(), columnsWithRowtime); - return new ResolvedSchema(columnsWithRowtime, watermarkSpecs, primaryKey, indexes); + final ImmutableColumnsConstraint immutableColumns = + resolveImmutableColumns( + schema.getImmutableColumns().orElse(null), primaryKey, columnsWithRowtime); + + return new ResolvedSchema( + columnsWithRowtime, watermarkSpecs, primaryKey, indexes, immutableColumns); } // -------------------------------------------------------------------------------------------- @@ -459,6 +465,74 @@ private void validateSingleIndex(Index index, Map columnsByNameL } } + private @Nullable ImmutableColumnsConstraint resolveImmutableColumns( + @Nullable UnresolvedImmutableColumns unresolvedImmutableColumns, + @Nullable UniqueConstraint primaryKey, + List columns) { + if (unresolvedImmutableColumns == null) { + return null; + } + + final ImmutableColumnsConstraint immutableColumns = + ImmutableColumnsConstraint.immutableColumns( + unresolvedImmutableColumns.getConstraintName(), + unresolvedImmutableColumns.getColumnNames()); + + validateImmutableColumns(immutableColumns, primaryKey, columns); + + return immutableColumns; + } + + private void validateImmutableColumns( + ImmutableColumnsConstraint immutableColumns, + @Nullable UniqueConstraint primaryKey, + List columns) { + if (primaryKey == null) { + throw new ValidationException( + String.format( + "Invalid immutable constraint '%s'. " + + "An immutable constraint must be defined on the table that contains primary key.", + immutableColumns.getName())); + } + + final Map columnsByNameLookup = + columns.stream().collect(Collectors.toMap(Column::getName, Function.identity())); + + final Set duplicateColumns = + immutableColumns.getColumns().stream() + .filter( + name -> + Collections.frequency(immutableColumns.getColumns(), name) + > 1) + .collect(Collectors.toSet()); + + if (!duplicateColumns.isEmpty()) { + throw new ValidationException( + String.format( + "Invalid immutable constraint '%s'. " + + "An immutable constraint must not contain duplicate columns. " + + "Found: %s", + immutableColumns.getName(), duplicateColumns)); + } + + for (String columnName : immutableColumns.getColumns()) { + Column column = columnsByNameLookup.get(columnName); + if (column == null) { + throw new ValidationException( + String.format( + "Invalid immutable constraint '%s'. Column '%s' does not exist.", + immutableColumns.getName(), columnName)); + } + + if (!column.isPhysical()) { + throw new ValidationException( + String.format( + "Invalid immutable constraint '%s'. Column '%s' is not a physical column.", + immutableColumns.getName(), columnName)); + } + } + } + private ResolvedExpression resolveExpression( List columns, Expression expression, @Nullable DataType outputDataType) { final LocalReferenceExpression[] localRefs = diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/internal/ShowCreateUtilTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/internal/ShowCreateUtilTest.java index 15464afc7e5d8..51e1a5479fd84 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/internal/ShowCreateUtilTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/internal/ShowCreateUtilTest.java @@ -30,6 +30,7 @@ import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogView; import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ImmutableColumnsConstraint; import org.apache.flink.table.catalog.IntervalFreshness; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; @@ -73,13 +74,24 @@ class ShowCreateUtilTest { Column.metadata("mt_column", DataTypes.STRING(), null, true)), List.of(), UniqueConstraint.primaryKey("pk", List.of("id")), - List.of()); + List.of(), + null); private static final ResolvedSchema TWO_COLUMNS_SCHEMA = ResolvedSchema.of( Column.physical("id", DataTypes.INT()), Column.physical("name", DataTypes.STRING())); + private static final ResolvedSchema TWO_COLUMNS_SCHEMA_WITH_PRIMARY_KEY_AND_IMMUTABLE_COLS = + new ResolvedSchema( + List.of( + Column.physical("id", DataTypes.INT()), + Column.physical("name", DataTypes.STRING())), + List.of(), + UniqueConstraint.primaryKey("pk", List.of("id")), + List.of(), + ImmutableColumnsConstraint.immutableColumns("imt", List.of("name"))); + @ParameterizedTest(name = "{index}: {2}") @MethodSource("argsForShowCreateTable") void showCreateTable( @@ -213,6 +225,23 @@ private static Collection argsForShowCreateTable() { + "COMMENT 'Table comment'\n" + "DISTRIBUTED BY RANGE(`1`, `10`) INTO 2 BUCKETS\n"); + addTemporaryAndPermanent( + argList, + createResolvedTable( + TWO_COLUMNS_SCHEMA_WITH_PRIMARY_KEY_AND_IMMUTABLE_COLS, + Collections.emptyMap(), + Collections.emptyList(), + TableDistribution.of( + TableDistribution.Kind.RANGE, 2, Arrays.asList("1", "10")), + "Table comment"), + "CREATE %sTABLE `catalogName`.`dbName`.`tableName` (\n" + + " `id` INT,\n" + + " `name` VARCHAR(2147483647),\n" + + " CONSTRAINT `pk` PRIMARY KEY (`id`) NOT ENFORCED\n" + + ")\n" + + "COMMENT 'Table comment'\n" + + "DISTRIBUTED BY RANGE(`1`, `10`) INTO 2 BUCKETS\n"); + final Map options = new HashMap<>(); options.put("option_key_a", "option_value_a"); options.put("option_key_b", "option_value_b"); @@ -315,6 +344,26 @@ private static Collection argsForShowCreateMaterializedTable() { + "REFRESH_MODE = CONTINUOUS\n" + "AS SELECT 1\n")); + argList.add( + Arguments.of( + createResolvedMaterialized( + TWO_COLUMNS_SCHEMA_WITH_PRIMARY_KEY_AND_IMMUTABLE_COLS, + null, + List.of(), + null, + IntervalFreshness.ofMinute("1"), + RefreshMode.CONTINUOUS, + "SELECT 1", + "SELECT 1"), + "CREATE MATERIALIZED TABLE `catalogName`.`dbName`.`materializedTableName` (\n" + + " `id` INT,\n" + + " `name` VARCHAR(2147483647),\n" + + " CONSTRAINT `pk` PRIMARY KEY (`id`) NOT ENFORCED\n" + + ")\n" + + "FRESHNESS = INTERVAL '1' MINUTE\n" + + "REFRESH_MODE = CONTINUOUS\n" + + "AS SELECT 1\n")); + argList.add( Arguments.of( createResolvedMaterialized( 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 05ddc34a23b41..92035ccc944f7 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 @@ -89,6 +89,7 @@ class CatalogBaseTableResolutionTest { .watermark("ts", WATERMARK_SQL) .primaryKeyNamed("primary_constraint", "id") .indexNamed("idx", Collections.singletonList("id")) + .immutableColumnsNamed("imt", Collections.singletonList("region")) .build(); private static final Schema MATERIALIZED_TABLE_SCHEMA = @@ -101,6 +102,7 @@ class CatalogBaseTableResolutionTest { .withComment("") // empty column comment .primaryKeyNamed("primary_constraint", "id") .indexNamed("idx", Collections.singletonList("id")) + .immutableColumnsNamed("imt", Collections.singletonList("region")) .build(); private static final TableSchema LEGACY_TABLE_SCHEMA = @@ -138,7 +140,9 @@ class CatalogBaseTableResolutionTest { UniqueConstraint.primaryKey( "primary_constraint", Collections.singletonList("id")), Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("id")))); + DefaultIndex.newIndex("idx", Collections.singletonList("id"))), + ImmutableColumnsConstraint.immutableColumns( + "imt", Collections.singletonList("region"))); private static final ResolvedSchema RESOLVED_MATERIALIZED_TABLE_SCHEMA = new ResolvedSchema( @@ -152,7 +156,9 @@ class CatalogBaseTableResolutionTest { UniqueConstraint.primaryKey( "primary_constraint", Collections.singletonList("id")), Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("id")))); + DefaultIndex.newIndex("idx", Collections.singletonList("id"))), + ImmutableColumnsConstraint.immutableColumns( + "imt", Collections.singletonList("region"))); private static final ContinuousRefreshHandler CONTINUOUS_REFRESH_HANDLER = new ContinuousRefreshHandler( @@ -171,7 +177,8 @@ class CatalogBaseTableResolutionTest { Column.physical("county", DataTypes.VARCHAR(200))), Collections.emptyList(), null, - Collections.emptyList()); + Collections.emptyList(), + null); @Test void testCatalogTableResolution() { @@ -397,6 +404,8 @@ private static Map catalogTableAsProperties() { properties.put("schema.primary-key.columns", "id"); properties.put("schema.index.0.name", "idx"); properties.put("schema.index.0.columns", "id"); + properties.put("schema.immutable.name", "imt"); + properties.put("schema.immutable.columns", "region"); properties.put("partition.keys.0.name", "region"); properties.put("partition.keys.1.name", "county"); properties.put("version", "12"); @@ -424,6 +433,8 @@ private static Map catalogMaterializedTableAsProperties() throws properties.put("schema.primary-key.columns", "id"); properties.put("schema.index.0.name", "idx"); properties.put("schema.index.0.columns", "id"); + properties.put("schema.immutable.name", "imt"); + properties.put("schema.immutable.columns", "region"); properties.put("freshness-interval", "30"); properties.put("freshness-unit", "SECOND"); properties.put("logical-refresh-mode", "CONTINUOUS"); diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java index 585101307967f..3519ade2907eb 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java @@ -90,6 +90,7 @@ class SchemaResolutionTest { .watermark("ts", WATERMARK_SQL) .columnByExpression("proctime", PROCTIME_SQL) .indexNamed("idx", Collections.singletonList("counter")) + .immutableColumnsNamed("imt", Collections.singletonList("payload")) .build(); // the type of ts_ltz is TIMESTAMP_LTZ @@ -141,8 +142,9 @@ void testSchemaResolution() { UniqueConstraint.primaryKey( "primary_constraint", Collections.singletonList("id")), Collections.singletonList( - DefaultIndex.newIndex( - "idx", Collections.singletonList("counter")))); + DefaultIndex.newIndex("idx", Collections.singletonList("counter"))), + ImmutableColumnsConstraint.immutableColumns( + "imt", Collections.singletonList("payload"))); final ResolvedSchema actualStreamSchema = resolveSchema(SCHEMA, true); { @@ -172,7 +174,8 @@ void testSchemaResolutionWithTimestampLtzRowtime() { WatermarkSpec.of("ts1", WATERMARK_RESOLVED_WITH_TS_LTZ)), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("id")))); + DefaultIndex.newIndex("idx", Collections.singletonList("id"))), + null); final ResolvedSchema actualStreamSchema = resolveSchema(SCHEMA_WITH_TS_LTZ, true); { @@ -202,7 +205,8 @@ void testSchemaResolutionWithSourceWatermark() { DataTypes.TIMESTAMP_LTZ(1)))), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("ts_ltz")))); + DefaultIndex.newIndex("idx", Collections.singletonList("ts_ltz"))), + null); final ResolvedSchema resolvedSchema = resolveSchema( Schema.newBuilder() @@ -352,6 +356,40 @@ void testSchemaResolutionErrors() { .build(), "Invalid index 'INDEX_ts'. " + "Column 'ts' is not a physical column or a metadata column."); + + // immutable constraints + + testError( + Schema.newBuilder().column("id", DataTypes.INT()).immutableColumns("id").build(), + "An immutable constraint must be defined on the table that contains primary key."); + + testError( + Schema.newBuilder() + .column("id", DataTypes.INT().notNull()) + .primaryKey("id") + .immutableColumns("INVALID") + .build(), + "Column 'INVALID' does not exist."); + + testError( + Schema.newBuilder() + .column("id", DataTypes.INT().notNull()) + .column("orig_ts", DataTypes.TIMESTAMP(3)) + .columnByExpression("ts", COMPUTED_SQL) + .primaryKey("id") + .immutableColumns("ts") + .build(), + "Column 'ts' is not a physical column."); + + testError( + Schema.newBuilder() + .column("id", DataTypes.INT().notNull()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .immutableColumns("name", "name") + .build(), + "Invalid immutable constraint 'IMMUTABLE_COLUMNS_name_name'. " + + "An immutable constraint must not contain duplicate columns. Found: [name]"); } @Test @@ -371,6 +409,7 @@ void testIndexNamedBuildingErrors() { .indexNamed("idx", null) .build()) .hasMessageContaining("Index column names must not be null."); + assertThatThrownBy( () -> Schema.newBuilder() @@ -380,6 +419,48 @@ void testIndexNamedBuildingErrors() { .hasMessageContaining("Index must be defined for at least a single column."); } + @Test + void testImmutableColumnsBuildingErrors() { + assertThatThrownBy( + () -> + Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .primaryKey("a") + .immutableColumnsNamed(null, Collections.singletonList("a")) + .build()) + .hasMessageContaining("Immutable constraint name must not be null."); + + assertThatThrownBy( + () -> + Schema.newBuilder() + .column("a", DataTypes.INT()) + .primaryKey("a") + .immutableColumnsNamed("imt", (String[]) null) + .build()) + .hasMessageContaining("Immutable column names must not be null."); + + assertThatThrownBy( + () -> + Schema.newBuilder() + .column("a", DataTypes.INT()) + .primaryKey("a") + .immutableColumnsNamed("idx", Collections.emptyList()) + .build()) + .hasMessageContaining( + "Immutable constraint must be defined for at least a single column."); + + assertThatThrownBy( + () -> + Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.INT().notNull()) + .primaryKey("a") + .immutableColumns("a") + .immutableColumns("b") + .build()) + .hasMessageContaining("Multiple immutable constraints are not supported."); + } + @Test void testUnresolvedSchemaString() { assertThat(SCHEMA.toString()) @@ -394,7 +475,8 @@ void testUnresolvedSchemaString() { + " `proctime` AS [PROCTIME()],\n" + " WATERMARK FOR `ts` AS [ts - INTERVAL '5' SECOND],\n" + " CONSTRAINT `primary_constraint` PRIMARY KEY (`id`) NOT ENFORCED,\n" - + " INDEX `idx` (`counter`)\n" + + " INDEX `idx` (`counter`),\n" + + " CONSTRAINT `imt` COLUMNS (`payload`) IMMUTABLE NOT ENFORCED\n" + ")"); } @@ -413,7 +495,8 @@ void testResolvedSchemaString() { + " `proctime` TIMESTAMP_LTZ(3) NOT NULL *PROCTIME* AS PROCTIME(),\n" + " WATERMARK FOR `ts`: TIMESTAMP(3) AS ts - INTERVAL '5' SECOND,\n" + " CONSTRAINT `primary_constraint` PRIMARY KEY (`id`) NOT ENFORCED,\n" - + " INDEX `idx` (`counter`)\n" + + " INDEX `idx` (`counter`),\n" + + " CONSTRAINT `imt` COLUMNS (`payload`) IMMUTABLE NOT ENFORCED\n" + ")"); } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java index 654e919dfe553..d22edb91b2523 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java @@ -24,6 +24,7 @@ import org.apache.flink.table.catalog.Column.MetadataColumn; import org.apache.flink.table.catalog.Column.PhysicalColumn; import org.apache.flink.table.catalog.Constraint; +import org.apache.flink.table.catalog.ImmutableColumnsConstraint; import org.apache.flink.table.catalog.Index; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.SchemaResolver; @@ -80,7 +81,12 @@ public final class Schema { private final List indexes; - /** Please use {@link #Schema(List, List, UnresolvedPrimaryKey, List)} instead. */ + private final @Nullable UnresolvedImmutableColumns immutableColumns; + + /** + * Please use {@link #Schema(List, List, UnresolvedPrimaryKey, List, + * UnresolvedImmutableColumns)} instead. + */ @Deprecated public Schema( List columns, @@ -94,10 +100,20 @@ public Schema( List watermarkSpecs, @Nullable UnresolvedPrimaryKey primaryKey, List indexes) { + this(columns, watermarkSpecs, primaryKey, indexes, null); + } + + public Schema( + List columns, + List watermarkSpecs, + @Nullable UnresolvedPrimaryKey primaryKey, + List indexes, + @Nullable UnresolvedImmutableColumns immutableColumns) { this.columns = Collections.unmodifiableList(columns); this.watermarkSpecs = Collections.unmodifiableList(watermarkSpecs); this.primaryKey = primaryKey; this.indexes = Collections.unmodifiableList(indexes); + this.immutableColumns = immutableColumns; } /** Builder for configuring and creating instances of {@link Schema}. */ @@ -134,6 +150,10 @@ public List getIndexes() { return indexes; } + public Optional getImmutableColumns() { + return Optional.ofNullable(immutableColumns); + } + /** Resolves the given {@link Schema} to a validated {@link ResolvedSchema}. */ public ResolvedSchema resolve(SchemaResolver resolver) { return resolver.resolve(this); @@ -150,6 +170,9 @@ public String toString() { if (!indexes.isEmpty()) { components.addAll(indexes); } + if (immutableColumns != null) { + components.add(immutableColumns); + } return components.stream() .map(Objects::toString) .map(s -> " " + s) @@ -168,12 +191,13 @@ public boolean equals(Object o) { return columns.equals(schema.columns) && watermarkSpecs.equals(schema.watermarkSpecs) && Objects.equals(primaryKey, schema.primaryKey) - && indexes.equals(schema.indexes); + && indexes.equals(schema.indexes) + && Objects.equals(immutableColumns, schema.immutableColumns); } @Override public int hashCode() { - return Objects.hash(columns, watermarkSpecs, primaryKey, indexes); + return Objects.hash(columns, watermarkSpecs, primaryKey, indexes, immutableColumns); } // -------------------------------------------------------------------------------------------- @@ -190,6 +214,8 @@ public static final class Builder { private final List indexes; + private @Nullable UnresolvedImmutableColumns immutableColumns; + private Builder() { columns = new ArrayList<>(); watermarkSpecs = new ArrayList<>(); @@ -206,6 +232,11 @@ public Builder fromSchema(Schema unresolvedSchema) { unresolvedSchema.primaryKey.getColumnNames()); } indexes.addAll(unresolvedSchema.indexes); + if (unresolvedSchema.immutableColumns != null) { + immutableColumnsNamed( + unresolvedSchema.immutableColumns.getConstraintName(), + unresolvedSchema.immutableColumns.getColumnNames()); + } return this; } @@ -215,6 +246,7 @@ public Builder fromResolvedSchema(ResolvedSchema resolvedSchema) { addResolvedWatermarkSpec(resolvedSchema.getWatermarkSpecs()); resolvedSchema.getPrimaryKey().ifPresent(this::addResolvedConstraint); addResolvedIndexes(resolvedSchema.getIndexes()); + resolvedSchema.getImmutableColumns().ifPresent(this::addResolvedConstraint); return this; } @@ -672,9 +704,87 @@ public Builder indexNamed(String indexName, List columnNames) { return this; } + /** + * Declares an immutable columns constraint for a list of given columns. Immutable columns + * constraint is used to identify which columns in a table are not allowed to be modified. + * Currently, this constraint is informational only and is not enforced. It can be utilized + * for optimization purposes. It is the responsibility of the data owner to ensure that + * these columns are unmodified. + * + *

The immutable columns will be assigned a generated name in the format {@code + * IMMUTABLE_COLUMNS_col1_col2}. + * + * @param columnNames columns that form the constraint for immutable columns + */ + public Builder immutableColumns(String... columnNames) { + Preconditions.checkNotNull(columnNames, "Immutable column names must not be null."); + return immutableColumns(Arrays.asList(columnNames)); + } + + /** + * Declares an immutable columns constraint for a list of given columns. Immutable columns + * constraint is used to identify which columns in a table are not allowed to be modified. + * Currently, this constraint is informational only and is not enforced. It can be utilized + * for optimization purposes. It is the responsibility of the data owner to ensure that + * these columns are unmodified. + * + *

The immutable columns will be assigned a generated name in the format {@code + * IMMUTABLE_COLUMNS_col1_col2}. + * + * @param columnNames columns that form the constraint for immutable columns + */ + public Builder immutableColumns(List columnNames) { + Preconditions.checkNotNull(columnNames, "Immutable column names must not be null."); + final String generatedConstraintName = + columnNames.stream().collect(Collectors.joining("_", "IMMUTABLE_COLUMNS_", "")); + return immutableColumnsNamed(generatedConstraintName, columnNames); + } + + /** + * Declares an immutable columns constraint for a list of given columns. Immutable columns + * constraint is used to identify which columns in a table are not allowed to be modified. + * Currently, this constraint is informational only and is not enforced. It can be utilized + * for optimization purposes. It is the responsibility of the data owner to ensure that + * these columns are unmodified. + * + * @param constraintName name for the immutable columns constraint, can be used to reference + * this constraint + * @param columnNames columns that form the constraint for immutable columns + */ + public Builder immutableColumnsNamed(String constraintName, String... columnNames) { + Preconditions.checkNotNull(columnNames, "Immutable column names must not be null."); + return immutableColumnsNamed(constraintName, Arrays.asList(columnNames)); + } + + /** + * Declares an immutable columns constraint for a list of given columns. Immutable columns + * constraint is used to identify which columns in a table are not allowed to be modified. + * Currently, this constraint is informational only and is not enforced. It can be utilized + * for optimization purposes. It is the responsibility of the data owner to ensure that + * these columns are unmodified. + * + * @param constraintName name for the immutable columns constraint, can be used to reference + * this constraint + * @param columnNames columns that form the constraint for immutable columns + */ + public Builder immutableColumnsNamed(String constraintName, List columnNames) { + Preconditions.checkState( + immutableColumns == null, "Multiple immutable constraints are not supported."); + Preconditions.checkNotNull( + constraintName, "Immutable constraint name must not be null."); + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(constraintName), + "Immutable constraint name must not be empty."); + Preconditions.checkArgument( + columnNames != null && !columnNames.isEmpty(), + "Immutable constraint must be defined for at least a single column."); + immutableColumns = new UnresolvedImmutableColumns(constraintName, columnNames); + return this; + } + /** Returns an instance of an unresolved {@link Schema}. */ public Schema build() { - return new Schema(columns, watermarkSpecs, primaryKey, indexes); + return new Schema(columns, watermarkSpecs, primaryKey, indexes, immutableColumns); } // ---------------------------------------------------------------------------------------- @@ -709,9 +819,13 @@ private void addResolvedWatermarkSpec(List specs) { s.getRowtimeAttribute(), s.getWatermarkExpression()))); } - private void addResolvedConstraint(UniqueConstraint constraint) { + private void addResolvedConstraint(Constraint constraint) { if (constraint.getType() == Constraint.ConstraintType.PRIMARY_KEY) { - primaryKeyNamed(constraint.getName(), constraint.getColumns()); + primaryKeyNamed(constraint.getName(), ((UniqueConstraint) constraint).getColumns()); + } else if (constraint.getType() == Constraint.ConstraintType.IMMUTABLE_COLUMNS) { + immutableColumnsNamed( + constraint.getName(), + ((ImmutableColumnsConstraint) constraint).getColumns()); } else { throw new IllegalArgumentException("Unsupported constraint type."); } @@ -1173,4 +1287,53 @@ public int hashCode() { return Objects.hash(indexName, columnNames); } } + + /** + * Declaration of a list of immutable columns that will be resolved to {@link + * ImmutableColumnsConstraint} during schema resolution. + */ + @PublicEvolving + public static final class UnresolvedImmutableColumns extends UnresolvedConstraint { + + private final List columnNames; + + public UnresolvedImmutableColumns(String constraintName, List columnNames) { + super(constraintName); + this.columnNames = columnNames; + } + + public List getColumnNames() { + return columnNames; + } + + @Override + public String toString() { + return String.format( + "%s COLUMNS (%s) IMMUTABLE NOT ENFORCED", + super.toString(), + columnNames.stream() + .map(EncodingUtils::escapeIdentifier) + .collect(Collectors.joining(", "))); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + UnresolvedImmutableColumns that = (UnresolvedImmutableColumns) o; + return columnNames.equals(that.columnNames); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), columnNames); + } + } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogPropertiesUtil.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogPropertiesUtil.java index 16e9ba8c22468..67b72bdeb68af 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogPropertiesUtil.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogPropertiesUtil.java @@ -376,6 +376,8 @@ public static CatalogModel deserializeCatalogModel(Map propertie private static final String PRIMARY_KEY = "primary-key"; + private static final String IMMUTABLE = "immutable"; + private static final String COLUMNS = "columns"; private static final String PARTITION = "partition"; @@ -399,6 +401,10 @@ public static CatalogModel deserializeCatalogModel(Map propertie private static final String PRIMARY_KEY_COLUMNS = compoundKey(PRIMARY_KEY, COLUMNS); + private static final String IMMUTABLE_NAME = compoundKey(IMMUTABLE, NAME); + + private static final String IMMUTABLE_COLUMNS = compoundKey(IMMUTABLE, COLUMNS); + private static final String INDEX = "index"; private static final String INDEX_NAME = "name"; @@ -515,6 +521,8 @@ private static Schema deserializeSchema(Map map, String schemaKe deserializePrimaryKey(map, schemaKey, builder); + deserializeImmutableCols(map, schemaKey, builder); + deserializeIndexes(map, schemaKey, builder); return builder.build(); @@ -545,6 +553,17 @@ private static void deserializePrimaryKey( } } + private static void deserializeImmutableCols( + Map map, String schemaKey, Builder builder) { + final String constraintNameKey = compoundKey(schemaKey, IMMUTABLE_NAME); + final String columnsKey = compoundKey(schemaKey, IMMUTABLE_COLUMNS); + if (map.containsKey(constraintNameKey)) { + final String constraintName = getValue(map, constraintNameKey); + final String[] columns = getValue(map, columnsKey, s -> s.split(",")); + builder.immutableColumnsNamed(constraintName, columns); + } + } + private static void deserializeWatermark( Map map, String schemaKey, Builder builder) { final String watermarkKey = compoundKey(schemaKey, WATERMARK); @@ -654,6 +673,8 @@ private static void serializeResolvedSchema( schema.getPrimaryKey().ifPresent(pk -> serializePrimaryKey(map, pk)); + schema.getImmutableColumns().ifPresent(ics -> serializeImmutableCols(map, ics)); + serializeIndexes(map, schema.getIndexes()); } @@ -679,6 +700,12 @@ private static void serializePrimaryKey(Map map, UniqueConstrain String.join(",", constraint.getColumns())); } + private static void serializeImmutableCols( + Map map, ImmutableColumnsConstraint constraint) { + map.put(compoundKey(SCHEMA, IMMUTABLE_NAME), constraint.getName()); + map.put(compoundKey(SCHEMA, IMMUTABLE_COLUMNS), String.join(",", constraint.getColumns())); + } + private static void serializeWatermarkSpecs( Map map, List specs, SqlFactory sqlFactory) { if (!specs.isEmpty()) { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Constraint.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Constraint.java index f0c363e9f022c..1f0c4359c64a2 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Constraint.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Constraint.java @@ -46,19 +46,23 @@ public interface Constraint { /** * Type of the constraint. * - *

Unique constraints: - * *

    - *
  • UNIQUE - is satisfied if and only if there do not exist two rows that have same - * non-null values in the unique columns - *
  • PRIMARY KEY - additionally to UNIQUE constraint, it requires none of the values in - * specified columns be a null value. Moreover there can be only a single PRIMARY KEY - * defined for a Table. + *
  • Unique constraints: + *
      + *
    • UNIQUE - is satisfied if and only if there do not exist two rows that have same + * non-null values in the unique columns + *
    • PRIMARY KEY - additionally to UNIQUE constraint, it requires none of the values + * in specified columns be a null value. Moreover there can be only a single PRIMARY + * KEY defined for a Table. + *
    + *
  • Immutable constraint - is satisfied iff these specific columns are not allowed to be + * modified. *
*/ @PublicEvolving enum ConstraintType { PRIMARY_KEY, - UNIQUE_KEY + UNIQUE_KEY, + IMMUTABLE_COLUMNS } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ImmutableColumnsConstraint.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ImmutableColumnsConstraint.java new file mode 100644 index 0000000000000..e1426d0b659ef --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ImmutableColumnsConstraint.java @@ -0,0 +1,97 @@ +/* + * 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.utils.EncodingUtils; + +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * Immutable columns constraint is used to identify which columns in a table are not allowed to be + * modified. + * + * @see ConstraintType + */ +@PublicEvolving +public final class ImmutableColumnsConstraint extends AbstractConstraint { + + private final List columns; + private final ConstraintType type; + + /** Creates a non enforced {@link ConstraintType#IMMUTABLE_COLUMNS} constraint. */ + public static ImmutableColumnsConstraint immutableColumns(String name, List columns) { + return new ImmutableColumnsConstraint( + name, false, ConstraintType.IMMUTABLE_COLUMNS, columns); + } + + private ImmutableColumnsConstraint( + String name, boolean enforced, ConstraintType type, List columns) { + super(name, enforced); + + this.columns = columns; + this.type = type; + + if (type != ConstraintType.IMMUTABLE_COLUMNS) { + throw new IllegalStateException("Unknown key type: " + getType()); + } + } + + public List getColumns() { + return columns; + } + + @Override + public ConstraintType getType() { + return ConstraintType.IMMUTABLE_COLUMNS; + } + + @Override + public String asSummaryString() { + return String.format( + "CONSTRAINT %s COLUMNS (%s) IMMUTABLE%s", + EncodingUtils.escapeIdentifier(getName()), + columns.stream() + .map(EncodingUtils::escapeIdentifier) + .collect(Collectors.joining(", ")), + isEnforced() ? "" : " NOT ENFORCED"); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + ImmutableColumnsConstraint that = (ImmutableColumnsConstraint) o; + return Objects.equals(columns, that.columns); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), columns); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ResolvedSchema.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ResolvedSchema.java index 916733b44b70a..b2871b6e56a40 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ResolvedSchema.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/ResolvedSchema.java @@ -64,8 +64,12 @@ public final class ResolvedSchema { private final List watermarkSpecs; private final @Nullable UniqueConstraint primaryKey; private final List indexes; + private final @Nullable ImmutableColumnsConstraint immutableColumns; - /** Please use {@link #ResolvedSchema(List, List, UniqueConstraint, List)} instead. */ + /** + * Please use {@link #ResolvedSchema(List, List, UniqueConstraint, List, + * ImmutableColumnsConstraint)} instead. + */ @Deprecated public ResolvedSchema( List columns, @@ -79,16 +83,31 @@ public ResolvedSchema( List watermarkSpecs, @Nullable UniqueConstraint primaryKey, List indexes) { + this(columns, watermarkSpecs, primaryKey, indexes, null); + } + + public ResolvedSchema( + List columns, + List watermarkSpecs, + @Nullable UniqueConstraint primaryKey, + List indexes, + @Nullable ImmutableColumnsConstraint immutableColumns) { this.columns = Preconditions.checkNotNull(columns, "Columns must not be null."); this.watermarkSpecs = Preconditions.checkNotNull(watermarkSpecs, "Watermark specs must not be null."); this.primaryKey = primaryKey; this.indexes = Preconditions.checkNotNull(indexes, "Indexes must not be null."); + this.immutableColumns = immutableColumns; + + Preconditions.checkArgument( + primaryKey != null || immutableColumns == null, + "Immutable constraint must be defined on the table that contains primary key."); } /** Shortcut for a resolved schema of only columns. */ public static ResolvedSchema of(List columns) { - return new ResolvedSchema(columns, Collections.emptyList(), null, Collections.emptyList()); + return new ResolvedSchema( + columns, Collections.emptyList(), null, Collections.emptyList(), null); } /** Shortcut for a resolved schema of only columns. */ @@ -106,7 +125,8 @@ public static ResolvedSchema physical( IntStream.range(0, columnNames.size()) .mapToObj(i -> Column.physical(columnNames.get(i), columnDataTypes.get(i))) .collect(Collectors.toList()); - return new ResolvedSchema(columns, Collections.emptyList(), null, Collections.emptyList()); + return new ResolvedSchema( + columns, Collections.emptyList(), null, Collections.emptyList(), null); } /** Shortcut for a resolved schema of only physical columns. */ @@ -180,6 +200,11 @@ public List getIndexes() { return indexes; } + /** Returns the constraint about immutable columns if it has been defined. */ + public Optional getImmutableColumns() { + return Optional.ofNullable(immutableColumns); + } + /** * Returns the primary key indexes in the {@link #toPhysicalRowDataType()}, if any, otherwise * returns an empty array. @@ -196,6 +221,24 @@ public int[] getPrimaryKeyIndexes() { .orElseGet(() -> new int[] {}); } + /** + * Returns the indexes of columns about the immutable constraint in the {@link + * #toPhysicalRowDataType()}, if any, otherwise returns an empty array. + */ + public int[] getImmutableColumnIndexes() { + final List columns = + getColumns().stream() + .filter(Column::isPhysical) + .map(Column::getName) + .collect(Collectors.toList()); + return getImmutableColumns() + .map(ImmutableColumnsConstraint::getColumns) + .map( + immutableColumns -> + immutableColumns.stream().mapToInt(columns::indexOf).toArray()) + .orElseGet(() -> new int[] {}); + } + /** * Converts all columns of this schema into a (possibly nested) row data type. * @@ -253,6 +296,9 @@ public String toString() { components.add(primaryKey); } components.addAll(indexes); + if (immutableColumns != null) { + components.add(immutableColumns); + } return components.stream() .map(Objects::toString) .map(s -> " " + s) @@ -271,12 +317,13 @@ public boolean equals(Object o) { return Objects.equals(columns, that.columns) && Objects.equals(watermarkSpecs, that.watermarkSpecs) && Objects.equals(primaryKey, that.primaryKey) - && Objects.equals(indexes, that.indexes); + && Objects.equals(indexes, that.indexes) + && Objects.equals(immutableColumns, that.immutableColumns); } @Override public int hashCode() { - return Objects.hash(columns, watermarkSpecs, primaryKey, indexes); + return Objects.hash(columns, watermarkSpecs, primaryKey, indexes, immutableColumns); } // -------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TableSchemaUtils.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TableSchemaUtils.java index ad541ff7d26a9..21cb1900f771f 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TableSchemaUtils.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TableSchemaUtils.java @@ -127,7 +127,8 @@ public static ResolvedSchema removeTimeAttributeFromResolvedSchema( .collect(Collectors.toList()), resolvedSchema.getWatermarkSpecs(), resolvedSchema.getPrimaryKey().orElse(null), - resolvedSchema.getIndexes()); + resolvedSchema.getIndexes(), + resolvedSchema.getImmutableColumns().orElse(null)); } /** diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/CatalogPropertiesUtilTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/CatalogPropertiesUtilTest.java index 6800f8e2cedd9..43eb735c4c9a6 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/CatalogPropertiesUtilTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/CatalogPropertiesUtilTest.java @@ -120,6 +120,7 @@ private static Stream> resolvedCatalogBaseTables() { .column("f2", DataTypes.STRING().getLogicalType().asSerializableString()) .primaryKey("f1") .indexNamed("f1", Collections.singletonList("f1")) + .immutableColumns("f2") .build(); final TableDistribution hashDist = @@ -137,8 +138,12 @@ private static Stream> resolvedCatalogBaseTables() { List indexes = Collections.singletonList( DefaultIndex.newIndex("f1", Collections.singletonList("f1"))); + final ImmutableColumnsConstraint immutableColumns = + ImmutableColumnsConstraint.immutableColumns( + "IMMUTABLE_COLUMNS_f2", Collections.singletonList("f2")); final ResolvedSchema resolvedSchema = - new ResolvedSchema(columns, Collections.emptyList(), primaryKey, indexes); + new ResolvedSchema( + columns, Collections.emptyList(), primaryKey, indexes, immutableColumns); return Stream.of( new ResolvedCatalogTable( diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/CatalogTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/CatalogTest.java index 937e99c46ef6f..228b5485f0168 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/CatalogTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/CatalogTest.java @@ -1623,7 +1623,8 @@ protected ResolvedSchema createSchema() { Collections.emptyList(), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("first")))); + DefaultIndex.newIndex("idx", Collections.singletonList("first"))), + null); } protected ResolvedSchema createAnotherSchema() { @@ -1635,7 +1636,8 @@ protected ResolvedSchema createAnotherSchema() { Collections.emptyList(), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("first")))); + DefaultIndex.newIndex("idx", Collections.singletonList("first"))), + null); } protected List createPartitionKeys() { diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/TestSchemaResolver.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/TestSchemaResolver.java index 454691c0921da..5ba20ef1fbc92 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/TestSchemaResolver.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/TestSchemaResolver.java @@ -51,7 +51,10 @@ public ResolvedSchema resolve(Schema schema) { final List indexes = resolveIndexes(schema.getIndexes()); - return new ResolvedSchema(columns, watermarkSpecs, primaryKey, indexes); + final ImmutableColumnsConstraint immutableColumns = + resolveImmutableColumns(schema.getImmutableColumns().orElse(null)); + + return new ResolvedSchema(columns, watermarkSpecs, primaryKey, indexes, immutableColumns); } private List resolveIndexes(List unresolvedIndexes) { @@ -124,6 +127,17 @@ private WatermarkSpec resolveWatermarkSpecs( unresolvedPrimaryKey.getConstraintName(), unresolvedPrimaryKey.getColumnNames()); } + private @Nullable ImmutableColumnsConstraint resolveImmutableColumns( + @Nullable Schema.UnresolvedImmutableColumns unresolvedImmutableColumns) { + if (unresolvedImmutableColumns == null) { + return null; + } + + return ImmutableColumnsConstraint.immutableColumns( + unresolvedImmutableColumns.getConstraintName(), + unresolvedImmutableColumns.getColumnNames()); + } + private ResolvedExpression resolveExpression(Expression expression) { if (expression instanceof ResolvedExpression) { return (ResolvedExpression) expression; diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/utils/TableSchemaUtilsTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/utils/TableSchemaUtilsTest.java index 97e6664834be1..4bfa97673cb68 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/utils/TableSchemaUtilsTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/utils/TableSchemaUtilsTest.java @@ -22,6 +22,7 @@ import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.DefaultIndex; +import org.apache.flink.table.catalog.ImmutableColumnsConstraint; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.catalog.WatermarkSpec; @@ -104,7 +105,9 @@ void testRemoveTimeAttribute() { WatermarkSpec.of("t", ResolvedExpressionMock.of(rowTimeType, "t"))), UniqueConstraint.primaryKey("test-pk", Collections.singletonList("id")), Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("id")))); + DefaultIndex.newIndex("idx", Collections.singletonList("id"))), + ImmutableColumnsConstraint.immutableColumns( + "test-imt", Collections.singletonList("t"))); assertThat(TableSchemaUtils.removeTimeAttributeFromResolvedSchema(schema)) .isEqualTo( new ResolvedSchema( @@ -124,6 +127,8 @@ void testRemoveTimeAttribute() { "test-pk", Collections.singletonList("id")), Collections.singletonList( DefaultIndex.newIndex( - "idx", Collections.singletonList("id"))))); + "idx", Collections.singletonList("id"))), + ImmutableColumnsConstraint.immutableColumns( + "test-imt", Collections.singletonList("t")))); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java index d293342abb487..a78f9f52ce08a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java @@ -533,6 +533,7 @@ private static void validateScanSourceForStreaming( hasChangelogMode && changelogMode.contains(RowKind.UPDATE_BEFORE); final boolean hasUpdateAfter = hasChangelogMode && changelogMode.contains(RowKind.UPDATE_AFTER); + final boolean hasDelete = hasChangelogMode && changelogMode.contains(RowKind.DELETE); if (!hasUpdateBefore && hasUpdateAfter) { // only UPDATE_AFTER if (!schema.getPrimaryKey().isPresent()) { @@ -564,6 +565,12 @@ private static void validateScanSourceForStreaming( tableDebugName)); } } + if (hasDelete) { + if (schema.getImmutableColumns().isPresent()) { + throw new ValidationException( + "The immutable constraint cannot be defined on the table with changelog mode [DELETE]."); + } + } } private static void validateScanSourceForBatch( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkMetadata.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkMetadata.java index d7cf6cfa92714..d35f2df3f3b6f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkMetadata.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkMetadata.java @@ -254,4 +254,19 @@ interface Handler extends MetadataHandler { Set getUpsertKeys(RelNode r, RelMetadataQuery mq); } } + + /** Metadata about which combinations of columns are unmodified corresponding each pk. */ + public interface ImmutableColumns extends Metadata { + Method METHOD = Types.lookupMethod(ImmutableColumns.class, "getImmutableColumns"); + + MetadataDef DEF = + MetadataDef.of(ImmutableColumns.class, ImmutableColumns.Handler.class, METHOD); + + ImmutableBitSet getImmutableColumns(); + + /** Handler API. */ + interface Handler extends MetadataHandler { + ImmutableBitSet getImmutableColumns(RelNode r, RelMetadataQuery mq); + } + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMdImmutableColumns.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMdImmutableColumns.java new file mode 100644 index 0000000000000..10dc43a5d6760 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMdImmutableColumns.java @@ -0,0 +1,311 @@ +/* + * 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.metadata; + +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.planner.plan.nodes.calcite.WatermarkAssigner; +import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLookupJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner; +import org.apache.flink.table.planner.plan.schema.TableSourceTable; + +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.plan.hep.HepRelVertex; +import org.apache.calcite.plan.volcano.RelSubset; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Calc; +import org.apache.calcite.rel.core.Exchange; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.metadata.MetadataDef; +import org.apache.calcite.rel.metadata.MetadataHandler; +import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider; +import org.apache.calcite.rel.metadata.RelMetadataProvider; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.Util; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** A metadata handler for {@link FlinkMetadata.ImmutableColumns}. */ +public class FlinkRelMdImmutableColumns implements MetadataHandler { + static final FlinkRelMdImmutableColumns INSTANCE = new FlinkRelMdImmutableColumns(); + + public static final RelMetadataProvider SOURCE = + ReflectiveRelMetadataProvider.reflectiveSource( + FlinkMetadata.ImmutableColumns.METHOD, INSTANCE); + + // ~ Constructors ----------------------------------------------------------- + + private FlinkRelMdImmutableColumns() {} + + // ~ Methods ---------------------------------------------------------------- + + public MetadataDef getDef() { + return FlinkMetadata.ImmutableColumns.DEF; + } + + public ImmutableBitSet getImmutableColumns(TableScan rel, RelMetadataQuery mq) { + return getTableImmutableColumns(rel.getTable()); + } + + public ImmutableBitSet getImmutableColumns(Project rel, RelMetadataQuery mq) { + FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(mq); + return guardByUpsertKeys( + rel, getProjectImmutableColumns(rel.getProjects(), rel.getInput(), fmq), fmq); + } + + public ImmutableBitSet getImmutableColumns(Filter rel, RelMetadataQuery mq) { + FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(mq); + return guardByUpsertKeys(rel, fmq.getImmutableColumns(rel.getInput()), fmq); + } + + public ImmutableBitSet getImmutableColumns(Calc rel, RelMetadataQuery mq) { + FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(mq); + List projects = + rel.getProgram().getProjectList().stream() + .map(localRef -> rel.getProgram().expandLocalRef(localRef)) + .collect(Collectors.toList()); + return guardByUpsertKeys( + rel, getProjectImmutableColumns(projects, rel.getInput(), fmq), fmq); + } + + public ImmutableBitSet getImmutableColumns(Exchange rel, RelMetadataQuery mq) { + FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(mq); + return guardByUpsertKeys(rel, fmq.getImmutableColumns(rel.getInput()), fmq); + } + + public ImmutableBitSet getImmutableColumns( + StreamPhysicalChangelogNormalize rel, RelMetadataQuery mq) { + FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(mq); + return guardByUpsertKeys(rel, fmq.getImmutableColumns(rel.getInput()), fmq); + } + + public ImmutableBitSet getImmutableColumns( + StreamPhysicalMiniBatchAssigner rel, RelMetadataQuery mq) { + FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(mq); + return guardByUpsertKeys(rel, fmq.getImmutableColumns(rel.getInput()), fmq); + } + + public ImmutableBitSet getImmutableColumns( + StreamPhysicalDropUpdateBefore rel, RelMetadataQuery mq) { + FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(mq); + return guardByUpsertKeys(rel, fmq.getImmutableColumns(rel.getInput()), fmq); + } + + public ImmutableBitSet getImmutableColumns(WatermarkAssigner rel, RelMetadataQuery mq) { + FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(mq); + return guardByUpsertKeys(rel, fmq.getImmutableColumns(rel.getInput()), fmq); + } + + public ImmutableBitSet getImmutableColumns(Join join, RelMetadataQuery mq) { + JoinRelType joinType = join.getJoinType(); + + FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(mq); + int leftFieldCount = join.getLeft().getRowType().getFieldCount(); + + return unionJoinImmutableCols( + join, + joinType, + () -> fmq.getImmutableColumns(join.getLeft()), + () -> fmq.getImmutableColumns(join.getRight()), + leftFieldCount, + fmq); + } + + public ImmutableBitSet getImmutableColumns(CommonPhysicalLookupJoin join, RelMetadataQuery mq) { + FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(mq); + int leftFieldCount = join.getInput().getRowType().getFieldCount(); + + return unionJoinImmutableCols( + join, + join.joinType(), + () -> fmq.getImmutableColumns(join.getInput()), + // TODO support propagating immutable columns from the lookup side + () -> null, // rightImmutableColsSupplier + leftFieldCount, + fmq); + } + + public ImmutableBitSet getImmutableColumns(HepRelVertex rel, RelMetadataQuery mq) { + FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(mq); + return guardByUpsertKeys(rel, fmq.getImmutableColumns(rel.getCurrentRel()), fmq); + } + + public ImmutableBitSet getImmutableColumns(RelSubset rel, RelMetadataQuery mq) { + FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(mq); + return guardByUpsertKeys( + rel, fmq.getImmutableColumns(Util.first(rel.getBest(), rel.getOriginal())), fmq); + } + + public ImmutableBitSet getImmutableColumns(RelNode rel, RelMetadataQuery mq) { + // Catch-all rule when none of the others apply. + // More nodes can be supported later, such as Expand, Aggregate, Window, Rank, etc. + return null; + } + + /** + * Guards the immutable columns by verifying that the node has upsert keys. Immutable columns + * are only meaningful "within each pk"; if no upsert key exists, the result is cleared. + */ + @Nullable + private ImmutableBitSet guardByUpsertKeys( + RelNode rel, @Nullable ImmutableBitSet immutableColumns, FlinkRelMetadataQuery fmq) { + if (immutableColumns == null || immutableColumns.isEmpty()) { + return immutableColumns; + } + Set upsertKeys = fmq.getUpsertKeys(rel); + if (upsertKeys == null || upsertKeys.isEmpty()) { + return null; + } + return immutableColumns; + } + + /** + * Unions left/right immutable columns for a join, respecting join type semantics: + * + *
    + *
  • SEMI / ANTI: output contains only left-side columns → propagate left immutable only + *
  • LEFT: right side may produce nulls → ignore right immutable + *
  • RIGHT: left side may produce nulls → ignore left immutable + *
  • FULL: both sides may produce nulls → ignore both + *
  • INNER: both sides preserved + *
+ * + *

Right-side indices are shifted by leftFieldCount before union. The result is guarded by + * upsert keys. + */ + @Nullable + private ImmutableBitSet unionJoinImmutableCols( + RelNode rel, + JoinRelType joinType, + Supplier leftImmutableColsSupplier, + Supplier rightImmutableColsSupplier, + int leftFieldCount, + FlinkRelMetadataQuery fmq) { + if (joinType == JoinRelType.SEMI || joinType == JoinRelType.ANTI) { + return guardByUpsertKeys(rel, leftImmutableColsSupplier.get(), fmq); + } + + // nullable side's columns may flip between value/null → not immutable + ImmutableBitSet leftImmutableColumns = + joinType.generatesNullsOnLeft() ? null : leftImmutableColsSupplier.get(); + ImmutableBitSet rightImmutableColumns = + joinType.generatesNullsOnRight() ? null : rightImmutableColsSupplier.get(); + + // shift right side indices by left field count + ImmutableBitSet shiftedRight = + rightImmutableColumns == null || rightImmutableColumns.isEmpty() + ? rightImmutableColumns + : ImmutableBitSet.of( + rightImmutableColumns.toList().stream() + .map(i -> i + leftFieldCount) + .collect(Collectors.toList())); + + // union left and right immutable columns + ImmutableBitSet result; + if (leftImmutableColumns != null && shiftedRight != null) { + result = leftImmutableColumns.union(shiftedRight); + } else { + result = Optional.ofNullable(leftImmutableColumns).orElse(shiftedRight); + } + return guardByUpsertKeys(rel, result, fmq); + } + + @Nullable + private ImmutableBitSet getTableImmutableColumns(RelOptTable relOptTable) { + if (!(relOptTable instanceof TableSourceTable)) { + return null; + } + + TableSourceTable tst = (TableSourceTable) relOptTable; + ResolvedSchema schema = tst.contextResolvedTable().getResolvedTable().getResolvedSchema(); + + if (schema.getPrimaryKey().isEmpty()) { + return null; + } + + // use relOptTable's type which may be projected based on original schema + List tableOutputFields = relOptTable.getRowType().getFieldNames(); + + // add pk + Set allImmutableFieldsInSchema = + new HashSet<>(schema.getPrimaryKey().get().getColumns()); + // add constraint for immutable columns + if (schema.getImmutableColumns().isPresent()) { + allImmutableFieldsInSchema.addAll(schema.getImmutableColumns().get().getColumns()); + } + + Set outputImmutableColumns = + allImmutableFieldsInSchema.stream() + .flatMap( + immutableField -> { + int fieldIdx = tableOutputFields.indexOf(immutableField); + if (fieldIdx >= 0) { + return Stream.of(fieldIdx); + } else { + return Stream.empty(); + } + }) + .collect(Collectors.toSet()); + + return ImmutableBitSet.of(outputImmutableColumns); + } + + @Nullable + private ImmutableBitSet getProjectImmutableColumns( + List projects, RelNode inputNode, FlinkRelMetadataQuery fmq) { + ImmutableBitSet inputImmutableColumns = fmq.getImmutableColumns(inputNode); + if (inputImmutableColumns == null || inputImmutableColumns.isEmpty()) { + return inputImmutableColumns; + } + + Map> mapInToOutPos = new HashMap<>(); + for (int i = 0; i < projects.size(); i++) { + RexNode projExpr = projects.get(i); + if (projExpr instanceof RexInputRef) { + mapInToOutPos + .computeIfAbsent( + ((RexInputRef) projExpr).getIndex(), k -> new ArrayList<>()) + .add(i); + } + } + return ImmutableBitSet.of( + inputImmutableColumns.toList().stream() + .flatMap(in -> mapInToOutPos.getOrDefault(in, List.of()).stream()) + .collect(Collectors.toList())); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMetadataQuery.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMetadataQuery.java index 5c1cf8de5d3aa..562ae680a910e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMetadataQuery.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMetadataQuery.java @@ -50,6 +50,7 @@ public class FlinkRelMetadataQuery extends RelMetadataQuery { private FlinkMetadata.ModifiedMonotonicity.Handler modifiedMonotonicityHandler; private FlinkMetadata.WindowProperties.Handler windowPropertiesHandler; private FlinkMetadata.UpsertKeys.Handler upsertKeysHandler; + private FlinkMetadata.ImmutableColumns.Handler immutableColumnsHandler; /** * Returns an instance of FlinkRelMetadataQuery. It ensures that cycles do not occur while @@ -85,6 +86,7 @@ private FlinkRelMetadataQuery() { this.modifiedMonotonicityHandler = HANDLERS.modifiedMonotonicityHandler; this.windowPropertiesHandler = HANDLERS.windowPropertiesHandler; this.upsertKeysHandler = HANDLERS.upsertKeysHandler; + this.immutableColumnsHandler = HANDLERS.immutableColumnsHandler; } /** Extended handlers. */ @@ -107,6 +109,8 @@ private static class Handlers { initialHandler(FlinkMetadata.WindowProperties.Handler.class); private FlinkMetadata.UpsertKeys.Handler upsertKeysHandler = initialHandler(FlinkMetadata.UpsertKeys.Handler.class); + private FlinkMetadata.ImmutableColumns.Handler immutableColumnsHandler = + initialHandler(FlinkMetadata.ImmutableColumns.Handler.class); } /** @@ -308,4 +312,21 @@ public Set getUpsertKeysInKeyGroupRange(RelNode rel, int[] part } return getUpsertKeys(rel); } + + /** + * Returns the columns that will never be updated upstream within each pk. + * + * @return the columns that will never be updated upstream within each pk, or null if this + * information cannot be determined (whereas empty set indicates that all columns may be + * updated) + */ + public ImmutableBitSet getImmutableColumns(RelNode rel) { + for (; ; ) { + try { + return immutableColumnsHandler.getImmutableColumns(rel, this); + } catch (JaninoRelMetadataProvider.NoHandler e) { + immutableColumnsHandler = revise(e.relClass, FlinkMetadata.ImmutableColumns.DEF); + } + } + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AbstractConstraintMixin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AbstractConstraintMixin.java new file mode 100644 index 0000000000000..20560ef2afbc3 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AbstractConstraintMixin.java @@ -0,0 +1,39 @@ +/* + * 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.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +/** Base mixin for {@code AbstractConstraint} subclasses. */ +@Internal +abstract class AbstractConstraintMixin { + + static final String NAME = "name"; + static final String ENFORCED = "enforced"; + + @JsonProperty(NAME) + public abstract String getName(); + + @JsonProperty(ENFORCED) + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public abstract boolean isEnforced(); +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/CompiledPlanSerdeUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/CompiledPlanSerdeUtil.java index cb227b3fd75ce..bb0ea8a35e24a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/CompiledPlanSerdeUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/CompiledPlanSerdeUtil.java @@ -28,6 +28,7 @@ import org.apache.flink.table.catalog.ContextResolvedModel; import org.apache.flink.table.catalog.ContextResolvedTable; import org.apache.flink.table.catalog.DefaultIndex; +import org.apache.flink.table.catalog.ImmutableColumnsConstraint; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ResolvedCatalogModel; import org.apache.flink.table.catalog.ResolvedCatalogTable; @@ -230,6 +231,8 @@ private static void registerDeserializers(SimpleModule module) { private static void registerMixins(SimpleModule module) { module.setMixInAnnotation(WatermarkSpec.class, WatermarkSpecMixin.class); module.setMixInAnnotation(UniqueConstraint.class, UniqueConstraintMixin.class); + module.setMixInAnnotation( + ImmutableColumnsConstraint.class, ImmutableColumnsConstraintMixin.class); module.setMixInAnnotation(DefaultIndex.class, DefaultIndexMixin.class); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ImmutableColumnsConstraintMixin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ImmutableColumnsConstraintMixin.java new file mode 100644 index 0000000000000..4957b9728a18e --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ImmutableColumnsConstraintMixin.java @@ -0,0 +1,49 @@ +/* + * 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.Constraint; +import org.apache.flink.table.catalog.ImmutableColumnsConstraint; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +/** Mixin for {@link ImmutableColumnsConstraint}. */ +@Internal +abstract class ImmutableColumnsConstraintMixin extends AbstractConstraintMixin { + + static final String TYPE = "type"; + static final String COLUMNS = "columns"; + + @JsonCreator + private ImmutableColumnsConstraintMixin( + @JsonProperty(NAME) String name, + @JsonProperty(ENFORCED) boolean enforced, + @JsonProperty(TYPE) Constraint.ConstraintType type, + @JsonProperty(COLUMNS) List columns) {} + + @JsonProperty(TYPE) + public abstract Constraint.ConstraintType getType(); + + @JsonProperty(COLUMNS) + public abstract List getColumns(); +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedSchemaJsonDeserializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedSchemaJsonDeserializer.java index 7795facd1e4e8..82c84ff9acced 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedSchemaJsonDeserializer.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedSchemaJsonDeserializer.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.DefaultIndex; +import org.apache.flink.table.catalog.ImmutableColumnsConstraint; import org.apache.flink.table.catalog.Index; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; @@ -39,6 +40,7 @@ import static org.apache.flink.table.planner.plan.nodes.exec.serde.CompiledPlanSerdeUtil.deserializeList; import static org.apache.flink.table.planner.plan.nodes.exec.serde.CompiledPlanSerdeUtil.deserializeListOrEmpty; import static org.apache.flink.table.planner.plan.nodes.exec.serde.ResolvedSchemaJsonSerializer.COLUMNS; +import static org.apache.flink.table.planner.plan.nodes.exec.serde.ResolvedSchemaJsonSerializer.IMMUTABLE_COLUMNS; import static org.apache.flink.table.planner.plan.nodes.exec.serde.ResolvedSchemaJsonSerializer.INDEXES; import static org.apache.flink.table.planner.plan.nodes.exec.serde.ResolvedSchemaJsonSerializer.PRIMARY_KEY; import static org.apache.flink.table.planner.plan.nodes.exec.serde.ResolvedSchemaJsonSerializer.WATERMARK_SPECS; @@ -69,7 +71,10 @@ public ResolvedSchema deserialize(JsonParser jsonParser, DeserializationContext deserializeFieldOrNull(jsonNode, PRIMARY_KEY, UniqueConstraint.class, codec, ctx); List indexes = deserializeListOrEmpty(jsonNode, INDEXES, DefaultIndex.class, codec, ctx); + ImmutableColumnsConstraint immutableColumns = + deserializeFieldOrNull( + jsonNode, IMMUTABLE_COLUMNS, ImmutableColumnsConstraint.class, codec, ctx); - return new ResolvedSchema(columns, watermarkSpecs, primaryKey, indexes); + return new ResolvedSchema(columns, watermarkSpecs, primaryKey, indexes, immutableColumns); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedSchemaJsonSerializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedSchemaJsonSerializer.java index f3d0c83df2b5b..1cd450e37d28d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedSchemaJsonSerializer.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ResolvedSchemaJsonSerializer.java @@ -43,6 +43,7 @@ final class ResolvedSchemaJsonSerializer extends StdSerializer { static final String WATERMARK_SPECS = "watermarkSpecs"; static final String PRIMARY_KEY = "primaryKey"; static final String INDEXES = "indexes"; + static final String IMMUTABLE_COLUMNS = "immutableColumns"; ResolvedSchemaJsonSerializer() { super(ResolvedSchema.class); @@ -67,6 +68,11 @@ public void serialize( jsonGenerator, PRIMARY_KEY, resolvedSchema.getPrimaryKey(), serializerProvider); serializeListIfNotEmpty( jsonGenerator, INDEXES, resolvedSchema.getIndexes(), serializerProvider); + serializeOptionalField( + jsonGenerator, + IMMUTABLE_COLUMNS, + resolvedSchema.getImmutableColumns(), + serializerProvider); jsonGenerator.writeEndObject(); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/UniqueConstraintMixin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/UniqueConstraintMixin.java index 62501577f0f14..46ecd9012e92f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/UniqueConstraintMixin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/UniqueConstraintMixin.java @@ -23,17 +23,14 @@ import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; /** Mixin for {@link UniqueConstraint}. */ @Internal -abstract class UniqueConstraintMixin { +abstract class UniqueConstraintMixin extends AbstractConstraintMixin { - static final String NAME = "name"; - static final String ENFORCED = "enforced"; static final String TYPE = "type"; static final String COLUMNS = "columns"; @@ -44,16 +41,9 @@ private UniqueConstraintMixin( @JsonProperty(TYPE) ConstraintType type, @JsonProperty(COLUMNS) List columns) {} - @JsonProperty(NAME) - public abstract String getName(); - @JsonProperty(TYPE) public abstract ConstraintType getType(); - @JsonProperty(ENFORCED) - @JsonInclude(JsonInclude.Include.NON_DEFAULT) - public abstract boolean isEnforced(); - @JsonProperty(COLUMNS) public abstract List getColumns(); } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkDefaultRelMetadataProvider.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkDefaultRelMetadataProvider.scala index f5d4d93832a65..135603d56c22a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkDefaultRelMetadataProvider.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkDefaultRelMetadataProvider.scala @@ -48,7 +48,8 @@ object FlinkDefaultRelMetadataProvider { RelMdPredicates.SOURCE, FlinkRelMdCollation.SOURCE, RelMdExplainVisibility.SOURCE, - FlinkRelMdWindowProperties.SOURCE + FlinkRelMdWindowProperties.SOURCE, + FlinkRelMdImmutableColumns.SOURCE ) ) } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeys.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeys.scala index ea52022297c82..7890b116b26ad 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeys.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeys.scala @@ -48,10 +48,13 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] { override def getDef: MetadataDef[UpsertKeys] = UpsertKeys.DEF def getUpsertKeys(rel: TableScan, mq: RelMetadataQuery): JSet[ImmutableBitSet] = { - rel.getTable match { + val baseKeys = rel.getTable match { case t: IntermediateRelTable => t.upsertKeys case _ => mq.getUniqueKeys(rel) } + enrichWithImmutableColumns( + baseKeys, + () => FlinkRelMetadataQuery.reuseOrCreate(mq).getImmutableColumns(rel)) } def getUpsertKeys(rel: Project, mq: RelMetadataQuery): JSet[ImmutableBitSet] = @@ -81,11 +84,13 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] { () => FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(rel.getInput)) def getUpsertKeys(rel: Exchange, mq: RelMetadataQuery): JSet[ImmutableBitSet] = { - val keys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(rel.getInput) + val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq) + val upsertKeys = fmq.getUpsertKeys(rel.getInput) + val immutableColumns = fmq.getImmutableColumns(rel.getInput) rel.getDistribution.getType match { case RelDistribution.Type.HASH_DISTRIBUTED => - filterKeys(keys, ImmutableBitSet.of(rel.getDistribution.getKeys)) - case RelDistribution.Type.SINGLETON => keys + filterKeys(upsertKeys, ImmutableBitSet.of(rel.getDistribution.getKeys), immutableColumns) + case RelDistribution.Type.SINGLETON => upsertKeys case t => throw new UnsupportedOperationException("Unsupported distribution type: " + t) } } @@ -95,19 +100,20 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] { case rank: StreamPhysicalRank if RankUtil.isDeduplication(rel) => ImmutableSet.of(ImmutableBitSet.of(rank.partitionKey.toArray.map(Integer.valueOf).toList)) case _ => - val inputKeys = filterKeys( - FlinkRelMetadataQuery - .reuseOrCreate(mq) - .getUpsertKeys(rel.getInput), - rel.partitionKey) + val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq) + val inputUpsertKeys = fmq.getUpsertKeys(rel.getInput) + val inputImmutableColumns = fmq.getImmutableColumns(rel.getInput) + val inputKeys = filterKeys(inputUpsertKeys, rel.partitionKey, inputImmutableColumns) FlinkRelMdUniqueKeys.INSTANCE.getRankUniqueKeys(rel, inputKeys) } } - def getUpsertKeys(rel: Sort, mq: RelMetadataQuery): JSet[ImmutableBitSet] = - filterKeys( - FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(rel.getInput), - ImmutableBitSet.of(rel.getCollation.getKeys)) + def getUpsertKeys(rel: Sort, mq: RelMetadataQuery): JSet[ImmutableBitSet] = { + val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq) + val upsertKeys = fmq.getUpsertKeys(rel.getInput) + val immutableColumns = fmq.getImmutableColumns(rel.getInput) + filterKeys(upsertKeys, ImmutableBitSet.of(rel.getCollation.getKeys), immutableColumns) + } def getUpsertKeys( rel: StreamPhysicalChangelogNormalize, @@ -206,20 +212,23 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] { rel: SingleRel, mq: RelMetadataQuery, distributionKeys: ImmutableBitSet*): JSet[ImmutableBitSet] = { - var inputKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(rel.getInput) + val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq) + var inputUpsertKeys = fmq.getUpsertKeys(rel.getInput) + val inputImmutableColumns = fmq.getImmutableColumns(rel.getInput) for (distributionKey <- distributionKeys) { - inputKeys = filterKeys(inputKeys, distributionKey) + inputUpsertKeys = filterKeys(inputUpsertKeys, distributionKey, inputImmutableColumns) } - inputKeys + inputUpsertKeys } def getUpsertKeys(join: Join, mq: RelMetadataQuery): JSet[ImmutableBitSet] = { val joinInfo = join.analyzeCondition() join.getJoinType match { case JoinRelType.SEMI | JoinRelType.ANTI => - filterKeys( - FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(join.getLeft), - joinInfo.leftSet()) + val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq) + val leftInputUpsertKeys = fmq.getUpsertKeys(join.getLeft) + val leftInputImmutableColumns = fmq.getImmutableColumns(join.getLeft) + filterKeys(leftInputUpsertKeys, joinInfo.leftSet(), leftInputImmutableColumns) case _ => getJoinUpsertKeys(joinInfo, join.getJoinType, join.getLeft, join.getRight, mq) } @@ -348,6 +357,8 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] { val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq) val leftKeys = fmq.getUpsertKeys(left) val rightKeys = fmq.getUpsertKeys(right) + val leftImmutableColumns = fmq.getImmutableColumns(left) + val rightImmutableColumns = fmq.getImmutableColumns(right) FlinkRelMdUniqueKeys.INSTANCE.getJoinUniqueKeys( joinRelType, @@ -356,8 +367,8 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] { // (the distribution keys), ensuring the result remains an upsert key. // Note: An Exchange typically applies this filtering already via fmq.getUpsertKeys(...). // We keep it here to be safe in case a join can appear without a preceding Exchange. - filterKeys(leftKeys, joinInfo.leftSet), - filterKeys(rightKeys, joinInfo.rightSet), + filterKeys(leftKeys, joinInfo.leftSet, leftImmutableColumns), + filterKeys(rightKeys, joinInfo.rightSet, rightImmutableColumns), isSideUnique(leftKeys, joinInfo.leftSet), isSideUnique(rightKeys, joinInfo.rightSet) ) @@ -392,17 +403,35 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] { * * Example: * - distributionKey = {k1} - * - keys = {{k1}, {k1, k2}, {k2}} + * - upsertKeys = {{k1}, {k1, k2}, {k2}} + * - immutableColumns = null * Result: {{k1}, {k1, k2}} (drops {k2}) + * + * Example: + * - distributionKey = {k1, k3} + * - upsertKeys = {{k1}, {k1, k2}, {k1, k3}, {k2}} + * - immutableColumns = {k3} + * Result: {{k1}, {k1, k2}, {k1, k3}} (drops {k2}) */ private def filterKeys( - keys: JSet[ImmutableBitSet], - distributionKey: ImmutableBitSet): JSet[ImmutableBitSet] = { - if (keys != null) { - keys.filter(k => k.contains(distributionKey)) - } else { - null + upsertKeys: JSet[ImmutableBitSet], + distributionKey: ImmutableBitSet, + immutableColumns: ImmutableBitSet): JSet[ImmutableBitSet] = { + if (upsertKeys == null) { + return null } + + upsertKeys.filter( + upsertKey => { + val key = + if (immutableColumns == null) { + upsertKey + } else { + upsertKey.union(immutableColumns) + } + + key.contains(distributionKey) + }) } /* @@ -426,6 +455,28 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] { // Catch-all rule when none of the others apply. def getUpsertKeys(rel: RelNode, mq: RelMetadataQuery): JSet[ImmutableBitSet] = null + + /** + * Enriches the given upsert keys with immutable columns as an additional upsert key. + * + * If baseKeys is null or empty, returns as-is without invoking the supplier (immutable columns + * are meaningless without existing upsert keys). + */ + private def enrichWithImmutableColumns( + baseKeys: JSet[ImmutableBitSet], + immutableColsSupplier: () => ImmutableBitSet): JSet[ImmutableBitSet] = { + if (baseKeys == null || baseKeys.isEmpty) { + return baseKeys + } + val immutableCols = immutableColsSupplier() + if (immutableCols != null && !immutableCols.isEmpty) { + val enriched = new util.HashSet[ImmutableBitSet](baseKeys) + enriched.add(immutableCols) + enriched + } else { + baseKeys + } + } } object FlinkRelMdUpsertKeys { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala index c660f00ad5af5..b11f9edc7c352 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala @@ -19,7 +19,6 @@ package org.apache.flink.table.planner.plan.optimize.program import org.apache.flink.legacy.table.sinks.{AppendStreamTableSink, RetractStreamTableSink, StreamTableSink, UpsertStreamTableSink} import org.apache.flink.table.api.{TableException, ValidationException} -import org.apache.flink.table.api.InsertConflictStrategy import org.apache.flink.table.api.InsertConflictStrategy.ConflictBehavior import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.api.config.ExecutionConfigOptions.UpsertMaterialize @@ -1559,10 +1558,11 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti // there are no upsert keys, so all columns are non-primary key columns true } else { - val upsertKey = upsertKeys.head - RexNodeExtractor - .extractRefInputFields(JavaScalaConversionUtil.toJava(Seq(condition))) - .exists(i => !upsertKey.get(i)) + val inputRefIndices = + RexNodeExtractor + .extractRefInputFields(JavaScalaConversionUtil.toJava(Seq(condition))) + val inputRefSet = ImmutableBitSet.of(inputRefIndices: _*) + !upsertKeys.stream().anyMatch(uk => uk.contains(inputRefSet)) } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/CatalogConstraintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/CatalogConstraintTest.java index 042bff697a09d..82786f3d4317a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/CatalogConstraintTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/catalog/CatalogConstraintTest.java @@ -22,6 +22,7 @@ import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.Column; @@ -38,6 +39,8 @@ import org.apache.calcite.util.ImmutableBitSet; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.util.Arrays; import java.util.Collections; @@ -46,6 +49,7 @@ import java.util.Map; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test for Catalog constraints. */ public class CatalogConstraintTest { @@ -57,30 +61,18 @@ public class CatalogConstraintTest { @BeforeEach void setup() { - EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + EnvironmentSettings settings = EnvironmentSettings.newInstance().inStreamingMode().build(); tEnv = TableEnvironment.create(settings); catalog = tEnv.getCatalog(tEnv.getCurrentCatalog()).orElse(null); assertThat(catalog).isNotNull(); } - @Test - void testWithPrimaryKey() throws Exception { - final Schema tableSchema = - Schema.newBuilder() - .fromResolvedSchema( - new ResolvedSchema( - Arrays.asList( - Column.physical("a", DataTypes.STRING()), - Column.physical("b", DataTypes.BIGINT().notNull()), - Column.physical("c", DataTypes.INT())), - Collections.emptyList(), - UniqueConstraint.primaryKey( - "primary_constraint", - Collections.singletonList("b")), - Collections.singletonList( - DefaultIndex.newIndex("idx", List.of("a", "b"))))) - .build(); - Map properties = buildCatalogTableProperties(); + @ParameterizedTest() + @ValueSource(booleans = {true, false}) + void testWithPrimaryKey(boolean containsPrimaryKey) throws Exception { + ResolvedSchema resolvedSchema = buildResolvedSchema(containsPrimaryKey); + final Schema tableSchema = Schema.newBuilder().fromResolvedSchema(resolvedSchema).build(); + Map properties = buildCatalogTableProperties(true); catalog.createTable( new ObjectPath(databaseName, "T1"), @@ -94,21 +86,62 @@ void testWithPrimaryKey() throws Exception { RelNode t1 = TableTestUtil.toRelNode(tEnv.sqlQuery("select * from T1")); FlinkRelMetadataQuery mq = FlinkRelMetadataQuery.reuseOrCreate(t1.getCluster().getMetadataQuery()); + if (containsPrimaryKey) { + assertThat(mq.getUniqueKeys(t1)).isEqualTo(ImmutableSet.of(ImmutableBitSet.of(1))); + } else { + assertThat(mq.getUniqueKeys(t1)).isEqualTo(ImmutableSet.of()); + } + } + + @ParameterizedTest() + @ValueSource(booleans = {true, false}) + void testWithImmutableColsConstraint(boolean containsImmutableColsConstraint) throws Exception { + final Schema.Builder schemaBuilder = + Schema.newBuilder().fromResolvedSchema(buildResolvedSchema(true)); + if (containsImmutableColsConstraint) { + schemaBuilder.immutableColumnsNamed("immutable_constraint", List.of("a")); + } + final Schema tableSchema = schemaBuilder.build(); + Map properties = buildCatalogTableProperties(false); + + catalog.createTable( + new ObjectPath(databaseName, "T1"), + CatalogTable.newBuilder() + .schema(tableSchema) + .comment("") + .options(properties) + .build(), + false); + + RelNode t1 = TableTestUtil.toRelNode(tEnv.sqlQuery("select * from T1")); + + FlinkRelMetadataQuery mq = + FlinkRelMetadataQuery.reuseOrCreate(t1.getCluster().getMetadataQuery()); + // unique keys are not changed assertThat(mq.getUniqueKeys(t1)).isEqualTo(ImmutableSet.of(ImmutableBitSet.of(1))); + + if (containsImmutableColsConstraint) { + assertThat((Iterable) mq.getImmutableColumns(t1)) + .isEqualTo(ImmutableBitSet.of(0, 1)); + assertThat(mq.getUpsertKeys(t1)) + .isEqualTo(ImmutableSet.of(ImmutableBitSet.of(1), ImmutableBitSet.of(0, 1))); + } else { + assertThat((Iterable) mq.getImmutableColumns(t1)) + .isEqualTo(ImmutableBitSet.of(1)); + assertThat(mq.getUpsertKeys(t1)).isEqualTo(ImmutableSet.of(ImmutableBitSet.of(1))); + } } @Test - void testWithoutPrimaryKey() throws Exception { - + void testImmutableColsConstraintDefinedOnSourceWithDelete() throws Exception { final Schema tableSchema = Schema.newBuilder() - .fromResolvedSchema( - ResolvedSchema.of( - Column.physical("a", DataTypes.BIGINT()), - Column.physical("b", DataTypes.STRING()), - Column.physical("c", DataTypes.INT()))) + .fromResolvedSchema(buildResolvedSchema(true)) + .immutableColumnsNamed("immutable_constraint", List.of("a")) .build(); - Map properties = buildCatalogTableProperties(); + + Map properties = buildCatalogTableProperties(false); + properties.put("changelog-mode", "I,UA,UB,D"); catalog.createTable( new ObjectPath(databaseName, "T1"), @@ -119,21 +152,44 @@ void testWithoutPrimaryKey() throws Exception { .build(), false); - RelNode t1 = TableTestUtil.toRelNode(tEnv.sqlQuery("select * from T1")); - FlinkRelMetadataQuery mq = - FlinkRelMetadataQuery.reuseOrCreate(t1.getCluster().getMetadataQuery()); - assertThat(mq.getUniqueKeys(t1)).isEqualTo(ImmutableSet.of()); + assertThatThrownBy(() -> TableTestUtil.toRelNode(tEnv.sqlQuery("select * from T1"))) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "The immutable constraint cannot be defined " + + "on the table with changelog mode [DELETE]."); } - private Map buildCatalogTableProperties() { - Map properties = new HashMap<>(); - properties.put("connector.type", "filesystem"); - properties.put("connector.property-version", "1"); - properties.put("connector.path", "/path/to/csv"); + private ResolvedSchema buildResolvedSchema(boolean containsPrimaryKey) { + return containsPrimaryKey + ? new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.STRING()), + Column.physical("b", DataTypes.BIGINT().notNull()), + Column.physical("c", DataTypes.INT())), + Collections.emptyList(), + UniqueConstraint.primaryKey( + "primary_constraint", Collections.singletonList("b")), + Collections.singletonList(DefaultIndex.newIndex("idx", List.of("a", "b"))), + null) + : ResolvedSchema.of( + Column.physical("a", DataTypes.BIGINT()), + Column.physical("b", DataTypes.STRING()), + Column.physical("c", DataTypes.INT())); + } - properties.put("format.type", "csv"); - properties.put("format.property-version", "1"); - properties.put("format.field-delimiter", ";"); + private Map buildCatalogTableProperties(boolean legacyTable) { + Map properties = new HashMap<>(); + if (legacyTable) { + properties.put("connector.type", "filesystem"); + properties.put("connector.property-version", "1"); + properties.put("connector.path", "/path/to/csv"); + + properties.put("format.type", "csv"); + properties.put("format.property-version", "1"); + properties.put("format.field-delimiter", ";"); + } else { + properties.put("connector", "values"); + } return properties; } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/lineage/TableLineageUtilsTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/lineage/TableLineageUtilsTest.java index 35d300ea35e7b..b0ffe87df9ad1 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/lineage/TableLineageUtilsTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/lineage/TableLineageUtilsTest.java @@ -54,7 +54,8 @@ class TableLineageUtilsTest { Column.physical("c", DataTypes.BOOLEAN())), Collections.emptyList(), null, - Collections.emptyList()); + Collections.emptyList(), + null); private static final Schema CATALOG_TABLE_SCHEMA = Schema.newBuilder().fromResolvedSchema(CATALOG_TABLE_RESOLVED_SCHEMA).build(); private static final Map TABLE_OPTIONS = new HashMap<>(); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java index 6f246c2fbb1c7..dd00b8c850652 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java @@ -39,6 +39,7 @@ import org.apache.flink.table.catalog.TableChange; import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.catalog.TableDistribution.Kind; +import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.catalog.WatermarkSpec; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; @@ -1469,9 +1470,9 @@ private static List createOrAlter(final String operation) { Column.physical("shop_id", DataTypes.BIGINT()), Column.physical("user_id", DataTypes.INT().notNull())), List.of(), - org.apache.flink.table.catalog.UniqueConstraint.primaryKey( - "PK_user_id", List.of("user_id")), - List.of())), + UniqueConstraint.primaryKey("PK_user_id", List.of("user_id")), + List.of(), + null)), Arguments.of( operation + "MATERIALIZED TABLE users_shops (PRIMARY KEY(user_id) NOT ENFORCED)" @@ -1482,9 +1483,9 @@ private static List createOrAlter(final String operation) { Column.physical("shop_id", DataTypes.INT().notNull()), Column.physical("user_id", DataTypes.INT().notNull())), List.of(), - org.apache.flink.table.catalog.UniqueConstraint.primaryKey( - "PK_user_id", List.of("user_id")), - List.of()))); + UniqueConstraint.primaryKey("PK_user_id", List.of("user_id")), + List.of(), + null))); } /** Boilerplate CatalogMaterializedTable builder for tests. */ diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReaderTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReaderTest.java index 302bcae32b62e..ce8a16f534b56 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReaderTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReaderTest.java @@ -102,7 +102,8 @@ void testGetFlinkPreparingTableBase() { Collections.emptyList(), Collections.emptyList(), null, - Collections.emptyList()); + Collections.emptyList(), + null); final CatalogTable catalogTable = ConnectorCatalogTable.source( new TestTableSource( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMdImmutableColumnsTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMdImmutableColumnsTest.java new file mode 100644 index 0000000000000..02d4988573943 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMdImmutableColumnsTest.java @@ -0,0 +1,527 @@ +/* + * 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.metadata; + +import org.apache.flink.table.planner.calcite.FlinkContext; +import org.apache.flink.table.planner.plan.nodes.calcite.LogicalWatermarkAssigner; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner; +import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Calc; +import org.apache.calcite.rel.core.JoinInfo; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.ImmutableIntList; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import scala.Option; + +import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapContext; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +/** Tests for {@link FlinkRelMdImmutableColumns}. */ +public class FlinkRelMdImmutableColumnsTest extends FlinkRelMdHandlerTestBase { + + // ------------------------------------------------------------------------------------- + // TableScan + // ------------------------------------------------------------------------------------- + + @Test + public void testGetImmutableColumnsOnTableScanWithImmutableCols() { + // Projected rowType (a,c,d): PK(a)=0, immutable(c)=1, immutable(d)=2 → {0, 1, 2} + assertEquals( + ImmutableBitSet.of(0, 1, 2), + mq().getImmutableColumns(tableWithImmutableColsLogicalScan())); + } + + @Test + public void testGetImmutableColumnsOnTableScanWithPkOnly() { + // TableSourceTable1: PK(a,b)={0,1}, no immutable constraint → {0, 1} + RelNode scan = relBuilder().scan("TableSourceTable1").build(); + assertEquals(ImmutableBitSet.of(0, 1), mq().getImmutableColumns(scan)); + } + + @Test + public void testGetImmutableColumnsOnTableScanWithSingleColPk() { + // TableSourceTable2: PK(b)={1} → {1} + RelNode scan = relBuilder().scan("TableSourceTable2").build(); + assertEquals(ImmutableBitSet.of(1), mq().getImmutableColumns(scan)); + } + + @Test + public void testGetImmutableColumnsOnTableScanWithoutPk() { + // TableSourceTable3: no PK → null + RelNode scan = relBuilder().scan("TableSourceTable3").build(); + assertNull(mq().getImmutableColumns(scan)); + } + + @Test + public void testGetImmutableColumnsOnNonTableSourceTableScan() { + // student uses MockMetaTable (not TableSourceTable) → null + assertNull(mq().getImmutableColumns(studentLogicalScan())); + } + + // ------------------------------------------------------------------------------------- + // Project + // ------------------------------------------------------------------------------------- + + @Test + public void testGetImmutableColumnsOnProjectKeepsAll() { + // Project: a(0), c(1), d(2) → output immutable = {0, 1, 2} + relBuilder().push(tableWithImmutableColsLogicalScan()); + RelNode project = + relBuilder() + .project( + relBuilder().field(0), // a → out 0 + relBuilder().field(1), // c → out 1 + relBuilder().field(2)) // d → out 2 + .build(); + assertEquals(ImmutableBitSet.of(0, 1, 2), mq().getImmutableColumns(project)); + } + + @Test + public void testGetImmutableColumnsOnProjectDropsImmutableCols() { + // Project with only expressions (no direct field refs) → none tracked → empty + relBuilder().push(tableWithImmutableColsLogicalScan()); + RelNode project = + relBuilder() + .project( + relBuilder() + .call( + SqlStdOperatorTable.PLUS, + relBuilder().field(0), + relBuilder().literal(1))) + .build(); + assertEquals(ImmutableBitSet.of(), mq().getImmutableColumns(project)); + } + + @Test + public void testGetImmutableColumnsOnProjectWithDuplicateRefs() { + // Project: a(0), a(0), c(1) → a maps to {0, 1}, c maps to {2} → {0, 1, 2} + relBuilder().push(tableWithImmutableColsLogicalScan()); + RelNode project = + relBuilder() + .project( + relBuilder().field(0), // a → out 0 + relBuilder().field(0), // a → out 1 + relBuilder().field(1)) // c → out 2 + .build(); + assertEquals(ImmutableBitSet.of(0, 1, 2), mq().getImmutableColumns(project)); + } + + @Test + public void testGetImmutableColumnsOnProjectWithExpression() { + // Project: a(0), a+1 (expression), c(1) + // Expressions (non-RexInputRef) are not tracked → a+1 is not immutable + relBuilder().push(tableWithImmutableColsLogicalScan()); + RelNode project = + relBuilder() + .project( + relBuilder().field(0), // a → out 0 + relBuilder() + .call( + SqlStdOperatorTable.PLUS, + relBuilder().field(0), + relBuilder().literal(1)), // a+1 → out 1 + relBuilder().field(1)) // c → out 2 + .build(); + // a→0, c→2 → {0, 2} + assertEquals(ImmutableBitSet.of(0, 2), mq().getImmutableColumns(project)); + } + + @Test + public void testGetImmutableColumnsOnProjectNullInput() { + // Project on student (MockMetaTable immutable = null) → null + relBuilder().push(studentLogicalScan()); + RelNode project = + relBuilder().project(relBuilder().field(0), relBuilder().field(1)).build(); + assertNull(mq().getImmutableColumns(project)); + } + + // ------------------------------------------------------------------------------------- + // Filter + // ------------------------------------------------------------------------------------- + + @Test + public void testGetImmutableColumnsOnFilter() { + // Filter passes through immutable columns unchanged + relBuilder().push(tableWithImmutableColsLogicalScan()); + RelNode filter = + relBuilder() + .filter( + relBuilder() + .call( + SqlStdOperatorTable.LESS_THAN, + relBuilder().field(0), + relBuilder().literal(100))) + .build(); + assertEquals(ImmutableBitSet.of(0, 1, 2), mq().getImmutableColumns(filter)); + } + + @Test + public void testGetImmutableColumnsOnFilterNullInput() { + // Filter on student (MockMetaTable immutable = null) → null + assertNull(mq().getImmutableColumns(logicalFilter())); + } + + // ------------------------------------------------------------------------------------- + // Calc + // ------------------------------------------------------------------------------------- + + @Test + public void testGetImmutableColumnsOnCalc() { + // Calc with projection [a(0), c(2)] and filter a < 100 + RelNode input = tableWithImmutableColsLogicalScan(); + relBuilder().push(input); + + RexNode proj0 = relBuilder().field(0); // a + RexNode proj1 = relBuilder().field(1); // c + List projects = Arrays.asList(proj0, proj1); + + RexNode condition = + relBuilder() + .call( + SqlStdOperatorTable.LESS_THAN, + relBuilder().field(0), + relBuilder().literal(100)); + List conditions = Collections.singletonList(condition); + + // Build a temp project to get the output row type + RelNode tempProject = relBuilder().project(proj0, proj1).build(); + RelDataType outputRowType = tempProject.getRowType(); + + Calc calc = createLogicalCalc(input, outputRowType, projects, conditions); + + // Input immutable: {0, 1, 2} → a→0, c→1 → {0, 1} + assertEquals(ImmutableBitSet.of(0, 1), mq().getImmutableColumns(calc)); + } + + // ------------------------------------------------------------------------------------- + // WatermarkAssigner + // ------------------------------------------------------------------------------------- + + @Test + public void testGetImmutableColumnsOnWatermarkAssigner() { + // Build a WatermarkAssigner on top of the immutable scan, using rowtime column (index 3) + RelNode input = tableWithImmutableColsLogicalScan(); + FlinkContext flinkContext = unwrapContext(cluster()); + RexNode watermarkExpr = + flinkContext + .getRexFactory() + .createSqlToRexConverter(input.getRowType(), null) + .convertToRexNode("rowtime - INTERVAL '10' SECOND"); + RelNode watermarkAssigner = + LogicalWatermarkAssigner.create( + cluster(), input, Collections.emptyList(), 3, watermarkExpr); + // Pass through: {0, 1, 2} + assertEquals(ImmutableBitSet.of(0, 1, 2), mq().getImmutableColumns(watermarkAssigner)); + } + + // ------------------------------------------------------------------------------------- + // MiniBatchAssigner + // ------------------------------------------------------------------------------------- + + @Test + public void testGetImmutableColumnsOnMiniBatchAssigner() { + RelNode input = tableWithImmutableColsStreamScan(); + RelNode miniBatchAssigner = + new StreamPhysicalMiniBatchAssigner(cluster(), streamPhysicalTraits(), input); + // Pass through: {0, 1, 2} + assertEquals(ImmutableBitSet.of(0, 1, 2), mq().getImmutableColumns(miniBatchAssigner)); + } + + // ------------------------------------------------------------------------------------- + // Exchange + // ------------------------------------------------------------------------------------- + + @Test + public void testGetImmutableColumnsOnExchange() { + RelNode scan = tableWithImmutableColsStreamScan(); + FlinkRelDistribution hash = FlinkRelDistribution.hash(new int[] {0}, true); + RelNode exchange = + new StreamPhysicalExchange( + cluster(), streamPhysicalTraits().replace(hash), scan, hash); + // Pass through: {0, 1, 2} + assertEquals(ImmutableBitSet.of(0, 1, 2), mq().getImmutableColumns(exchange)); + } + + // ------------------------------------------------------------------------------------- + // ChangelogNormalize + // ------------------------------------------------------------------------------------- + + @Test + public void testGetImmutableColumnsOnChangelogNormalize() { + RelNode scan = tableWithImmutableColsStreamScan(); + RelNode changelogNormalize = + new StreamPhysicalChangelogNormalize( + cluster(), + streamPhysicalTraits(), + scan, + new int[] {0}, + null, + false, + new RexNode[] {}); + // Pass through: {0, 1, 2} + assertEquals(ImmutableBitSet.of(0, 1, 2), mq().getImmutableColumns(changelogNormalize)); + } + + // ------------------------------------------------------------------------------------- + // DropUpdateBefore + // ------------------------------------------------------------------------------------- + + @Test + public void testGetImmutableColumnsOnDropUpdateBefore() { + RelNode scan = tableWithImmutableColsStreamScan(); + RelNode dropUpdateBefore = + new StreamPhysicalDropUpdateBefore(cluster(), streamPhysicalTraits(), scan); + // Pass through: {0, 1, 2} + assertEquals(ImmutableBitSet.of(0, 1, 2), mq().getImmutableColumns(dropUpdateBefore)); + } + + // ------------------------------------------------------------------------------------- + // Join + // ------------------------------------------------------------------------------------- + + @Test + public void testGetImmutableColumnsOnInnerJoin1() { + // Left: projected_table_source_table_with_immutable_cols (a,c,d,rowtime) → + // immutable={0,1,2} + // Right: TableSourceTable1 (a,b,c,d) → immutable={0,1} + // Right shifted by 4: {4,5} + // Union: {0,1,2,4,5} + RelNode join = + relBuilder() + .scan("projected_table_source_table_with_immutable_cols") + .scan("TableSourceTable1") + .join( + JoinRelType.INNER, + relBuilder() + .call( + SqlStdOperatorTable.EQUALS, + relBuilder().field(2, 0, 0), + relBuilder().field(2, 1, 0))) + .build(); + assertEquals(ImmutableBitSet.of(0, 1, 2, 4, 5), mq().getImmutableColumns(join)); + } + + @Test + public void testGetImmutableColumnsOnInnerJoin2() { + // Left: projected_table_source_table_with_immutable_cols → {0,1,2} + // Right: projected_table_source_table_with_immutable_cols → {0,1,2} shifted by 4 → {4,5,6} + // Union: {0,1,2,4,5,6} + RelNode join = + relBuilder() + .scan("projected_table_source_table_with_immutable_cols") + .scan("projected_table_source_table_with_immutable_cols") + .join( + JoinRelType.INNER, + relBuilder() + .call( + SqlStdOperatorTable.EQUALS, + relBuilder().field(2, 0, 0), + relBuilder().field(2, 1, 0))) + .build(); + assertEquals(ImmutableBitSet.of(0, 1, 2, 4, 5, 6), mq().getImmutableColumns(join)); + } + + @Test + public void testGetImmutableColumnsOnInnerJoinWhileOneSideNoUpsertKey() { + // Left: TableSourceTable3 → immutable & pk = null + // Right: projected_table_source_table_with_immutable_cols → {0,1,2} shifted by 7 → {7,8,9} + // Join has no upsert keys → guarded to null + RelNode join = + relBuilder() + .scan("TableSourceTable3") + .scan("projected_table_source_table_with_immutable_cols") + .join( + JoinRelType.INNER, + relBuilder() + .call( + SqlStdOperatorTable.EQUALS, + relBuilder().field(2, 0, 0), + relBuilder().field(2, 1, 0))) + .build(); + assertNull(mq().getImmutableColumns(join)); + + // Left: projected_table_source_table_with_immutable_cols → {0,1,2} + // Right: TableSourceTable3 → immutable & pk = null + // Join has no upsert keys → guarded to null + join = + relBuilder() + .scan("projected_table_source_table_with_immutable_cols") + .scan("TableSourceTable3") + .join( + JoinRelType.INNER, + relBuilder() + .call( + SqlStdOperatorTable.EQUALS, + relBuilder().field(2, 0, 0), + relBuilder().field(2, 1, 0))) + .build(); + + assertNull(mq().getImmutableColumns(join)); + + // Left: TableSourceTable3 → immutable & pk = null + // Right: TableSourceTable3 → immutable & pk = null + // Join has no upsert keys → guarded to null + join = + relBuilder() + .scan("TableSourceTable3") + .scan("TableSourceTable3") + .join( + JoinRelType.INNER, + relBuilder() + .call( + SqlStdOperatorTable.EQUALS, + relBuilder().field(2, 0, 0), + relBuilder().field(2, 1, 0))) + .build(); + + assertNull(mq().getImmutableColumns(join)); + } + + @Test + public void testGetImmutableColumnsOnLeftJoin() { + // LEFT JOIN: right side may produce nulls → ignore right immutable + // Left: projected_table_source_table_with_immutable_cols → {0,1,2} + // Result: {0,1,2} + RelNode join = + relBuilder() + .scan("projected_table_source_table_with_immutable_cols") + .scan("projected_table_source_table_with_immutable_cols") + .join( + JoinRelType.LEFT, + relBuilder() + .call( + SqlStdOperatorTable.EQUALS, + relBuilder().field(2, 0, 0), + relBuilder().field(2, 1, 0))) + .build(); + assertEquals(ImmutableBitSet.of(0, 1, 2), mq().getImmutableColumns(join)); + } + + @Test + public void testGetImmutableColumnsOnRightJoin() { + // RIGHT JOIN: left side may produce nulls → ignore left immutable + // Right: projected_table_source_table_with_immutable_cols → shifted by 4 → {4,5,6} + // Result: {4,5,6} + RelNode join = + relBuilder() + .scan("projected_table_source_table_with_immutable_cols") + .scan("projected_table_source_table_with_immutable_cols") + .join( + JoinRelType.RIGHT, + relBuilder() + .call( + SqlStdOperatorTable.EQUALS, + relBuilder().field(2, 0, 0), + relBuilder().field(2, 1, 0))) + .build(); + assertEquals(ImmutableBitSet.of(4, 5, 6), mq().getImmutableColumns(join)); + } + + @Test + public void testGetImmutableColumnsOnFullJoin() { + // FULL JOIN: both sides may produce nulls → both ignored → null + RelNode join = + relBuilder() + .scan("projected_table_source_table_with_immutable_cols") + .scan("projected_table_source_table_with_immutable_cols") + .join( + JoinRelType.FULL, + relBuilder() + .call( + SqlStdOperatorTable.EQUALS, + relBuilder().field(2, 0, 0), + relBuilder().field(2, 1, 0))) + .build(); + assertNull(mq().getImmutableColumns(join)); + } + + // ------------------------------------------------------------------------------------- + // Lookup Join + // ------------------------------------------------------------------------------------- + + @Test + public void testGetImmutableColumnsOnLookupJoinWithImmutableCols() { + // Left: projected_table_source_table_with_immutable_cols → {0,1,2} + // Right projected_table_source_table_with_immutable_cols = {0,1,2}, ignored + // Result: {0,1,2} + TableScan src = tableWithImmutableColsStreamScan(); + StreamPhysicalLookupJoin lookupJoin = + getStreamLookupJoinsWithImmutableCols( + src, + src.getTable(), + JoinInfo.of(ImmutableIntList.of(0), ImmutableIntList.of(0)), + JoinRelType.INNER, + Option.empty()); + assertEquals(ImmutableBitSet.of(0, 1, 2), mq().getImmutableColumns(lookupJoin)); + + lookupJoin = + getStreamLookupJoinsWithImmutableCols( + src, + src.getTable(), + JoinInfo.of(ImmutableIntList.of(0), ImmutableIntList.of(0)), + JoinRelType.LEFT, + Option.empty()); + assertEquals(ImmutableBitSet.of(0, 1, 2), mq().getImmutableColumns(lookupJoin)); + + // join without lookup side's pk + // Result: null + lookupJoin = + getStreamLookupJoinsWithImmutableCols( + src, + src.getTable(), + JoinInfo.of(ImmutableIntList.of(0), ImmutableIntList.of(1)), + JoinRelType.LEFT, + Option.empty()); + assertNull(mq().getImmutableColumns(lookupJoin)); + } + + // ------------------------------------------------------------------------------------- + // Default (catch-all) + // ------------------------------------------------------------------------------------- + + @Test + public void testGetImmutableColumnsOnDefault() { + // TestRel has no specific handler → catch-all returns null + assertNull(mq().getImmutableColumns(testRel())); + } + + @Test + public void testGetImmutableColumnsOnValues() { + // LogicalValues has no specific handler → catch-all returns null + assertNull(mq().getImmutableColumns(logicalValues())); + assertNull(mq().getImmutableColumns(emptyValues())); + } +} 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 6417922b10732..0ba42e3dd68c4 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 @@ -83,7 +83,8 @@ public class ContextResolvedTableSerdeTest { Collections.emptyList(), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("a")))); + DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null); private static final Schema CATALOG_TABLE_SCHEMA = Schema.newBuilder().fromResolvedSchema(CATALOG_TABLE_RESOLVED_SCHEMA).build(); @@ -416,7 +417,8 @@ void withDifferentSchema() throws Exception { null, Collections.singletonList( DefaultIndex.newIndex( - "idx", Collections.singletonList("a")))); + "idx", Collections.singletonList("a"))), + null); final ContextResolvedTable spec = ContextResolvedTable.permanent( PERMANENT_TABLE_IDENTIFIER, diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java index 461b1346e7bea..cc151c093821c 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java @@ -90,7 +90,8 @@ static Stream testDynamicTableSinkSpecSerde() { Collections.emptyList(), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("a")))); + DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null); final CatalogTable catalogTable1 = CatalogTable.newBuilder() .schema(Schema.newBuilder().fromResolvedSchema(resolvedSchema1).build()) @@ -122,7 +123,8 @@ static Stream testDynamicTableSinkSpecSerde() { Collections.emptyList(), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("a")))); + DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null); final CatalogTable catalogTable2 = CatalogTable.newBuilder() .schema(Schema.newBuilder().fromResolvedSchema(resolvedSchema2).build()) @@ -160,7 +162,8 @@ static Stream testDynamicTableSinkSpecSerde() { Collections.emptyList(), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("a")))); + DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null); final CatalogTable catalogTable3 = CatalogTable.newBuilder() .schema(Schema.newBuilder().fromResolvedSchema(resolvedSchema3).build()) @@ -195,7 +198,8 @@ static Stream testDynamicTableSinkSpecSerde() { Collections.emptyList(), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("a")))); + DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null); final CatalogTable catalogTable4 = CatalogTable.newBuilder() .schema(Schema.newBuilder().fromResolvedSchema(resolvedSchema4).build()) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java index c32eaae35ce2c..6fd5942f19e43 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java @@ -107,7 +107,8 @@ public static Stream testDynamicTableSinkSpecSerde() { Collections.emptyList(), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("a")))); + DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null); final CatalogTable catalogTable1 = CatalogTable.newBuilder() @@ -146,7 +147,8 @@ public static Stream testDynamicTableSinkSpecSerde() { Collections.emptyList(), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("a")))); + DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null); final CatalogTable catalogTable2 = CatalogTable.newBuilder() @@ -393,7 +395,8 @@ static ResolvedCatalogTable tableWithOnlyPhysicalColumns(Map opt Collections.emptyList(), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("a")))); + DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null); return new ResolvedCatalogTable( CatalogTable.newBuilder() 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 9e6162c9cccf0..323221bca370d 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 @@ -26,6 +26,7 @@ import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.DefaultIndex; import org.apache.flink.table.catalog.ExternalCatalogTable; +import org.apache.flink.table.catalog.ImmutableColumnsConstraint; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.TableDistribution; @@ -96,7 +97,9 @@ class ResolvedCatalogTableSerdeTest { Collections.singletonList(WatermarkSpec.of("b", REX_NODE_EXPRESSION)), UniqueConstraint.primaryKey("myPrimaryKey", Arrays.asList("a", "c")), Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("b")))); + DefaultIndex.newIndex("idx", Collections.singletonList("b"))), + ImmutableColumnsConstraint.immutableColumns( + "imt", Collections.singletonList("d"))); private static final ResolvedCatalogTable FULL_RESOLVED_CATALOG_TABLE = new ResolvedCatalogTable( @@ -126,7 +129,8 @@ static Stream testResolvedCatalogTableSerde() { Collections.emptyList(), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("a")))); + DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null); return Stream.of( FULL_RESOLVED_CATALOG_TABLE, diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/TemporalTableSourceSpecSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/TemporalTableSourceSpecSerdeTest.java index 7eb6132a75d80..78804d8d0a289 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/TemporalTableSourceSpecSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/TemporalTableSourceSpecSerdeTest.java @@ -79,7 +79,8 @@ public static Stream testTemporalTableSourceSpecSerde() Collections.emptyList(), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("a")))); + DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null); final CatalogTable catalogTable1 = CatalogTable.newBuilder() diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/VectorSearchTableSourceSpecSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/VectorSearchTableSourceSpecSerdeTest.java index 8604882b7de98..cef6487cc8b8c 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/VectorSearchTableSourceSpecSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/VectorSearchTableSourceSpecSerdeTest.java @@ -78,7 +78,8 @@ public static Stream testVectorSearchTableSourceSpe Collections.singletonList(Column.physical("a", DataTypes.BIGINT())), Collections.emptyList(), null, - Collections.emptyList()); + Collections.emptyList(), + null); final CatalogTable catalogTable1 = CatalogTable.newBuilder() diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/DataStreamJavaITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/DataStreamJavaITCase.java index 94ebdba3d049c..98310727df5ab 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/DataStreamJavaITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/DataStreamJavaITCase.java @@ -370,7 +370,8 @@ void testFromAndToDataStreamEventTime() throws Exception { TIMESTAMP_LTZ(3), "`SOURCE_WATERMARK`()"))), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("f0"))))); + DefaultIndex.newIndex("idx", Collections.singletonList("f0"))), + null)); tableEnv.createTemporaryView("t", table); @@ -644,7 +645,8 @@ void testToDataStreamCustomEventTime() throws Exception { TIMESTAMP(3), "`SOURCE_WATERMARK`()"))), null, Collections.singletonList( - DefaultIndex.newIndex("idx", Collections.singletonList("f0"))))); + DefaultIndex.newIndex("idx", Collections.singletonList("f0"))), + null)); final DataStream rowtimeStream = tableEnv.toDataStream(table) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/ImmutableColConstraintTestUtils.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/ImmutableColConstraintTestUtils.java new file mode 100644 index 0000000000000..cdf9e6efcd008 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/ImmutableColConstraintTestUtils.java @@ -0,0 +1,66 @@ +/* + * 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.utils; + +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ImmutableColumnsConstraint; +import org.apache.flink.table.catalog.ObjectPath; + +import java.util.List; + +/** + * Utils about {@link ImmutableColumnsConstraint} for tests. + * + *

This utils can be removed after we support syntax to define immutable columns constraint in + * DDL. + */ +public class ImmutableColConstraintTestUtils { + + public static void addImmutableColConstraint( + Catalog catalog, String databaseName, String tableName, String... immutableCols) + throws Exception { + ObjectPath tablePath = new ObjectPath(databaseName, tableName); + CatalogTable originalTable = (CatalogTable) catalog.getTable(tablePath); + catalog.dropTable(tablePath, false); + + Schema.UnresolvedImmutableColumns immutableColumns = + new Schema.UnresolvedImmutableColumns("imt", List.of(immutableCols)); + + Schema schema = originalTable.getUnresolvedSchema(); + schema = + new Schema( + schema.getColumns(), + schema.getWatermarkSpecs(), + schema.getPrimaryKey().orElse(null), + schema.getIndexes(), + immutableColumns); + + CatalogTable newTable = + CatalogTable.newBuilder() + .schema(schema) + .comment(originalTable.getComment()) + .partitionKeys(originalTable.getPartitionKeys()) + .options(originalTable.getOptions()) + .build(); + + catalog.createTable(tablePath, newTable, false); + } +} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.xml index 72ac683e6199c..071900e4bada2 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.xml @@ -16,6 +16,130 @@ See the License for the specific language governing permissions and limitations under the License. --> + + + 'Tom' and score > 90 +]]> + + + ($2, _UTF-16LE'Tom'), >($0, 90))]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +]]> + + + (name, 'Tom'), >(score, 90))], changelogMode=[I,UB,UA]) ++- TableSourceScan(table=[[default_catalog, default_database, src, filter=[]]], fields=[score, note, name, id], changelogMode=[I,UB,UA]) +]]> + + + + + 'Tom' +]]> + + + ($0, _UTF-16LE'Tom')]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +]]> + + + (name, 'Tom')], changelogMode=[I,UB,UA]) ++- TableSourceScan(table=[[default_catalog, default_database, src, filter=[]]], fields=[name, score, id], changelogMode=[I,UB,UA]) +]]> + + + + + 5 +]]> + + + ($2, 5)]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +]]> + + + (id, 5)], changelogMode=[I,UA]) ++- DropUpdateBefore(changelogMode=[I,UA]) + +- TableSourceScan(table=[[default_catalog, default_database, src, filter=[]]], fields=[name, score, id], changelogMode=[I,UB,UA]) +]]> + + + + + 90 and name = 'Tom' and id > 0 +]]> + + + ($0, 90), =($2, _UTF-16LE'Tom'), >($3, 0))]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +]]> + + + (score, 90), =(name, 'Tom'), >(id, 0))], changelogMode=[I,UA]) ++- DropUpdateBefore(changelogMode=[I,UA]) + +- TableSourceScan(table=[[default_catalog, default_database, src, filter=[]]], fields=[score, note, name, id], changelogMode=[I,UB,UA]) +]]> + + + + + 'Tom' +]]> + + + ($0, _UTF-16LE'Tom')]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +]]> + + + (name, 'Tom')], changelogMode=[I,UA]) ++- DropUpdateBefore(changelogMode=[I,UA]) + +- TableSourceScan(table=[[default_catalog, default_database, src, filter=[]]], fields=[name, score, id], changelogMode=[I,UB,UA]) +]]> + + + + + 'Tom' and score > 90 +]]> + + + ($2, _UTF-16LE'Tom'), >($0, 90))]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +]]> + + + (name, 'Tom'), >(score, 90))], changelogMode=[I,UA]) ++- DropUpdateBefore(changelogMode=[I,UA]) + +- TableSourceScan(table=[[default_catalog, default_database, src, filter=[]]], fields=[score, note, name, id], changelogMode=[I,UB,UA]) +]]> + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala index 3e8bf996630a1..88dd7214062e2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala @@ -216,6 +216,15 @@ class FlinkRelMdHandlerTestBase { protected lazy val temporalTableStreamScan: StreamPhysicalDataStreamScan = createDataStreamScan(ImmutableList.of("TemporalTable4"), streamPhysicalTraits) + protected lazy val tableWithImmutableColsLogicalScan: LogicalTableScan = + createTableSourceTable( + ImmutableList.of("projected_table_source_table_with_immutable_cols"), + logicalTraits) + protected lazy val tableWithImmutableColsStreamScan: StreamPhysicalDataStreamScan = + createTableSourceTable( + ImmutableList.of("projected_table_source_table_with_immutable_cols"), + streamPhysicalTraits) + private lazy val valuesType = relBuilder.getTypeFactory .builder() .add("a", SqlTypeName.BIGINT) @@ -2694,6 +2703,25 @@ class FlinkRelMdHandlerTestBase { (batchLookupJoin, streamLookupJoin) } + protected def getStreamLookupJoinsWithImmutableCols( + leftInput: RelNode, + temporalTable: RelOptTable, + joinInfo: JoinInfo, + joinType: JoinRelType, + calcOnTemporalTable: Option[RexProgram]): StreamPhysicalLookupJoin = { + new StreamPhysicalLookupJoin( + cluster, + streamPhysicalTraits, + leftInput, + temporalTable, + calcOnTemporalTable, + joinInfo, + joinType, + Option.empty[RelHint], + false + ) + } + // select * from MyTable1 join MyTable4 on MyTable1.b = MyTable4.a protected lazy val logicalInnerJoinOnUniqueKeys: RelNode = relBuilder .scan("MyTable1") diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeysTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeysTest.scala index 3b7524e1bd3cd..996ef4f2de8bf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeysTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeysTest.scala @@ -17,17 +17,25 @@ */ package org.apache.flink.table.planner.plan.metadata +import org.apache.flink.table.planner.plan.`trait`.FlinkRelDistribution import org.apache.flink.table.planner.plan.nodes.calcite.LogicalExpand -import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalOverAggregate +import org.apache.flink.table.planner.plan.nodes.physical.stream.{StreamPhysicalExchange, StreamPhysicalOverAggregate, StreamPhysicalRank, StreamPhysicalTableSourceScan} import org.apache.flink.table.planner.plan.schema.TableSourceTable -import org.apache.flink.table.planner.plan.utils.ExpandUtil +import org.apache.flink.table.planner.plan.utils.{ExpandUtil, RankProcessStrategy} +import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankType} import com.google.common.collect.{ImmutableList, ImmutableSet} import org.apache.calcite.prepare.CalciteCatalogReader -import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.`type`.RelDataTypeFieldImpl +import org.apache.calcite.rel.{RelCollations, RelFieldCollation, RelNode} +import org.apache.calcite.rel.core.{JoinRelType, Window} import org.apache.calcite.rel.hint.RelHint +import org.apache.calcite.rex.{RexInputRef, RexNode, RexWindowBounds} import org.apache.calcite.sql.`type`.SqlTypeName.VARCHAR -import org.apache.calcite.sql.fun.SqlStdOperatorTable.{EQUALS, LESS_THAN} +import org.apache.calcite.sql.SqlWindow +import org.apache.calcite.sql.fun.SqlStdOperatorTable.{EQUALS, LESS_THAN, MAX} +import org.apache.calcite.sql.parser.SqlParserPos import org.apache.calcite.util.ImmutableBitSet import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -477,6 +485,182 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(intermediateScan).toSet) } + @Test + def testGetUpsertKeysOnTableScanWithImmutableCols(): Unit = { + // Immutable columns: {0, 1, 2} (PK 'a' + immutable 'c', 'd') + assertEquals( + toBitSet(Array(0), Array(0, 1, 2)), + mq.getUpsertKeys(tableWithImmutableColsLogicalScan).toSet) + } + + @Test + def testGetUpsertKeysOnExchangeWithImmutableCols(): Unit = { + // Hash exchange on column 1 (c, immutable) + val hash1 = FlinkRelDistribution.hash(Array(1), requireStrict = true) + val exchange1 = new StreamPhysicalExchange( + cluster, + streamPhysicalTraits.replace(hash1), + tableWithImmutableColsStreamScan, + hash1) + assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(exchange1).toSet) + + // Hash exchange on column 3 (rowtime, NOT immutable) + val hash3 = FlinkRelDistribution.hash(Array(3), requireStrict = true) + val exchange3 = new StreamPhysicalExchange( + cluster, + streamPhysicalTraits.replace(hash3), + tableWithImmutableColsStreamScan, + hash3) + assertEquals(toBitSet(), mq.getUpsertKeys(exchange3).toSet) + } + + @Test + def testGetUpsertKeysOnSortWithImmutableCols(): Unit = { + // Sort on column 1 (c, immutable) + relBuilder.push(tableWithImmutableColsLogicalScan) + val sort1 = relBuilder.sort(relBuilder.field(1)).build() + assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(sort1).toSet) + + // Sort on column 3 (rowtime, NOT immutable) + relBuilder.push(tableWithImmutableColsLogicalScan) + val sort3 = relBuilder.sort(relBuilder.field(3)).build() + assertEquals(toBitSet(), mq.getUpsertKeys(sort3).toSet) + } + + @Test + def testGetUpsertKeysOnRankWithImmutableCols(): Unit = { + def buildRank(partitionKey: Int): RelNode = { + val hash = FlinkRelDistribution.hash(Array(partitionKey), requireStrict = true) + val exchange = new StreamPhysicalExchange( + cluster, + tableWithImmutableColsStreamScan.getTraitSet.replace(hash), + tableWithImmutableColsStreamScan, + hash) + new StreamPhysicalRank( + cluster, + streamPhysicalTraits, + exchange, + ImmutableBitSet.of(partitionKey), + RelCollations.of(2), + RankType.RANK, + new ConstantRankRange(1, 5), + new RelDataTypeFieldImpl("rk", 4, longType), + true, + RankProcessStrategy.UNDEFINED_STRATEGY, + false + ) + } + + // Rank partitioned by column 1 (c, immutable) + val rank1 = buildRank(1) + assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(rank1).toSet) + + // Rank partitioned by column 3 (rowtime, NOT immutable) + val rank3 = buildRank(3) + assertEquals(toBitSet(), mq.getUpsertKeys(rank3).toSet) + } + + @Test + def testGetUpsertKeysOnOverAggWithImmutableCols(): Unit = { + def buildOverAgg(partitionKey: Int): RelNode = { + val inputRowType = tableWithImmutableColsStreamScan.getRowType + val rowtimeType = inputRowType.getFieldList.get(3).getType + + val group = new Window.Group( + ImmutableBitSet.of(partitionKey), + true, + RexWindowBounds.create(SqlWindow.createUnboundedPreceding(new SqlParserPos(0, 0)), null), + RexWindowBounds.create(SqlWindow.createCurrentRow(new SqlParserPos(0, 0)), null), + RelCollations.of( + new RelFieldCollation( + 2, + RelFieldCollation.Direction.ASCENDING, + RelFieldCollation.NullDirection.FIRST)), + ImmutableList.of( + new Window.RexWinAggCall( + MAX, + rowtimeType, + ImmutableList.of[RexNode](new RexInputRef(3, rowtimeType)), + 0, + false, + false + ) + ) + ) + + val outputBuilder = typeFactory.builder() + inputRowType.getFieldList.forEach(f => outputBuilder.add(f.getName, f.getType)) + outputBuilder.add("max_rowtime", rowtimeType) + val outputRowType = outputBuilder.build() + + val logicalOverAgg = new FlinkLogicalOverAggregate( + cluster, + flinkLogicalTraits, + tableWithImmutableColsLogicalScan, + ImmutableList.of(), + outputRowType, + ImmutableList.of(group) + ) + + val hash = FlinkRelDistribution.hash(Array(partitionKey), requireStrict = true) + val exchange = new StreamPhysicalExchange( + cluster, + tableWithImmutableColsStreamScan.getTraitSet.replace(hash), + tableWithImmutableColsStreamScan, + hash) + + new StreamPhysicalOverAggregate( + cluster, + streamPhysicalTraits, + exchange, + outputRowType, + logicalOverAgg + ) + } + + // Over agg partitioned by column 1 (c, immutable) + val over1 = buildOverAgg(1) + assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(over1).toSet) + + // Over agg partitioned by column 3 (rowtime, NOT immutable) + val over3 = buildOverAgg(3) + assertEquals(toBitSet(), mq.getUpsertKeys(over3).toSet) + } + + @Test + def testGetUpsertKeysOnSemiAntiJoinWithImmutableCols(): Unit = { + // SEMI join on left.c(1) = right.c(1) + // Left upsert keys: {{0}, {0,1,2}}, left immutable: {0,1,2} + val join1 = relBuilder + .scan("projected_table_source_table_with_immutable_cols") + .scan("projected_table_source_table_with_immutable_cols") + .join( + JoinRelType.SEMI, + relBuilder.call(EQUALS, relBuilder.field(2, 0, 1), relBuilder.field(2, 1, 1))) + .build() + assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(join1).toSet) + } + + @Test + def testGetUpsertKeysOnInnerJoinWithImmutableCols(): Unit = { + // Inner join on left.c(1) = right.c(1) + // Both sides: upsert keys = {{0}, {0,1,2}}, immutable = {0,1,2} + // filterKeys on both sides with join key {1}: both retain {{0}, {0,1,2}} + // Neither side is unique on {1}, so only concatenated keys survive + // Right shifted by 4: {{4}, {4,5,6}} + // Concat: {0}x{4}, {0}x{4,5,6}, {0,1,2}x{4}, {0,1,2}x{4,5,6} + val join = relBuilder + .scan("projected_table_source_table_with_immutable_cols") + .scan("projected_table_source_table_with_immutable_cols") + .join( + JoinRelType.INNER, + relBuilder.call(EQUALS, relBuilder.field(2, 0, 1), relBuilder.field(2, 1, 1))) + .build() + assertEquals( + toBitSet(Array(0, 4), Array(0, 4, 5, 6), Array(0, 1, 2, 4), Array(0, 1, 2, 4, 5, 6)), + mq.getUpsertKeys(join).toSet) + } + private def toBitSet(keys: Array[Int]*): Set[ImmutableBitSet] = { keys.map(k => ImmutableBitSet.of(k: _*)).toSet } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/MetadataTestUtil.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/MetadataTestUtil.scala index e8c321236c251..8f5f94572e5a1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/MetadataTestUtil.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/MetadataTestUtil.scala @@ -70,6 +70,9 @@ object MetadataTestUtil { rootSchema.add( "projected_table_source_table_with_partial_pk", createProjectedTableSourceTableWithPartialCompositePrimaryKey()) + rootSchema.add( + "projected_table_source_table_with_immutable_cols", + createProjectedTableSourceTableWithImmutableCols()) rootSchema } @@ -321,7 +324,8 @@ object MetadataTestUtil { ), Collections.emptyList(), UniqueConstraint.primaryKey("PK_1", util.Arrays.asList("a", "d")), - Collections.singletonList(DefaultIndex.newIndex("idx", Collections.singletonList("a")))) + Collections.singletonList(DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null) val catalogTable = getCatalogTable(resolvedSchema) @@ -366,7 +370,8 @@ object MetadataTestUtil { ), Collections.emptyList(), UniqueConstraint.primaryKey("PK_1", util.Arrays.asList("a", "b")), - Collections.singletonList(DefaultIndex.newIndex("idx", Collections.singletonList("a")))) + Collections.singletonList(DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null) val typeFactory = new FlinkTypeFactory(Thread.currentThread().getContextClassLoader) val rowType = typeFactory.buildRelNodeRowType( @@ -394,7 +399,8 @@ object MetadataTestUtil { ), Collections.emptyList(), UniqueConstraint.primaryKey("PK_1", util.Arrays.asList("b")), - Collections.singletonList(DefaultIndex.newIndex("idx", Collections.singletonList("a")))) + Collections.singletonList(DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null) val catalogTable = getCatalogTable(resolvedSchema) @@ -424,7 +430,8 @@ object MetadataTestUtil { ), Collections.emptyList(), null, - Collections.singletonList(DefaultIndex.newIndex("idx", Collections.singletonList("a")))) + Collections.singletonList(DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null) val catalogTable = getCatalogTable(resolvedSchema) @@ -451,7 +458,8 @@ object MetadataTestUtil { Column.physical("b", DataTypes.BIGINT().notNull())), Collections.emptyList(), UniqueConstraint.primaryKey("PK_1", util.Arrays.asList("a", "b")), - Collections.singletonList(DefaultIndex.newIndex("idx", Collections.singletonList("a")))) + Collections.singletonList(DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + null) val catalogTable = getCatalogTable(resolvedSchema) @@ -472,6 +480,46 @@ object MetadataTestUtil { flinkContext) } + private def createProjectedTableSourceTableWithImmutableCols(): Table = { + val resolvedSchema = new ResolvedSchema( + util.Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical("b", DataTypes.INT().notNull()), + Column.physical("c", DataTypes.STRING().notNull()), + Column.physical("d", DataTypes.BIGINT().notNull()), + Column.physical("rowtime", DataTypes.TIMESTAMP(3)) + ), + Collections.emptyList(), + UniqueConstraint.primaryKey("PK_1", util.Arrays.asList("a")), + Collections.singletonList(DefaultIndex.newIndex("idx", Collections.singletonList("a"))), + ImmutableColumnsConstraint.immutableColumns("imt", util.Arrays.asList("c", "d"))) + + val catalogTable = getCatalogTable(resolvedSchema) + + // projected: drop column b, keep a, c, d, rowtime + val typeFactory = new FlinkTypeFactory(Thread.currentThread().getContextClassLoader) + val rowType = typeFactory.buildRelNodeRowType( + Seq("a", "c", "d", "rowtime"), + Seq( + new BigIntType(false), + new VarCharType(false, 100), + new BigIntType(false), + new TimestampType(true, TimestampKind.ROWTIME, 3))) + + new MockTableSourceTable( + rowType, + new TestTableSource(), + true, + ContextResolvedTable.temporary( + ObjectIdentifier.of( + "default_catalog", + "default_database", + "projected_table_source_table_with_immutable_cols"), + new ResolvedCatalogTable(catalogTable, resolvedSchema) + ), + flinkContext) + } + private def getCatalogTable(resolvedSchema: ResolvedSchema) = { CatalogTable .newBuilder() @@ -568,4 +616,13 @@ class MockTableSourceTable( call: SqlCall, parent: SqlNode, config: CalciteConnectionConfig): Boolean = false + + def copy(newTableSource: DynamicTableSource, newRowType: RelDataType): MockTableSourceTable = { + new MockTableSourceTable( + newRowType, + newTableSource, + isStreamingMode, + contextResolvedTable, + flinkContext) + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala index b37bf494dcc7b..156890a947330 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.rules.physical.stream import org.apache.flink.table.api.ExplainDetail import org.apache.flink.table.api.config.{AggregatePhaseStrategy, OptimizerConfigOptions} import org.apache.flink.table.planner.plan.optimize.program.FlinkChangelogModeInferenceProgram +import org.apache.flink.table.planner.utils.ImmutableColConstraintTestUtils.addImmutableColConstraint import org.apache.flink.table.planner.utils.TableTestBase import org.junit.jupiter.api.{BeforeEach, Test} @@ -265,4 +266,165 @@ class ChangelogModeInferenceTest extends TableTestBase { util.verifyRelPlan(statementSet, ExplainDetail.CHANGELOG_MODE) } + @Test + def testFilterSubsetOfUpsertKey(): Unit = { + util.tableEnv.executeSql(""" + |create table src ( + | name string, + | score int, + | id int primary key not enforced + |) with ( + | 'connector' = 'values', + | 'changelog-mode' = 'I,UA,UB' + |) + |""".stripMargin) + + val catalog = util.tableEnv.getCatalog(util.tableEnv.getCurrentCatalog).get() + addImmutableColConstraint(catalog, util.tableEnv.getCurrentDatabase, "src", "name") + + val sql = + """ + |select * from src where name <> 'Tom' + |""".stripMargin + + // upsert keys: {{name, id}, {id}} + // filter references: {name} + // upsert key {name, id} contains {name}, so UB can be dropped + util.verifyRelPlan(sql, ExplainDetail.CHANGELOG_MODE) + } + + @Test + def testFilterSubsetOfUpsertKey2(): Unit = { + util.tableEnv.executeSql(""" + |create table src ( + | score int, + | note string, + | name string, + | id int, + | primary key (name, id) not enforced + |) with ( + | 'connector' = 'values', + | 'changelog-mode' = 'I,UA,UB' + |) + |""".stripMargin) + + val catalog = util.tableEnv.getCatalog(util.tableEnv.getCurrentCatalog).get() + addImmutableColConstraint(catalog, util.tableEnv.getCurrentDatabase, "src", "score") + + val sql = + """ + |select * from src where name <> 'Tom' and score > 90 + |""".stripMargin + + // upsert keys: {{score, name, id}, {name, id}} + // filter references: {score, name} + // upsert key {score, name, id} contains {score, name}, so UB can be dropped + util.verifyRelPlan(sql, ExplainDetail.CHANGELOG_MODE) + } + + @Test + def testFilterNotContainedByAnyUpsertKey(): Unit = { + util.tableEnv.executeSql(""" + |create table src ( + | score int, + | note string, + | name string, + | id int, + | primary key (id) not enforced + |) with ( + | 'connector' = 'values', + | 'changelog-mode' = 'I,UA,UB' + |) + |""".stripMargin) + + val catalog = util.tableEnv.getCatalog(util.tableEnv.getCurrentCatalog).get() + addImmutableColConstraint(catalog, util.tableEnv.getCurrentDatabase, "src", "score") + + val sql = + """ + |select * from src where name <> 'Tom' and score > 90 + |""".stripMargin + + // upsert keys: {{score, id}, {id}} + // filter references: {score, name} + // upsert key {score, id} or {id} does not contain {score, name}, so UB cannot be dropped + util.verifyRelPlan(sql, ExplainDetail.CHANGELOG_MODE) + } + + @Test + def testFilterOneEntireUpsertKey(): Unit = { + util.tableEnv.executeSql(""" + |create table src ( + | score int, + | note string, + | name string, + | id int, + | primary key (name, id) not enforced + |) with ( + | 'connector' = 'values', + | 'changelog-mode' = 'I,UA,UB' + |) + |""".stripMargin) + + val catalog = util.tableEnv.getCatalog(util.tableEnv.getCurrentCatalog).get() + addImmutableColConstraint(catalog, util.tableEnv.getCurrentDatabase, "src", "score") + + val sql = + """ + |select * from src where score > 90 and name = 'Tom' and id > 0 + |""".stripMargin + + // upsert keys: {{score, name, id}, {name, id}} + // filter references: {score, name, id} + // upsert key {score, name, id} fully contains {score, name, id}, so UB can be dropped + util.verifyRelPlan(sql, ExplainDetail.CHANGELOG_MODE) + } + + @Test + def testFilterOnPrimaryKeyOnly(): Unit = { + util.tableEnv.executeSql(""" + |create table src ( + | name string, + | score int, + | id int primary key not enforced + |) with ( + | 'connector' = 'values', + | 'changelog-mode' = 'I,UA,UB' + |) + |""".stripMargin) + + val sql = + """ + |select * from src where id > 5 + |""".stripMargin + + // upsert keys: {{id}} + // filter references: {id} + // upsert key {id} fully contains {id}, so UB can be dropped + util.verifyRelPlan(sql, ExplainDetail.CHANGELOG_MODE) + } + + @Test + def testFilterOnNonUpsertKeyColOnly(): Unit = { + util.tableEnv.executeSql(""" + |create table src ( + | name string, + | score int, + | id int primary key not enforced + |) with ( + | 'connector' = 'values', + | 'changelog-mode' = 'I,UA,UB' + |) + |""".stripMargin) + + val sql = + """ + |select * from src where name <> 'Tom' + |""".stripMargin + + // upsert keys: {{id}} + // filter references: {name} + // upsert key {id} does not contain {name}, so UB cannot be dropped + util.verifyRelPlan(sql, ExplainDetail.CHANGELOG_MODE) + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala index c4d8e7962c08e..f1c7258392477 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.stream.sql.join import org.apache.flink.table.api._ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableFunc1, TableTestBase} +import org.apache.flink.table.planner.utils.ImmutableColConstraintTestUtils.addImmutableColConstraint import org.assertj.core.api.Assertions.assertThatThrownBy import org.junit.jupiter.api.Test @@ -510,6 +511,132 @@ class JoinTest extends TableTestBase { ) } + @Test + def testJoinOutputUpsertKeyInSinkPkWhileJoinOnImmutableCols(): Unit = { + val catalog = util.tableEnv.getCatalog(util.tableEnv.getCurrentCatalog).get() + + util.tableEnv.executeSql(""" + |create table source_city ( + | id varchar, + | city_name varchar, + | primary key (id) not enforced + |) with ( + | 'connector' = 'values', + | 'changelog-mode' = 'I,UA,UB' + |) + |""".stripMargin) + addImmutableColConstraint(catalog, util.tableEnv.getCurrentDatabase, "source_city", "city_name") + + util.tableEnv.executeSql(""" + |create table source_city_detail ( + | city_id varchar, + | city_name varchar, + | city_detail varchar, + | primary key (city_id) not enforced + |) with ( + | 'connector' = 'values', + | 'changelog-mode' = 'I,UA,UB' + |) + |""".stripMargin) + addImmutableColConstraint( + catalog, + util.tableEnv.getCurrentDatabase, + "source_city_detail", + "city_name") + + util.tableEnv.executeSql(""" + |create table sink ( + | id varchar, + | city_id varchar, + | city_name varchar, + | city_detail varchar, + | primary key (id, city_id) not enforced + |) with ( + | 'connector' = 'values' + | ,'sink-insert-only' = 'false' + |) + |""".stripMargin) + + // verify UB should be dropped and no upsertMaterialize + util.verifyExplainInsert( + """ + |insert into sink + |select t1.id, t2.city_id, t1.city_name, t2.city_detail + | from source_city t1 + | join source_city_detail t2 on t1.city_name = t2.city_name + |on conflict do deduplicate + |""".stripMargin, + ExplainDetail.CHANGELOG_MODE + ) + } + + @Test + def testJoinOutputUpsertKeyInSinkPkWhileJoinOnPartOfImmutableCols(): Unit = { + val catalog = util.tableEnv.getCatalog(util.tableEnv.getCurrentCatalog).get() + + util.tableEnv.executeSql(""" + |create table source_city ( + | id varchar, + | city_name varchar, + | city_no int, + | primary key (id) not enforced + |) with ( + | 'connector' = 'values', + | 'changelog-mode' = 'I,UA,UB' + |) + |""".stripMargin) + addImmutableColConstraint( + catalog, + util.tableEnv.getCurrentDatabase, + "source_city", + "city_name", + "city_no") + + util.tableEnv.executeSql(""" + |create table source_city_detail ( + | city_id varchar, + | city_name varchar, + | city_no int, + | city_detail varchar, + | primary key (city_id) not enforced + |) with ( + | 'connector' = 'values', + | 'changelog-mode' = 'I,UA,UB' + |) + |""".stripMargin) + addImmutableColConstraint( + catalog, + util.tableEnv.getCurrentDatabase, + "source_city_detail", + "city_name", + "city_no") + + util.tableEnv.executeSql(""" + |create table sink ( + | id varchar, + | city_id varchar, + | city_name varchar, + | city_detail varchar, + | primary key (id, city_id) not enforced + |) with ( + | 'connector' = 'values' + | ,'sink-insert-only' = 'false' + |) + |""".stripMargin) + + // verify UB should be dropped and no upsertMaterialize on sink + util.verifyExplainInsert( + """ + |insert into sink + |select t1.id, t2.city_id, t1.city_name, t2.city_detail + | from source_city t1 + | join source_city_detail t2 on t1.city_name = t2.city_name + |on conflict do deduplicate + |""".stripMargin, + ExplainDetail.CHANGELOG_MODE + ) + } + @Test def testInnerJoinWithFilterPushDown(): Unit = { util.verifyExecPlan(""" diff --git a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTest.java b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTest.java index 9be9499428c9f..7a117ef58c262 100644 --- a/flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTest.java +++ b/flink-test-utils-parent/flink-table-filesystem-test-utils/src/test/java/org/apache/flink/table/file/testutils/catalog/TestFileSystemCatalogTest.java @@ -29,6 +29,7 @@ import org.apache.flink.table.catalog.CatalogMaterializedTable.RefreshStatus; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ImmutableColumnsConstraint; import org.apache.flink.table.catalog.IntervalFreshness; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; @@ -69,13 +70,20 @@ public class TestFileSystemCatalogTest extends TestFileSystemCatalogTestBase { Column.physical("age", DataTypes.INT()), Column.physical("tss", DataTypes.TIMESTAMP(3)), Column.physical("partition", DataTypes.VARCHAR(10))); - private static final UniqueConstraint CONSTRAINTS = + private static final UniqueConstraint PK_CONSTRAINT = UniqueConstraint.primaryKey("primary_constraint", Collections.singletonList("id")); + private static final ImmutableColumnsConstraint IMMUTABLE_COLS_CONSTRAINT = + ImmutableColumnsConstraint.immutableColumns( + "imt_constraint", Collections.singletonList("name")); private static final List PARTITION_KEYS = Collections.singletonList("partition"); private static final ResolvedSchema CREATE_RESOLVED_SCHEMA = new ResolvedSchema( - CREATE_COLUMNS, Collections.emptyList(), CONSTRAINTS, Collections.emptyList()); + CREATE_COLUMNS, + Collections.emptyList(), + PK_CONSTRAINT, + Collections.emptyList(), + IMMUTABLE_COLS_CONSTRAINT); private static final Schema CREATE_SCHEMA = Schema.newBuilder().fromResolvedSchema(CREATE_RESOLVED_SCHEMA).build();