From 423f4529367a5ac7b4f8b1fd68c88286ef1b1182 Mon Sep 17 00:00:00 2001 From: yuxiqian <34335406+yuxiqian@users.noreply.github.com> Date: Tue, 30 Apr 2024 18:41:02 +0800 Subject: [PATCH] [FLINK-35272][cdc][runtime] Transform projection & filter feature overhaul --- .../flink/cdc/common/utils/SchemaUtils.java | 182 ++++ .../cdc/common/utils/SchemaUtilsTest.java | 268 +++++ .../composer/flink/FlinkPipelineComposer.java | 8 +- .../flink/translator/TransformTranslator.java | 40 +- .../pipeline/tests/TransformE2eITCase.java | 460 ++++++++- .../src/test/resources/ddl/transform_test.sql | 20 +- ...erator.java => PostTransformOperator.java} | 234 ++--- ...essor.java => PostTransformProcessor.java} | 91 +- .../operators/transform/PostTransformers.java | 53 + ...perator.java => PreTransformOperator.java} | 132 ++- .../transform/PreTransformProcessor.java | 111 +++ .../operators/transform/PreTransformers.java | 53 + .../operators/transform/ProjectionColumn.java | 27 + .../transform/ProjectionColumnProcessor.java | 37 +- .../operators/transform/TableInfo.java | 39 +- .../transform/TransformFilterProcessor.java | 9 +- .../operators/transform/TransformRule.java | 77 ++ .../cdc/runtime/parser/TransformParser.java | 219 ++++- ...st.java => PostTransformOperatorTest.java} | 324 +++++- .../transform/PreTransformOperatorTest.java | 520 ++++++++++ .../TransformSchemaOperatorTest.java | 228 ----- .../UnifiedTransformOperatorTest.java | 919 ++++++++++++++++++ .../runtime/parser/TransformParserTest.java | 109 +++ 23 files changed, 3588 insertions(+), 572 deletions(-) rename flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/{TransformDataOperator.java => PostTransformOperator.java} (67%) rename flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/{TransformProjectionProcessor.java => PostTransformProcessor.java} (66%) create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformers.java rename flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/{TransformSchemaOperator.java => PreTransformOperator.java} (69%) create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformProcessor.java create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformers.java create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformRule.java rename flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/{TransformDataOperatorTest.java => PostTransformOperatorTest.java} (69%) create mode 100644 flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorTest.java delete mode 100644 flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/TransformSchemaOperatorTest.java create mode 100644 flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/UnifiedTransformOperatorTest.java diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/SchemaUtils.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/SchemaUtils.java index 107e63d821c..ea4ce46e337 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/SchemaUtils.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/SchemaUtils.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.common.utils; import org.apache.flink.cdc.common.annotation.PublicEvolving; +import org.apache.flink.cdc.common.annotation.VisibleForTesting; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.event.AddColumnEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; @@ -26,11 +27,21 @@ import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypeFamily; +import org.apache.flink.cdc.common.types.DataTypeRoot; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.DecimalType; + +import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collections; import java.util.LinkedList; import java.util.List; +import java.util.Objects; import java.util.stream.Collectors; +import java.util.stream.IntStream; /** Utils for {@link Schema} to perform the ability of evolution. */ @PublicEvolving @@ -56,6 +67,177 @@ public static List createFieldGetters(List colum return fieldGetters; } + /** Restore original data fields from RecordData structure. */ + public static List restoreOriginalData( + @Nullable RecordData recordData, List fieldGetters) { + if (recordData == null) { + return Collections.emptyList(); + } + List actualFields = new ArrayList<>(); + for (RecordData.FieldGetter fieldGetter : fieldGetters) { + actualFields.add(fieldGetter.getFieldOrNull(recordData)); + } + return actualFields; + } + + /** Merge compatible upstream schemas. */ + public static Schema mergeCompatibleSchemas(List schemas) { + if (schemas.isEmpty()) { + return null; + } else if (schemas.size() == 1) { + return schemas.get(0); + } else { + Schema outputSchema = null; + for (Schema schema : schemas) { + outputSchema = mergeSchema(outputSchema, schema); + } + return outputSchema; + } + } + + /** Try to combine two schemas with potential incompatible type. */ + @VisibleForTesting + public static Schema mergeSchema(@Nullable Schema lhs, Schema rhs) { + if (lhs == null) { + return rhs; + } + if (lhs.getColumnCount() != rhs.getColumnCount()) { + throw new IllegalStateException( + String.format( + "Unable to merge schema %s and %s with different column counts.", + lhs, rhs)); + } + if (!lhs.primaryKeys().equals(rhs.primaryKeys())) { + throw new IllegalStateException( + String.format( + "Unable to merge schema %s and %s with different primary keys.", + lhs, rhs)); + } + if (!lhs.partitionKeys().equals(rhs.partitionKeys())) { + throw new IllegalStateException( + String.format( + "Unable to merge schema %s and %s with different partition keys.", + lhs, rhs)); + } + if (!lhs.options().equals(rhs.options())) { + throw new IllegalStateException( + String.format( + "Unable to merge schema %s and %s with different options.", lhs, rhs)); + } + if (!Objects.equals(lhs.comment(), rhs.comment())) { + throw new IllegalStateException( + String.format( + "Unable to merge schema %s and %s with different comments.", lhs, rhs)); + } + + List leftColumns = lhs.getColumns(); + List rightColumns = rhs.getColumns(); + + List mergedColumns = + IntStream.range(0, lhs.getColumnCount()) + .mapToObj(i -> mergeColumn(leftColumns.get(i), rightColumns.get(i))) + .collect(Collectors.toList()); + + return lhs.copy(mergedColumns); + } + + /** Try to combine two columns with potential incompatible type. */ + @VisibleForTesting + public static Column mergeColumn(Column lhs, Column rhs) { + if (!Objects.equals(lhs.getName(), rhs.getName())) { + throw new IllegalStateException( + String.format( + "Unable to merge column %s and %s with different name.", lhs, rhs)); + } + if (!Objects.equals(lhs.getComment(), rhs.getComment())) { + throw new IllegalStateException( + String.format( + "Unable to merge column %s and %s with different comments.", lhs, rhs)); + } + return lhs.copy(mergeDataType(lhs.getType(), rhs.getType())); + } + + /** Try to combine given data types to a compatible wider data type. */ + @VisibleForTesting + public static DataType mergeDataType(DataType lhs, DataType rhs) { + // Ignore nullability during data type merge + boolean nullable = lhs.isNullable() || rhs.isNullable(); + lhs = lhs.notNull(); + rhs = rhs.notNull(); + + DataType mergedType; + if (lhs.equals(rhs)) { + // identical type + mergedType = rhs; + } else if (lhs.is(DataTypeFamily.INTEGER_NUMERIC) + && rhs.is(DataTypeFamily.INTEGER_NUMERIC)) { + mergedType = DataTypes.BIGINT(); + } else if (lhs.is(DataTypeFamily.CHARACTER_STRING) + && rhs.is(DataTypeFamily.CHARACTER_STRING)) { + mergedType = DataTypes.STRING(); + } else if (lhs.is(DataTypeFamily.APPROXIMATE_NUMERIC) + && rhs.is(DataTypeFamily.APPROXIMATE_NUMERIC)) { + mergedType = DataTypes.DOUBLE(); + } else if (lhs.is(DataTypeRoot.DECIMAL) && rhs.is(DataTypeRoot.DECIMAL)) { + // Merge two decimal types + DecimalType lhsDecimal = (DecimalType) lhs; + DecimalType rhsDecimal = (DecimalType) rhs; + int resultIntDigits = + Math.max( + lhsDecimal.getPrecision() - lhsDecimal.getScale(), + rhsDecimal.getPrecision() - rhsDecimal.getScale()); + int resultScale = Math.max(lhsDecimal.getScale(), rhsDecimal.getScale()); + mergedType = DataTypes.DECIMAL(resultIntDigits + resultScale, resultScale); + } else if (lhs.is(DataTypeRoot.DECIMAL) && rhs.is(DataTypeFamily.EXACT_NUMERIC)) { + // Merge decimal and int + DecimalType lhsDecimal = (DecimalType) lhs; + mergedType = + DataTypes.DECIMAL( + Math.max( + lhsDecimal.getPrecision(), + lhsDecimal.getScale() + getNumericPrecision(rhs)), + lhsDecimal.getScale()); + } else if (rhs.is(DataTypeRoot.DECIMAL) && lhs.is(DataTypeFamily.EXACT_NUMERIC)) { + // Merge decimal and int + DecimalType rhsDecimal = (DecimalType) rhs; + mergedType = + DataTypes.DECIMAL( + Math.max( + rhsDecimal.getPrecision(), + rhsDecimal.getScale() + getNumericPrecision(lhs)), + rhsDecimal.getScale()); + } else { + throw new IllegalStateException( + String.format("Incompatible types: \"%s\" and \"%s\"", lhs, rhs)); + } + + if (nullable) { + return mergedType.nullable(); + } else { + return mergedType.notNull(); + } + } + + @VisibleForTesting + public static int getNumericPrecision(DataType dataType) { + if (dataType.is(DataTypeFamily.EXACT_NUMERIC)) { + if (dataType.is(DataTypeRoot.TINYINT)) { + return 3; + } else if (dataType.is(DataTypeRoot.SMALLINT)) { + return 5; + } else if (dataType.is(DataTypeRoot.INTEGER)) { + return 10; + } else if (dataType.is(DataTypeRoot.BIGINT)) { + return 19; + } else if (dataType.is(DataTypeRoot.DECIMAL)) { + return ((DecimalType) dataType).getPrecision(); + } + } + + throw new IllegalArgumentException( + "Failed to get precision of non-exact decimal type " + dataType); + } + /** apply SchemaChangeEvent to the old schema and return the schema after changing. */ public static Schema applySchemaChangeEvent(Schema schema, SchemaChangeEvent event) { if (event instanceof AddColumnEvent) { diff --git a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaUtilsTest.java b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaUtilsTest.java index 83fb358b2ce..126ef90b6c0 100644 --- a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaUtilsTest.java +++ b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaUtilsTest.java @@ -163,4 +163,272 @@ public void testApplySchemaChangeEvent() { .physicalColumn("newCol4", DataTypes.VARCHAR(10)) .build()); } + + @Test + public void testGetNumericPrecision() { + Assert.assertEquals(SchemaUtils.getNumericPrecision(DataTypes.TINYINT()), 3); + Assert.assertEquals(SchemaUtils.getNumericPrecision(DataTypes.SMALLINT()), 5); + Assert.assertEquals(SchemaUtils.getNumericPrecision(DataTypes.INT()), 10); + Assert.assertEquals(SchemaUtils.getNumericPrecision(DataTypes.BIGINT()), 19); + Assert.assertEquals(SchemaUtils.getNumericPrecision(DataTypes.DECIMAL(10, 2)), 10); + Assert.assertEquals(SchemaUtils.getNumericPrecision(DataTypes.DECIMAL(17, 0)), 17); + Assert.assertThrows( + IllegalArgumentException.class, + () -> SchemaUtils.getNumericPrecision(DataTypes.STRING())); + } + + @Test + public void testMergeDataType() { + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.BINARY(17), DataTypes.BINARY(17)), + DataTypes.BINARY(17)); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.VARBINARY(17), DataTypes.VARBINARY(17)), + DataTypes.VARBINARY(17)); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.BYTES(), DataTypes.BYTES()), DataTypes.BYTES()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.BOOLEAN(), DataTypes.BOOLEAN()), + DataTypes.BOOLEAN()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.INT(), DataTypes.INT()), DataTypes.INT()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.TINYINT(), DataTypes.TINYINT()), + DataTypes.TINYINT()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.SMALLINT(), DataTypes.SMALLINT()), + DataTypes.SMALLINT()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.BIGINT(), DataTypes.BIGINT()), + DataTypes.BIGINT()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.FLOAT(), DataTypes.FLOAT()), DataTypes.FLOAT()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.DOUBLE(), DataTypes.DOUBLE()), + DataTypes.DOUBLE()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.CHAR(17), DataTypes.CHAR(17)), + DataTypes.CHAR(17)); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.VARCHAR(17), DataTypes.VARCHAR(17)), + DataTypes.VARCHAR(17)); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.STRING(), DataTypes.STRING()), + DataTypes.STRING()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.DECIMAL(17, 7), DataTypes.DECIMAL(17, 7)), + DataTypes.DECIMAL(17, 7)); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.DATE(), DataTypes.DATE()), DataTypes.DATE()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.TIME(), DataTypes.TIME()), DataTypes.TIME()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.TIME(6), DataTypes.TIME(6)), DataTypes.TIME(6)); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.TIMESTAMP(), DataTypes.TIMESTAMP()), + DataTypes.TIMESTAMP()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.TIMESTAMP(3), DataTypes.TIMESTAMP(3)), + DataTypes.TIMESTAMP(3)); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.TIMESTAMP_TZ(), DataTypes.TIMESTAMP_TZ()), + DataTypes.TIMESTAMP_TZ()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.TIMESTAMP_TZ(3), DataTypes.TIMESTAMP_TZ(3)), + DataTypes.TIMESTAMP_TZ(3)); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.TIMESTAMP_LTZ(), DataTypes.TIMESTAMP_LTZ()), + DataTypes.TIMESTAMP_LTZ()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.TIMESTAMP_LTZ(3), DataTypes.TIMESTAMP_LTZ(3)), + DataTypes.TIMESTAMP_LTZ(3)); + Assert.assertEquals( + SchemaUtils.mergeDataType( + DataTypes.ARRAY(DataTypes.INT()), DataTypes.ARRAY(DataTypes.INT())), + DataTypes.ARRAY(DataTypes.INT())); + Assert.assertEquals( + SchemaUtils.mergeDataType( + DataTypes.MAP(DataTypes.INT(), DataTypes.STRING()), + DataTypes.MAP(DataTypes.INT(), DataTypes.STRING())), + DataTypes.MAP(DataTypes.INT(), DataTypes.STRING())); + + // Test compatible widening cast + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.INT(), DataTypes.BIGINT()), DataTypes.BIGINT()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.VARCHAR(17), DataTypes.STRING()), + DataTypes.STRING()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.FLOAT(), DataTypes.DOUBLE()), + DataTypes.DOUBLE()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.INT(), DataTypes.DECIMAL(4, 0)), + DataTypes.DECIMAL(10, 0)); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.INT(), DataTypes.DECIMAL(10, 5)), + DataTypes.DECIMAL(15, 5)); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.BIGINT(), DataTypes.DECIMAL(10, 5)), + DataTypes.DECIMAL(24, 5)); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.DECIMAL(5, 4), DataTypes.DECIMAL(10, 2)), + DataTypes.DECIMAL(12, 4)); + + // Test merging with nullability + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.INT().notNull(), DataTypes.INT().notNull()), + DataTypes.INT().notNull()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.INT().nullable(), DataTypes.INT().notNull()), + DataTypes.INT().nullable()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.INT().notNull(), DataTypes.INT().nullable()), + DataTypes.INT().nullable()); + Assert.assertEquals( + SchemaUtils.mergeDataType(DataTypes.INT().nullable(), DataTypes.INT().nullable()), + DataTypes.INT().nullable()); + + // incompatible type merges test + Assert.assertThrows( + IllegalStateException.class, + () -> SchemaUtils.mergeDataType(DataTypes.INT(), DataTypes.DOUBLE())); + Assert.assertThrows( + IllegalStateException.class, + () -> SchemaUtils.mergeDataType(DataTypes.DECIMAL(17, 0), DataTypes.DOUBLE())); + Assert.assertThrows( + IllegalStateException.class, + () -> SchemaUtils.mergeDataType(DataTypes.INT(), DataTypes.STRING())); + } + + @Test + public void testMergeColumn() { + // Test normal merges + Assert.assertEquals( + SchemaUtils.mergeColumn( + Column.physicalColumn("Column1", DataTypes.INT()), + Column.physicalColumn("Column1", DataTypes.BIGINT())), + Column.physicalColumn("Column1", DataTypes.BIGINT())); + + Assert.assertEquals( + SchemaUtils.mergeColumn( + Column.physicalColumn("Column2", DataTypes.FLOAT()), + Column.physicalColumn("Column2", DataTypes.DOUBLE())), + Column.physicalColumn("Column2", DataTypes.DOUBLE())); + + // Test merging columns with incompatible types + Assert.assertThrows( + IllegalStateException.class, + () -> + SchemaUtils.mergeColumn( + Column.physicalColumn("Column3", DataTypes.INT()), + Column.physicalColumn("Column3", DataTypes.STRING()))); + + // Test merging with incompatible names + Assert.assertThrows( + IllegalStateException.class, + () -> + SchemaUtils.mergeColumn( + Column.physicalColumn("Column4", DataTypes.INT()), + Column.physicalColumn("AnotherColumn4", DataTypes.INT()))); + } + + @Test + public void testMergeSchema() { + // Test normal merges + Assert.assertEquals( + SchemaUtils.mergeSchema( + Schema.newBuilder() + .physicalColumn("Column1", DataTypes.INT()) + .physicalColumn("Column2", DataTypes.DOUBLE()) + .primaryKey("Column1") + .partitionKey("Column2") + .build(), + Schema.newBuilder() + .physicalColumn("Column1", DataTypes.BIGINT()) + .physicalColumn("Column2", DataTypes.FLOAT()) + .primaryKey("Column1") + .partitionKey("Column2") + .build()), + Schema.newBuilder() + .physicalColumn("Column1", DataTypes.BIGINT()) + .physicalColumn("Column2", DataTypes.DOUBLE()) + .primaryKey("Column1") + .partitionKey("Column2") + .build()); + + // Test merging with incompatible types + Assert.assertThrows( + IllegalStateException.class, + () -> + SchemaUtils.mergeSchema( + Schema.newBuilder() + .physicalColumn("Column1", DataTypes.INT()) + .physicalColumn("Column2", DataTypes.DOUBLE()) + .primaryKey("Column1") + .partitionKey("Column2") + .build(), + Schema.newBuilder() + .physicalColumn("Column1", DataTypes.STRING()) + .physicalColumn("Column2", DataTypes.STRING()) + .primaryKey("Column1") + .partitionKey("Column2") + .build())); + + // Test merging with incompatible column names + Assert.assertThrows( + IllegalStateException.class, + () -> + SchemaUtils.mergeSchema( + Schema.newBuilder() + .physicalColumn("Column1", DataTypes.INT()) + .physicalColumn("Column2", DataTypes.DOUBLE()) + .primaryKey("Column1") + .partitionKey("Column2") + .build(), + Schema.newBuilder() + .physicalColumn("NotColumn1", DataTypes.INT()) + .physicalColumn("NotColumn2", DataTypes.DOUBLE()) + .primaryKey("NotColumn1") + .partitionKey("NotColumn2") + .build())); + + // Test merging with different column counts + Assert.assertThrows( + IllegalStateException.class, + () -> + SchemaUtils.mergeSchema( + Schema.newBuilder() + .physicalColumn("Column1", DataTypes.INT()) + .physicalColumn("Column2", DataTypes.DOUBLE()) + .physicalColumn("Column3", DataTypes.STRING()) + .primaryKey("Column1") + .partitionKey("Column2") + .build(), + Schema.newBuilder() + .physicalColumn("NotColumn1", DataTypes.INT()) + .physicalColumn("NotColumn2", DataTypes.DOUBLE()) + .primaryKey("NotColumn1") + .partitionKey("NotColumn2") + .build())); + + // Test merging with incompatible schema metadata + Assert.assertThrows( + IllegalStateException.class, + () -> + SchemaUtils.mergeSchema( + Schema.newBuilder() + .physicalColumn("Column1", DataTypes.INT()) + .physicalColumn("Column2", DataTypes.DOUBLE()) + .primaryKey("Column1") + .partitionKey("Column2") + .option("Key1", "Value1") + .build(), + Schema.newBuilder() + .physicalColumn("Column1", DataTypes.INT()) + .physicalColumn("Column2", DataTypes.DOUBLE()) + .primaryKey("Column2") + .partitionKey("Column1") + .option("Key2", "Value2") + .build())); + } } diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposer.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposer.java index 73910d59be7..964ea204717 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposer.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposer.java @@ -100,9 +100,9 @@ public PipelineExecution compose(PipelineDef pipelineDef) { DataStream stream = sourceTranslator.translate(pipelineDef.getSource(), env, pipelineDef.getConfig()); - // Build TransformSchemaOperator for processing Schema Event + // Build PreTransformOperator for processing Schema Event TransformTranslator transformTranslator = new TransformTranslator(); - stream = transformTranslator.translateSchema(stream, pipelineDef.getTransforms()); + stream = transformTranslator.translatePreTransform(stream, pipelineDef.getTransforms()); // Schema operator SchemaOperatorTranslator schemaOperatorTranslator = @@ -117,9 +117,9 @@ public PipelineExecution compose(PipelineDef pipelineDef) { OperatorIDGenerator schemaOperatorIDGenerator = new OperatorIDGenerator(schemaOperatorTranslator.getSchemaOperatorUid()); - // Build TransformDataOperator for processing Data Event + // Build PostTransformOperator for processing Data Event stream = - transformTranslator.translateData( + transformTranslator.translatePostTransform( stream, pipelineDef.getTransforms(), schemaOperatorIDGenerator.generate(), diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/TransformTranslator.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/TransformTranslator.java index 53400f628f3..6feec31d920 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/TransformTranslator.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/TransformTranslator.java @@ -19,8 +19,8 @@ import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.composer.definition.TransformDef; -import org.apache.flink.cdc.runtime.operators.transform.TransformDataOperator; -import org.apache.flink.cdc.runtime.operators.transform.TransformSchemaOperator; +import org.apache.flink.cdc.runtime.operators.transform.PostTransformOperator; +import org.apache.flink.cdc.runtime.operators.transform.PreTransformOperator; import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.datastream.DataStream; @@ -28,34 +28,35 @@ import java.util.List; /** - * Translator used to build {@link TransformSchemaOperator} and {@link TransformDataOperator} for - * event transform. + * Translator used to build {@link PreTransformOperator} and {@link PostTransformOperator} for event + * transform. */ public class TransformTranslator { - public DataStream translateSchema( + public DataStream translatePreTransform( DataStream input, List transforms) { if (transforms.isEmpty()) { return input; } - TransformSchemaOperator.Builder transformSchemaFunctionBuilder = - TransformSchemaOperator.newBuilder(); + PreTransformOperator.Builder preTransformFunctionBuilder = + PreTransformOperator.newBuilder(); for (TransformDef transform : transforms) { if (transform.isValidProjection()) { - transformSchemaFunctionBuilder.addTransform( + preTransformFunctionBuilder.addTransform( transform.getSourceTable(), - transform.getProjection().get(), + transform.getProjection().orElse(null), + transform.getFilter().orElse(null), transform.getPrimaryKeys(), transform.getPartitionKeys(), transform.getTableOptions()); } } return input.transform( - "Transform:Schema", new EventTypeInfo(), transformSchemaFunctionBuilder.build()); + "Transform:Schema", new EventTypeInfo(), preTransformFunctionBuilder.build()); } - public DataStream translateData( + public DataStream translatePostTransform( DataStream input, List transforms, OperatorID schemaOperatorID, @@ -64,19 +65,22 @@ public DataStream translateData( return input; } - TransformDataOperator.Builder transformDataFunctionBuilder = - TransformDataOperator.newBuilder(); + PostTransformOperator.Builder postTransformFunctionBuilder = + PostTransformOperator.newBuilder(); for (TransformDef transform : transforms) { if (transform.isValidProjection() || transform.isValidFilter()) { - transformDataFunctionBuilder.addTransform( + postTransformFunctionBuilder.addTransform( transform.getSourceTable(), transform.isValidProjection() ? transform.getProjection().get() : null, - transform.isValidFilter() ? transform.getFilter().get() : null); + transform.isValidFilter() ? transform.getFilter().get() : null, + transform.getPrimaryKeys(), + transform.getPartitionKeys(), + transform.getTableOptions()); } } - transformDataFunctionBuilder.addSchemaOperatorID(schemaOperatorID); - transformDataFunctionBuilder.addTimezone(timezone); + postTransformFunctionBuilder.addSchemaOperatorID(schemaOperatorID); + postTransformFunctionBuilder.addTimezone(timezone); return input.transform( - "Transform:Data", new EventTypeInfo(), transformDataFunctionBuilder.build()); + "Transform:Data", new EventTypeInfo(), postTransformFunctionBuilder.build()); } } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java index 2f896d3344e..91eb529b5db 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java @@ -22,8 +22,10 @@ import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; -import org.apache.flink.cdc.runtime.operators.transform.TransformSchemaOperator; +import org.apache.flink.cdc.runtime.operators.transform.PostTransformOperator; +import org.apache.flink.cdc.runtime.operators.transform.PreTransformOperator; +import org.jetbrains.annotations.NotNull; import org.junit.After; import org.junit.Before; import org.junit.ClassRule; @@ -44,7 +46,7 @@ import java.util.List; import java.util.concurrent.TimeoutException; -/** E2e tests for the {@link TransformSchemaOperator}. */ +/** E2e tests for the {@link PreTransformOperator} and {@link PostTransformOperator}. */ @RunWith(Parameterized.class) public class TransformE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(TransformE2eITCase.class); @@ -143,7 +145,7 @@ public void testHeteroSchemaTransform() throws Exception { List expectedEvents = Arrays.asList( String.format( - "CreateTableEvent{tableId=%s.terminus, schema=columns={`ID` INT NOT NULL,`VERSION` STRING}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.terminus, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", transformRenameDatabase.getDatabaseName()), String.format( "DataChangeEvent{tableId=%s.terminus, before=[], after=[1009, 8.1], op=INSERT, meta=()}", @@ -177,7 +179,7 @@ public void testHeteroSchemaTransform() throws Exception { mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); Statement stat = conn.createStatement()) { stat.execute("UPDATE TABLEALPHA SET VERSION='100' WHERE id=1009;"); - stat.execute("INSERT INTO TABLEALPHA VALUES (3007, '7', 79);"); + stat.execute("INSERT INTO TABLEALPHA VALUES (3007, '7', 79, 16, 'IINA');"); stat.execute("DELETE FROM TABLEBETA WHERE id=2011;"); } catch (SQLException e) { LOG.error("Update table for CDC failed.", e); @@ -207,7 +209,7 @@ public void testHeteroSchemaTransform() throws Exception { } @Test - public void testMultipleHittingTable() throws Exception { + public void testMultipleTransformRule() throws Exception { String pipelineJob = String.format( "source:\n" @@ -222,9 +224,16 @@ public void testMultipleHittingTable() throws Exception { + "\n" + "sink:\n" + " type: values\n" + + "route:\n" + "transform:\n" - + " - source-table: %s.TABLE\\.*\n" - + " projection: \\*, ID + 1000 as UID, VERSION AS NEWVERSION\n" + + " - source-table: %s.\\.*\n" + + " projection: ID, VERSION, 'Type-A' AS CATEGORY\n" + + " " + + getString() + + "ID > 1008\n" + + " - source-table: %s.\\.*\n" + + " projection: ID, VERSION, 'Type-B' AS CATEGORY\n" + + " filter: ID <= 1008\n" + "\n" + "pipeline:\n" + " parallelism: 1", @@ -232,6 +241,7 @@ public void testMultipleHittingTable() throws Exception { MYSQL_TEST_USER, MYSQL_TEST_PASSWORD, transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName(), transformRenameDatabase.getDatabaseName()); Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); @@ -243,34 +253,34 @@ public void testMultipleHittingTable() throws Exception { List expectedEvents = Arrays.asList( String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`PRICEALPHA` INT,`UID` INT,`NEWVERSION` STRING}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CATEGORY` STRING}, primaryKeys=ID, options=()}", transformRenameDatabase.getDatabaseName()), String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 99, 2010, 10], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8, Type-B], op=INSERT, meta=()}", transformRenameDatabase.getDatabaseName()), String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 59, 2011, 11], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, Type-A], op=INSERT, meta=()}", transformRenameDatabase.getDatabaseName()), String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8, 199, 2008, 8], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, Type-A], op=INSERT, meta=()}", transformRenameDatabase.getDatabaseName()), String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 0, 2009, 8.1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, Type-A], op=INSERT, meta=()}", transformRenameDatabase.getDatabaseName()), String.format( - "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`CODENAMESBETA` STRING,`UID` INT,`NEWVERSION` STRING}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CATEGORY` STRING}, primaryKeys=ID, options=()}", transformRenameDatabase.getDatabaseName()), String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Sonoma, 3014, 14], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Type-A], op=INSERT, meta=()}", transformRenameDatabase.getDatabaseName()), String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Monterey, 3012, 12], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Type-A], op=INSERT, meta=()}", transformRenameDatabase.getDatabaseName()), String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Ventura, 3013, 13], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Type-A], op=INSERT, meta=()}", transformRenameDatabase.getDatabaseName()), String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Big Sur, 3011, 11], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Type-A], op=INSERT, meta=()}", transformRenameDatabase.getDatabaseName())); validateResult(expectedEvents); LOG.info("Begin incremental reading stage."); @@ -286,7 +296,137 @@ public void testMultipleHittingTable() throws Exception { mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); Statement stat = conn.createStatement()) { stat.execute("UPDATE TABLEALPHA SET VERSION='100' WHERE id=1009;"); - stat.execute("INSERT INTO TABLEALPHA VALUES (3007, '7', 79);"); + stat.execute("INSERT INTO TABLEALPHA VALUES (3007, '7', 79, 16, 'IINA');"); + stat.execute("DELETE FROM TABLEBETA WHERE id=2011;"); + } catch (SQLException e) { + LOG.error("Update table for CDC failed.", e); + throw e; + } + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, Type-A], after=[1009, 100, Type-A], op=UPDATE, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, Type-A], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Type-A], after=[], op=DELETE, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + String stdout = taskManagerConsumer.toUtf8String(); + System.out.println(stdout); + } + + private static @NotNull String getString() { + return "filter: "; + } + + @Test + public void testAssortedSchemaTransform() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.\\.*\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "route:\n" + + " - source-table: %s.\\.*\n" + + " sink-table: %s.terminus\n" + + "transform:\n" + + " - source-table: %s.TABLEALPHA\n" + + " projection: ID, CONCAT('v', VERSION) AS VERSION, LOWER(NAMEALPHA) AS NAME\n" + + " filter: AGEALPHA < 19\n" + + " - source-table: %s.TABLEBETA\n" + + " projection: ID, CONCAT('v', VERSION) AS VERSION, LOWER(NAMEBETA) AS NAME\n" + + "\n" + + "pipeline:\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + waitUntilSpecificEvent( + String.format( + "CreateTableEvent{tableId=%s.terminus, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`NAME` STRING}, primaryKeys=ID, options=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.terminus, before=[], after=[1008, v8, alice], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.terminus, before=[], after=[1009, v8.1, bob], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.terminus, before=[], after=[2011, v11, eva], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.terminus, before=[], after=[2012, v12, fred], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.terminus, before=[], after=[2013, v13, gus], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.terminus, before=[], after=[2014, v14, henry], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + LOG.info("Begin incremental reading stage."); + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + transformRenameDatabase.getDatabaseName()); + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stat = conn.createStatement()) { + stat.execute("UPDATE TABLEALPHA SET VERSION='100' WHERE id=1009;"); + stat.execute("INSERT INTO TABLEALPHA VALUES (3007, '7', 79, 16, 'IINA');"); stat.execute("DELETE FROM TABLEBETA WHERE id=2011;"); } catch (SQLException e) { LOG.error("Update table for CDC failed.", e); @@ -295,19 +435,19 @@ public void testMultipleHittingTable() throws Exception { waitUntilSpecificEvent( String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 0, 2009, 8.1], after=[1009, 100, 0, 2009, 100], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[1009, v8.1, bob], after=[1009, v100, bob], op=UPDATE, meta=()}", transformRenameDatabase.getDatabaseName()), 6000L); waitUntilSpecificEvent( String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 79, 4007, 7], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[], after=[3007, v7, iina], op=INSERT, meta=()}", transformRenameDatabase.getDatabaseName()), 6000L); waitUntilSpecificEvent( String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Big Sur, 3011, 11], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[2011, v11, eva], after=[], op=DELETE, meta=()}", transformRenameDatabase.getDatabaseName()), 6000L); @@ -315,6 +455,284 @@ public void testMultipleHittingTable() throws Exception { System.out.println(stdout); } + @Test + public void testWildcardSchemaTransform() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.\\.*\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "transform:\n" + + " - source-table: %s.TABLEALPHA\n" + + " projection: \\*, CONCAT('v', VERSION) AS VERSION, LOWER(NAMEALPHA) AS NAME\n" + + " filter: AGEALPHA < 19\n" + + " - source-table: %s.TABLEBETA\n" + + " projection: \\*, CONCAT('v', VERSION) AS VERSION, LOWER(NAMEBETA) AS NAME\n" + + "\n" + + "pipeline:\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + waitUntilSpecificEvent( + String.format( + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`NAME` STRING}, primaryKeys=ID, options=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, v8, 199, 17, Alice, alice], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, v8.1, 0, 18, Bob, bob], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`NAME` STRING}, primaryKeys=ID, options=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, v11, Big Sur, 21, Eva, eva], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, v12, Monterey, 22, Fred, fred], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, v13, Ventura, 23, Gus, gus], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, v14, Sonoma, 24, Henry, henry], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + LOG.info("Begin incremental reading stage."); + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + transformRenameDatabase.getDatabaseName()); + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stat = conn.createStatement()) { + stat.execute("UPDATE TABLEALPHA SET VERSION='100' WHERE id=1009;"); + stat.execute("INSERT INTO TABLEALPHA VALUES (3007, '7', 79, 16, 'IINA');"); + stat.execute("DELETE FROM TABLEBETA WHERE id=2011;"); + } catch (SQLException e) { + LOG.error("Update table for CDC failed.", e); + throw e; + } + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, v8.1, 0, 18, Bob, bob], after=[1009, v100, 0, 18, Bob, bob], op=UPDATE, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, v7, 79, 16, IINA, iina], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, v11, Big Sur, 21, Eva, eva], after=[], op=DELETE, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + String stdout = taskManagerConsumer.toUtf8String(); + System.out.println(stdout); + } + + @Test + public void testWildcardWithMetadataColumnTransform() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.\\.*\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "transform:\n" + + " - source-table: %s.TABLEALPHA\n" + + " projection: \\*, __namespace_name__ || '.' || __schema_name__ || '.' || __table_name__ AS identifier_name\n" + + "transform:\n" + + " - source-table: %s.TABLEBETA\n" + + " projection: \\*, __namespace_name__ || '.' || __schema_name__ || '.' || __table_name__ AS identifier_name\n" + + "\n" + + "pipeline:\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + Thread.sleep(10000L); + System.out.println(taskManagerConsumer.toUtf8String()); + } + + @Test + public void testMultipleHittingTable() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.\\.*\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "transform:\n" + + " - source-table: %s.TABLE\\.*\n" + + " projection: \\*, ID + 1000 as UID, VERSION AS NEWVERSION\n" + + "\n" + + "pipeline:\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + List expectedEvents = + Arrays.asList( + String.format( + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`UID` INT,`NEWVERSION` STRING}, primaryKeys=ID, options=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8, 199, 17, Alice, 2008, 8], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 0, 18, Bob, 2009, 8.1], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 99, 19, Carol, 2010, 10], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 59, 20, Dave, 2011, 11], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`UID` INT,`NEWVERSION` STRING}, primaryKeys=ID, options=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Big Sur, 21, Eva, 3011, 11], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Monterey, 22, Fred, 3012, 12], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Ventura, 23, Gus, 3013, 13], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Sonoma, 24, Henry, 3014, 14], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName())); + validateResult(expectedEvents); + LOG.info("Begin incremental reading stage."); + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + transformRenameDatabase.getDatabaseName()); + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stat = conn.createStatement()) { + stat.execute("UPDATE TABLEALPHA SET VERSION='100' WHERE id=1009;"); + stat.execute("INSERT INTO TABLEALPHA VALUES (3007, '7', 79, 25, 'IINA');"); + stat.execute("DELETE FROM TABLEBETA WHERE id=2011;"); + } catch (SQLException e) { + LOG.error("Update table for CDC failed.", e); + throw e; + } + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 0, 18, Bob, 2009, 8.1], after=[1009, 100, 0, 18, Bob, 2009, 100], op=UPDATE, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 79, 25, IINA, 4007, 7], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Big Sur, 21, Eva, 3011, 11], after=[], op=DELETE, meta=()}", + transformRenameDatabase.getDatabaseName()), + 6000L); + } + private void validateResult(List expectedEvents) throws Exception { for (String event : expectedEvents) { waitUntilSpecificEvent(event, 6000L); diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/transform_test.sql b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/transform_test.sql index 9e61a265f03..8989ad2c804 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/transform_test.sql +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/transform_test.sql @@ -19,13 +19,15 @@ CREATE TABLE TABLEALPHA ( ID INT NOT NULL, VERSION VARCHAR(17), PRICEALPHA INT, + AGEALPHA INT, + NAMEALPHA VARCHAR(128), PRIMARY KEY (ID) ); -INSERT INTO TABLEALPHA VALUES (1008, '8', 199); -INSERT INTO TABLEALPHA VALUES (1009, '8.1', 0); -INSERT INTO TABLEALPHA VALUES (1010, '10', 99); -INSERT INTO TABLEALPHA VALUES (1011, '11', 59); +INSERT INTO TABLEALPHA VALUES (1008, '8', 199, 17, 'Alice'); +INSERT INTO TABLEALPHA VALUES (1009, '8.1', 0, 18, 'Bob'); +INSERT INTO TABLEALPHA VALUES (1010, '10', 99, 19, 'Carol'); +INSERT INTO TABLEALPHA VALUES (1011, '11', 59, 20, 'Dave'); DROP TABLE IF EXISTS TABLEBETA; @@ -33,10 +35,12 @@ CREATE TABLE TABLEBETA ( ID INT NOT NULL, VERSION VARCHAR(17), CODENAMESBETA VARCHAR(17), + AGEBETA INT, + NAMEBETA VARCHAR(128), PRIMARY KEY (ID) ); -INSERT INTO TABLEBETA VALUES (2011, '11', 'Big Sur'); -INSERT INTO TABLEBETA VALUES (2012, '12', 'Monterey'); -INSERT INTO TABLEBETA VALUES (2013, '13', 'Ventura'); -INSERT INTO TABLEBETA VALUES (2014, '14', 'Sonoma'); \ No newline at end of file +INSERT INTO TABLEBETA VALUES (2011, '11', 'Big Sur', 21, 'Eva'); +INSERT INTO TABLEBETA VALUES (2012, '12', 'Monterey', 22, 'Fred'); +INSERT INTO TABLEBETA VALUES (2013, '13', 'Ventura', 23, 'Gus'); +INSERT INTO TABLEBETA VALUES (2014, '14', 'Sonoma', 24, 'Henry'); \ No newline at end of file diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformDataOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperator.java similarity index 67% rename from flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformDataOperator.java rename to flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperator.java index 20479c87322..e14228a5c2e 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformDataOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperator.java @@ -18,8 +18,7 @@ package org.apache.flink.cdc.runtime.operators.transform; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; @@ -30,9 +29,7 @@ import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.schema.Selectors; import org.apache.flink.cdc.common.utils.SchemaUtils; -import org.apache.flink.cdc.common.utils.StringUtils; import org.apache.flink.cdc.runtime.operators.sink.SchemaEvolutionClient; -import org.apache.flink.cdc.runtime.parser.TransformParser; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.graph.StreamConfig; @@ -52,52 +49,67 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; -/** A data process function that applies user-defined transform logics. */ -public class TransformDataOperator extends AbstractStreamOperator +/** + * A data process function that performs column filtering, calculated column evaluation & final + * projection. + */ +public class PostTransformOperator extends AbstractStreamOperator implements OneInputStreamOperator { private SchemaEvolutionClient schemaEvolutionClient; private final OperatorID schemaOperatorID; private final String timezone; - private final List> transformRules; - private transient List< - Tuple4< - Selectors, - Optional, - Optional, - Boolean>> - transforms; + private final List transformRules; + private transient List transforms; /** keep the relationship of TableId and table information. */ private final Map tableInfoMap; - private transient Map, TransformProjectionProcessor> + private transient Map, PostTransformProcessor> transformProjectionProcessorMap; private transient Map, TransformFilterProcessor> transformFilterProcessorMap; - public static TransformDataOperator.Builder newBuilder() { - return new TransformDataOperator.Builder(); + public static PostTransformOperator.Builder newBuilder() { + return new PostTransformOperator.Builder(); } - /** Builder of {@link TransformDataOperator}. */ + /** Builder of {@link PostTransformOperator}. */ public static class Builder { - private final List> transformRules = new ArrayList<>(); + private final List transformRules = new ArrayList<>(); private OperatorID schemaOperatorID; private String timezone; - public TransformDataOperator.Builder addTransform( + public PostTransformOperator.Builder addTransform( + String tableInclusions, + @Nullable String projection, + @Nullable String filter, + String primaryKey, + String partitionKey, + String tableOptions) { + transformRules.add( + new TransformRule( + tableInclusions, + projection, + filter, + primaryKey, + partitionKey, + tableOptions)); + return this; + } + + public PostTransformOperator.Builder addTransform( String tableInclusions, @Nullable String projection, @Nullable String filter) { - transformRules.add(Tuple3.of(tableInclusions, projection, filter)); + transformRules.add(new TransformRule(tableInclusions, projection, filter, "", "", "")); return this; } - public TransformDataOperator.Builder addSchemaOperatorID(OperatorID schemaOperatorID) { + public PostTransformOperator.Builder addSchemaOperatorID(OperatorID schemaOperatorID) { this.schemaOperatorID = schemaOperatorID; return this; } - public TransformDataOperator.Builder addTimezone(String timezone) { + public PostTransformOperator.Builder addTimezone(String timezone) { if (PipelineOptions.PIPELINE_LOCAL_TIME_ZONE.defaultValue().equals(timezone)) { this.timezone = ZoneId.systemDefault().toString(); } else { @@ -106,15 +118,13 @@ public TransformDataOperator.Builder addTimezone(String timezone) { return this; } - public TransformDataOperator build() { - return new TransformDataOperator(transformRules, schemaOperatorID, timezone); + public PostTransformOperator build() { + return new PostTransformOperator(transformRules, schemaOperatorID, timezone); } } - private TransformDataOperator( - List> transformRules, - OperatorID schemaOperatorID, - String timezone) { + private PostTransformOperator( + List transformRules, OperatorID schemaOperatorID, String timezone) { this.transformRules = transformRules; this.schemaOperatorID = schemaOperatorID; this.timezone = timezone; @@ -142,24 +152,22 @@ public void open() throws Exception { transformRules.stream() .map( tuple3 -> { - String tableInclusions = tuple3.f0; - String projection = tuple3.f1; - String filterExpression = tuple3.f2; + String tableInclusions = tuple3.getTableInclusions(); + String projection = tuple3.getProjection(); + String filterExpression = tuple3.getFilter(); Selectors selectors = new Selectors.SelectorsBuilder() .includeTables(tableInclusions) .build(); - return new Tuple4<>( + return new PostTransformers( selectors, - TransformProjection.of(projection), - TransformFilter.of(filterExpression), - containFilteredComputedColumn( - projection, filterExpression)); + TransformProjection.of(projection).orElse(null), + TransformFilter.of(filterExpression).orElse(null)); }) .collect(Collectors.toList()); - this.transformFilterProcessorMap = new ConcurrentHashMap<>(); this.transformProjectionProcessorMap = new ConcurrentHashMap<>(); + this.transformFilterProcessorMap = new ConcurrentHashMap<>(); } @Override @@ -196,16 +204,22 @@ public void processElement(StreamRecord element) throws Exception { private SchemaChangeEvent cacheSchema(SchemaChangeEvent event) throws Exception { TableId tableId = event.tableId(); - Schema newSchema; + Schema schema; if (event instanceof CreateTableEvent) { - newSchema = ((CreateTableEvent) event).getSchema(); + CreateTableEvent createTableEvent = (CreateTableEvent) event; + schema = createTableEvent.getSchema(); } else { - newSchema = + schema = SchemaUtils.applySchemaChangeEvent( getTableInfoFromSchemaEvolutionClient(tableId).getSchema(), event); } - transformSchema(tableId, newSchema); - tableInfoMap.put(tableId, TableInfo.of(tableId, newSchema)); + + Schema projectedSchema = transformSchema(tableId, schema); + tableInfoMap.put(tableId, TableInfo.of(tableId, projectedSchema, schema)); + + if (event instanceof CreateTableEvent) { + return new CreateTableEvent(event.tableId(), projectedSchema); + } return event; } @@ -214,7 +228,8 @@ private TableInfo getTableInfoFromSchemaEvolutionClient(TableId tableId) throws if (tableInfo == null) { Optional schemaOptional = schemaEvolutionClient.getLatestSchema(tableId); if (schemaOptional.isPresent()) { - tableInfo = TableInfo.of(tableId, schemaOptional.get()); + Schema projectedSchema = transformSchema(tableId, schemaOptional.get()); + tableInfo = TableInfo.of(tableId, projectedSchema, schemaOptional.get()); } else { throw new RuntimeException( "Could not find schema message from SchemaRegistry for " + tableId); @@ -223,67 +238,50 @@ private TableInfo getTableInfoFromSchemaEvolutionClient(TableId tableId) throws return tableInfo; } - private void transformSchema(TableId tableId, Schema schema) throws Exception { - for (Tuple4, Optional, Boolean> - transform : transforms) { - Selectors selectors = transform.f0; - if (selectors.isMatch(tableId) && transform.f1.isPresent()) { - TransformProjection transformProjection = transform.f1.get(); + private Schema transformSchema(TableId tableId, Schema schema) throws Exception { + List newSchemas = new ArrayList<>(); + for (PostTransformers transform : transforms) { + Selectors selectors = transform.getSelectors(); + if (selectors.isMatch(tableId) && transform.getProjection().isPresent()) { + TransformProjection transformProjection = transform.getProjection().get(); + TransformFilter transformFilter = transform.getFilter().orElse(null); if (transformProjection.isValid()) { if (!transformProjectionProcessorMap.containsKey( Tuple2.of(tableId, transformProjection))) { transformProjectionProcessorMap.put( Tuple2.of(tableId, transformProjection), - TransformProjectionProcessor.of(transformProjection)); + PostTransformProcessor.of(transformProjection, transformFilter)); } - TransformProjectionProcessor transformProjectionProcessor = + PostTransformProcessor postTransformProcessor = transformProjectionProcessorMap.get( Tuple2.of(tableId, transformProjection)); // update the columns of projection and add the column of projection into Schema - transformProjectionProcessor.processSchemaChangeEvent(schema); + newSchemas.add(postTransformProcessor.processSchemaChangeEvent(schema)); } } } + if (newSchemas.isEmpty()) { + return schema; + } + + return SchemaUtils.mergeCompatibleSchemas(newSchemas); } private Optional processDataChangeEvent(DataChangeEvent dataChangeEvent) throws Exception { TableId tableId = dataChangeEvent.tableId(); + TableInfo tableInfo = getTableInfoFromSchemaEvolutionClient(tableId); List> transformedDataChangeEventOptionalList = new ArrayList<>(); long epochTime = System.currentTimeMillis(); - for (Tuple4, Optional, Boolean> - transform : transforms) { - Selectors selectors = transform.f0; - Boolean isPreProjection = transform.f3; + for (PostTransformers transform : transforms) { + Selectors selectors = transform.getSelectors(); + if (selectors.isMatch(tableId)) { Optional dataChangeEventOptional = Optional.of(dataChangeEvent); - Optional transformProjectionOptional = transform.f1; - if (isPreProjection - && transformProjectionOptional.isPresent() - && transformProjectionOptional.get().isValid()) { - TransformProjection transformProjection = transformProjectionOptional.get(); - if (!transformProjectionProcessorMap.containsKey( - Tuple2.of(tableId, transformProjection)) - || !transformProjectionProcessorMap - .get(Tuple2.of(tableId, transformProjection)) - .hasTableInfo()) { - transformProjectionProcessorMap.put( - Tuple2.of(tableId, transformProjection), - TransformProjectionProcessor.of( - getTableInfoFromSchemaEvolutionClient(tableId), - transformProjection, - timezone)); - } - TransformProjectionProcessor transformProjectionProcessor = - transformProjectionProcessorMap.get( - Tuple2.of(tableId, transformProjection)); - dataChangeEventOptional = - processProjection( - transformProjectionProcessor, - dataChangeEventOptional.get(), - epochTime); - } - Optional transformFilterOptional = transform.f2; + Optional transformProjectionOptional = + transform.getProjection(); + Optional transformFilterOptional = transform.getFilter(); + if (transformFilterOptional.isPresent() && transformFilterOptional.get().isVaild()) { TransformFilter transformFilter = transformFilterOptional.get(); @@ -291,10 +289,7 @@ private Optional processDataChangeEvent(DataChangeEvent dataCha Tuple2.of(tableId, transformFilter))) { transformFilterProcessorMap.put( Tuple2.of(tableId, transformFilter), - TransformFilterProcessor.of( - getTableInfoFromSchemaEvolutionClient(tableId), - transformFilter, - timezone)); + TransformFilterProcessor.of(tableInfo, transformFilter, timezone)); } TransformFilterProcessor transformFilterProcessor = transformFilterProcessorMap.get(Tuple2.of(tableId, transformFilter)); @@ -304,8 +299,7 @@ private Optional processDataChangeEvent(DataChangeEvent dataCha dataChangeEventOptional.get(), epochTime); } - if (!isPreProjection - && dataChangeEventOptional.isPresent() + if (dataChangeEventOptional.isPresent() && transformProjectionOptional.isPresent() && transformProjectionOptional.get().isValid()) { TransformProjection transformProjection = transformProjectionOptional.get(); @@ -316,30 +310,32 @@ private Optional processDataChangeEvent(DataChangeEvent dataCha .hasTableInfo()) { transformProjectionProcessorMap.put( Tuple2.of(tableId, transformProjection), - TransformProjectionProcessor.of( - getTableInfoFromSchemaEvolutionClient(tableId), + PostTransformProcessor.of( + tableInfo, transformProjection, + transformFilterOptional.orElse(null), timezone)); } - TransformProjectionProcessor transformProjectionProcessor = + PostTransformProcessor postTransformProcessor = transformProjectionProcessorMap.get( Tuple2.of(tableId, transformProjection)); dataChangeEventOptional = processProjection( - transformProjectionProcessor, + postTransformProcessor, dataChangeEventOptional.get(), epochTime); } transformedDataChangeEventOptionalList.add(dataChangeEventOptional); } } + if (transformedDataChangeEventOptionalList.isEmpty()) { - return Optional.of(dataChangeEvent); + return processPostProjection(tableInfo, dataChangeEvent); } else { for (Optional dataChangeEventOptional : transformedDataChangeEventOptionalList) { if (dataChangeEventOptional.isPresent()) { - return dataChangeEventOptional; + return processPostProjection(tableInfo, dataChangeEventOptional.get()); } } return Optional.empty(); @@ -371,39 +367,49 @@ private Optional processFilter( } private Optional processProjection( - TransformProjectionProcessor transformProjectionProcessor, + PostTransformProcessor postTransformProcessor, DataChangeEvent dataChangeEvent, - long epochTime) - throws Exception { + long epochTime) { BinaryRecordData before = (BinaryRecordData) dataChangeEvent.before(); BinaryRecordData after = (BinaryRecordData) dataChangeEvent.after(); if (before != null) { BinaryRecordData projectedBefore = - transformProjectionProcessor.processData(before, epochTime); + postTransformProcessor.processData(before, epochTime); dataChangeEvent = DataChangeEvent.projectBefore(dataChangeEvent, projectedBefore); } if (after != null) { - BinaryRecordData projectedAfter = - transformProjectionProcessor.processData(after, epochTime); + BinaryRecordData projectedAfter = postTransformProcessor.processData(after, epochTime); dataChangeEvent = DataChangeEvent.projectAfter(dataChangeEvent, projectedAfter); } return Optional.of(dataChangeEvent); } - private boolean containFilteredComputedColumn(String projection, String filter) { - boolean contain = false; - if (StringUtils.isNullOrWhitespaceOnly(projection) - || StringUtils.isNullOrWhitespaceOnly(filter)) { - return contain; + private Optional processPostProjection( + TableInfo tableInfo, DataChangeEvent dataChangeEvent) throws Exception { + BinaryRecordData before = (BinaryRecordData) dataChangeEvent.before(); + BinaryRecordData after = (BinaryRecordData) dataChangeEvent.after(); + if (before != null) { + BinaryRecordData projectedBefore = projectRecord(tableInfo, before); + dataChangeEvent = DataChangeEvent.projectBefore(dataChangeEvent, projectedBefore); } - List computedColumnNames = TransformParser.parseComputedColumnNames(projection); - List filteredColumnNames = TransformParser.parseFilterColumnNameList(filter); - for (String computedColumnName : computedColumnNames) { - if (filteredColumnNames.contains(computedColumnName)) { - return true; - } + if (after != null) { + BinaryRecordData projectedAfter = projectRecord(tableInfo, after); + dataChangeEvent = DataChangeEvent.projectAfter(dataChangeEvent, projectedAfter); } - return contain; + return Optional.of(dataChangeEvent); + } + + private BinaryRecordData projectRecord(TableInfo tableInfo, BinaryRecordData recordData) { + List valueList = new ArrayList<>(); + RecordData.FieldGetter[] fieldGetters = tableInfo.getFieldGetters(); + + for (RecordData.FieldGetter fieldGetter : fieldGetters) { + valueList.add(fieldGetter.getFieldOrNull(recordData)); + } + + return tableInfo + .getRecordDataGenerator() + .generate(valueList.toArray(new Object[valueList.size()])); } private void clearOperator() { diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformProjectionProcessor.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformProcessor.java similarity index 66% rename from flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformProjectionProcessor.java rename to flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformProcessor.java index 7049bbdfda5..7b533aa7317 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformProjectionProcessor.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformProcessor.java @@ -19,19 +19,22 @@ import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; -import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.DataType; import org.apache.flink.cdc.runtime.parser.TransformParser; import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; /** * The processor of transform projection applies to process a row of filtering tables. @@ -41,26 +44,29 @@ *
    *
  • CreateTableEvent: add the user-defined computed columns into Schema. *
  • SchemaChangeEvent: update the columns of TransformProjection. - *
  • DataChangeEvent: Fill data field to row in TransformSchemaOperator. Process the data column + *
  • DataChangeEvent: Fill data field to row in PreTransformOperator. Process the data column * and the user-defined expression computed columns. *
*/ -public class TransformProjectionProcessor { - private static final Logger LOG = LoggerFactory.getLogger(TransformProjectionProcessor.class); +public class PostTransformProcessor { + private static final Logger LOG = LoggerFactory.getLogger(PostTransformProcessor.class); private TableInfo tableInfo; private TableChangeInfo tableChangeInfo; private TransformProjection transformProjection; + private @Nullable TransformFilter transformFilter; private String timezone; private Map projectionColumnProcessorMap; - public TransformProjectionProcessor( + public PostTransformProcessor( TableInfo tableInfo, TableChangeInfo tableChangeInfo, TransformProjection transformProjection, + @Nullable TransformFilter transformFilter, String timezone) { this.tableInfo = tableInfo; this.tableChangeInfo = tableChangeInfo; this.transformProjection = transformProjection; + this.transformFilter = transformFilter; this.timezone = timezone; this.projectionColumnProcessorMap = new ConcurrentHashMap<>(); } @@ -73,47 +79,59 @@ public boolean hasTableInfo() { return this.tableInfo != null; } - public static TransformProjectionProcessor of( - TableInfo tableInfo, TransformProjection transformProjection, String timezone) { - return new TransformProjectionProcessor(tableInfo, null, transformProjection, timezone); + public static PostTransformProcessor of( + TableInfo tableInfo, + TransformProjection transformProjection, + TransformFilter transformFilter, + String timezone) { + return new PostTransformProcessor( + tableInfo, null, transformProjection, transformFilter, timezone); } - public static TransformProjectionProcessor of( - TableChangeInfo tableChangeInfo, TransformProjection transformProjection) { - return new TransformProjectionProcessor(null, tableChangeInfo, transformProjection, null); + public static PostTransformProcessor of( + TableChangeInfo tableChangeInfo, + TransformProjection transformProjection, + TransformFilter transformFilter) { + return new PostTransformProcessor( + null, tableChangeInfo, transformProjection, transformFilter, null); } - public static TransformProjectionProcessor of(TransformProjection transformProjection) { - return new TransformProjectionProcessor(null, null, transformProjection, null); + public static PostTransformProcessor of( + TransformProjection transformProjection, TransformFilter transformFilter) { + return new PostTransformProcessor(null, null, transformProjection, transformFilter, null); } - public CreateTableEvent processCreateTableEvent(CreateTableEvent createTableEvent) { - List projectionColumns = - TransformParser.generateProjectionColumns( - transformProjection.getProjection(), - createTableEvent.getSchema().getColumns()); - transformProjection.setProjectionColumns(projectionColumns); - List allColumnList = transformProjection.getAllColumnList(); - // add the column of projection into Schema - Schema schema = createTableEvent.getSchema().copy(allColumnList); - return new CreateTableEvent(createTableEvent.tableId(), schema); - } - - public void processSchemaChangeEvent(Schema schema) { + public Schema processSchemaChangeEvent(Schema schema) { List projectionColumns = TransformParser.generateProjectionColumns( transformProjection.getProjection(), schema.getColumns()); transformProjection.setProjectionColumns(projectionColumns); + return schema.copy( + projectionColumns.stream() + .map(ProjectionColumn::getColumn) + .collect(Collectors.toList())); } - public BinaryRecordData processFillDataField(BinaryRecordData data) { + public BinaryRecordData processData(BinaryRecordData after, long epochTime) { List valueList = new ArrayList<>(); - for (Column column : tableChangeInfo.getTransformedSchema().getColumns()) { + for (Column column : tableInfo.getSchema().getColumns()) { boolean isProjectionColumn = false; for (ProjectionColumn projectionColumn : transformProjection.getProjectionColumns()) { if (column.getName().equals(projectionColumn.getColumnName()) && projectionColumn.isValidTransformedProjectionColumn()) { - valueList.add(null); + if (!projectionColumnProcessorMap.containsKey( + projectionColumn.getColumnName())) { + projectionColumnProcessorMap.put( + projectionColumn.getColumnName(), + ProjectionColumnProcessor.of( + tableInfo, projectionColumn, timezone)); + } + ProjectionColumnProcessor projectionColumnProcessor = + projectionColumnProcessorMap.get(projectionColumn.getColumnName()); + valueList.add( + DataTypeConverter.convert( + projectionColumnProcessor.evaluate(after, epochTime), + projectionColumn.getDataType())); isProjectionColumn = true; break; } @@ -122,17 +140,18 @@ public BinaryRecordData processFillDataField(BinaryRecordData data) { valueList.add( getValueFromBinaryRecordData( column.getName(), - data, - tableChangeInfo.getOriginalSchema().getColumns(), - tableChangeInfo.getFieldGetters())); + column.getType(), + after, + tableInfo.getOriginalSchema().getColumns(), + tableInfo.getOriginalFieldGetters())); } } - return tableChangeInfo + return tableInfo .getRecordDataGenerator() .generate(valueList.toArray(new Object[valueList.size()])); } - public BinaryRecordData processData(BinaryRecordData after, long epochTime) { + public BinaryRecordData preProcessData(BinaryRecordData after, long epochTime) { List valueList = new ArrayList<>(); for (Column column : tableInfo.getSchema().getColumns()) { boolean isProjectionColumn = false; @@ -160,6 +179,7 @@ public BinaryRecordData processData(BinaryRecordData after, long epochTime) { valueList.add( getValueFromBinaryRecordData( column.getName(), + column.getType(), after, tableInfo.getSchema().getColumns(), tableInfo.getFieldGetters())); @@ -172,13 +192,14 @@ public BinaryRecordData processData(BinaryRecordData after, long epochTime) { private Object getValueFromBinaryRecordData( String columnName, + DataType expectedType, BinaryRecordData binaryRecordData, List columns, RecordData.FieldGetter[] fieldGetters) { for (int i = 0; i < columns.size(); i++) { if (columnName.equals(columns.get(i).getName())) { return DataTypeConverter.convert( - fieldGetters[i].getFieldOrNull(binaryRecordData), columns.get(i).getType()); + fieldGetters[i].getFieldOrNull(binaryRecordData), expectedType); } } return null; diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformers.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformers.java new file mode 100644 index 00000000000..9daea231e70 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformers.java @@ -0,0 +1,53 @@ +/* + * 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.cdc.runtime.operators.transform; + +import org.apache.flink.cdc.common.schema.Selectors; + +import javax.annotation.Nullable; + +import java.util.Optional; + +/** Transformation rules used by {@link PostTransformOperator}. */ +public class PostTransformers { + private final Selectors selectors; + + private final Optional projection; + private final Optional filter; + + public PostTransformers( + Selectors selectors, + @Nullable TransformProjection projection, + @Nullable TransformFilter filter) { + this.selectors = selectors; + this.projection = projection != null ? Optional.of(projection) : Optional.empty(); + this.filter = filter != null ? Optional.of(filter) : Optional.empty(); + } + + public Selectors getSelectors() { + return selectors; + } + + public Optional getProjection() { + return projection; + } + + public Optional getFilter() { + return filter; + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformSchemaOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java similarity index 69% rename from flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformSchemaOperator.java rename to flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java index 230fc1a6ffd..b5faf9e5317 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformSchemaOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; @@ -44,51 +43,63 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; -/** A schema process function that applies user-defined transform logics. */ -public class TransformSchemaOperator extends AbstractStreamOperator +/** + * A data process function that filters out columns which aren't (directly & indirectly) referenced. + */ +public class PreTransformOperator extends AbstractStreamOperator implements OneInputStreamOperator { - private final List> transformRules; - private transient List>> transforms; + private final List transformRules; + private transient List transforms; private final Map tableChangeInfoMap; - private transient Map processorMap; private final List> schemaMetadataTransformers; private transient ListState state; + private Map preTransformProcessorMap; - public static TransformSchemaOperator.Builder newBuilder() { - return new TransformSchemaOperator.Builder(); + public static PreTransformOperator.Builder newBuilder() { + return new PreTransformOperator.Builder(); } - /** Builder of {@link TransformSchemaOperator}. */ + /** Builder of {@link PreTransformOperator}. */ public static class Builder { - private final List> transformRules = - new ArrayList<>(); + private final List transformRules = new ArrayList<>(); + + public PreTransformOperator.Builder addTransform( + String tableInclusions, @Nullable String projection, @Nullable String filter) { + transformRules.add(new TransformRule(tableInclusions, projection, filter, "", "", "")); + return this; + } - public TransformSchemaOperator.Builder addTransform( + public PreTransformOperator.Builder addTransform( String tableInclusions, @Nullable String projection, + @Nullable String filter, String primaryKey, String partitionKey, String tableOption) { transformRules.add( - Tuple5.of(tableInclusions, projection, primaryKey, partitionKey, tableOption)); + new TransformRule( + tableInclusions, + projection, + filter, + primaryKey, + partitionKey, + tableOption)); return this; } - public TransformSchemaOperator build() { - return new TransformSchemaOperator(transformRules); + public PreTransformOperator build() { + return new PreTransformOperator(transformRules); } } - private TransformSchemaOperator( - List> transformRules) { + private PreTransformOperator(List transformRules) { this.transformRules = transformRules; this.tableChangeInfoMap = new ConcurrentHashMap<>(); - this.processorMap = new ConcurrentHashMap<>(); + this.preTransformProcessorMap = new ConcurrentHashMap<>(); this.schemaMetadataTransformers = new ArrayList<>(); this.chainingStrategy = ChainingStrategy.ALWAYS; } @@ -97,21 +108,26 @@ private TransformSchemaOperator( public void open() throws Exception { super.open(); transforms = new ArrayList<>(); - for (Tuple5 transformRule : transformRules) { - String tableInclusions = transformRule.f0; - String projection = transformRule.f1; - String primaryKeys = transformRule.f2; - String partitionKeys = transformRule.f3; - String tableOptions = transformRule.f4; + for (TransformRule transformRule : transformRules) { + String tableInclusions = transformRule.getTableInclusions(); + String projection = transformRule.getProjection(); + String filter = transformRule.getFilter(); + String primaryKeys = transformRule.getPrimaryKey(); + String partitionKeys = transformRule.getPartitionKey(); + String tableOptions = transformRule.getTableOption(); Selectors selectors = new Selectors.SelectorsBuilder().includeTables(tableInclusions).build(); - transforms.add(new Tuple2<>(selectors, TransformProjection.of(projection))); + transforms.add( + new PreTransformers( + selectors, + TransformProjection.of(projection).orElse(null), + TransformFilter.of(filter).orElse(null))); schemaMetadataTransformers.add( new Tuple2<>( selectors, new SchemaMetadataTransform(primaryKeys, partitionKeys, tableOptions))); } - this.processorMap = new ConcurrentHashMap<>(); + this.preTransformProcessorMap = new ConcurrentHashMap<>(); } @Override @@ -197,6 +213,7 @@ private SchemaChangeEvent cacheChangeSchema(SchemaChangeEvent event) { private CreateTableEvent transformCreateTableEvent(CreateTableEvent createTableEvent) { TableId tableId = createTableEvent.tableId(); + TableChangeInfo tableChangeInfo = tableChangeInfoMap.get(tableId); for (Tuple2 transform : schemaMetadataTransformers) { Selectors selectors = transform.f0; @@ -209,19 +226,22 @@ private CreateTableEvent transformCreateTableEvent(CreateTableEvent createTableE } } - for (Tuple2> transform : transforms) { - Selectors selectors = transform.f0; - if (selectors.isMatch(tableId) && transform.f1.isPresent()) { - TransformProjection transformProjection = transform.f1.get(); + for (PreTransformers transform : transforms) { + Selectors selectors = transform.getSelectors(); + if (selectors.isMatch(tableId) && transform.getProjection().isPresent()) { + TransformProjection transformProjection = transform.getProjection().get(); + TransformFilter transformFilter = transform.getFilter().orElse(null); if (transformProjection.isValid()) { - if (!processorMap.containsKey(tableId)) { - processorMap.put( - tableId, TransformProjectionProcessor.of(transformProjection)); + if (!preTransformProcessorMap.containsKey(tableId)) { + preTransformProcessorMap.put( + tableId, + new PreTransformProcessor( + tableChangeInfo, transformProjection, transformFilter)); } - TransformProjectionProcessor transformProjectionProcessor = - processorMap.get(tableId); - // update the columns of projection and add the column of projection into Schema - return transformProjectionProcessor.processCreateTableEvent(createTableEvent); + PreTransformProcessor preTransformProcessor = + preTransformProcessorMap.get(tableId); + // filter out unreferenced columns in pre-transform process + return preTransformProcessor.preTransformCreateTableEvent(createTableEvent); } } } @@ -252,12 +272,14 @@ private Schema transformSchemaMetaData( private DataChangeEvent processDataChangeEvent(DataChangeEvent dataChangeEvent) throws Exception { TableId tableId = dataChangeEvent.tableId(); - for (Tuple2> transform : transforms) { - Selectors selectors = transform.f0; - if (selectors.isMatch(tableId) && transform.f1.isPresent()) { - TransformProjection transformProjection = transform.f1.get(); + for (PreTransformers transform : transforms) { + Selectors selectors = transform.getSelectors(); + + if (selectors.isMatch(tableId) && transform.getProjection().isPresent()) { + TransformProjection transformProjection = transform.getProjection().get(); + TransformFilter transformFilter = transform.getFilter().orElse(null); if (transformProjection.isValid()) { - return processProjection(transformProjection, dataChangeEvent); + return processProjection(transformProjection, transformFilter, dataChangeEvent); } } } @@ -265,25 +287,27 @@ private DataChangeEvent processDataChangeEvent(DataChangeEvent dataChangeEvent) } private DataChangeEvent processProjection( - TransformProjection transformProjection, DataChangeEvent dataChangeEvent) - throws Exception { + TransformProjection transformProjection, + @Nullable TransformFilter transformFilter, + DataChangeEvent dataChangeEvent) { TableId tableId = dataChangeEvent.tableId(); TableChangeInfo tableChangeInfo = tableChangeInfoMap.get(tableId); - if (!processorMap.containsKey(tableId) || !processorMap.get(tableId).hasTableChangeInfo()) { - processorMap.put( - tableId, TransformProjectionProcessor.of(tableChangeInfo, transformProjection)); + if (!preTransformProcessorMap.containsKey(tableId) + || !preTransformProcessorMap.get(tableId).hasTableChangeInfo()) { + preTransformProcessorMap.put( + tableId, + new PreTransformProcessor( + tableChangeInfo, transformProjection, transformFilter)); } - TransformProjectionProcessor transformProjectionProcessor = processorMap.get(tableId); + PreTransformProcessor preTransformProcessor = preTransformProcessorMap.get(tableId); BinaryRecordData before = (BinaryRecordData) dataChangeEvent.before(); BinaryRecordData after = (BinaryRecordData) dataChangeEvent.after(); if (before != null) { - BinaryRecordData projectedBefore = - transformProjectionProcessor.processFillDataField(before); + BinaryRecordData projectedBefore = preTransformProcessor.processFillDataField(before); dataChangeEvent = DataChangeEvent.projectBefore(dataChangeEvent, projectedBefore); } if (after != null) { - BinaryRecordData projectedAfter = - transformProjectionProcessor.processFillDataField(after); + BinaryRecordData projectedAfter = preTransformProcessor.processFillDataField(after); dataChangeEvent = DataChangeEvent.projectAfter(dataChangeEvent, projectedAfter); } return dataChangeEvent; @@ -291,7 +315,7 @@ private DataChangeEvent processProjection( private void clearOperator() { this.transforms = null; - this.processorMap = null; + this.preTransformProcessorMap = null; this.state = null; } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformProcessor.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformProcessor.java new file mode 100644 index 00000000000..cd985dce731 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformProcessor.java @@ -0,0 +1,111 @@ +/* + * 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.cdc.runtime.operators.transform; + +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.data.binary.BinaryRecordData; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.runtime.parser.TransformParser; +import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.List; + +/** + * The processor of pre-transform projection in {@link PreTransformOperator}. + * + *

A pre-transform projection processor handles: + * + *

    + *
  • CreateTableEvent: removes unused (unreferenced) columns from given schema. + *
  • SchemaChangeEvent: update the columns of TransformProjection. + *
  • DataChangeEvent: omits unused columns in data row. + *
+ */ +public class PreTransformProcessor { + private TableChangeInfo tableChangeInfo; + private TransformProjection transformProjection; + private @Nullable TransformFilter transformFilter; + + public PreTransformProcessor( + TableChangeInfo tableChangeInfo, + TransformProjection transformProjection, + @Nullable TransformFilter transformFilter) { + this.tableChangeInfo = tableChangeInfo; + this.transformProjection = transformProjection; + this.transformFilter = transformFilter; + } + + public boolean hasTableChangeInfo() { + return this.tableChangeInfo != null; + } + + public CreateTableEvent preTransformCreateTableEvent(CreateTableEvent createTableEvent) { + List preTransformColumns = + TransformParser.generateReferencedColumns( + transformProjection.getProjection(), + transformFilter != null ? transformFilter.getExpression() : null, + createTableEvent.getSchema().getColumns()); + Schema schema = createTableEvent.getSchema().copy(preTransformColumns); + return new CreateTableEvent(createTableEvent.tableId(), schema); + } + + public BinaryRecordData processFillDataField(BinaryRecordData data) { + List valueList = new ArrayList<>(); + for (Column column : tableChangeInfo.getTransformedSchema().getColumns()) { + boolean isProjectionColumn = false; + for (ProjectionColumn projectionColumn : transformProjection.getProjectionColumns()) { + if (column.getName().equals(projectionColumn.getColumnName()) + && projectionColumn.isValidTransformedProjectionColumn()) { + valueList.add(null); + isProjectionColumn = true; + break; + } + } + if (!isProjectionColumn) { + valueList.add( + getValueFromBinaryRecordData( + column.getName(), + data, + tableChangeInfo.getOriginalSchema().getColumns(), + tableChangeInfo.getFieldGetters())); + } + } + return tableChangeInfo + .getRecordDataGenerator() + .generate(valueList.toArray(new Object[valueList.size()])); + } + + private Object getValueFromBinaryRecordData( + String columnName, + BinaryRecordData binaryRecordData, + List columns, + RecordData.FieldGetter[] fieldGetters) { + for (int i = 0; i < columns.size(); i++) { + if (columnName.equals(columns.get(i).getName())) { + return DataTypeConverter.convert( + fieldGetters[i].getFieldOrNull(binaryRecordData), columns.get(i).getType()); + } + } + return null; + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformers.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformers.java new file mode 100644 index 00000000000..9df7103b244 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformers.java @@ -0,0 +1,53 @@ +/* + * 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.cdc.runtime.operators.transform; + +import org.apache.flink.cdc.common.schema.Selectors; + +import javax.annotation.Nullable; + +import java.util.Optional; + +/** Transformation rules used by {@link PreTransformOperator}. */ +public class PreTransformers { + private final Selectors selectors; + + private final Optional projection; + private final Optional filter; + + public PreTransformers( + Selectors selectors, + @Nullable TransformProjection projection, + @Nullable TransformFilter filter) { + this.selectors = selectors; + this.projection = projection != null ? Optional.of(projection) : Optional.empty(); + this.filter = filter != null ? Optional.of(filter) : Optional.empty(); + } + + public Selectors getSelectors() { + return selectors; + } + + public Optional getProjection() { + return projection; + } + + public Optional getFilter() { + return filter; + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/ProjectionColumn.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/ProjectionColumn.java index 500e36c99d4..4bbe3d558da 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/ProjectionColumn.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/ProjectionColumn.java @@ -22,6 +22,7 @@ import org.apache.flink.cdc.common.utils.StringUtils; import java.io.Serializable; +import java.util.ArrayList; import java.util.List; /** @@ -59,6 +60,14 @@ public ProjectionColumn( this.originalColumnNames = originalColumnNames; } + public ProjectionColumn copy() { + return new ProjectionColumn( + column.copy(column.getName()), + expression, + scriptExpression, + new ArrayList<>(originalColumnNames)); + } + public Column getColumn() { return column; } @@ -103,4 +112,22 @@ public static ProjectionColumn of( scriptExpression, originalColumnNames); } + + @Override + public String toString() { + return "ProjectionColumn{" + + "column=" + + column + + ", expression='" + + expression + + '\'' + + ", scriptExpression='" + + scriptExpression + + '\'' + + ", originalColumnNames=" + + originalColumnNames + + ", transformExpressionKey=" + + transformExpressionKey + + '}'; + } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/ProjectionColumnProcessor.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/ProjectionColumnProcessor.java index 6c5202342d8..a9b95749a3f 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/ProjectionColumnProcessor.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/ProjectionColumnProcessor.java @@ -75,30 +75,36 @@ public Object evaluate(BinaryRecordData after, long epochTime) { private Object[] generateParams(BinaryRecordData after, long epochTime) { List params = new ArrayList<>(); - List columns = tableInfo.getSchema().getColumns(); - RecordData.FieldGetter[] fieldGetters = tableInfo.getFieldGetters(); + List columns = tableInfo.getOriginalSchema().getColumns(); + RecordData.FieldGetter[] fieldGetters = tableInfo.getOriginalFieldGetters(); for (String originalColumnName : projectionColumn.getOriginalColumnNames()) { - if (originalColumnName.equals(TransformParser.DEFAULT_NAMESPACE_NAME)) { - params.add(tableInfo.getNamespace()); - continue; - } - if (originalColumnName.equals(TransformParser.DEFAULT_SCHEMA_NAME)) { - params.add(tableInfo.getSchemaName()); - continue; - } - if (originalColumnName.equals(TransformParser.DEFAULT_TABLE_NAME)) { - params.add(tableInfo.getTableName()); - continue; + switch (originalColumnName) { + case TransformParser.DEFAULT_NAMESPACE_NAME: + params.add(tableInfo.getNamespace()); + continue; + case TransformParser.DEFAULT_SCHEMA_NAME: + params.add(tableInfo.getSchemaName()); + continue; + case TransformParser.DEFAULT_TABLE_NAME: + params.add(tableInfo.getTableName()); + continue; } + + boolean argumentFound = false; for (int i = 0; i < columns.size(); i++) { Column column = columns.get(i); if (column.getName().equals(originalColumnName)) { params.add( DataTypeConverter.convertToOriginal( fieldGetters[i].getFieldOrNull(after), column.getType())); + argumentFound = true; break; } } + if (!argumentFound) { + throw new IllegalArgumentException( + "Failed to evaluate argument " + originalColumnName); + } } params.add(timezone); params.add(epochTime); @@ -108,12 +114,11 @@ private Object[] generateParams(BinaryRecordData after, long epochTime) { private TransformExpressionKey generateTransformExpressionKey() { List argumentNames = new ArrayList<>(); List> paramTypes = new ArrayList<>(); - List columns = tableInfo.getSchema().getColumns(); + List columns = tableInfo.getOriginalSchema().getColumns(); String scriptExpression = projectionColumn.getScriptExpression(); List originalColumnNames = projectionColumn.getOriginalColumnNames(); for (String originalColumnName : originalColumnNames) { - for (int i = 0; i < columns.size(); i++) { - Column column = columns.get(i); + for (Column column : columns) { if (column.getName().equals(originalColumnName)) { argumentNames.add(originalColumnName); paramTypes.add(DataTypeConverter.convertOriginalClass(column.getType())); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TableInfo.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TableInfo.java index 93bfcafec9e..8950c577fde 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TableInfo.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TableInfo.java @@ -31,16 +31,22 @@ public class TableInfo { private TableId tableId; private Schema schema; private RecordData.FieldGetter[] fieldGetters; + private Schema originalSchema; + private RecordData.FieldGetter[] originalFieldGetters; private BinaryRecordDataGenerator recordDataGenerator; public TableInfo( TableId tableId, Schema schema, RecordData.FieldGetter[] fieldGetters, + Schema originalSchema, + RecordData.FieldGetter[] originalFieldGetters, BinaryRecordDataGenerator recordDataGenerator) { this.tableId = tableId; this.schema = schema; this.fieldGetters = fieldGetters; + this.originalSchema = originalSchema; + this.originalFieldGetters = originalFieldGetters; this.recordDataGenerator = recordDataGenerator; } @@ -68,23 +74,40 @@ public Schema getSchema() { return schema; } + public Schema getOriginalSchema() { + return originalSchema; + } + public RecordData.FieldGetter[] getFieldGetters() { return fieldGetters; } + public RecordData.FieldGetter[] getOriginalFieldGetters() { + return originalFieldGetters; + } + public BinaryRecordDataGenerator getRecordDataGenerator() { return recordDataGenerator; } - public static TableInfo of(TableId tableId, Schema schema) { - List fieldGetters = - SchemaUtils.createFieldGetters(schema.getColumns()); - BinaryRecordDataGenerator recordDataGenerator = - new BinaryRecordDataGenerator(DataTypeConverter.toRowType(schema.getColumns())); + public static TableInfo of(TableId tableId, Schema projectedSchema, Schema originalSchema) { + + List projectedFieldGetters = + SchemaUtils.createFieldGetters(projectedSchema.getColumns()); + + List originalFieldGetters = + SchemaUtils.createFieldGetters(originalSchema.getColumns()); + + BinaryRecordDataGenerator projectedRecordDataGenerator = + new BinaryRecordDataGenerator( + DataTypeConverter.toRowType(projectedSchema.getColumns())); + return new TableInfo( tableId, - schema, - fieldGetters.toArray(new RecordData.FieldGetter[0]), - recordDataGenerator); + projectedSchema, + projectedFieldGetters.toArray(new RecordData.FieldGetter[0]), + originalSchema, + originalFieldGetters.toArray(new RecordData.FieldGetter[0]), + projectedRecordDataGenerator); } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformFilterProcessor.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformFilterProcessor.java index 7eaefc3c84b..933d8ddf66a 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformFilterProcessor.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformFilterProcessor.java @@ -70,8 +70,8 @@ public boolean process(BinaryRecordData after, long epochTime) { private Object[] generateParams(BinaryRecordData after, long epochTime) { List params = new ArrayList<>(); - List columns = tableInfo.getSchema().getColumns(); - RecordData.FieldGetter[] fieldGetters = tableInfo.getFieldGetters(); + List columns = tableInfo.getOriginalSchema().getColumns(); + RecordData.FieldGetter[] fieldGetters = tableInfo.getOriginalFieldGetters(); for (String columnName : transformFilter.getColumnNames()) { if (columnName.equals(TransformParser.DEFAULT_NAMESPACE_NAME)) { params.add(tableInfo.getNamespace()); @@ -103,12 +103,11 @@ private Object[] generateParams(BinaryRecordData after, long epochTime) { private TransformExpressionKey generateTransformExpressionKey() { List argumentNames = new ArrayList<>(); List> paramTypes = new ArrayList<>(); - List columns = tableInfo.getSchema().getColumns(); + List columns = tableInfo.getOriginalSchema().getColumns(); String scriptExpression = transformFilter.getScriptExpression(); List columnNames = transformFilter.getColumnNames(); for (String columnName : columnNames) { - for (int i = 0; i < columns.size(); i++) { - Column column = columns.get(i); + for (Column column : columns) { if (column.getName().equals(columnName)) { argumentNames.add(columnName); paramTypes.add(DataTypeConverter.convertOriginalClass(column.getType())); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformRule.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformRule.java new file mode 100644 index 00000000000..27b2652266d --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformRule.java @@ -0,0 +1,77 @@ +/* + * 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.cdc.runtime.operators.transform; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +import static org.apache.flink.cdc.runtime.parser.TransformParser.normalizeFilter; + +/** A rule defining pre-transformations where filtered rows and irrelevant columns are removed. */ +public class TransformRule implements Serializable { + private static final long serialVersionUID = 1L; + + private final String tableInclusions; + private final @Nullable String projection; + private final @Nullable String filter; + private final String primaryKey; + private final String partitionKey; + private final String tableOption; + + public TransformRule( + String tableInclusions, + @Nullable String projection, + @Nullable String filter, + String primaryKey, + String partitionKey, + String tableOption) { + this.tableInclusions = tableInclusions; + this.projection = projection; + this.filter = normalizeFilter(projection, filter); + this.primaryKey = primaryKey; + this.partitionKey = partitionKey; + this.tableOption = tableOption; + } + + public String getTableInclusions() { + return tableInclusions; + } + + @Nullable + public String getProjection() { + return projection; + } + + @Nullable + public String getFilter() { + return filter; + } + + public String getPrimaryKey() { + return primaryKey; + } + + public String getPartitionKey() { + return partitionKey; + } + + public String getTableOption() { + return tableOption; + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/TransformParser.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/TransformParser.java index 23cf4b376d4..4db3f354fc4 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/TransformParser.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/TransformParser.java @@ -21,7 +21,6 @@ import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.types.DataType; import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.cdc.common.utils.StringUtils; import org.apache.flink.cdc.runtime.operators.transform.ProjectionColumn; import org.apache.flink.cdc.runtime.parser.metadata.TransformSchemaFactory; import org.apache.flink.cdc.runtime.parser.metadata.TransformSqlOperatorTable; @@ -41,15 +40,16 @@ import org.apache.calcite.rel.type.RelDataTypeSystem; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlSelect; -import org.apache.calcite.sql.fun.SqlCase; import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; +import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlTypeFactoryImpl; import org.apache.calcite.sql.util.SqlOperatorTables; import org.apache.calcite.sql.validate.SqlConformanceEnum; @@ -60,14 +60,21 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.cdc.common.utils.StringUtils.isNullOrWhitespaceOnly; /** Use Flink's calcite parser to parse the statement of flink cdc pipeline transform. */ public class TransformParser { @@ -141,16 +148,86 @@ public static SqlSelect parseSelect(String statement) { } } - // Parse all columns + // Returns referenced columns (directly and indirectly) by projection and filter expression. + // For example, given projection expression "a, c, upper(x) as d, y as e", filter expression "z + // > 0", and columns array [a, b, c, x, y, z], returns referenced column array [a, c, x, y, z]. + public static List generateReferencedColumns( + String projectionExpression, @Nullable String filterExpression, List columns) { + if (isNullOrWhitespaceOnly(projectionExpression)) { + return new ArrayList<>(); + } + + Set referencedColumnNames = new HashSet<>(); + + SqlSelect sqlProject = parseProjectionExpression(projectionExpression); + if (!sqlProject.getSelectList().isEmpty()) { + for (SqlNode sqlNode : sqlProject.getSelectList()) { + if (sqlNode instanceof SqlBasicCall) { + SqlBasicCall sqlBasicCall = (SqlBasicCall) sqlNode; + if (SqlKind.AS.equals(sqlBasicCall.getOperator().kind)) { + referencedColumnNames.addAll( + parseColumnNameList(sqlBasicCall.getOperandList().get(0))); + } else { + throw new ParseException( + "Unrecognized projection expression: " + + sqlBasicCall + + ". Should be AS "); + } + } else if (sqlNode instanceof SqlIdentifier) { + SqlIdentifier sqlIdentifier = (SqlIdentifier) sqlNode; + if (sqlIdentifier.isStar()) { + // wildcard star character matches all columns + return columns; + } + referencedColumnNames.add( + sqlIdentifier.names.get(sqlIdentifier.names.size() - 1)); + } + } + } + + if (!isNullOrWhitespaceOnly(projectionExpression)) { + SqlSelect sqlFilter = parseFilterExpression(filterExpression); + referencedColumnNames.addAll(parseColumnNameList(sqlFilter.getWhere())); + } + + return columns.stream() + .filter(e -> referencedColumnNames.contains(e.getName())) + .collect(Collectors.toList()); + } + + // Expands wildcard character * to full column list. + // For example, given projection expression "a AS new_a, *, c as new_c" + // and schema [a, b, c], expand it to [a as new_a, a, b, c, c as new_c]. + // This step is necessary since passing wildcard to sqlToRel will capture + // unexpected metadata columns. + private static void expandWildcard(SqlSelect sqlSelect, List columns) { + List expandedNodes = new ArrayList<>(); + for (SqlNode sqlNode : sqlSelect.getSelectList().getList()) { + if (sqlNode instanceof SqlIdentifier && ((SqlIdentifier) sqlNode).isStar()) { + expandedNodes.addAll( + columns.stream() + .map(c -> new SqlIdentifier(c.getName(), SqlParserPos.QUOTED_ZERO)) + .collect(Collectors.toList())); + } else { + expandedNodes.add(sqlNode); + } + } + sqlSelect.setSelectList(new SqlNodeList(expandedNodes, SqlParserPos.ZERO)); + } + + // Returns projected columns based on given projection expression. + // For example, given projection expression "a, b, c, upper(a) as d, b as e" and columns array + // [a, b, c, x, y, z], returns projection column array [a, b, c, d, e]. public static List generateProjectionColumns( String projectionExpression, List columns) { - if (StringUtils.isNullOrWhitespaceOnly(projectionExpression)) { + if (isNullOrWhitespaceOnly(projectionExpression)) { return new ArrayList<>(); } SqlSelect sqlSelect = parseProjectionExpression(projectionExpression); if (sqlSelect.getSelectList().isEmpty()) { return new ArrayList<>(); } + expandWildcard(sqlSelect, columns); RelNode relNode = sqlToRel(columns, sqlSelect); Map relDataTypeMap = relNode.getRowType().getFieldList().stream() @@ -158,18 +235,23 @@ public static List generateProjectionColumns( Collectors.toMap( RelDataTypeField::getName, RelDataTypeField::getType)); + Map rawDataTypeMap = + columns.stream().collect(Collectors.toMap(Column::getName, Column::getType)); + Map isNotNullMap = columns.stream() .collect( Collectors.toMap( Column::getName, column -> !column.getType().isNullable())); + List projectionColumns = new ArrayList<>(); + for (SqlNode sqlNode : sqlSelect.getSelectList()) { if (sqlNode instanceof SqlBasicCall) { SqlBasicCall sqlBasicCall = (SqlBasicCall) sqlNode; if (SqlKind.AS.equals(sqlBasicCall.getOperator().kind)) { Optional transformOptional = Optional.empty(); - String columnName = null; + String columnName; List operandList = sqlBasicCall.getOperandList(); if (operandList.size() == 2) { transformOptional = Optional.of(operandList.get(0)); @@ -177,7 +259,11 @@ public static List generateProjectionColumns( if (sqlNode1 instanceof SqlIdentifier) { SqlIdentifier sqlIdentifier = (SqlIdentifier) sqlNode1; columnName = sqlIdentifier.names.get(sqlIdentifier.names.size() - 1); + } else { + columnName = null; } + } else { + columnName = null; } if (isMetadataColumn(columnName)) { continue; @@ -209,14 +295,25 @@ public static List generateProjectionColumns( projectionColumns.add(projectionColumn); } } else { - throw new ParseException("Unrecognized projection: " + sqlBasicCall.toString()); + throw new ParseException( + "Unrecognized projection expression: " + + sqlBasicCall + + ". Should be AS "); } } else if (sqlNode instanceof SqlIdentifier) { SqlIdentifier sqlIdentifier = (SqlIdentifier) sqlNode; String columnName = sqlIdentifier.names.get(sqlIdentifier.names.size() - 1); - DataType columnType = - DataTypeConverter.convertCalciteRelDataTypeToDataType( - relDataTypeMap.get(columnName)); + DataType columnType; + if (rawDataTypeMap.containsKey(columnName)) { + columnType = rawDataTypeMap.get(columnName); + } else if (relDataTypeMap.containsKey(columnName)) { + columnType = + DataTypeConverter.convertCalciteRelDataTypeToDataType( + relDataTypeMap.get(columnName)); + } else { + throw new RuntimeException( + String.format("Failed to deduce column %s type", columnName)); + } if (isMetadataColumn(columnName)) { projectionColumns.add( ProjectionColumn.of( @@ -244,7 +341,7 @@ public static List generateProjectionColumns( } public static String translateFilterExpressionToJaninoExpression(String filterExpression) { - if (StringUtils.isNullOrWhitespaceOnly(filterExpression)) { + if (isNullOrWhitespaceOnly(filterExpression)) { return ""; } SqlSelect sqlSelect = TransformParser.parseFilterExpression(filterExpression); @@ -257,7 +354,7 @@ public static String translateFilterExpressionToJaninoExpression(String filterEx public static List parseComputedColumnNames(String projection) { List columnNames = new ArrayList<>(); - if (StringUtils.isNullOrWhitespaceOnly(projection)) { + if (isNullOrWhitespaceOnly(projection)) { return columnNames; } SqlSelect sqlSelect = parseProjectionExpression(projection); @@ -298,7 +395,7 @@ public static List parseComputedColumnNames(String projection) { } public static List parseFilterColumnNameList(String filterExpression) { - if (StringUtils.isNullOrWhitespaceOnly(filterExpression)) { + if (isNullOrWhitespaceOnly(filterExpression)) { return new ArrayList<>(); } SqlSelect sqlSelect = parseFilterExpression(filterExpression); @@ -315,12 +412,12 @@ private static List parseColumnNameList(SqlNode sqlNode) { SqlIdentifier sqlIdentifier = (SqlIdentifier) sqlNode; String columnName = sqlIdentifier.names.get(sqlIdentifier.names.size() - 1); columnNameList.add(columnName); - } else if (sqlNode instanceof SqlBasicCall) { - SqlBasicCall sqlBasicCall = (SqlBasicCall) sqlNode; - findSqlIdentifier(sqlBasicCall.getOperandList(), columnNameList); - } else if (sqlNode instanceof SqlCase) { - SqlCase sqlCase = (SqlCase) sqlNode; - findSqlIdentifier(sqlCase.getWhenOperands().getList(), columnNameList); + } else if (sqlNode instanceof SqlCall) { + SqlCall sqlCall = (SqlCall) sqlNode; + findSqlIdentifier(sqlCall.getOperandList(), columnNameList); + } else if (sqlNode instanceof SqlNodeList) { + SqlNodeList sqlNodeList = (SqlNodeList) sqlNode; + findSqlIdentifier(sqlNodeList.getList(), columnNameList); } return columnNameList; } @@ -331,13 +428,12 @@ private static void findSqlIdentifier(List sqlNodes, List colum SqlIdentifier sqlIdentifier = (SqlIdentifier) sqlNode; String columnName = sqlIdentifier.names.get(sqlIdentifier.names.size() - 1); columnNameList.add(columnName); - } else if (sqlNode instanceof SqlBasicCall) { - SqlBasicCall sqlBasicCall = (SqlBasicCall) sqlNode; - findSqlIdentifier(sqlBasicCall.getOperandList(), columnNameList); - } else if (sqlNode instanceof SqlCase) { - SqlCase sqlCase = (SqlCase) sqlNode; - SqlNodeList whenOperands = sqlCase.getWhenOperands(); - findSqlIdentifier(whenOperands.getList(), columnNameList); + } else if (sqlNode instanceof SqlCall) { + SqlCall sqlCall = (SqlCall) sqlNode; + findSqlIdentifier(sqlCall.getOperandList(), columnNameList); + } else if (sqlNode instanceof SqlNodeList) { + SqlNodeList sqlNodeList = (SqlNodeList) sqlNode; + findSqlIdentifier(sqlNodeList.getList(), columnNameList); } } } @@ -384,10 +480,81 @@ public static SqlSelect parseFilterExpression(String filterExpression) { StringBuilder statement = new StringBuilder(); statement.append("SELECT * FROM "); statement.append(DEFAULT_TABLE); - if (!StringUtils.isNullOrWhitespaceOnly(filterExpression)) { + if (!isNullOrWhitespaceOnly(filterExpression)) { statement.append(" WHERE "); statement.append(filterExpression); } return parseSelect(statement.toString()); } + + public static SqlNode rewriteExpression(SqlNode sqlNode, Map replaceMap) { + if (sqlNode instanceof SqlCall) { + SqlCall sqlCall = (SqlCall) sqlNode; + + List operands = sqlCall.getOperandList(); + IntStream.range(0, sqlCall.operandCount()) + .forEach( + i -> + sqlCall.setOperand( + i, rewriteExpression(operands.get(i), replaceMap))); + return sqlCall; + } else if (sqlNode instanceof SqlIdentifier) { + SqlIdentifier sqlIdentifier = (SqlIdentifier) sqlNode; + if (sqlIdentifier.names.size() == 1) { + String name = sqlIdentifier.names.get(0); + if (replaceMap.containsKey(name)) { + return replaceMap.get(name); + } + } + return sqlIdentifier; + } else if (sqlNode instanceof SqlNodeList) { + SqlNodeList sqlNodeList = (SqlNodeList) sqlNode; + IntStream.range(0, sqlNodeList.size()) + .forEach( + i -> + sqlNodeList.set( + i, rewriteExpression(sqlNodeList.get(i), replaceMap))); + return sqlNodeList; + } else { + return sqlNode; + } + } + + // Filter expression might hold reference to a calculated column, which causes confusion about + // the sequence of projection and filtering operations. This function rewrites filtering about + // calculated columns to circumvent this problem. + public static String normalizeFilter(String projection, String filter) { + if (isNullOrWhitespaceOnly(projection) || isNullOrWhitespaceOnly(filter)) { + return filter; + } + + SqlSelect sqlSelect = parseProjectionExpression(projection); + if (sqlSelect.getSelectList().isEmpty()) { + return filter; + } + + Map calculatedExpression = new HashMap<>(); + for (SqlNode sqlNode : sqlSelect.getSelectList()) { + if (sqlNode instanceof SqlBasicCall) { + SqlBasicCall sqlBasicCall = (SqlBasicCall) sqlNode; + if (SqlKind.AS.equals(sqlBasicCall.getOperator().kind)) { + List operandList = sqlBasicCall.getOperandList(); + if (operandList.size() == 2) { + SqlIdentifier alias = (SqlIdentifier) operandList.get(1); + String name = alias.names.get(alias.names.size() - 1); + SqlNode expression = operandList.get(0); + calculatedExpression.put(name, expression); + } + } + } + } + + SqlNode sqlFilter = parseFilterExpression(filter).getWhere(); + sqlFilter = rewriteExpression(sqlFilter, calculatedExpression); + if (sqlFilter != null) { + return sqlFilter.toString(); + } else { + return filter; + } + } } diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/TransformDataOperatorTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperatorTest.java similarity index 69% rename from flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/TransformDataOperatorTest.java rename to flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperatorTest.java index 09d12e82302..15a9dd83e0a 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/TransformDataOperatorTest.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperatorTest.java @@ -33,11 +33,12 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; +import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; import java.math.BigDecimal; -/** Unit tests for the {@link TransformDataOperator}. */ -public class TransformDataOperatorTest { +/** Unit tests for the {@link PostTransformOperator}. */ +public class PostTransformOperatorTest { private static final TableId CUSTOMERS_TABLEID = TableId.tableId("my_company", "my_branch", "customers"); private static final Schema CUSTOMERS_SCHEMA = @@ -70,12 +71,17 @@ public class TransformDataOperatorTest { private static final TableId METADATA_TABLEID = TableId.tableId("my_company", "my_branch", "metadata_table"); private static final Schema METADATA_SCHEMA = + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .primaryKey("col1") + .build(); + private static final Schema EXPECTED_METADATA_SCHEMA = Schema.newBuilder() .physicalColumn("col1", DataTypes.STRING()) .physicalColumn("identifier_name", DataTypes.STRING()) - .physicalColumn("__namespace_name__", DataTypes.STRING()) - .physicalColumn("__schema_name__", DataTypes.STRING()) - .physicalColumn("__table_name__", DataTypes.STRING()) + .physicalColumn("__namespace_name__", DataTypes.STRING().notNull()) + .physicalColumn("__schema_name__", DataTypes.STRING().notNull()) + .physicalColumn("__table_name__", DataTypes.STRING().notNull()) .primaryKey("col1") .build(); @@ -124,16 +130,66 @@ public class TransformDataOperatorTest { .primaryKey("col1") .build(); + private static final TableId REDUCE_TABLEID = + TableId.tableId("my_company", "my_branch", "reduce_table"); + + private static final Schema REDUCE_SCHEMA = + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("ref1", DataTypes.STRING()) + .physicalColumn("ref2", DataTypes.INT()) + .primaryKey("id") + .partitionKey("id") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + + private static final Schema EXPECTED_REDUCE_SCHEMA = + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("uid", DataTypes.STRING()) + .physicalColumn("newage", DataTypes.INT()) + .physicalColumn("ref1", DataTypes.STRING()) + .physicalColumn("seventeen", DataTypes.INT()) + .primaryKey("id") + .partitionKey("id") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + + private static final TableId WILDCARD_TABLEID = + TableId.tableId("my_company", "my_branch", "wildcard_table"); + + private static final Schema WILDCARD_SCHEMA = + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .primaryKey("id") + .partitionKey("id") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + + private static final Schema EXPECTED_WILDCARD_SCHEMA = + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("newage", DataTypes.INT()) + .primaryKey("id") + .partitionKey("id") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + @Test void testDataChangeEventTransform() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( CUSTOMERS_TABLEID.identifier(), "*, concat(col1,col2) col12", "col1 = '1'") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -206,7 +262,6 @@ void testDataChangeEventTransform() throws Exception { Assertions.assertThat( transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) .isEqualTo(new StreamRecord<>(insertEventExpect)); - transform.processElement(new StreamRecord<>(insertEventIgnored)); transform.processElement(new StreamRecord<>(updateEvent)); Assertions.assertThat( transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) @@ -215,8 +270,8 @@ void testDataChangeEventTransform() throws Exception { @Test void testDataChangeEventTransformTwice() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( CUSTOMERS_TABLEID.identifier(), "*, concat(col1, '1') col12", @@ -226,7 +281,7 @@ void testDataChangeEventTransformTwice() throws Exception { "*, concat(col1, '2') col12", "col1 = '2'") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -320,11 +375,11 @@ void testDataChangeEventTransformTwice() throws Exception { @Test void testDataChangeEventTransformProjectionDataTypeConvert() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() - .addTransform(DATATYPE_TABLEID.identifier(), "*", null) + PostTransformOperator transform = + PostTransformOperator.newBuilder() + .addTransform(DATATYPE_TABLEID.identifier(), "*", null, null, null, null) .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -366,14 +421,14 @@ void testDataChangeEventTransformProjectionDataTypeConvert() throws Exception { @Test void testMetadataTransform() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( METADATA_TABLEID.identifier(), "*, __namespace_name__ || '.' || __schema_name__ || '.' || __table_name__ identifier_name, __namespace_name__, __schema_name__, __table_name__", " __table_name__ = 'metadata_table' ") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -382,16 +437,17 @@ void testMetadataTransform() throws Exception { CreateTableEvent createTableEvent = new CreateTableEvent(METADATA_TABLEID, METADATA_SCHEMA); BinaryRecordDataGenerator recordDataGenerator = new BinaryRecordDataGenerator(((RowType) METADATA_SCHEMA.toRowDataType())); + BinaryRecordDataGenerator expectedRecordDataGenerator = + new BinaryRecordDataGenerator(((RowType) EXPECTED_METADATA_SCHEMA.toRowDataType())); // Insert DataChangeEvent insertEvent = DataChangeEvent.insertEvent( METADATA_TABLEID, - recordDataGenerator.generate( - new Object[] {new BinaryStringData("1"), null, null, null, null})); + recordDataGenerator.generate(new Object[] {new BinaryStringData("1")})); DataChangeEvent insertEventExpect = DataChangeEvent.insertEvent( METADATA_TABLEID, - recordDataGenerator.generate( + expectedRecordDataGenerator.generate( new Object[] { new BinaryStringData("1"), new BinaryStringData("my_company.my_branch.metadata_table"), @@ -404,7 +460,7 @@ void testMetadataTransform() throws Exception { transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) .isEqualTo( new StreamRecord<>( - new CreateTableEvent(METADATA_TABLEID, METADATA_SCHEMA))); + new CreateTableEvent(METADATA_TABLEID, EXPECTED_METADATA_SCHEMA))); transform.processElement(new StreamRecord<>(insertEvent)); Assertions.assertThat( transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) @@ -413,14 +469,14 @@ void testMetadataTransform() throws Exception { @Test void testMetadataASTransform() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( METADATA_AS_TABLEID.identifier(), "sid, name, UPPER(name) as name_upper, __table_name__ as tbname", "sid < 3") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -460,8 +516,8 @@ void testMetadataASTransform() throws Exception { @Test void testTimestampTransform() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( TIMESTAMP_TABLEID.identifier(), "col1, IF(LOCALTIME = CURRENT_TIME, 1, 0) as time_equal," @@ -470,7 +526,7 @@ void testTimestampTransform() throws Exception { "LOCALTIMESTAMP = CURRENT_TIMESTAMP") .addTimezone("GMT") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -505,8 +561,8 @@ void testTimestampTransform() throws Exception { @Test void testTimestampDiffTransform() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( TIMESTAMPDIFF_TABLEID.identifier(), "col1, TIMESTAMP_DIFF('SECOND', LOCALTIMESTAMP, CURRENT_TIMESTAMP) as second_diff," @@ -518,7 +574,7 @@ void testTimestampDiffTransform() throws Exception { null) .addTimezone("GMT-8:00") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -595,15 +651,15 @@ void testBuildInFunctionTransform() throws Exception { } private void testExpressionConditionTransform(String expression) throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( CONDITION_TABLEID.identifier(), "col1, IF(" + expression + ", true, false) as condition_result", expression) .addTimezone("GMT") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -635,4 +691,202 @@ private void testExpressionConditionTransform(String expression) throws Exceptio transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) .isEqualTo(new StreamRecord<>(insertEventExpect)); } + + @Test + public void testReduceSchemaTransform() throws Exception { + PostTransformOperator transform = + PostTransformOperator.newBuilder() + .addTransform( + REDUCE_TABLEID.identifier(), + "id, upper(id) as uid, age + 1 as newage, lower(ref1) as ref1, 17 as seventeen", + "newage > 17 and ref2 > 17") + .addTimezone("GMT") + .build(); + EventOperatorTestHarness + transformFunctionEventEventOperatorTestHarness = + new EventOperatorTestHarness<>(transform, 1); + // Initialization + transformFunctionEventEventOperatorTestHarness.open(); + // Create table + CreateTableEvent createTableEvent = new CreateTableEvent(REDUCE_TABLEID, REDUCE_SCHEMA); + BinaryRecordDataGenerator recordDataGenerator = + new BinaryRecordDataGenerator(((RowType) REDUCE_SCHEMA.toRowDataType())); + BinaryRecordDataGenerator expectedRecordDataGenerator = + new BinaryRecordDataGenerator(((RowType) EXPECTED_REDUCE_SCHEMA.toRowDataType())); + // Insert + DataChangeEvent insertEvent = + DataChangeEvent.insertEvent( + REDUCE_TABLEID, + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + 17, + new BinaryStringData("Reference"), + 42 + })); + + DataChangeEvent insertEventExpect = + DataChangeEvent.insertEvent( + REDUCE_TABLEID, + expectedRecordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + new BinaryStringData("ID001"), + 18, + new BinaryStringData("reference"), + 17 + })); + + // Update + DataChangeEvent updateEvent = + DataChangeEvent.updateEvent( + REDUCE_TABLEID, + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + 17, + new BinaryStringData("Reference"), + 42 + }), + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + 18, + new BinaryStringData("UpdatedReference"), + 41 + })); + + DataChangeEvent updateEventExpect = + DataChangeEvent.updateEvent( + REDUCE_TABLEID, + expectedRecordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + new BinaryStringData("ID001"), + 18, + new BinaryStringData("reference"), + 17 + }), + expectedRecordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + new BinaryStringData("ID001"), + 19, + new BinaryStringData("updatedreference"), + 17 + })); + + transform.processElement(new StreamRecord<>(createTableEvent)); + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo( + new StreamRecord<>( + new CreateTableEvent(REDUCE_TABLEID, EXPECTED_REDUCE_SCHEMA))); + transform.processElement(new StreamRecord<>(insertEvent)); + + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo(new StreamRecord<>(insertEventExpect)); + + transform.processElement(new StreamRecord<>(updateEvent)); + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo(new StreamRecord<>(updateEventExpect)); + } + + @Test + public void testWildcardSchemaTransform() throws Exception { + PostTransformOperator transform = + PostTransformOperator.newBuilder() + .addTransform( + WILDCARD_TABLEID.identifier(), + "*, age + 1 as newage", + "newage > 17") + .addTimezone("GMT") + .build(); + EventOperatorTestHarness + transformFunctionEventEventOperatorTestHarness = + new EventOperatorTestHarness<>(transform, 1); + // Initialization + transformFunctionEventEventOperatorTestHarness.open(); + // Create table + CreateTableEvent createTableEvent = new CreateTableEvent(WILDCARD_TABLEID, WILDCARD_SCHEMA); + BinaryRecordDataGenerator recordDataGenerator = + new BinaryRecordDataGenerator(((RowType) WILDCARD_SCHEMA.toRowDataType())); + BinaryRecordDataGenerator expectedRecordDataGenerator = + new BinaryRecordDataGenerator(((RowType) EXPECTED_WILDCARD_SCHEMA.toRowDataType())); + // Insert + DataChangeEvent insertEvent = + DataChangeEvent.insertEvent( + WILDCARD_TABLEID, + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + 17, + new BinaryStringData("Alice"), + })); + + DataChangeEvent insertEventExpect = + DataChangeEvent.insertEvent( + WILDCARD_TABLEID, + expectedRecordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + 17, + new BinaryStringData("Alice"), + 18 + })); + + // Update + DataChangeEvent updateEvent = + DataChangeEvent.updateEvent( + WILDCARD_TABLEID, + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + 17, + new BinaryStringData("Alice"), + }), + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + 18, + new BinaryStringData("Arisu"), + })); + + DataChangeEvent updateEventExpect = + DataChangeEvent.updateEvent( + WILDCARD_TABLEID, + expectedRecordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + 17, + new BinaryStringData("Alice"), + 18 + }), + expectedRecordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + 18, + new BinaryStringData("Arisu"), + 19 + })); + + transform.processElement(new StreamRecord<>(createTableEvent)); + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo( + new StreamRecord<>( + new CreateTableEvent(WILDCARD_TABLEID, EXPECTED_WILDCARD_SCHEMA))); + transform.processElement(new StreamRecord<>(insertEvent)); + + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo(new StreamRecord<>(insertEventExpect)); + + transform.processElement(new StreamRecord<>(updateEvent)); + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo(new StreamRecord<>(updateEventExpect)); + } } diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorTest.java new file mode 100644 index 00000000000..c447d51c24c --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorTest.java @@ -0,0 +1,520 @@ +/* + * 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.cdc.runtime.operators.transform; + +import org.apache.flink.cdc.common.data.binary.BinaryStringData; +import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.RowType; +import org.apache.flink.cdc.runtime.testutils.operators.EventOperatorTestHarness; +import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; + +import java.util.Collections; + +/** Unit tests for the {@link PreTransformOperator}. */ +public class PreTransformOperatorTest { + private static final TableId CUSTOMERS_TABLEID = + TableId.tableId("my_company", "my_branch", "customers"); + private static final Schema CUSTOMERS_SCHEMA = + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .primaryKey("col1") + .build(); + private static final Schema CUSTOMERS_LATEST_SCHEMA = + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .physicalColumn("col3", DataTypes.STRING()) + .primaryKey("col1") + .build(); + private static final Schema EXPECT_SCHEMA = + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .primaryKey("col2") + .partitionKey("col12") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + private static final Schema EXPECT_LATEST_SCHEMA = + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .physicalColumn("col3", DataTypes.STRING()) + .primaryKey("col2") + .partitionKey("col12") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + + private static final Schema NULLABILITY_SCHEMA = + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("name", DataTypes.STRING()) + .primaryKey("id") + .partitionKey("id") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + + private static final Schema EXPECTED_NULLABILITY_SCHEMA = + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("name", DataTypes.STRING()) + .primaryKey("id") + .partitionKey("id") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + + private static final Schema REFERENCED_SCHEMA = + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("ref1", DataTypes.STRING()) + .physicalColumn("ref2", DataTypes.INT()) + .primaryKey("id") + .partitionKey("id") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + + private static final Schema EXPECTED_REFERENCED_SCHEMA = + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("ref1", DataTypes.STRING()) + .physicalColumn("ref2", DataTypes.INT()) + .primaryKey("id") + .partitionKey("id") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + + private static final Schema WILDCARD_SCHEMA = + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .primaryKey("id") + .partitionKey("id") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + + private static final Schema EXPECTED_WILDCARD_SCHEMA = + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .primaryKey("id") + .partitionKey("id") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + + private static final TableId METADATA_TABLEID = + TableId.tableId("my_company", "my_branch", "metadata_table"); + private static final Schema METADATA_SCHEMA = + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .primaryKey("id") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + + private static final Schema EXPECTED_METADATA_SCHEMA = + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .primaryKey("id") + .options(ImmutableMap.of("key1", "value1", "key2", "value2")) + .build(); + + private static final TableId METADATA_AS_TABLEID = + TableId.tableId("my_company", "my_branch", "metadata_as_table"); + private static final Schema METADATA_AS_SCHEMA = + Schema.newBuilder() + .physicalColumn("sid", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("name_upper", DataTypes.STRING()) + .physicalColumn("tbname", DataTypes.STRING()) + .primaryKey("sid") + .build(); + + @Test + void testEventTransform() throws Exception { + PreTransformOperator transform = + PreTransformOperator.newBuilder() + .addTransform( + CUSTOMERS_TABLEID.identifier(), + "*, concat(col1,col2) col12", + null, + "col2", + "col12", + "key1=value1,key2=value2") + .build(); + EventOperatorTestHarness + transformFunctionEventEventOperatorTestHarness = + new EventOperatorTestHarness<>(transform, 1); + // Initialization + transformFunctionEventEventOperatorTestHarness.open(); + // Create table + CreateTableEvent createTableEvent = + new CreateTableEvent(CUSTOMERS_TABLEID, CUSTOMERS_SCHEMA); + // Add column + AddColumnEvent.ColumnWithPosition columnWithPosition = + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("col3", DataTypes.STRING())); + AddColumnEvent addColumnEvent = + new AddColumnEvent( + CUSTOMERS_TABLEID, Collections.singletonList(columnWithPosition)); + BinaryRecordDataGenerator recordDataGenerator = + new BinaryRecordDataGenerator(((RowType) CUSTOMERS_LATEST_SCHEMA.toRowDataType())); + BinaryRecordDataGenerator recordDataGeneratorExpect = + new BinaryRecordDataGenerator(((RowType) EXPECT_LATEST_SCHEMA.toRowDataType())); + // Insert + DataChangeEvent insertEvent = + DataChangeEvent.insertEvent( + CUSTOMERS_TABLEID, + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("1"), + new BinaryStringData("2"), + new BinaryStringData("3"), + })); + DataChangeEvent insertEventExpect = + DataChangeEvent.insertEvent( + CUSTOMERS_TABLEID, + recordDataGeneratorExpect.generate( + new Object[] { + new BinaryStringData("1"), + new BinaryStringData("2"), + new BinaryStringData("3") + })); + + // Update + DataChangeEvent updateEvent = + DataChangeEvent.updateEvent( + CUSTOMERS_TABLEID, + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("1"), + new BinaryStringData("2"), + new BinaryStringData("3") + }), + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("1"), + new BinaryStringData("3"), + new BinaryStringData("3") + })); + DataChangeEvent updateEventExpect = + DataChangeEvent.updateEvent( + CUSTOMERS_TABLEID, + recordDataGeneratorExpect.generate( + new Object[] { + new BinaryStringData("1"), + new BinaryStringData("2"), + new BinaryStringData("3") + }), + recordDataGeneratorExpect.generate( + new Object[] { + new BinaryStringData("1"), + new BinaryStringData("3"), + new BinaryStringData("3") + })); + transform.processElement(new StreamRecord<>(createTableEvent)); + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo( + new StreamRecord<>(new CreateTableEvent(CUSTOMERS_TABLEID, EXPECT_SCHEMA))); + transform.processElement(new StreamRecord<>(addColumnEvent)); + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo(new StreamRecord<>(addColumnEvent)); + transform.processElement(new StreamRecord<>(insertEvent)); + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo(new StreamRecord<>(insertEventExpect)); + transform.processElement(new StreamRecord<>(updateEvent)); + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo(new StreamRecord<>(updateEventExpect)); + } + + @Test + public void testNullabilityColumn() throws Exception { + PreTransformOperator transform = + PreTransformOperator.newBuilder() + .addTransform( + CUSTOMERS_TABLEID.identifier(), + "id, upper(id) uid, name, upper(name) uname", + null, + "id", + "id", + "key1=value1,key2=value2") + .build(); + EventOperatorTestHarness + transformFunctionEventEventOperatorTestHarness = + new EventOperatorTestHarness<>(transform, 1); + // Initialization + transformFunctionEventEventOperatorTestHarness.open(); + // Create table + CreateTableEvent createTableEvent = + new CreateTableEvent(CUSTOMERS_TABLEID, NULLABILITY_SCHEMA); + transform.processElement(new StreamRecord<>(createTableEvent)); + + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo( + new StreamRecord<>( + new CreateTableEvent( + CUSTOMERS_TABLEID, EXPECTED_NULLABILITY_SCHEMA))); + } + + @Test + public void testReduceTransformColumn() throws Exception { + PreTransformOperator transform = + PreTransformOperator.newBuilder() + .addTransform( + CUSTOMERS_TABLEID.identifier(), + "id, upper(id) as uid, age + 1 as newage, lower(ref1) as ref1", + "newage > 17 and ref2 > 17", + "id", + "id", + "key1=value1,key2=value2") + .build(); + EventOperatorTestHarness + transformFunctionEventEventOperatorTestHarness = + new EventOperatorTestHarness<>(transform, 1); + // Initialization + transformFunctionEventEventOperatorTestHarness.open(); + // Create table + CreateTableEvent createTableEvent = + new CreateTableEvent(CUSTOMERS_TABLEID, REFERENCED_SCHEMA); + transform.processElement(new StreamRecord<>(createTableEvent)); + + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo( + new StreamRecord<>( + new CreateTableEvent( + CUSTOMERS_TABLEID, EXPECTED_REFERENCED_SCHEMA))); + + BinaryRecordDataGenerator recordDataGenerator = + new BinaryRecordDataGenerator(((RowType) REFERENCED_SCHEMA.toRowDataType())); + BinaryRecordDataGenerator recordDataGeneratorExpect = + new BinaryRecordDataGenerator( + ((RowType) EXPECTED_REFERENCED_SCHEMA.toRowDataType())); + + // Insert + DataChangeEvent insertEvent = + DataChangeEvent.insertEvent( + CUSTOMERS_TABLEID, + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + 17, + new BinaryStringData("Alice"), + new BinaryStringData("Reference"), + 42, + })); + DataChangeEvent insertEventExpect = + DataChangeEvent.insertEvent( + CUSTOMERS_TABLEID, + recordDataGeneratorExpect.generate( + new Object[] { + new BinaryStringData("id001"), + 17, + new BinaryStringData("Reference"), + 42 + })); + + // Update + DataChangeEvent updateEvent = + DataChangeEvent.updateEvent( + CUSTOMERS_TABLEID, + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + 17, + new BinaryStringData("Alice"), + new BinaryStringData("Reference"), + 42, + }), + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), + 18, + new BinaryStringData("Arisu"), + new BinaryStringData("UpdatedReference"), + 41, + })); + DataChangeEvent updateEventExpect = + DataChangeEvent.updateEvent( + CUSTOMERS_TABLEID, + recordDataGeneratorExpect.generate( + new Object[] { + new BinaryStringData("id001"), + 17, + new BinaryStringData("Reference"), + 42 + }), + recordDataGeneratorExpect.generate( + new Object[] { + new BinaryStringData("id001"), + 18, + new BinaryStringData("UpdatedReference"), + 41 + })); + + transform.processElement(new StreamRecord<>(insertEvent)); + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo(new StreamRecord<>(insertEventExpect)); + + transform.processElement(new StreamRecord<>(updateEvent)); + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo(new StreamRecord<>(updateEventExpect)); + } + + @Test + public void testWildcardTransformColumn() throws Exception { + PreTransformOperator transform = + PreTransformOperator.newBuilder() + .addTransform( + CUSTOMERS_TABLEID.identifier(), + "*, age + 1 as newage", + "newage > 17", + "id", + "id", + "key1=value1,key2=value2") + .build(); + EventOperatorTestHarness + transformFunctionEventEventOperatorTestHarness = + new EventOperatorTestHarness<>(transform, 1); + // Initialization + transformFunctionEventEventOperatorTestHarness.open(); + // Create table + CreateTableEvent createTableEvent = + new CreateTableEvent(CUSTOMERS_TABLEID, WILDCARD_SCHEMA); + transform.processElement(new StreamRecord<>(createTableEvent)); + + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo( + new StreamRecord<>( + new CreateTableEvent(CUSTOMERS_TABLEID, EXPECTED_WILDCARD_SCHEMA))); + + BinaryRecordDataGenerator recordDataGenerator = + new BinaryRecordDataGenerator(((RowType) WILDCARD_SCHEMA.toRowDataType())); + BinaryRecordDataGenerator recordDataGeneratorExpect = + new BinaryRecordDataGenerator(((RowType) EXPECTED_WILDCARD_SCHEMA.toRowDataType())); + + // Insert + DataChangeEvent insertEvent = + DataChangeEvent.insertEvent( + CUSTOMERS_TABLEID, + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), 17, new BinaryStringData("Alice") + })); + DataChangeEvent insertEventExpect = + DataChangeEvent.insertEvent( + CUSTOMERS_TABLEID, + recordDataGeneratorExpect.generate( + new Object[] { + new BinaryStringData("id001"), + 17, + new BinaryStringData("Alice"), + })); + + // Update + DataChangeEvent updateEvent = + DataChangeEvent.updateEvent( + CUSTOMERS_TABLEID, + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), 17, new BinaryStringData("Alice") + }), + recordDataGenerator.generate( + new Object[] { + new BinaryStringData("id001"), 18, new BinaryStringData("Arisu") + })); + DataChangeEvent updateEventExpect = + DataChangeEvent.updateEvent( + CUSTOMERS_TABLEID, + recordDataGeneratorExpect.generate( + new Object[] { + new BinaryStringData("id001"), + 17, + new BinaryStringData("Alice"), + }), + recordDataGeneratorExpect.generate( + new Object[] { + new BinaryStringData("id001"), + 18, + new BinaryStringData("Arisu"), + })); + + transform.processElement(new StreamRecord<>(insertEvent)); + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo(new StreamRecord<>(insertEventExpect)); + + transform.processElement(new StreamRecord<>(updateEvent)); + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo(new StreamRecord<>(updateEventExpect)); + } + + @Test + void testMetadataTransform() throws Exception { + PreTransformOperator transform = + PreTransformOperator.newBuilder() + .addTransform( + METADATA_TABLEID.identifier(), + "*, __namespace_name__ || '.' || __schema_name__ || '.' || __table_name__ identifier_name, __namespace_name__, __schema_name__, __table_name__", + " __table_name__ = 'metadata_table' ") + .build(); + + EventOperatorTestHarness + transformFunctionEventEventOperatorTestHarness = + new EventOperatorTestHarness<>(transform, 1); + // Initialization + transformFunctionEventEventOperatorTestHarness.open(); + // Create table + CreateTableEvent createTableEvent = new CreateTableEvent(METADATA_TABLEID, METADATA_SCHEMA); + transform.processElement(new StreamRecord<>(createTableEvent)); + + Assertions.assertThat( + transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) + .isEqualTo( + new StreamRecord<>( + new CreateTableEvent(METADATA_TABLEID, EXPECTED_METADATA_SCHEMA))); + } +} diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/TransformSchemaOperatorTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/TransformSchemaOperatorTest.java deleted file mode 100644 index 82992a64e6b..00000000000 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/TransformSchemaOperatorTest.java +++ /dev/null @@ -1,228 +0,0 @@ -/* - * 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.cdc.runtime.operators.transform; - -import org.apache.flink.cdc.common.data.binary.BinaryStringData; -import org.apache.flink.cdc.common.event.AddColumnEvent; -import org.apache.flink.cdc.common.event.CreateTableEvent; -import org.apache.flink.cdc.common.event.DataChangeEvent; -import org.apache.flink.cdc.common.event.Event; -import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.common.schema.Column; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.cdc.common.types.RowType; -import org.apache.flink.cdc.runtime.testutils.operators.EventOperatorTestHarness; -import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; - -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.Test; -import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; - -import java.util.Collections; - -/** Unit tests for the {@link TransformSchemaOperator}. */ -public class TransformSchemaOperatorTest { - private static final TableId CUSTOMERS_TABLEID = - TableId.tableId("my_company", "my_branch", "customers"); - private static final Schema CUSTOMERS_SCHEMA = - Schema.newBuilder() - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .primaryKey("col1") - .build(); - private static final Schema CUSTOMERS_LATEST_SCHEMA = - Schema.newBuilder() - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col3", DataTypes.STRING()) - .primaryKey("col1") - .build(); - private static final Schema EXPECT_SCHEMA = - Schema.newBuilder() - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col12", DataTypes.STRING()) - .primaryKey("col2") - .partitionKey("col12") - .options(ImmutableMap.of("key1", "value1", "key2", "value2")) - .build(); - private static final Schema EXPECT_LATEST_SCHEMA = - Schema.newBuilder() - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col12", DataTypes.STRING()) - .physicalColumn("col3", DataTypes.STRING()) - .primaryKey("col2") - .partitionKey("col12") - .options(ImmutableMap.of("key1", "value1", "key2", "value2")) - .build(); - - private static final Schema NULLABILITY_SCHEMA = - Schema.newBuilder() - .physicalColumn("id", DataTypes.STRING().notNull()) - .physicalColumn("name", DataTypes.STRING()) - .primaryKey("id") - .partitionKey("id") - .options(ImmutableMap.of("key1", "value1", "key2", "value2")) - .build(); - - private static final Schema EXPECTED_NULLABILITY_SCHEMA = - Schema.newBuilder() - .physicalColumn("id", DataTypes.STRING().notNull()) - .physicalColumn("uid", DataTypes.STRING()) - .physicalColumn("name", DataTypes.STRING()) - .physicalColumn("uname", DataTypes.STRING()) - .primaryKey("id") - .partitionKey("id") - .options(ImmutableMap.of("key1", "value1", "key2", "value2")) - .build(); - - @Test - void testEventTransform() throws Exception { - TransformSchemaOperator transform = - TransformSchemaOperator.newBuilder() - .addTransform( - CUSTOMERS_TABLEID.identifier(), - "*, concat(col1,col2) col12", - "col2", - "col12", - "key1=value1,key2=value2") - .build(); - EventOperatorTestHarness - transformFunctionEventEventOperatorTestHarness = - new EventOperatorTestHarness<>(transform, 1); - // Initialization - transformFunctionEventEventOperatorTestHarness.open(); - // Create table - CreateTableEvent createTableEvent = - new CreateTableEvent(CUSTOMERS_TABLEID, CUSTOMERS_SCHEMA); - // Add column - AddColumnEvent.ColumnWithPosition columnWithPosition = - new AddColumnEvent.ColumnWithPosition( - Column.physicalColumn("col3", DataTypes.STRING())); - AddColumnEvent addColumnEvent = - new AddColumnEvent( - CUSTOMERS_TABLEID, Collections.singletonList(columnWithPosition)); - BinaryRecordDataGenerator recordDataGenerator = - new BinaryRecordDataGenerator(((RowType) CUSTOMERS_LATEST_SCHEMA.toRowDataType())); - BinaryRecordDataGenerator recordDataGeneratorExpect = - new BinaryRecordDataGenerator(((RowType) EXPECT_LATEST_SCHEMA.toRowDataType())); - // Insert - DataChangeEvent insertEvent = - DataChangeEvent.insertEvent( - CUSTOMERS_TABLEID, - recordDataGenerator.generate( - new Object[] { - new BinaryStringData("1"), - new BinaryStringData("2"), - new BinaryStringData("3"), - })); - DataChangeEvent insertEventExpect = - DataChangeEvent.insertEvent( - CUSTOMERS_TABLEID, - recordDataGeneratorExpect.generate( - new Object[] { - new BinaryStringData("1"), - new BinaryStringData("2"), - null, - new BinaryStringData("3") - })); - - // Update - DataChangeEvent updateEvent = - DataChangeEvent.updateEvent( - CUSTOMERS_TABLEID, - recordDataGenerator.generate( - new Object[] { - new BinaryStringData("1"), - new BinaryStringData("2"), - new BinaryStringData("3") - }), - recordDataGenerator.generate( - new Object[] { - new BinaryStringData("1"), - new BinaryStringData("3"), - new BinaryStringData("3") - })); - DataChangeEvent updateEventExpect = - DataChangeEvent.updateEvent( - CUSTOMERS_TABLEID, - recordDataGeneratorExpect.generate( - new Object[] { - new BinaryStringData("1"), - new BinaryStringData("2"), - null, - new BinaryStringData("3") - }), - recordDataGeneratorExpect.generate( - new Object[] { - new BinaryStringData("1"), - new BinaryStringData("3"), - null, - new BinaryStringData("3") - })); - transform.processElement(new StreamRecord<>(createTableEvent)); - Assertions.assertThat( - transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) - .isEqualTo( - new StreamRecord<>(new CreateTableEvent(CUSTOMERS_TABLEID, EXPECT_SCHEMA))); - transform.processElement(new StreamRecord<>(addColumnEvent)); - Assertions.assertThat( - transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) - .isEqualTo(new StreamRecord<>(addColumnEvent)); - transform.processElement(new StreamRecord<>(insertEvent)); - Assertions.assertThat( - transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) - .isEqualTo(new StreamRecord<>(insertEventExpect)); - transform.processElement(new StreamRecord<>(updateEvent)); - Assertions.assertThat( - transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) - .isEqualTo(new StreamRecord<>(updateEventExpect)); - } - - @Test - public void testNullabilityColumn() throws Exception { - TransformSchemaOperator transform = - TransformSchemaOperator.newBuilder() - .addTransform( - CUSTOMERS_TABLEID.identifier(), - "id, upper(id) uid, name, upper(name) uname", - "id", - "id", - "key1=value1,key2=value2") - .build(); - EventOperatorTestHarness - transformFunctionEventEventOperatorTestHarness = - new EventOperatorTestHarness<>(transform, 1); - // Initialization - transformFunctionEventEventOperatorTestHarness.open(); - // Create table - CreateTableEvent createTableEvent = - new CreateTableEvent(CUSTOMERS_TABLEID, NULLABILITY_SCHEMA); - transform.processElement(new StreamRecord<>(createTableEvent)); - - Assertions.assertThat( - transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) - .isEqualTo( - new StreamRecord<>( - new CreateTableEvent( - CUSTOMERS_TABLEID, EXPECTED_NULLABILITY_SCHEMA))); - } -} diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/UnifiedTransformOperatorTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/UnifiedTransformOperatorTest.java new file mode 100644 index 00000000000..b83ac688dc6 --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/UnifiedTransformOperatorTest.java @@ -0,0 +1,919 @@ +/* + * 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.cdc.runtime.operators.transform; + +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.data.binary.BinaryStringData; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.RowType; +import org.apache.flink.cdc.common.utils.SchemaUtils; +import org.apache.flink.cdc.runtime.testutils.operators.EventOperatorTestHarness; +import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +/** Unit tests for the {@link PreTransformOperator} and {@link PostTransformOperator}. */ +public class UnifiedTransformOperatorTest { + + /** Defines a unified transform test cases. */ + static class UnifiedTransformTestCase { + + private static final Logger LOG = LoggerFactory.getLogger(UnifiedTransformTestCase.class); + + private final TableId tableId; + private final String projectionExpression; + private final String filterExpression; + + private Schema sourceSchema; + private Schema preTransformedSchema; + private Schema postTransformedSchema; + + private final List sourceEvents; + private final List preTransformedEvents; + private final List postTransformedEvents; + + private final List sourceFieldGetters; + private final List preTransformedFieldGetters; + private final List postTransformedFieldGetters; + + private PreTransformOperator preTransformOperator; + private PostTransformOperator postTransformOperator; + + private final BinaryRecordDataGenerator sourceRecordGenerator; + private final BinaryRecordDataGenerator preTransformedRecordGenerator; + private final BinaryRecordDataGenerator postTransformedRecordGenerator; + + private EventOperatorTestHarness preTransformOperatorHarness; + private EventOperatorTestHarness postTransformOperatorHarness; + + public static UnifiedTransformTestCase of( + TableId tableId, + String projectionExpression, + String filterExpression, + Schema sourceSchema, + Schema preTransformedSchema, + Schema postTransformedSchema) { + return new UnifiedTransformTestCase( + tableId, + projectionExpression, + filterExpression, + sourceSchema, + preTransformedSchema, + postTransformedSchema); + } + + private Object[] stringify(Object... objects) { + return Arrays.stream(objects) + .map(o -> o instanceof String ? new BinaryStringData((String) o) : o) + .toArray(); + } + + public UnifiedTransformTestCase insertSource(Object... record) { + sourceEvents.add( + DataChangeEvent.insertEvent( + tableId, sourceRecordGenerator.generate(stringify(record)))); + return this; + } + + public UnifiedTransformTestCase insertPreTransformed() { + preTransformedEvents.add(null); + return this; + } + + public UnifiedTransformTestCase insertPreTransformed(Object... record) { + preTransformedEvents.add( + DataChangeEvent.insertEvent( + tableId, preTransformedRecordGenerator.generate(stringify(record)))); + return this; + } + + public UnifiedTransformTestCase insertPostTransformed() { + postTransformedEvents.add(null); + return this; + } + + public UnifiedTransformTestCase insertPostTransformed(Object... record) { + postTransformedEvents.add( + DataChangeEvent.insertEvent( + tableId, postTransformedRecordGenerator.generate(stringify(record)))); + return this; + } + + public UnifiedTransformTestCase updateSource(Object[] beforeRecord, Object[] afterRecord) { + sourceEvents.add( + DataChangeEvent.updateEvent( + tableId, + sourceRecordGenerator.generate(stringify(beforeRecord)), + sourceRecordGenerator.generate(stringify(afterRecord)))); + return this; + } + + public UnifiedTransformTestCase updatePreTransformed() { + preTransformedEvents.add(null); + return this; + } + + public UnifiedTransformTestCase updatePreTransformed( + Object[] beforeRecord, Object[] afterRecord) { + preTransformedEvents.add( + DataChangeEvent.updateEvent( + tableId, + preTransformedRecordGenerator.generate(stringify(beforeRecord)), + preTransformedRecordGenerator.generate(stringify(afterRecord)))); + return this; + } + + public UnifiedTransformTestCase updatePostTransformed() { + postTransformedEvents.add(null); + return this; + } + + public UnifiedTransformTestCase updatePostTransformed( + Object[] beforeRecord, Object[] afterRecord) { + postTransformedEvents.add( + DataChangeEvent.updateEvent( + tableId, + postTransformedRecordGenerator.generate(stringify(beforeRecord)), + postTransformedRecordGenerator.generate(stringify(afterRecord)))); + return this; + } + + public UnifiedTransformTestCase deleteSource(Object... record) { + sourceEvents.add( + DataChangeEvent.deleteEvent( + tableId, sourceRecordGenerator.generate(stringify(record)))); + return this; + } + + public UnifiedTransformTestCase deletePreTransformed() { + preTransformedEvents.add(null); + return this; + } + + public UnifiedTransformTestCase deletePreTransformed(Object... record) { + preTransformedEvents.add( + DataChangeEvent.deleteEvent( + tableId, preTransformedRecordGenerator.generate(stringify(record)))); + return this; + } + + public UnifiedTransformTestCase deletePostTransformed() { + postTransformedEvents.add(null); + return this; + } + + public UnifiedTransformTestCase deletePostTransformed(Object... record) { + postTransformedEvents.add( + DataChangeEvent.deleteEvent( + tableId, postTransformedRecordGenerator.generate(stringify(record)))); + return this; + } + + private UnifiedTransformTestCase( + TableId tableId, + String projectionExpression, + String filterExpression, + Schema sourceSchema, + Schema preTransformedSchema, + Schema postTransformedSchema) { + this.tableId = tableId; + this.projectionExpression = projectionExpression; + this.filterExpression = filterExpression; + + this.sourceSchema = sourceSchema; + this.preTransformedSchema = preTransformedSchema; + this.postTransformedSchema = postTransformedSchema; + + this.sourceRecordGenerator = + new BinaryRecordDataGenerator(((RowType) sourceSchema.toRowDataType())); + this.preTransformedRecordGenerator = + new BinaryRecordDataGenerator(((RowType) preTransformedSchema.toRowDataType())); + this.postTransformedRecordGenerator = + new BinaryRecordDataGenerator( + ((RowType) postTransformedSchema.toRowDataType())); + + this.sourceEvents = new ArrayList<>(); + this.preTransformedEvents = new ArrayList<>(); + this.postTransformedEvents = new ArrayList<>(); + + this.sourceEvents.add(new CreateTableEvent(tableId, sourceSchema)); + this.preTransformedEvents.add(new CreateTableEvent(tableId, preTransformedSchema)); + this.postTransformedEvents.add(new CreateTableEvent(tableId, postTransformedSchema)); + + this.sourceFieldGetters = SchemaUtils.createFieldGetters(sourceSchema); + this.preTransformedFieldGetters = SchemaUtils.createFieldGetters(preTransformedSchema); + this.postTransformedFieldGetters = + SchemaUtils.createFieldGetters(postTransformedSchema); + } + + private UnifiedTransformTestCase initializeHarness() throws Exception { + preTransformOperator = + PreTransformOperator.newBuilder() + .addTransform( + tableId.identifier(), projectionExpression, filterExpression) + .build(); + preTransformOperatorHarness = new EventOperatorTestHarness<>(preTransformOperator, 1); + preTransformOperatorHarness.open(); + + postTransformOperator = + PostTransformOperator.newBuilder() + .addTransform( + tableId.identifier(), projectionExpression, filterExpression) + .build(); + postTransformOperatorHarness = new EventOperatorTestHarness<>(postTransformOperator, 1); + postTransformOperatorHarness.open(); + return this; + } + + private void destroyHarness() throws Exception { + if (preTransformOperatorHarness != null) { + preTransformOperatorHarness.close(); + } + if (postTransformOperatorHarness != null) { + postTransformOperatorHarness.close(); + } + } + + private void logBinaryDataContents( + String prefix, Event event, List fieldGetters) { + LOG.info("{}: {}", prefix, event); + if (event instanceof DataChangeEvent) { + LOG.info( + " Before Record Data: {}", + SchemaUtils.restoreOriginalData( + ((DataChangeEvent) event).before(), fieldGetters)); + LOG.info( + " After Record Data: {}", + SchemaUtils.restoreOriginalData( + ((DataChangeEvent) event).after(), fieldGetters)); + } + } + + public UnifiedTransformTestCase runTests() throws Exception { + for (int i = 0; i < sourceEvents.size(); i++) { + Event sourceEvent = sourceEvents.get(i); + logBinaryDataContents("Source Event", sourceEvent, sourceFieldGetters); + + preTransformOperator.processElement(new StreamRecord<>(sourceEvent)); + + Event expectedPreTransformEvent = preTransformedEvents.get(i); + Event actualPreTransformEvent = + Optional.ofNullable(preTransformOperatorHarness.getOutputRecords().poll()) + .map(StreamRecord::getValue) + .orElse(null); + + logBinaryDataContents( + "Expected PreTransform ", + expectedPreTransformEvent, + preTransformedFieldGetters); + logBinaryDataContents( + " Actual PreTransform ", + actualPreTransformEvent, + preTransformedFieldGetters); + Assertions.assertThat(actualPreTransformEvent).isEqualTo(expectedPreTransformEvent); + + postTransformOperator.processElement( + new StreamRecord<>(preTransformedEvents.get(i))); + Event expectedPostTransformEvent = postTransformedEvents.get(i); + Event actualPostTransformEvent = + Optional.ofNullable(postTransformOperatorHarness.getOutputRecords().poll()) + .map(StreamRecord::getValue) + .orElse(null); + logBinaryDataContents( + "Expected PostTransform", + expectedPostTransformEvent, + postTransformedFieldGetters); + logBinaryDataContents( + " Actual PostTransform", + actualPostTransformEvent, + postTransformedFieldGetters); + Assertions.assertThat(actualPostTransformEvent) + .isEqualTo(expectedPostTransformEvent); + } + + sourceEvents.clear(); + preTransformedEvents.clear(); + postTransformedEvents.clear(); + return this; + } + } + + @Test + public void testDataChangeEventTransform() throws Exception { + TableId tableId = TableId.tableId("my_company", "my_branch", "data_changes"); + UnifiedTransformTestCase.of( + tableId, + "id, age, id + age as computed", + "id > 100", + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("computed", DataTypes.INT()) + .primaryKey("id") + .build()) + .initializeHarness() + .insertSource(1000, "Alice", 17) + .insertPreTransformed(1000, 17) + .insertPostTransformed(1000, 17, 1017) + .insertSource(2000, "Bob", 18) + .insertPreTransformed(2000, 18) + .insertPostTransformed(2000, 18, 2018) + .updateSource(new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePreTransformed(new Object[] {2000, 18}, new Object[] {2000, 16}) + .updatePostTransformed(new Object[] {2000, 18, 2018}, new Object[] {2000, 16, 2016}) + // filtered out data row + .insertSource(50, "Carol", 19) + .insertPreTransformed(50, 19) + .insertPostTransformed() + .deleteSource(1000, "Alice", 17) + .deletePreTransformed(1000, 17) + .deletePostTransformed(1000, 17, 1017) + .runTests() + .destroyHarness(); + } + + @Test + public void testSchemaNullabilityTransform() throws Exception { + TableId tableId = TableId.tableId("my_company", "my_branch", "schema_nullability"); + UnifiedTransformTestCase.of( + tableId, + "id, name, age, id + age as computed", + "id > 100", + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT().notNull()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT().notNull()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT().notNull()) + .physicalColumn("computed", DataTypes.INT()) + .primaryKey("id") + .build()) + .initializeHarness() + .insertSource(1000, "Alice", 17) + .insertPreTransformed(1000, "Alice", 17) + .insertPostTransformed(1000, "Alice", 17, 1017) + .insertSource(2000, "Bob", 18) + .insertPreTransformed(2000, "Bob", 18) + .insertPostTransformed(2000, "Bob", 18, 2018) + .updateSource(new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePreTransformed( + new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePostTransformed( + new Object[] {2000, "Bob", 18, 2018}, + new Object[] {2000, "Barcarolle", 16, 2016}) + // filtered out data row + .insertSource(50, "Carol", 19) + .insertPreTransformed(50, "Carol", 19) + .insertPostTransformed() + .deleteSource(1000, "Alice", 17) + .deletePreTransformed(1000, "Alice", 17) + .deletePostTransformed(1000, "Alice", 17, 1017) + .runTests() + .destroyHarness(); + } + + @Test + public void testReduceColumnsTransform() throws Exception { + TableId tableId = TableId.tableId("my_company", "my_branch", "reduce_column"); + UnifiedTransformTestCase.of( + tableId, + "id, upper(id) as uid, age + 1 as newage, lower(ref1) as lowerref", + "newage > 17 and ref2 > 17", + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("name", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT().notNull()) + .physicalColumn("ref1", DataTypes.STRING()) + .physicalColumn("ref2", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("age", DataTypes.INT().notNull()) + .physicalColumn("ref1", DataTypes.STRING()) + .physicalColumn("ref2", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("uid", DataTypes.STRING()) + .physicalColumn("newage", DataTypes.INT()) + .physicalColumn("lowerref", DataTypes.STRING()) + .primaryKey("id") + .build()) + .initializeHarness() + .insertSource("id001", "Alice", 17, "Reference001", 2021) + .insertPreTransformed("id001", 17, "Reference001", 2021) + .insertPostTransformed("id001", "ID001", 18, "reference001") + // this data record is filtered out since newage <= 17 + .insertSource("id002", "Bob", 15, "Reference002", 2017) + .insertPreTransformed("id002", 15, "Reference002", 2017) + .insertPostTransformed() + // this data record is filtered out since ref2 <= 17 + .insertSource("id003", "Bill", 18, "Reference003", 0) + .insertPreTransformed("id003", 18, "Reference003", 0) + .insertPostTransformed() + .insertSource("id004", "Carol", 18, "Reference004", 2018) + .insertPreTransformed("id004", 18, "Reference004", 2018) + .insertPostTransformed("id004", "ID004", 19, "reference004") + // test update event transform + .updateSource( + new Object[] {"id004", "Carol", 18, "Reference004", 2018}, + new Object[] {"id004", "Colin", 18, "NeoReference004", 2018}) + .updatePreTransformed( + new Object[] {"id004", 18, "Reference004", 2018}, + new Object[] {"id004", 18, "NeoReference004", 2018}) + .updatePostTransformed( + new Object[] {"id004", "ID004", 19, "reference004"}, + new Object[] {"id004", "ID004", 19, "neoreference004"}) + // updated value to a filtered out condition + .updateSource( + new Object[] {"id004", "Colin", 18, "NeoReference004", 2018}, + new Object[] {"id004", "Colin", 10, "NeoReference004", 2018}) + .updatePreTransformed( + new Object[] {"id004", 18, "NeoReference004", 2018}, + new Object[] {"id004", 10, "NeoReference004", 2018}) + .updatePostTransformed() + .deleteSource("id001", "Alice", 17, "Reference001", 2021) + .deletePreTransformed("id001", 17, "Reference001", 2021) + .deletePostTransformed("id001", "ID001", 18, "reference001") + .runTests() + .destroyHarness(); + } + + @Test + public void testWildcardTransform() throws Exception { + TableId tableId = TableId.tableId("my_company", "my_branch", "wildcard"); + UnifiedTransformTestCase.of( + tableId, + "*, id + age as computed", + "id > 100", + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("computed", DataTypes.INT()) + .primaryKey("id") + .build()) + .initializeHarness() + .insertSource(1000, "Alice", 17) + .insertPreTransformed(1000, "Alice", 17) + .insertPostTransformed(1000, "Alice", 17, 1017) + .insertSource(2000, "Bob", 18) + .insertPreTransformed(2000, "Bob", 18) + .insertPostTransformed(2000, "Bob", 18, 2018) + .updateSource(new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePreTransformed( + new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePostTransformed( + new Object[] {2000, "Bob", 18, 2018}, + new Object[] {2000, "Barcarolle", 16, 2016}) + // filtered out data row + .insertSource(50, "Carol", 19) + .insertPreTransformed(50, "Carol", 19) + .insertPostTransformed() + .deleteSource(1000, "Alice", 17) + .deletePreTransformed(1000, "Alice", 17) + .deletePostTransformed(1000, "Alice", 17, 1017) + .runTests() + .destroyHarness(); + + UnifiedTransformTestCase.of( + tableId, + "id + age as computed, *", + "id > 100", + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("computed", DataTypes.INT()) + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build()) + .initializeHarness() + .insertSource(1000, "Alice", 17) + .insertPreTransformed(1000, "Alice", 17) + .insertPostTransformed(1017, 1000, "Alice", 17) + .insertSource(2000, "Bob", 18) + .insertPreTransformed(2000, "Bob", 18) + .insertPostTransformed(2018, 2000, "Bob", 18) + .updateSource(new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePreTransformed( + new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePostTransformed( + new Object[] {2018, 2000, "Bob", 18}, + new Object[] {2016, 2000, "Barcarolle", 16}) + // filtered out data row + .insertSource(50, "Carol", 19) + .insertPreTransformed(50, "Carol", 19) + .insertPostTransformed() + .deleteSource(1000, "Alice", 17) + .deletePreTransformed(1000, "Alice", 17) + .deletePostTransformed(1017, 1000, "Alice", 17) + .runTests() + .destroyHarness(); + } + + @Test + public void testMetadataTransform() throws Exception { + TableId tableId = TableId.tableId("my_company", "my_branch", "metadata"); + UnifiedTransformTestCase.of( + tableId, + "*, __namespace_name__, __schema_name__, __table_name__", + "id > 100", + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("__namespace_name__", DataTypes.STRING().notNull()) + .physicalColumn("__schema_name__", DataTypes.STRING().notNull()) + .physicalColumn("__table_name__", DataTypes.STRING().notNull()) + .primaryKey("id") + .build()) + .initializeHarness() + .insertSource(1000, "Alice", 17) + .insertPreTransformed(1000, "Alice", 17) + .insertPostTransformed(1000, "Alice", 17, "my_company", "my_branch", "metadata") + .insertSource(2000, "Bob", 18) + .insertPreTransformed(2000, "Bob", 18) + .insertPostTransformed(2000, "Bob", 18, "my_company", "my_branch", "metadata") + .updateSource(new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePreTransformed( + new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePostTransformed( + new Object[] {2000, "Bob", 18, "my_company", "my_branch", "metadata"}, + new Object[] { + 2000, "Barcarolle", 16, "my_company", "my_branch", "metadata" + }) + // filtered out data row + .insertSource(50, "Carol", 19) + .insertPreTransformed(50, "Carol", 19) + .insertPostTransformed() + .deleteSource(1000, "Alice", 17) + .deletePreTransformed(1000, "Alice", 17) + .deletePostTransformed(1000, "Alice", 17, "my_company", "my_branch", "metadata") + .runTests() + .destroyHarness(); + } + + @Test + public void testCalculatedMetadataTransform() throws Exception { + TableId tableId = TableId.tableId("my_company", "my_branch", "metadata_transform"); + UnifiedTransformTestCase.of( + tableId, + "*, __namespace_name__ || '.' || __schema_name__ || '.' || __table_name__ AS identifier_name", + "id > 100", + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("identifier_name", DataTypes.STRING()) + .primaryKey("id") + .build()) + .initializeHarness() + .insertSource(1000, "Alice", 17) + .insertPreTransformed(1000, "Alice", 17) + .insertPostTransformed(1000, "Alice", 17, "my_company.my_branch.metadata_transform") + .insertSource(2000, "Bob", 18) + .insertPreTransformed(2000, "Bob", 18) + .insertPostTransformed(2000, "Bob", 18, "my_company.my_branch.metadata_transform") + .updateSource(new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePreTransformed( + new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePostTransformed( + new Object[] {2000, "Bob", 18, "my_company.my_branch.metadata_transform"}, + new Object[] { + 2000, "Barcarolle", 16, "my_company.my_branch.metadata_transform" + }) + // filtered out data row + .insertSource(50, "Carol", 19) + .insertPreTransformed(50, "Carol", 19) + .insertPostTransformed() + .deleteSource(1000, "Alice", 17) + .deletePreTransformed(1000, "Alice", 17) + .deletePostTransformed(1000, "Alice", 17, "my_company.my_branch.metadata_transform") + .runTests() + .destroyHarness(); + + UnifiedTransformTestCase.of( + tableId, + "__namespace_name__ || '.' || __schema_name__ || '.' || __table_name__ AS identifier_name, *", + "id > 100", + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("identifier_name", DataTypes.STRING()) + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build()) + .initializeHarness() + .insertSource(1000, "Alice", 17) + .insertPreTransformed(1000, "Alice", 17) + .insertPostTransformed("my_company.my_branch.metadata_transform", 1000, "Alice", 17) + .insertSource(2000, "Bob", 18) + .insertPreTransformed(2000, "Bob", 18) + .insertPostTransformed("my_company.my_branch.metadata_transform", 2000, "Bob", 18) + .updateSource(new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePreTransformed( + new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePostTransformed( + new Object[] {"my_company.my_branch.metadata_transform", 2000, "Bob", 18}, + new Object[] { + "my_company.my_branch.metadata_transform", 2000, "Barcarolle", 16 + }) + // filtered out data row + .insertSource(50, "Carol", 19) + .insertPreTransformed(50, "Carol", 19) + .insertPostTransformed() + .deleteSource(1000, "Alice", 17) + .deletePreTransformed(1000, "Alice", 17) + .deletePostTransformed("my_company.my_branch.metadata_transform", 1000, "Alice", 17) + .runTests() + .destroyHarness(); + } + + @Test + public void testMetadataAndCalculatedTransform() throws Exception { + TableId tableId = TableId.tableId("my_company", "my_branch", "metadata_transform"); + UnifiedTransformTestCase.of( + tableId, + "*, __namespace_name__ || '.' || __schema_name__ || '.' || __table_name__ AS identifier_name, __namespace_name__, __schema_name__, __table_name__", + "id > 100", + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .physicalColumn("identifier_name", DataTypes.STRING()) + .physicalColumn("__namespace_name__", DataTypes.STRING().notNull()) + .physicalColumn("__schema_name__", DataTypes.STRING().notNull()) + .physicalColumn("__table_name__", DataTypes.STRING().notNull()) + .primaryKey("id") + .build()) + .initializeHarness() + .insertSource(1000, "Alice", 17) + .insertPreTransformed(1000, "Alice", 17) + .insertPostTransformed( + 1000, + "Alice", + 17, + "my_company.my_branch.metadata_transform", + "my_company", + "my_branch", + "metadata_transform") + .insertSource(2000, "Bob", 18) + .insertPreTransformed(2000, "Bob", 18) + .insertPostTransformed( + 2000, + "Bob", + 18, + "my_company.my_branch.metadata_transform", + "my_company", + "my_branch", + "metadata_transform") + .updateSource(new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePreTransformed( + new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePostTransformed( + new Object[] { + 2000, + "Bob", + 18, + "my_company.my_branch.metadata_transform", + "my_company", + "my_branch", + "metadata_transform" + }, + new Object[] { + 2000, + "Barcarolle", + 16, + "my_company.my_branch.metadata_transform", + "my_company", + "my_branch", + "metadata_transform" + }) + // filtered out data row + .insertSource(50, "Carol", 19) + .insertPreTransformed(50, "Carol", 19) + .insertPostTransformed() + .deleteSource(1000, "Alice", 17) + .deletePreTransformed(1000, "Alice", 17) + .deletePostTransformed( + 1000, + "Alice", + 17, + "my_company.my_branch.metadata_transform", + "my_company", + "my_branch", + "metadata_transform") + .runTests() + .destroyHarness(); + + UnifiedTransformTestCase.of( + tableId, + "__namespace_name__ || '.' || __schema_name__ || '.' || __table_name__ AS identifier_name, __namespace_name__, __schema_name__, __table_name__, *", + "id > 100", + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("identifier_name", DataTypes.STRING()) + .physicalColumn("__namespace_name__", DataTypes.STRING().notNull()) + .physicalColumn("__schema_name__", DataTypes.STRING().notNull()) + .physicalColumn("__table_name__", DataTypes.STRING().notNull()) + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build()) + .initializeHarness() + .insertSource(1000, "Alice", 17) + .insertPreTransformed(1000, "Alice", 17) + .insertPostTransformed( + "my_company.my_branch.metadata_transform", + "my_company", + "my_branch", + "metadata_transform", + 1000, + "Alice", + 17) + .insertSource(2000, "Bob", 18) + .insertPreTransformed(2000, "Bob", 18) + .insertPostTransformed( + "my_company.my_branch.metadata_transform", + "my_company", + "my_branch", + "metadata_transform", + 2000, + "Bob", + 18) + .updateSource(new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePreTransformed( + new Object[] {2000, "Bob", 18}, new Object[] {2000, "Barcarolle", 16}) + .updatePostTransformed( + new Object[] { + "my_company.my_branch.metadata_transform", + "my_company", + "my_branch", + "metadata_transform", + 2000, + "Bob", + 18 + }, + new Object[] { + "my_company.my_branch.metadata_transform", + "my_company", + "my_branch", + "metadata_transform", + 2000, + "Barcarolle", + 16 + }) + // filtered out data row + .insertSource(50, "Carol", 19) + .insertPreTransformed(50, "Carol", 19) + .insertPostTransformed() + .deleteSource(1000, "Alice", 17) + .deletePreTransformed(1000, "Alice", 17) + .deletePostTransformed( + "my_company.my_branch.metadata_transform", + "my_company", + "my_branch", + "metadata_transform", + 1000, + "Alice", + 17) + .runTests() + .destroyHarness(); + } +} diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/parser/TransformParserTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/parser/TransformParserTest.java index 9dffeb84a17..d2e7aa74e05 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/parser/TransformParserTest.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/parser/TransformParserTest.java @@ -17,8 +17,11 @@ package org.apache.flink.cdc.runtime.parser; +import org.apache.flink.api.common.io.ParseException; +import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.runtime.operators.transform.ProjectionColumn; import org.apache.flink.cdc.runtime.parser.metadata.TransformSchemaFactory; import org.apache.flink.cdc.runtime.parser.metadata.TransformSqlOperatorTable; @@ -46,6 +49,7 @@ import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -266,6 +270,111 @@ public void testTranslateFilterToJaninoExpression() { "(valueEquals(id, 1) ? \"a\" : valueEquals(id, 2) ? \"b\" : \"c\")"); } + @Test + public void testGenerateProjectionColumns() { + List testColumns = + Arrays.asList( + Column.physicalColumn("id", DataTypes.INT(), "id"), + Column.physicalColumn("name", DataTypes.STRING(), "string"), + Column.physicalColumn("age", DataTypes.INT(), "age"), + Column.physicalColumn("address", DataTypes.STRING(), "address"), + Column.physicalColumn("weight", DataTypes.DOUBLE(), "weight"), + Column.physicalColumn("height", DataTypes.DOUBLE(), "height")); + + List result = + TransformParser.generateProjectionColumns( + "id, upper(name) as name, age + 1 as newage, weight / (height * height) as bmi", + testColumns); + + List expected = + Arrays.asList( + "ProjectionColumn{column=`id` INT, expression='null', scriptExpression='null', originalColumnNames=null, transformExpressionKey=null}", + "ProjectionColumn{column=`name` STRING, expression='UPPER(`TB`.`name`)', scriptExpression='upper(name)', originalColumnNames=[name], transformExpressionKey=null}", + "ProjectionColumn{column=`newage` INT, expression='`TB`.`age` + 1', scriptExpression='age + 1', originalColumnNames=[age], transformExpressionKey=null}", + "ProjectionColumn{column=`bmi` DOUBLE, expression='`TB`.`weight` / (`TB`.`height` * `TB`.`height`)', scriptExpression='weight / height * height', originalColumnNames=[weight, height, height], transformExpressionKey=null}"); + Assert.assertEquals(result.toString(), "[" + String.join(", ", expected) + "]"); + + List metadataResult = + TransformParser.generateProjectionColumns( + "*, __namespace_name__, __schema_name__, __table_name__", testColumns); + + List metadataExpected = + Arrays.asList( + "ProjectionColumn{column=`id` INT, expression='null', scriptExpression='null', originalColumnNames=null, transformExpressionKey=null}", + "ProjectionColumn{column=`name` STRING, expression='null', scriptExpression='null', originalColumnNames=null, transformExpressionKey=null}", + "ProjectionColumn{column=`age` INT, expression='null', scriptExpression='null', originalColumnNames=null, transformExpressionKey=null}", + "ProjectionColumn{column=`address` STRING, expression='null', scriptExpression='null', originalColumnNames=null, transformExpressionKey=null}", + "ProjectionColumn{column=`weight` DOUBLE, expression='null', scriptExpression='null', originalColumnNames=null, transformExpressionKey=null}", + "ProjectionColumn{column=`height` DOUBLE, expression='null', scriptExpression='null', originalColumnNames=null, transformExpressionKey=null}", + "ProjectionColumn{column=`__namespace_name__` STRING NOT NULL, expression='__namespace_name__', scriptExpression='__namespace_name__', originalColumnNames=[__namespace_name__], transformExpressionKey=null}", + "ProjectionColumn{column=`__schema_name__` STRING NOT NULL, expression='__schema_name__', scriptExpression='__schema_name__', originalColumnNames=[__schema_name__], transformExpressionKey=null}", + "ProjectionColumn{column=`__table_name__` STRING NOT NULL, expression='__table_name__', scriptExpression='__table_name__', originalColumnNames=[__table_name__], transformExpressionKey=null}"); + Assert.assertEquals( + metadataResult.toString(), "[" + String.join(", ", metadataExpected) + "]"); + + // calculated columns must use AS to provide an alias name + Assert.assertThrows( + ParseException.class, + () -> { + TransformParser.generateProjectionColumns("id, 1 + 1", testColumns); + }); + } + + @Test + public void testGenerateReferencedColumns() { + List testColumns = + Arrays.asList( + Column.physicalColumn("id", DataTypes.INT(), "id"), + Column.physicalColumn("name", DataTypes.STRING(), "string"), + Column.physicalColumn("age", DataTypes.INT(), "age"), + Column.physicalColumn("address", DataTypes.STRING(), "address"), + Column.physicalColumn("weight", DataTypes.DOUBLE(), "weight"), + Column.physicalColumn("height", DataTypes.DOUBLE(), "height"), + Column.physicalColumn("birthday", DataTypes.DATE(), "birthday")); + + List result = + TransformParser.generateReferencedColumns( + "id, upper(name) as name, age + 1 as newage, weight / (height * height) as bmi", + "bmi > 17 and char_length(address) > 10", + testColumns); + + List expected = + Arrays.asList( + "`id` INT 'id'", + "`name` STRING 'string'", + "`age` INT 'age'", + "`address` STRING 'address'", + "`weight` DOUBLE 'weight'", + "`height` DOUBLE 'height'"); + Assert.assertEquals(result.toString(), "[" + String.join(", ", expected) + "]"); + + // calculated columns must use AS to provide an alias name + Assert.assertThrows( + ParseException.class, + () -> { + TransformParser.generateReferencedColumns("id, 1 + 1", null, testColumns); + }); + } + + @Test + public void testNormalizeFilter() { + Assert.assertEquals( + TransformParser.normalizeFilter("a, b, c, d", "a > 0 and b > 0"), + "`a` > 0 AND `b` > 0"); + Assert.assertEquals(TransformParser.normalizeFilter("a, b, c, d", null), null); + Assert.assertEquals( + TransformParser.normalizeFilter( + "abs(a) as cal_a, char_length(b) as cal_b, c, d", + "a > 4 and cal_a > 8 and cal_b < 17 and c != d"), + "`a` > 4 AND ABS(`a`) > 8 AND CHAR_LENGTH(`b`) < 17 AND `c` <> `d`"); + + Assert.assertEquals( + TransformParser.normalizeFilter( + "x, y, z, 1 - x as u, 1 - y as v, 1 - z as w", + "concat(u, concat(v, concat(w, x), y), z) != 10"), + "`concat`(1 - `x`, `concat`(1 - `y`, `concat`(1 - `z`, `x`), `y`), `z`) <> 10"); + } + private void testFilterExpression(String expression, String expressionExpect) { String janinoExpression = TransformParser.translateFilterExpressionToJaninoExpression(expression);