From 81d916fc73fa2a3800202bc6eebe9d8274e4a258 Mon Sep 17 00:00:00 2001 From: yuxiqian <34335406+yuxiqian@users.noreply.github.com> Date: Thu, 8 Aug 2024 21:01:00 +0800 Subject: [PATCH] [FLINK-35272][cdc-runtime] Transform supports omitting and renaming computed column This closes #3285. --- .../connectors/pipeline-connectors/doris.md | 2 +- .../content.zh/docs/core-concept/transform.md | 4 +- .../connectors/pipeline-connectors/doris.md | 2 +- docs/content/docs/core-concept/transform.md | 4 +- .../flink/cdc/common/utils/SchemaUtils.java | 186 ++++ .../cdc/common/utils/SchemaUtilsTest.java | 395 +++++-- .../composer/flink/FlinkPipelineComposer.java | 8 +- .../flink/translator/TransformTranslator.java | 40 +- .../flink/FlinkPipelineComposerITCase.java | 201 +++- .../flink/FlinkPipelineTransformITCase.java | 6 + .../pipeline/tests/MySqlToDorisE2eITCase.java | 107 ++ .../cdc/pipeline/tests/RouteE2eITCase.java | 91 +- .../pipeline/tests/TransformE2eITCase.java | 588 +++++++++-- .../test/resources/ddl/data_types_test.sql | 91 ++ .../src/test/resources/ddl/transform_test.sql | 20 +- .../tests/TableChangeInfoMigrationMock.java | 19 +- .../operators/schema/SchemaOperator.java | 3 +- .../transform/PostTransformChangeInfo.java | 117 +++ ...erator.java => PostTransformOperator.java} | 285 +++-- .../operators/transform/PostTransformer.java | 53 + ...eInfo.java => PreTransformChangeInfo.java} | 89 +- ...perator.java => PreTransformOperator.java} | 188 ++-- .../transform/PreTransformProcessor.java | 138 +++ .../operators/transform/PreTransformer.java | 53 + .../operators/transform/ProjectionColumn.java | 27 + .../transform/ProjectionColumnProcessor.java | 47 +- .../operators/transform/TableInfo.java | 90 -- .../transform/TransformFilterProcessor.java | 118 ++- .../TransformProjectionProcessor.java | 165 ++- .../operators/transform/TransformRule.java | 77 ++ .../cdc/runtime/parser/TransformParser.java | 219 +++- .../partitioning/PrePartitionOperator.java | 3 +- ...st.java => PostTransformOperatorTest.java} | 358 ++++++- .../transform/PreTransformOperatorTest.java | 520 +++++++++ .../TransformSchemaOperatorTest.java | 228 ---- .../UnifiedTransformOperatorTest.java | 985 ++++++++++++++++++ .../runtime/parser/TransformParserTest.java | 167 ++- 37 files changed, 4624 insertions(+), 1070 deletions(-) create mode 100644 flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/data_types_test.sql create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformChangeInfo.java rename flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/{TransformDataOperator.java => PostTransformOperator.java} (57%) create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformer.java rename flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/{TableChangeInfo.java => PreTransformChangeInfo.java} (63%) rename flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/{TransformSchemaOperator.java => PreTransformOperator.java} (55%) 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/PreTransformer.java delete mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TableInfo.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} (83%) 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/docs/content.zh/docs/connectors/pipeline-connectors/doris.md b/docs/content.zh/docs/connectors/pipeline-connectors/doris.md index bdbb5fe2c..9bb3ee975 100644 --- a/docs/content.zh/docs/connectors/pipeline-connectors/doris.md +++ b/docs/content.zh/docs/connectors/pipeline-connectors/doris.md @@ -186,7 +186,7 @@ pipeline: CDC type - Doris type + Doris type NOTE diff --git a/docs/content.zh/docs/core-concept/transform.md b/docs/content.zh/docs/core-concept/transform.md index a34db3387..73db8196a 100644 --- a/docs/content.zh/docs/core-concept/transform.md +++ b/docs/content.zh/docs/core-concept/transform.md @@ -252,7 +252,9 @@ transform: Tips: The format of table-options is `key1=value1,key2=value2`. ## Classification mapping -Multiple transform rules can be defined to classify input data rows and apply different processings. For example, we may define a transform rule as follows: +Multiple transform rules can be defined to classify input data rows and apply different processing. +Only the first matched transform rule will apply. +For example, we may define a transform rule as follows: ```yaml transform: diff --git a/docs/content/docs/connectors/pipeline-connectors/doris.md b/docs/content/docs/connectors/pipeline-connectors/doris.md index b5048eaf9..6b0c233d4 100644 --- a/docs/content/docs/connectors/pipeline-connectors/doris.md +++ b/docs/content/docs/connectors/pipeline-connectors/doris.md @@ -186,7 +186,7 @@ pipeline: Flink CDC Type - Doris Type + Doris Type Note diff --git a/docs/content/docs/core-concept/transform.md b/docs/content/docs/core-concept/transform.md index a34db3387..73db8196a 100644 --- a/docs/content/docs/core-concept/transform.md +++ b/docs/content/docs/core-concept/transform.md @@ -252,7 +252,9 @@ transform: Tips: The format of table-options is `key1=value1,key2=value2`. ## Classification mapping -Multiple transform rules can be defined to classify input data rows and apply different processings. For example, we may define a transform rule as follows: +Multiple transform rules can be defined to classify input data rows and apply different processing. +Only the first matched transform rule will apply. +For example, we may define a transform rule as follows: ```yaml transform: 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 107e63d82..68cdcd0e1 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.RenameColumnEvent; 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,181 @@ public class SchemaUtils { 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 inferWiderSchema(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 = inferWiderSchema(outputSchema, schema); + } + return outputSchema; + } + } + + /** Try to combine two schemas with potential incompatible type. */ + @VisibleForTesting + public static Schema inferWiderSchema(@Nullable Schema lSchema, Schema rSchema) { + if (lSchema == null) { + return rSchema; + } + if (lSchema.getColumnCount() != rSchema.getColumnCount()) { + throw new IllegalStateException( + String.format( + "Unable to merge schema %s and %s with different column counts.", + lSchema, rSchema)); + } + if (!lSchema.primaryKeys().equals(rSchema.primaryKeys())) { + throw new IllegalStateException( + String.format( + "Unable to merge schema %s and %s with different primary keys.", + lSchema, rSchema)); + } + if (!lSchema.partitionKeys().equals(rSchema.partitionKeys())) { + throw new IllegalStateException( + String.format( + "Unable to merge schema %s and %s with different partition keys.", + lSchema, rSchema)); + } + if (!lSchema.options().equals(rSchema.options())) { + throw new IllegalStateException( + String.format( + "Unable to merge schema %s and %s with different options.", + lSchema, rSchema)); + } + if (!Objects.equals(lSchema.comment(), rSchema.comment())) { + throw new IllegalStateException( + String.format( + "Unable to merge schema %s and %s with different comments.", + lSchema, rSchema)); + } + + List leftColumns = lSchema.getColumns(); + List rightColumns = rSchema.getColumns(); + + List mergedColumns = + IntStream.range(0, lSchema.getColumnCount()) + .mapToObj(i -> inferWiderColumn(leftColumns.get(i), rightColumns.get(i))) + .collect(Collectors.toList()); + + return lSchema.copy(mergedColumns); + } + + /** Try to combine two columns with potential incompatible type. */ + @VisibleForTesting + public static Column inferWiderColumn(Column lColumn, Column rColumn) { + if (!Objects.equals(lColumn.getName(), rColumn.getName())) { + throw new IllegalStateException( + String.format( + "Unable to merge column %s and %s with different name.", + lColumn, rColumn)); + } + if (!Objects.equals(lColumn.getComment(), rColumn.getComment())) { + throw new IllegalStateException( + String.format( + "Unable to merge column %s and %s with different comments.", + lColumn, rColumn)); + } + return lColumn.copy(inferWiderType(lColumn.getType(), rColumn.getType())); + } + + /** Try to combine given data types to a compatible wider data type. */ + @VisibleForTesting + public static DataType inferWiderType(DataType lType, DataType rType) { + // Ignore nullability during data type merge + boolean nullable = lType.isNullable() || rType.isNullable(); + lType = lType.notNull(); + rType = rType.notNull(); + + DataType mergedType; + if (lType.equals(rType)) { + // identical type + mergedType = rType; + } else if (lType.is(DataTypeFamily.INTEGER_NUMERIC) + && rType.is(DataTypeFamily.INTEGER_NUMERIC)) { + mergedType = DataTypes.BIGINT(); + } else if (lType.is(DataTypeFamily.CHARACTER_STRING) + && rType.is(DataTypeFamily.CHARACTER_STRING)) { + mergedType = DataTypes.STRING(); + } else if (lType.is(DataTypeFamily.APPROXIMATE_NUMERIC) + && rType.is(DataTypeFamily.APPROXIMATE_NUMERIC)) { + mergedType = DataTypes.DOUBLE(); + } else if (lType.is(DataTypeRoot.DECIMAL) && rType.is(DataTypeRoot.DECIMAL)) { + // Merge two decimal types + DecimalType lhsDecimal = (DecimalType) lType; + DecimalType rhsDecimal = (DecimalType) rType; + 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 (lType.is(DataTypeRoot.DECIMAL) && rType.is(DataTypeFamily.EXACT_NUMERIC)) { + // Merge decimal and int + DecimalType lhsDecimal = (DecimalType) lType; + mergedType = + DataTypes.DECIMAL( + Math.max( + lhsDecimal.getPrecision(), + lhsDecimal.getScale() + getNumericPrecision(rType)), + lhsDecimal.getScale()); + } else if (rType.is(DataTypeRoot.DECIMAL) && lType.is(DataTypeFamily.EXACT_NUMERIC)) { + // Merge decimal and int + DecimalType rhsDecimal = (DecimalType) rType; + mergedType = + DataTypes.DECIMAL( + Math.max( + rhsDecimal.getPrecision(), + rhsDecimal.getScale() + getNumericPrecision(lType)), + rhsDecimal.getScale()); + } else { + throw new IllegalStateException( + String.format("Incompatible types: \"%s\" and \"%s\"", lType, rType)); + } + + 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 83fb358b2..8a508a890 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 @@ -27,8 +27,8 @@ import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.types.DataType; import org.apache.flink.cdc.common.types.DataTypes; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; @@ -55,13 +55,13 @@ public class SchemaUtilsTest { Column.physicalColumn("col3", DataTypes.STRING()))); AddColumnEvent addColumnEvent = new AddColumnEvent(tableId, addedColumns); schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col3", DataTypes.STRING()) - .build()); + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .physicalColumn("col3", DataTypes.STRING()) + .build()); // add new column before existed column addedColumns = new ArrayList<>(); @@ -72,14 +72,14 @@ public class SchemaUtilsTest { "col3")); addColumnEvent = new AddColumnEvent(tableId, addedColumns); schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col4", DataTypes.STRING()) - .physicalColumn("col3", DataTypes.STRING()) - .build()); + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .physicalColumn("col4", DataTypes.STRING()) + .physicalColumn("col3", DataTypes.STRING()) + .build()); // add new column after existed column addedColumns = new ArrayList<>(); @@ -90,15 +90,15 @@ public class SchemaUtilsTest { "col4")); addColumnEvent = new AddColumnEvent(tableId, addedColumns); schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col4", DataTypes.STRING()) - .physicalColumn("col5", DataTypes.STRING()) - .physicalColumn("col3", DataTypes.STRING()) - .build()); + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .physicalColumn("col4", DataTypes.STRING()) + .physicalColumn("col5", DataTypes.STRING()) + .physicalColumn("col3", DataTypes.STRING()) + .build()); // add column in first position addedColumns = new ArrayList<>(); @@ -109,29 +109,29 @@ public class SchemaUtilsTest { null)); addColumnEvent = new AddColumnEvent(tableId, addedColumns); schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col0", DataTypes.STRING()) - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col4", DataTypes.STRING()) - .physicalColumn("col5", DataTypes.STRING()) - .physicalColumn("col3", DataTypes.STRING()) - .build()); + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col0", DataTypes.STRING()) + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .physicalColumn("col4", DataTypes.STRING()) + .physicalColumn("col5", DataTypes.STRING()) + .physicalColumn("col3", DataTypes.STRING()) + .build()); // drop columns DropColumnEvent dropColumnEvent = new DropColumnEvent(tableId, Arrays.asList("col3", "col5")); schema = SchemaUtils.applySchemaChangeEvent(schema, dropColumnEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col0", DataTypes.STRING()) - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col4", DataTypes.STRING()) - .build()); + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col0", DataTypes.STRING()) + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .physicalColumn("col4", DataTypes.STRING()) + .build()); // rename columns Map nameMapping = new HashMap<>(); @@ -139,14 +139,14 @@ public class SchemaUtilsTest { nameMapping.put("col4", "newCol4"); RenameColumnEvent renameColumnEvent = new RenameColumnEvent(tableId, nameMapping); schema = SchemaUtils.applySchemaChangeEvent(schema, renameColumnEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col0", DataTypes.STRING()) - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("newCol2", DataTypes.STRING()) - .physicalColumn("newCol4", DataTypes.STRING()) - .build()); + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col0", DataTypes.STRING()) + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("newCol2", DataTypes.STRING()) + .physicalColumn("newCol4", DataTypes.STRING()) + .build()); // alter column types Map typeMapping = new HashMap<>(); @@ -154,13 +154,288 @@ public class SchemaUtilsTest { typeMapping.put("newCol4", DataTypes.VARCHAR(10)); AlterColumnTypeEvent alterColumnTypeEvent = new AlterColumnTypeEvent(tableId, typeMapping); schema = SchemaUtils.applySchemaChangeEvent(schema, alterColumnTypeEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col0", DataTypes.STRING()) - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("newCol2", DataTypes.VARCHAR(10)) - .physicalColumn("newCol4", DataTypes.VARCHAR(10)) - .build()); + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col0", DataTypes.STRING()) + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("newCol2", DataTypes.VARCHAR(10)) + .physicalColumn("newCol4", DataTypes.VARCHAR(10)) + .build()); + } + + @Test + public void testGetNumericPrecision() { + Assertions.assertThat(SchemaUtils.getNumericPrecision(DataTypes.TINYINT())).isEqualTo(3); + Assertions.assertThat(SchemaUtils.getNumericPrecision(DataTypes.SMALLINT())).isEqualTo(5); + Assertions.assertThat(SchemaUtils.getNumericPrecision(DataTypes.INT())).isEqualTo(10); + Assertions.assertThat(SchemaUtils.getNumericPrecision(DataTypes.BIGINT())).isEqualTo(19); + Assertions.assertThat(SchemaUtils.getNumericPrecision(DataTypes.DECIMAL(10, 2))) + .isEqualTo(10); + Assertions.assertThat(SchemaUtils.getNumericPrecision(DataTypes.DECIMAL(17, 0))) + .isEqualTo(17); + Assertions.assertThatThrownBy(() -> SchemaUtils.getNumericPrecision(DataTypes.STRING())) + .isExactlyInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Failed to get precision of non-exact decimal type"); + } + + @Test + public void testInferWiderType() { + Assertions.assertThat( + SchemaUtils.inferWiderType(DataTypes.BINARY(17), DataTypes.BINARY(17))) + .isEqualTo(DataTypes.BINARY(17)); + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.VARBINARY(17), DataTypes.VARBINARY(17))) + .isEqualTo(DataTypes.VARBINARY(17)); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.BYTES(), DataTypes.BYTES())) + .isEqualTo(DataTypes.BYTES()); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.BOOLEAN(), DataTypes.BOOLEAN())) + .isEqualTo(DataTypes.BOOLEAN()); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.INT(), DataTypes.INT())) + .isEqualTo(DataTypes.INT()); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.TINYINT(), DataTypes.TINYINT())) + .isEqualTo(DataTypes.TINYINT()); + Assertions.assertThat( + SchemaUtils.inferWiderType(DataTypes.SMALLINT(), DataTypes.SMALLINT())) + .isEqualTo(DataTypes.SMALLINT()); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.BIGINT(), DataTypes.BIGINT())) + .isEqualTo(DataTypes.BIGINT()); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.FLOAT(), DataTypes.FLOAT())) + .isEqualTo(DataTypes.FLOAT()); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.DOUBLE(), DataTypes.DOUBLE())) + .isEqualTo(DataTypes.DOUBLE()); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.CHAR(17), DataTypes.CHAR(17))) + .isEqualTo(DataTypes.CHAR(17)); + Assertions.assertThat( + SchemaUtils.inferWiderType(DataTypes.VARCHAR(17), DataTypes.VARCHAR(17))) + .isEqualTo(DataTypes.VARCHAR(17)); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.STRING(), DataTypes.STRING())) + .isEqualTo(DataTypes.STRING()); + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.DECIMAL(17, 7), DataTypes.DECIMAL(17, 7))) + .isEqualTo(DataTypes.DECIMAL(17, 7)); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.DATE(), DataTypes.DATE())) + .isEqualTo(DataTypes.DATE()); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.TIME(), DataTypes.TIME())) + .isEqualTo(DataTypes.TIME()); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.TIME(6), DataTypes.TIME(6))) + .isEqualTo(DataTypes.TIME(6)); + Assertions.assertThat( + SchemaUtils.inferWiderType(DataTypes.TIMESTAMP(), DataTypes.TIMESTAMP())) + .isEqualTo(DataTypes.TIMESTAMP()); + Assertions.assertThat( + SchemaUtils.inferWiderType(DataTypes.TIMESTAMP(3), DataTypes.TIMESTAMP(3))) + .isEqualTo(DataTypes.TIMESTAMP(3)); + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.TIMESTAMP_TZ(), DataTypes.TIMESTAMP_TZ())) + .isEqualTo(DataTypes.TIMESTAMP_TZ()); + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.TIMESTAMP_TZ(3), DataTypes.TIMESTAMP_TZ(3))) + .isEqualTo(DataTypes.TIMESTAMP_TZ(3)); + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.TIMESTAMP_LTZ(), DataTypes.TIMESTAMP_LTZ())) + .isEqualTo(DataTypes.TIMESTAMP_LTZ()); + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.TIMESTAMP_LTZ(3), DataTypes.TIMESTAMP_LTZ(3))) + .isEqualTo(DataTypes.TIMESTAMP_LTZ(3)); + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.ARRAY(DataTypes.INT()), DataTypes.ARRAY(DataTypes.INT()))) + .isEqualTo(DataTypes.ARRAY(DataTypes.INT())); + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.MAP(DataTypes.INT(), DataTypes.STRING()), + DataTypes.MAP(DataTypes.INT(), DataTypes.STRING()))) + .isEqualTo(DataTypes.MAP(DataTypes.INT(), DataTypes.STRING())); + + // Test compatible widening cast + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.INT(), DataTypes.BIGINT())) + .isEqualTo(DataTypes.BIGINT()); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.VARCHAR(17), DataTypes.STRING())) + .isEqualTo(DataTypes.STRING()); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.FLOAT(), DataTypes.DOUBLE())) + .isEqualTo(DataTypes.DOUBLE()); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.INT(), DataTypes.DECIMAL(4, 0))) + .isEqualTo(DataTypes.DECIMAL(10, 0)); + Assertions.assertThat(SchemaUtils.inferWiderType(DataTypes.INT(), DataTypes.DECIMAL(10, 5))) + .isEqualTo(DataTypes.DECIMAL(15, 5)); + Assertions.assertThat( + SchemaUtils.inferWiderType(DataTypes.BIGINT(), DataTypes.DECIMAL(10, 5))) + .isEqualTo(DataTypes.DECIMAL(24, 5)); + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.DECIMAL(5, 4), DataTypes.DECIMAL(10, 2))) + .isEqualTo(DataTypes.DECIMAL(12, 4)); + + // Test merging with nullability + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.INT().notNull(), DataTypes.INT().notNull())) + .isEqualTo(DataTypes.INT().notNull()); + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.INT().nullable(), DataTypes.INT().notNull())) + .isEqualTo(DataTypes.INT().nullable()); + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.INT().notNull(), DataTypes.INT().nullable())) + .isEqualTo(DataTypes.INT().nullable()); + Assertions.assertThat( + SchemaUtils.inferWiderType( + DataTypes.INT().nullable(), DataTypes.INT().nullable())) + .isEqualTo(DataTypes.INT().nullable()); + + // incompatible type merges test + Assertions.assertThatThrownBy( + () -> SchemaUtils.inferWiderType(DataTypes.INT(), DataTypes.DOUBLE())) + .isExactlyInstanceOf(IllegalStateException.class); + + Assertions.assertThatThrownBy( + () -> + SchemaUtils.inferWiderType( + DataTypes.DECIMAL(17, 0), DataTypes.DOUBLE())) + .isExactlyInstanceOf(IllegalStateException.class); + Assertions.assertThatThrownBy( + () -> SchemaUtils.inferWiderType(DataTypes.INT(), DataTypes.STRING())) + .isExactlyInstanceOf(IllegalStateException.class); + } + + @Test + public void testInferWiderColumn() { + // Test normal merges + Assertions.assertThat( + SchemaUtils.inferWiderColumn( + Column.physicalColumn("Column1", DataTypes.INT()), + Column.physicalColumn("Column1", DataTypes.BIGINT()))) + .isEqualTo(Column.physicalColumn("Column1", DataTypes.BIGINT())); + + Assertions.assertThat( + SchemaUtils.inferWiderColumn( + Column.physicalColumn("Column2", DataTypes.FLOAT()), + Column.physicalColumn("Column2", DataTypes.DOUBLE()))) + .isEqualTo(Column.physicalColumn("Column2", DataTypes.DOUBLE())); + + // Test merging columns with incompatible types + Assertions.assertThatThrownBy( + () -> + SchemaUtils.inferWiderColumn( + Column.physicalColumn("Column3", DataTypes.INT()), + Column.physicalColumn("Column3", DataTypes.STRING()))) + .isExactlyInstanceOf(IllegalStateException.class); + + // Test merging with incompatible names + Assertions.assertThatThrownBy( + () -> + SchemaUtils.inferWiderColumn( + Column.physicalColumn("Column4", DataTypes.INT()), + Column.physicalColumn("AnotherColumn4", DataTypes.INT()))) + .isExactlyInstanceOf(IllegalStateException.class); + } + + @Test + public void testInferWiderSchema() { + // Test normal merges + Assertions.assertThat( + SchemaUtils.inferWiderSchema( + 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())) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("Column1", DataTypes.BIGINT()) + .physicalColumn("Column2", DataTypes.DOUBLE()) + .primaryKey("Column1") + .partitionKey("Column2") + .build()); + + // Test merging with incompatible types + Assertions.assertThatThrownBy( + () -> + SchemaUtils.inferWiderSchema( + 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())) + .isExactlyInstanceOf(IllegalStateException.class); + + // Test merging with incompatible column names + Assertions.assertThatThrownBy( + () -> + SchemaUtils.inferWiderSchema( + 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())) + .isExactlyInstanceOf(IllegalStateException.class); + + // Test merging with different column counts + Assertions.assertThatThrownBy( + () -> + SchemaUtils.inferWiderSchema( + 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())) + .isExactlyInstanceOf(IllegalStateException.class); + + // Test merging with incompatible schema metadata + Assertions.assertThatThrownBy( + () -> + SchemaUtils.inferWiderSchema( + 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())) + .isExactlyInstanceOf(IllegalStateException.class); } } 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 a6f5e65a8..f00941b12 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 @@ -104,9 +104,9 @@ public class FlinkPipelineComposer implements PipelineComposer { 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 = @@ -119,9 +119,9 @@ public class FlinkPipelineComposer implements PipelineComposer { 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 53400f628..6feec31d9 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 @@ package org.apache.flink.cdc.composer.flink.translator; 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 org.apache.flink.streaming.api.datastream.DataStream; 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 class TransformTranslator { 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-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposerITCase.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposerITCase.java index bf3e9ab46..39c55a6f8 100644 --- a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposerITCase.java +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposerITCase.java @@ -360,7 +360,7 @@ class FlinkPipelineComposerITCase { new TransformDef( "default_namespace.default_schema.table1", "*,concat(col1,'1') as col12", - "col1 = '1'", + "col1 = '1' OR col1 = '999'", "col1", "col12", "key1=value1", @@ -728,6 +728,205 @@ class FlinkPipelineComposerITCase { "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[6, Frank, 30, student, null, male], op=INSERT, meta=()}"); } + @Test + void testTransformMergingWithRoute() throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.CUSTOM_SOURCE_EVENTS); + + TableId myTable1 = TableId.tableId("default_namespace", "default_schema", "mytable1"); + TableId myTable2 = TableId.tableId("default_namespace", "default_schema", "mytable2"); + Schema table1Schema = + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.INT()) + .primaryKey("id") + .build(); + Schema table2Schema = + Schema.newBuilder() + .physicalColumn("id", DataTypes.BIGINT()) + .physicalColumn("name", DataTypes.VARCHAR(255)) + .physicalColumn("age", DataTypes.TINYINT()) + .physicalColumn("description", DataTypes.STRING()) + .primaryKey("id") + .build(); + + // Create test dataset: + // Create table 1 [id, name, age] + // Table 1: +I[1, Alice, 18] + // Table 1: +I[2, Bob, 20] + // Table 1: -U[2, Bob, 20] +U[2, Bob, 30] + // Create table 2 [id, name, age, description] + // Table 2: +I[3, Charlie, 15, student] + // Table 2: +I[4, Donald, 25, student] + // Table 2: -D[4, Donald, 25, student] + // Rename column for table 1: name -> last_name + // Add column for table 2: gender + // Table 1: +I[5, Eliza, 24] + // Table 2: +I[6, Frank, 30, student, male] + List events = new ArrayList<>(); + BinaryRecordDataGenerator table1dataGenerator = + new BinaryRecordDataGenerator( + table1Schema.getColumnDataTypes().toArray(new DataType[0])); + BinaryRecordDataGenerator table2dataGenerator = + new BinaryRecordDataGenerator( + table2Schema.getColumnDataTypes().toArray(new DataType[0])); + events.add(new CreateTableEvent(myTable1, table1Schema)); + events.add( + DataChangeEvent.insertEvent( + myTable1, + table1dataGenerator.generate( + new Object[] {1, BinaryStringData.fromString("Alice"), 18}))); + events.add( + DataChangeEvent.insertEvent( + myTable1, + table1dataGenerator.generate( + new Object[] {2, BinaryStringData.fromString("Bob"), 20}))); + events.add( + DataChangeEvent.updateEvent( + myTable1, + table1dataGenerator.generate( + new Object[] {2, BinaryStringData.fromString("Bob"), 20}), + table1dataGenerator.generate( + new Object[] {2, BinaryStringData.fromString("Bob"), 30}))); + events.add(new CreateTableEvent(myTable2, table2Schema)); + events.add( + DataChangeEvent.insertEvent( + myTable2, + table2dataGenerator.generate( + new Object[] { + 3L, + BinaryStringData.fromString("Charlie"), + (byte) 15, + BinaryStringData.fromString("student") + }))); + events.add( + DataChangeEvent.insertEvent( + myTable2, + table2dataGenerator.generate( + new Object[] { + 4L, + BinaryStringData.fromString("Donald"), + (byte) 25, + BinaryStringData.fromString("student") + }))); + events.add( + DataChangeEvent.deleteEvent( + myTable2, + table2dataGenerator.generate( + new Object[] { + 4L, + BinaryStringData.fromString("Donald"), + (byte) 25, + BinaryStringData.fromString("student") + }))); + // events.add(new RenameColumnEvent(myTable1, ImmutableMap.of("name", "last_name"))); + events.add( + new AddColumnEvent( + myTable2, + Collections.singletonList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("gender", DataTypes.STRING()))))); + events.add( + DataChangeEvent.insertEvent( + myTable1, + table1dataGenerator.generate( + new Object[] {5, BinaryStringData.fromString("Eliza"), 24}))); + events.add( + DataChangeEvent.insertEvent( + myTable2, + new BinaryRecordDataGenerator( + new DataType[] { + DataTypes.BIGINT(), + DataTypes.VARCHAR(255), + DataTypes.TINYINT(), + DataTypes.STRING(), + DataTypes.STRING() + }) + .generate( + new Object[] { + 6L, + BinaryStringData.fromString("Frank"), + (byte) 30, + BinaryStringData.fromString("student"), + BinaryStringData.fromString("male") + }))); + + ValuesDataSourceHelper.setSourceEvents(Collections.singletonList(events)); + + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + // Setup value sink + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + // Setup transform + List transformDef = + Collections.singletonList( + new TransformDef( + "default_namespace.default_schema.mytable[0-9]", + "*,'last_name' as last_name", + null, + null, + null, + null, + "")); + + // Setup route + TableId mergedTable = TableId.tableId("default_namespace", "default_schema", "merged"); + List routeDef = + Collections.singletonList( + new RouteDef( + "default_namespace.default_schema.mytable[0-9]", + mergedTable.toString(), + null, + null)); + + // Setup pipeline + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef(sourceDef, sinkDef, routeDef, transformDef, pipelineConfig); + + // Execute the pipeline + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + Schema mergedTableSchema = ValuesDatabase.getTableSchema(mergedTable); + assertThat(mergedTableSchema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("id", DataTypes.BIGINT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("age", DataTypes.BIGINT()) + .physicalColumn("last_name", DataTypes.STRING()) + .physicalColumn("description", DataTypes.STRING()) + .physicalColumn("gender", DataTypes.STRING()) + .primaryKey("id") + .build()); + String[] outputEvents = outCaptor.toString().trim().split("\n"); + assertThat(outputEvents) + .containsExactly( + "CreateTableEvent{tableId=default_namespace.default_schema.merged, schema=columns={`id` INT,`name` STRING,`age` INT,`last_name` STRING}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[1, Alice, 18, last_name], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[2, Bob, 20, last_name], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[2, Bob, 20, last_name], after=[2, Bob, 30, last_name], op=UPDATE, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.merged, addedColumns=[ColumnWithPosition{column=`description` STRING, position=LAST, existedColumnName=null}]}", + "AlterColumnTypeEvent{tableId=default_namespace.default_schema.merged, nameMapping={age=BIGINT, id=BIGINT}}", + "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[3, Charlie, 15, last_name, student], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[4, Donald, 25, last_name, student], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[4, Donald, 25, last_name, student], after=[], op=DELETE, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.merged, addedColumns=[ColumnWithPosition{column=`gender` STRING, position=LAST, existedColumnName=null}]}", + "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[5, Eliza, 24, last_name, null, null], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[6, Frank, 30, male, student, last_name], op=INSERT, meta=()}"); + } + @ParameterizedTest @EnumSource void testRouteWithReplaceSymbol(ValuesDataSink.SinkApi sinkApi) throws Exception { diff --git a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineTransformITCase.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineTransformITCase.java index 77f89a37a..b50dbae47 100644 --- a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineTransformITCase.java +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineTransformITCase.java @@ -262,6 +262,12 @@ class FlinkPipelineTransformITCase { String localTimestamp = tokens.get(4); assertThat(currentTimestamp).isEqualTo(nowTimestamp).isEqualTo(localTimestamp); + // If timestamp millisecond part is .000, it will be truncated to yyyy-MM-dd'T'HH:mm:ss + // format. Manually append this for the following checks. + if (currentTimestamp.length() == 19) { + currentTimestamp += ".000"; + } + Instant instant = LocalDateTime.parse( currentTimestamp, diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToDorisE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToDorisE2eITCase.java index 39997cdb6..0e58c5ed2 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToDorisE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToDorisE2eITCase.java @@ -48,6 +48,7 @@ import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -93,6 +94,9 @@ public class MySqlToDorisE2eITCase extends PipelineTestEnvironment { protected final UniqueDatabase mysqlInventoryDatabase = new UniqueDatabase(MYSQL, "mysql_inventory", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + protected final UniqueDatabase complexDataTypesDatabase = + new UniqueDatabase(MYSQL, "data_types_test", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + @BeforeClass public static void initializeContainers() { LOG.info("Starting containers..."); @@ -128,6 +132,9 @@ public class MySqlToDorisE2eITCase extends PipelineTestEnvironment { super.before(); mysqlInventoryDatabase.createAndInitialize(); createDorisDatabase(mysqlInventoryDatabase.getDatabaseName()); + + complexDataTypesDatabase.createAndInitialize(); + createDorisDatabase(complexDataTypesDatabase.getDatabaseName()); } private static boolean checkBackendAvailability() { @@ -159,6 +166,9 @@ public class MySqlToDorisE2eITCase extends PipelineTestEnvironment { super.after(); mysqlInventoryDatabase.dropDatabase(); dropDorisDatabase(mysqlInventoryDatabase.getDatabaseName()); + + complexDataTypesDatabase.dropDatabase(); + dropDorisDatabase(complexDataTypesDatabase.getDatabaseName()); } @Test @@ -287,6 +297,103 @@ public class MySqlToDorisE2eITCase extends PipelineTestEnvironment { "112 | finally | null | 2.14 | null | null | null")); } + @Test + public void testComplexDataTypes() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: mysql\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: doris\n" + + " fenodes: doris:8030\n" + + " benodes: doris:8040\n" + + " username: %s\n" + + " password: \"%s\"\n" + + " table.create.properties.replication_num: 1\n" + + "\n" + + "transform:\n" + + " - source-table: %s.DATA_TYPES_TABLE\n" + + " projection: \\*, 'fine' AS FINE\n" + + " filter: id <> 3 AND id <> 4\n" + + "pipeline:\n" + + " parallelism: 1", + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + complexDataTypesDatabase.getDatabaseName(), + DORIS.getUsername(), + DORIS.getPassword(), + complexDataTypesDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path dorisCdcConnector = TestUtils.getResource("doris-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, dorisCdcConnector, mysqlDriverJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + validateSinkResult( + complexDataTypesDatabase.getDatabaseName(), + "DATA_TYPES_TABLE", + 52, + Collections.singletonList( + "1 | 127 | 255 | 255 | 32767 | 65535 | 65535 | 8388607 | 16777215 | 16777215 | 2147483647 | 4294967295 | 4294967295 | 2147483647 | 9223372036854775807 | Hello World | abc | 123.102 | 123.102 | 123.103 | 123.104 | 404.4443 | 404.4444 | 404.4445 | 123.4567 | 123.4568 | 123.4569 | 346 | 34567892.1 | 0 | 1 | 1 | 2020-07-17 | 2020-07-17 18:00:22.0 | 2020-07-17 18:00:22.0 | 2020-07-17 18:00:22 | text | EA== | EA== | EA== | EA== | 2021 | red | {\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0} | {\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0} | {\"coordinates\":[[3,0],[3,3],[3,5]],\"type\":\"LineString\",\"srid\":0} | {\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0} | {\"coordinates\":[[1,1],[2,2]],\"type\":\"MultiPoint\",\"srid\":0} | {\"coordinates\":[[[1,1],[2,2],[3,3]],[[4,4],[5,5]]],\"type\":\"MultiLineString\",\"srid\":0} | {\"coordinates\":[[[[0,0],[10,0],[10,10],[0,10],[0,0]]],[[[5,5],[7,5],[7,7],[5,7],[5,5]]]],\"type\":\"MultiPolygon\",\"srid\":0} | {\"geometries\":[{\"type\":\"Point\",\"coordinates\":[10,10]},{\"type\":\"Point\",\"coordinates\":[30,30]},{\"type\":\"LineString\",\"coordinates\":[[15,15],[20,20]]}],\"type\":\"GeometryCollection\",\"srid\":0} | fine")); + + LOG.info("Begin incremental reading stage."); + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + complexDataTypesDatabase.getDatabaseName()); + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stat = conn.createStatement()) { + + // Insert id = 2, 3, 4, 5 + for (int i = 2; i < 6; i++) { + stat.execute( + "INSERT INTO DATA_TYPES_TABLE\n" + + "VALUES (" + + i + + ", 127, 255, 255, 32767, 65535, 65535, 8388607, 16777215, 16777215, 2147483647,\n" + + " 4294967295, 4294967295, 2147483647, 9223372036854775807,\n" + + " 'Hello World', 'abc', 123.102, 123.102, 123.103, 123.104, 404.4443, 404.4444, 404.4445,\n" + + " 123.4567, 123.4568, 123.4569, 345.6, 34567892.1, 0, 1, true,\n" + + " '2020-07-17', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', '2020-07-17 18:00:22',\n" + + " 'text', UNHEX(HEX(16)), UNHEX(HEX(16)), UNHEX(HEX(16)), UNHEX(HEX(16)), 2021,\n" + + " 'red',\n" + + " ST_GeomFromText('POINT(1 1)'),\n" + + " ST_GeomFromText('POLYGON((1 1, 2 1, 2 2, 1 2, 1 1))'),\n" + + " ST_GeomFromText('LINESTRING(3 0, 3 3, 3 5)'),\n" + + " ST_GeomFromText('POLYGON((1 1, 2 1, 2 2, 1 2, 1 1))'),\n" + + " ST_GeomFromText('MULTIPOINT((1 1),(2 2))'),\n" + + " ST_GeomFromText('MultiLineString((1 1,2 2,3 3),(4 4,5 5))'),\n" + + " ST_GeomFromText('MULTIPOLYGON(((0 0, 10 0, 10 10, 0 10, 0 0)), ((5 5, 7 5, 7 7, 5 7, 5 5)))'),\n" + + " ST_GeomFromText('GEOMETRYCOLLECTION(POINT(10 10), POINT(30 30), LINESTRING(15 15, 20 20))'));"); + } + + validateSinkResult( + complexDataTypesDatabase.getDatabaseName(), + "DATA_TYPES_TABLE", + 52, + Arrays.asList( + "1 | 127 | 255 | 255 | 32767 | 65535 | 65535 | 8388607 | 16777215 | 16777215 | 2147483647 | 4294967295 | 4294967295 | 2147483647 | 9223372036854775807 | Hello World | abc | 123.102 | 123.102 | 123.103 | 123.104 | 404.4443 | 404.4444 | 404.4445 | 123.4567 | 123.4568 | 123.4569 | 346 | 34567892.1 | 0 | 1 | 1 | 2020-07-17 | 2020-07-17 18:00:22.0 | 2020-07-17 18:00:22.0 | 2020-07-17 18:00:22 | text | EA== | EA== | EA== | EA== | 2021 | red | {\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0} | {\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0} | {\"coordinates\":[[3,0],[3,3],[3,5]],\"type\":\"LineString\",\"srid\":0} | {\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0} | {\"coordinates\":[[1,1],[2,2]],\"type\":\"MultiPoint\",\"srid\":0} | {\"coordinates\":[[[1,1],[2,2],[3,3]],[[4,4],[5,5]]],\"type\":\"MultiLineString\",\"srid\":0} | {\"coordinates\":[[[[0,0],[10,0],[10,10],[0,10],[0,0]]],[[[5,5],[7,5],[7,7],[5,7],[5,5]]]],\"type\":\"MultiPolygon\",\"srid\":0} | {\"geometries\":[{\"type\":\"Point\",\"coordinates\":[10,10]},{\"type\":\"Point\",\"coordinates\":[30,30]},{\"type\":\"LineString\",\"coordinates\":[[15,15],[20,20]]}],\"type\":\"GeometryCollection\",\"srid\":0} | fine", + "2 | 127 | 255 | 255 | 32767 | 65535 | 65535 | 8388607 | 16777215 | 16777215 | 2147483647 | 4294967295 | 4294967295 | 2147483647 | 9223372036854775807 | Hello World | abc | 123.102 | 123.102 | 123.103 | 123.104 | 404.4443 | 404.4444 | 404.4445 | 123.4567 | 123.4568 | 123.4569 | 346 | 34567892.1 | 0 | 1 | 1 | 2020-07-17 | 2020-07-17 18:00:22.0 | 2020-07-17 18:00:22.0 | 2020-07-17 18:00:22 | text | EA== | EA== | EA== | EA== | 2021 | red | {\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0} | {\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0} | {\"coordinates\":[[3,0],[3,3],[3,5]],\"type\":\"LineString\",\"srid\":0} | {\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0} | {\"coordinates\":[[1,1],[2,2]],\"type\":\"MultiPoint\",\"srid\":0} | {\"coordinates\":[[[1,1],[2,2],[3,3]],[[4,4],[5,5]]],\"type\":\"MultiLineString\",\"srid\":0} | {\"coordinates\":[[[[0,0],[10,0],[10,10],[0,10],[0,0]]],[[[5,5],[7,5],[7,7],[5,7],[5,5]]]],\"type\":\"MultiPolygon\",\"srid\":0} | {\"geometries\":[{\"type\":\"Point\",\"coordinates\":[10,10]},{\"type\":\"Point\",\"coordinates\":[30,30]},{\"type\":\"LineString\",\"coordinates\":[[15,15],[20,20]]}],\"type\":\"GeometryCollection\",\"srid\":0} | fine", + "5 | 127 | 255 | 255 | 32767 | 65535 | 65535 | 8388607 | 16777215 | 16777215 | 2147483647 | 4294967295 | 4294967295 | 2147483647 | 9223372036854775807 | Hello World | abc | 123.102 | 123.102 | 123.103 | 123.104 | 404.4443 | 404.4444 | 404.4445 | 123.4567 | 123.4568 | 123.4569 | 346 | 34567892.1 | 0 | 1 | 1 | 2020-07-17 | 2020-07-17 18:00:22.0 | 2020-07-17 18:00:22.0 | 2020-07-17 18:00:22 | text | EA== | EA== | EA== | EA== | 2021 | red | {\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0} | {\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0} | {\"coordinates\":[[3,0],[3,3],[3,5]],\"type\":\"LineString\",\"srid\":0} | {\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0} | {\"coordinates\":[[1,1],[2,2]],\"type\":\"MultiPoint\",\"srid\":0} | {\"coordinates\":[[[1,1],[2,2],[3,3]],[[4,4],[5,5]]],\"type\":\"MultiLineString\",\"srid\":0} | {\"coordinates\":[[[[0,0],[10,0],[10,10],[0,10],[0,0]]],[[[5,5],[7,5],[7,7],[5,7],[5,5]]]],\"type\":\"MultiPolygon\",\"srid\":0} | {\"geometries\":[{\"type\":\"Point\",\"coordinates\":[10,10]},{\"type\":\"Point\",\"coordinates\":[30,30]},{\"type\":\"LineString\",\"coordinates\":[[15,15],[20,20]]}],\"type\":\"GeometryCollection\",\"srid\":0} | fine")); + } catch (SQLException e) { + LOG.error("Update table for CDC failed.", e); + throw e; + } + } + public static void createDorisDatabase(String databaseName) { try { Container.ExecResult rs = diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/RouteE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/RouteE2eITCase.java index e674d57d0..ab7ed178e 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/RouteE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/RouteE2eITCase.java @@ -22,7 +22,6 @@ import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; 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.junit.After; import org.junit.Assert; @@ -43,7 +42,7 @@ import java.sql.Statement; import java.time.Duration; import java.util.concurrent.TimeoutException; -/** E2e tests for the {@link TransformSchemaOperator}. */ +/** E2e tests for routing features. */ @RunWith(Parameterized.class) public class RouteE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(RouteE2eITCase.class); @@ -624,6 +623,94 @@ public class RouteE2eITCase extends PipelineTestEnvironment { "DataChangeEvent{tableId=%s.TABLEDELTA, before=[], after=[10004], op=INSERT, meta=()}"); } + @Test + public void testMergeTableRouteWithTransform() 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.\\.*\n" + + " projection: \\*, 'extras' AS EXTRAS\n" + + "route:\n" + + " - source-table: %s.\\.*\n" + + " sink-table: %s.ALL\n" + + "\n" + + "pipeline:\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + routeTestDatabase.getDatabaseName(), + routeTestDatabase.getDatabaseName(), + routeTestDatabase.getDatabaseName(), + routeTestDatabase.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.ALL, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`EXTRAS` STRING}, primaryKeys=ID, options=()}", + routeTestDatabase.getDatabaseName())); + + validateResult( + "DataChangeEvent{tableId=%s.ALL, before=[], after=[1008, 8, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[1009, 8.1, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[1010, 10, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[1011, 11, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[2011, 11, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[2012, 12, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[2013, 13, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[2014, 14, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[3015, Amber, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[3016, Black, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[3017, Cyan, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[3018, Denim, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[4019, Yosemite, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[4020, El Capitan, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[4021, Sierra, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[4022, High Sierra, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[4023, Mojave, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[4024, Catalina, extras], op=INSERT, meta=()}"); + + LOG.info("Begin incremental reading stage."); + + // generate binlogs + generateIncrementalChanges(); + + validateResult( + "DataChangeEvent{tableId=%s.ALL, before=[], after=[3007, 7, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[2014, 14, extras], after=[2014, 2014, extras], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[3019, Emerald, extras], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[4024, Catalina, extras], after=[], op=DELETE, meta=()}"); + + LOG.info("Begin schema changing stage."); + generateSchemaChanges(); + + validateResult( + "AddColumnEvent{tableId=%s.ALL, addedColumns=[ColumnWithPosition{column=`NAME` VARCHAR(17), position=LAST, existedColumnName=null}]}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[10001, 12, Derrida, extras], op=INSERT, meta=()}", + "AddColumnEvent{tableId=%s.ALL, addedColumns=[ColumnWithPosition{column=`VERSION_EX` VARCHAR(17), position=LAST, existedColumnName=null}]}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[10002, null, extras, null, 15], op=INSERT, meta=()}", + "AlterColumnTypeEvent{tableId=%s.ALL, nameMapping={VERSION=STRING}}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[10003, null, extras, null, Fluorite], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.ALL, before=[], after=[10004, null, extras, null, null], op=INSERT, meta=()}"); + } + @Test public void testReplacementSymbol() throws Exception { String pipelineJob = 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 0691e8806..0dad9f363 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,7 +22,8 @@ import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; 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.junit.After; import org.junit.Assert; @@ -50,7 +51,7 @@ import java.util.List; import java.util.concurrent.TimeoutException; import java.util.stream.Stream; -/** 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); @@ -62,7 +63,6 @@ public class TransformE2eITCase extends PipelineTestEnvironment { protected static final String MYSQL_TEST_PASSWORD = "mysqlpw"; protected static final String MYSQL_DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; protected static final String INTER_CONTAINER_MYSQL_ALIAS = "mysql"; - protected static final long EVENT_WAITING_TIMEOUT = 60000L; @ClassRule public static final MySqlContainer MYSQL = @@ -78,19 +78,19 @@ public class TransformE2eITCase extends PipelineTestEnvironment { .withNetworkAliases(INTER_CONTAINER_MYSQL_ALIAS) .withLogConsumer(new Slf4jLogConsumer(LOG)); - protected final UniqueDatabase transformRenameDatabase = + protected final UniqueDatabase transformTestDatabase = new UniqueDatabase(MYSQL, "transform_test", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); @Before public void before() throws Exception { super.before(); - transformRenameDatabase.createAndInitialize(); + transformTestDatabase.createAndInitialize(); } @After public void after() { super.after(); - transformRenameDatabase.dropDatabase(); + transformTestDatabase.dropDatabase(); } @Test @@ -106,7 +106,6 @@ public class TransformE2eITCase extends PipelineTestEnvironment { + " tables: %s.\\.*\n" + " server-id: 5400-5404\n" + " server-time-zone: UTC\n" - + "\n" + "sink:\n" + " type: values\n" + "route:\n" @@ -118,57 +117,125 @@ public class TransformE2eITCase extends PipelineTestEnvironment { + " filter: ID > 1008\n" + " - source-table: %s.TABLEBETA\n" + " projection: ID, VERSION\n" + + "pipeline:\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + transformTestDatabase.getDatabaseName(), + transformTestDatabase.getDatabaseName(), + transformTestDatabase.getDatabaseName(), + transformTestDatabase.getDatabaseName(), + transformTestDatabase.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` VARCHAR(17)}, primaryKeys=ID, options=()}", + transformTestDatabase.getDatabaseName()), + 60000L); + + validateEvents( + "DataChangeEvent{tableId=%s.terminus, before=[], after=[1009, 8.1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[], after=[1010, 10], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[], after=[1011, 11], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[], after=[2011, 11], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[], after=[2012, 12], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[], after=[2014, 14], op=INSERT, meta=()}"); + + LOG.info("Begin incremental reading stage."); + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + transformTestDatabase.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; + } + + validateEvents( + "DataChangeEvent{tableId=%s.terminus, before=[], after=[3007, 7], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[1009, 8.1], after=[1009, 100], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[2011, 11], after=[], op=DELETE, meta=()}"); + } + + @Test + public void testMultipleTransformRule() 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" + + "transform:\n" + + " - source-table: %s.\\.*\n" + + " projection: ID, VERSION, 'Type-A' AS CATEGORY\n" + + " filter: ID > 1008\n" + + " - source-table: %s.\\.*\n" + + " projection: ID, VERSION, 'Type-B' AS CATEGORY\n" + + " filter: ID <= 1008\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()); + transformTestDatabase.getDatabaseName(), + transformTestDatabase.getDatabaseName(), + transformTestDatabase.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( - "DataChangeEvent{tableId=%s.terminus, before=[], after=[1011, 11], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName())); + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CATEGORY` STRING}, primaryKeys=ID, options=()}", + transformTestDatabase.getDatabaseName()), + 60000L); waitUntilSpecificEvent( String.format( - "DataChangeEvent{tableId=%s.terminus, before=[], after=[2014, 14], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName())); - - List expectedEvents = - Arrays.asList( - String.format( - "CreateTableEvent{tableId=%s.terminus, schema=columns={`ID` INT NOT NULL,`VERSION` STRING}, primaryKeys=ID, options=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.terminus, before=[], after=[1009, 8.1], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.terminus, before=[], after=[1010, 10], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.terminus, before=[], after=[1011, 11], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.terminus, before=[], after=[2011, 11], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.terminus, before=[], after=[2012, 12], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.terminus, before=[], after=[2014, 14], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName())); - validateResult(expectedEvents); + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CATEGORY` STRING}, primaryKeys=ID, options=()}", + transformTestDatabase.getDatabaseName()), + 60000L); + + validateEvents( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8, Type-B], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, Type-A], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, Type-A], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, Type-A], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Type-A], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Type-A], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Type-A], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Type-A], op=INSERT, meta=()}"); + LOG.info("Begin incremental reading stage."); // generate binlogs String mysqlJdbcUrl = @@ -176,36 +243,272 @@ public class TransformE2eITCase extends PipelineTestEnvironment { "jdbc:mysql://%s:%s/%s", MYSQL.getHost(), MYSQL.getDatabasePort(), - transformRenameDatabase.getDatabaseName()); + transformTestDatabase.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);"); + 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; } + validateEvents( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, Type-A], after=[1009, 100, Type-A], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, Type-A], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Type-A], after=[], op=DELETE, meta=()}"); + } + + @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" + + "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" + + "pipeline:\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + transformTestDatabase.getDatabaseName(), + transformTestDatabase.getDatabaseName(), + transformTestDatabase.getDatabaseName(), + transformTestDatabase.getDatabaseName(), + transformTestDatabase.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( - "DataChangeEvent{tableId=%s.terminus, before=[], after=[3007, 7], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName())); + "CreateTableEvent{tableId=%s.terminus, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`NAME` STRING}, primaryKeys=ID, options=()}", + transformTestDatabase.getDatabaseName()), + 60000L); + + validateEvents( + "DataChangeEvent{tableId=%s.terminus, before=[], after=[1008, v8, alice], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[], after=[1009, v8.1, bob], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[], after=[2011, v11, eva], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[], after=[2012, v12, fred], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[], after=[2013, v13, gus], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[], after=[2014, v14, henry], op=INSERT, meta=()}"); + + LOG.info("Begin incremental reading stage."); + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + transformTestDatabase.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; + } + + validateEvents( + "DataChangeEvent{tableId=%s.terminus, before=[1009, v8.1, bob], after=[1009, v100, bob], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[], after=[3007, v7, iina], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.terminus, before=[2011, v11, eva], after=[], op=DELETE, meta=()}"); + } + + @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" + + "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" + + "pipeline:\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + transformTestDatabase.getDatabaseName(), + transformTestDatabase.getDatabaseName(), + transformTestDatabase.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( - "DataChangeEvent{tableId=%s.terminus, before=[1009, 8.1], after=[1009, 100], op=UPDATE, meta=()}", - transformRenameDatabase.getDatabaseName())); + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`NAME` STRING}, primaryKeys=ID, options=()}", + transformTestDatabase.getDatabaseName()), + 60000L); waitUntilSpecificEvent( String.format( - "DataChangeEvent{tableId=%s.terminus, before=[2011, 11], after=[], op=DELETE, meta=()}", - transformRenameDatabase.getDatabaseName())); + "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=()}", + transformTestDatabase.getDatabaseName()), + 60000L); + + validateEvents( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, v8, 199, 17, Alice, alice], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, v8.1, 0, 18, Bob, bob], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, v11, Big Sur, 21, Eva, eva], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, v12, Monterey, 22, Fred, fred], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, v13, Ventura, 23, Gus, gus], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, v14, Sonoma, 24, Henry, henry], op=INSERT, meta=()}"); - String stdout = taskManagerConsumer.toUtf8String(); - System.out.println(stdout); + LOG.info("Begin incremental reading stage."); + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + transformTestDatabase.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; + } + + validateEvents( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, v8.1, 0, 18, Bob, bob], after=[1009, v100, 0, 18, Bob, bob], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, v7, 79, 16, IINA, iina], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, v11, Big Sur, 21, Eva, eva], after=[], op=DELETE, meta=()}"); + } + + @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" + + "sink:\n" + + " type: values\n" + + "transform:\n" + + " - source-table: %s.TABLEALPHA\n" + + " projection: \\*, __namespace_name__ || '.' || __schema_name__ || '.' || __table_name__ AS identifier_name\n" + + " - source-table: %s.TABLEBETA\n" + + " projection: \\*, __namespace_name__ || '.' || __schema_name__ || '.' || __table_name__ AS identifier_name\n" + + "pipeline:\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + transformTestDatabase.getDatabaseName(), + transformTestDatabase.getDatabaseName(), + transformTestDatabase.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` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`identifier_name` STRING}, primaryKeys=ID, options=()}", + transformTestDatabase.getDatabaseName()), + 60000L); + + waitUntilSpecificEvent( + String.format( + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`identifier_name` STRING}, primaryKeys=ID, options=()}", + transformTestDatabase.getDatabaseName()), + 60000L); + + validateEvents( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8, 199, 17, Alice, null.%s.TABLEALPHA], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 99, 19, Carol, null.%s.TABLEALPHA], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 0, 18, Bob, null.%s.TABLEALPHA], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 59, 20, Dave, null.%s.TABLEALPHA], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Monterey, 22, Fred, null.%s.TABLEBETA], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Big Sur, 21, Eva, null.%s.TABLEBETA], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Sonoma, 24, Henry, null.%s.TABLEBETA], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Ventura, 23, Gus, null.%s.TABLEBETA], op=INSERT, meta=()}"); + + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + transformTestDatabase.getDatabaseName()); + insertBinlogEvents(mysqlJdbcUrl); + + validateEvents( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 0, 18, Bob, null.%s.TABLEALPHA], after=[1009, 100, 0, 18, Bob, null.%s.TABLEALPHA], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 79, 16, IINA, null.%s.TABLEALPHA], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Big Sur, 21, Eva, null.%s.TABLEBETA], after=[], op=DELETE, meta=()}"); + } + + private static void insertBinlogEvents(String mysqlJdbcUrl) throws SQLException { + 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; + } } @Test @@ -221,20 +524,18 @@ public class TransformE2eITCase extends PipelineTestEnvironment { + " 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()); + transformTestDatabase.getDatabaseName(), + transformTestDatabase.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"); @@ -244,41 +545,26 @@ public class TransformE2eITCase extends PipelineTestEnvironment { waitUntilSpecificEvent( String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`PRICEALPHA` INT,`UID` INT,`NEWVERSION` STRING}, primaryKeys=ID, options=()}", - transformRenameDatabase.getDatabaseName())); - 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=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 99, 2010, 10], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 59, 2011, 11], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8, 199, 2008, 8], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 0, 2009, 8.1], 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=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Sonoma, 3014, 14], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Monterey, 3012, 12], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Ventura, 3013, 13], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Big Sur, 3011, 11], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName())); - validateResult(expectedEvents); + "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=()}", + transformTestDatabase.getDatabaseName()), + 60000L); + + waitUntilSpecificEvent( + 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=()}", + transformTestDatabase.getDatabaseName()), + 60000L); + + validateEvents( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8, 199, 17, Alice, 2008, 8], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 0, 18, Bob, 2009, 8.1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 99, 19, Carol, 2010, 10], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 59, 20, Dave, 2011, 11], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Big Sur, 21, Eva, 3011, 11], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Monterey, 22, Fred, 3012, 12], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Ventura, 23, Gus, 3013, 13], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Sonoma, 24, Henry, 3014, 14], op=INSERT, meta=()}"); + LOG.info("Begin incremental reading stage."); // generate binlogs String mysqlJdbcUrl = @@ -286,33 +572,119 @@ public class TransformE2eITCase extends PipelineTestEnvironment { "jdbc:mysql://%s:%s/%s", MYSQL.getHost(), MYSQL.getDatabasePort(), - transformRenameDatabase.getDatabaseName()); + transformTestDatabase.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);"); + 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( + validateEvents( + "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=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 79, 25, IINA, 4007, 7], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Big Sur, 21, Eva, 3011, 11], after=[], op=DELETE, meta=()}"); + } + + @Test + public void testTransformWithCast() throws Exception { + String pipelineJob = String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 0, 2009, 8.1], after=[1009, 100, 0, 2009, 100], op=UPDATE, meta=()}", - transformRenameDatabase.getDatabaseName())); + "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" + + "sink:\n" + + " type: values\n" + + "transform:\n" + + " - source-table: %s.TABLEALPHA\n" + + " projection: ID, CAST(VERSION AS DOUBLE) + 100 AS VERSION, CAST(AGEALPHA AS VARCHAR) || ' - ' || NAMEALPHA AS IDENTIFIER\n" + + " filter: AGEALPHA < 19\n" + + " - source-table: %s.TABLEBETA\n" + + " projection: ID, CAST(VERSION AS DOUBLE) + 100 AS VERSION, CAST(AGEBETA AS VARCHAR) || ' - ' || NAMEBETA AS IDENTIFIER\n" + + "pipeline:\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + transformTestDatabase.getDatabaseName(), + transformTestDatabase.getDatabaseName(), + transformTestDatabase.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( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 79, 4007, 7], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName())); - + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` DOUBLE,`IDENTIFIER` STRING}, primaryKeys=ID, options=()}", + transformTestDatabase.getDatabaseName()), + 60000L); waitUntilSpecificEvent( String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Big Sur, 3011, 11], after=[], op=DELETE, meta=()}", - transformRenameDatabase.getDatabaseName())); + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` DOUBLE,`IDENTIFIER` STRING}, primaryKeys=ID, options=()}", + transformTestDatabase.getDatabaseName()), + 60000L); + + validateEvents( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 108.0, 17 - Alice], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 108.1, 18 - Bob], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 111.0, 21 - Eva], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 112.0, 22 - Fred], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 113.0, 23 - Gus], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 114.0, 24 - Henry], op=INSERT, meta=()}"); + + LOG.info("Begin incremental reading stage."); + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + transformTestDatabase.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; + } + + validateEvents( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 108.1, 18 - Bob], after=[1009, 200.0, 18 - Bob], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 107.0, 16 - IINA], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 111.0, 21 - Eva], after=[], op=DELETE, meta=()}"); + + String stdout = taskManagerConsumer.toUtf8String(); + System.out.println(stdout); + } + + private void validateEvents(String... expectedEvents) throws Exception { + for (String event : expectedEvents) { + waitUntilSpecificEvent( + String.format( + event, + transformTestDatabase.getDatabaseName(), + transformTestDatabase.getDatabaseName(), + transformTestDatabase.getDatabaseName()), + 20000L); + } } @Test @@ -341,8 +713,8 @@ public class TransformE2eITCase extends PipelineTestEnvironment { INTER_CONTAINER_MYSQL_ALIAS, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD, - transformRenameDatabase.getDatabaseName(), - transformRenameDatabase.getDatabaseName()); + transformTestDatabase.getDatabaseName(), + transformTestDatabase.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"); @@ -355,13 +727,13 @@ public class TransformE2eITCase extends PipelineTestEnvironment { private void validateResult(List expectedEvents) throws Exception { for (String event : expectedEvents) { - waitUntilSpecificEvent(event); + waitUntilSpecificEvent(event, 6000L); } } - private void waitUntilSpecificEvent(String event) throws Exception { + private void waitUntilSpecificEvent(String event, long timeout) throws Exception { boolean result = false; - long endTimeout = System.currentTimeMillis() + TransformE2eITCase.EVENT_WAITING_TIMEOUT; + long endTimeout = System.currentTimeMillis() + timeout; while (System.currentTimeMillis() < endTimeout) { String stdout = taskManagerConsumer.toUtf8String(); if (stdout.contains(event)) { @@ -441,6 +813,12 @@ public class TransformE2eITCase extends PipelineTestEnvironment { Assert.assertEquals(currentTimestamp, nowTimestamp); Assert.assertEquals(currentTimestamp, localTimestamp); + // If timestamp millisecond part is .000, it will be truncated to yyyy-MM-dd'T'HH:mm:ss + // format. Manually append this for the following checks. + if (currentTimestamp.length() == 19) { + currentTimestamp += ".000"; + } + Instant instant = LocalDateTime.parse( currentTimestamp, diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/data_types_test.sql b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/data_types_test.sql new file mode 100644 index 000000000..4c7621250 --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/data_types_test.sql @@ -0,0 +1,91 @@ +-- 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. + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: column_type_test +-- ---------------------------------------------------------------------------------------------------------------- + +CREATE TABLE DATA_TYPES_TABLE +( + id INT, + tiny_c TINYINT, + tiny_un_c TINYINT UNSIGNED, + tiny_un_z_c TINYINT UNSIGNED ZEROFILL, + small_c SMALLINT, + small_un_c SMALLINT UNSIGNED, + small_un_z_c SMALLINT UNSIGNED ZEROFILL, + medium_c MEDIUMINT, + medium_un_c MEDIUMINT UNSIGNED, + medium_un_z_c MEDIUMINT UNSIGNED ZEROFILL, + int_c INTEGER, + int_un_c INTEGER UNSIGNED, + int_un_z_c INTEGER UNSIGNED ZEROFILL, + int11_c INT(11), + big_c BIGINT, + varchar_c VARCHAR(255), + char_c CHAR(3), + real_c REAL, + float_c FLOAT, + float_un_c FLOAT UNSIGNED, + float_un_z_c FLOAT UNSIGNED ZEROFILL, + double_c DOUBLE, + double_un_c DOUBLE UNSIGNED, + double_un_z_c DOUBLE UNSIGNED ZEROFILL, + decimal_c DECIMAL(8, 4), + decimal_un_c DECIMAL(8, 4) UNSIGNED, + decimal_un_z_c DECIMAL(8, 4) UNSIGNED ZEROFILL, + numeric_c NUMERIC(6, 0), + big_decimal_c DECIMAL(65, 1), + bit1_c BIT, + tiny1_c TINYINT(1), + boolean_c BOOLEAN, + date_c DATE, + datetime3_c DATETIME(3), + datetime6_c DATETIME(6), + timestamp_c TIMESTAMP, + text_c TEXT, + tiny_blob_c TINYBLOB, + blob_c BLOB, + medium_blob_c MEDIUMBLOB, + long_blob_c LONGBLOB, + year_c YEAR, + enum_c enum('red', 'white') default 'red', + point_c POINT, + geometry_c GEOMETRY, + linestring_c LINESTRING, + polygon_c POLYGON, + multipoint_c MULTIPOINT, + multiline_c MULTILINESTRING, + multipolygon_c MULTIPOLYGON, + geometrycollection_c GEOMCOLLECTION, + PRIMARY KEY (id) +) DEFAULT CHARSET=utf8; + +INSERT INTO DATA_TYPES_TABLE +VALUES (1, 127, 255, 255, 32767, 65535, 65535, 8388607, 16777215, 16777215, 2147483647, + 4294967295, 4294967295, 2147483647, 9223372036854775807, + 'Hello World', 'abc', 123.102, 123.102, 123.103, 123.104, 404.4443, 404.4444, 404.4445, + 123.4567, 123.4568, 123.4569, 345.6, 34567892.1, 0, 1, true, + '2020-07-17', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', '2020-07-17 18:00:22', + 'text', UNHEX(HEX(16)), UNHEX(HEX(16)), UNHEX(HEX(16)), UNHEX(HEX(16)), 2021, + 'red', + ST_GeomFromText('POINT(1 1)'), + ST_GeomFromText('POLYGON((1 1, 2 1, 2 2, 1 2, 1 1))'), + ST_GeomFromText('LINESTRING(3 0, 3 3, 3 5)'), + ST_GeomFromText('POLYGON((1 1, 2 1, 2 2, 1 2, 1 1))'), + ST_GeomFromText('MULTIPOINT((1 1),(2 2))'), + ST_GeomFromText('MultiLineString((1 1,2 2,3 3),(4 4,5 5))'), + ST_GeomFromText('MULTIPOLYGON(((0 0, 10 0, 10 10, 0 10, 0 0)), ((5 5, 7 5, 7 7, 5 7, 5 5)))'), + ST_GeomFromText('GEOMETRYCOLLECTION(POINT(10 10), POINT(30 30), LINESTRING(15 15, 20 20))')); \ No newline at end of file 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 9e61a265f..8989ad2c8 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-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java b/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java index 6a14a2be2..1317c73eb 100644 --- a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java +++ b/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java @@ -20,7 +20,7 @@ package org.apache.flink.cdc.migration.tests; 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.runtime.operators.transform.TableChangeInfo; +import org.apache.flink.cdc.runtime.operators.transform.PreTransformChangeInfo; /** Dummy classes for migration test. Called via reflection. */ public class TableChangeInfoMigrationMock implements MigrationMockBase { @@ -34,27 +34,28 @@ public class TableChangeInfoMigrationMock implements MigrationMockBase { .primaryKey("id", "name") .build(); - public TableChangeInfo generateDummyObject() { - return TableChangeInfo.of(DUMMY_TABLE_ID, DUMMY_SCHEMA, DUMMY_SCHEMA); + public PreTransformChangeInfo generateDummyObject() { + return PreTransformChangeInfo.of(DUMMY_TABLE_ID, DUMMY_SCHEMA, DUMMY_SCHEMA); } @Override public int getSerializerVersion() { - return TableChangeInfo.SERIALIZER.getVersion(); + return PreTransformChangeInfo.SERIALIZER.getVersion(); } @Override public byte[] serializeObject() throws Exception { - return TableChangeInfo.SERIALIZER.serialize(generateDummyObject()); + return PreTransformChangeInfo.SERIALIZER.serialize(generateDummyObject()); } @Override public boolean deserializeAndCheckObject(int version, byte[] bytes) throws Exception { - TableChangeInfo expected = generateDummyObject(); - TableChangeInfo actual = TableChangeInfo.SERIALIZER.deserialize(version, bytes); + PreTransformChangeInfo expected = generateDummyObject(); + PreTransformChangeInfo actual = + PreTransformChangeInfo.SERIALIZER.deserialize(version, bytes); return expected.getTableId().equals(actual.getTableId()) - && expected.getOriginalSchema().equals(actual.getOriginalSchema()) - && expected.getTransformedSchema().equals(actual.getTransformedSchema()); + && expected.getSourceSchema().equals(actual.getSourceSchema()) + && expected.getPreTransformedSchema().equals(actual.getPreTransformedSchema()); } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperator.java index bcb447688..d3a9e158b 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperator.java @@ -75,6 +75,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import java.io.Serializable; import java.time.Duration; import java.util.ArrayList; import java.util.List; @@ -92,7 +93,7 @@ import static org.apache.flink.cdc.common.pipeline.PipelineOptions.DEFAULT_SCHEM */ @Internal public class SchemaOperator extends AbstractStreamOperator - implements OneInputStreamOperator { + implements OneInputStreamOperator, Serializable { private static final long serialVersionUID = 1L; diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformChangeInfo.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformChangeInfo.java new file mode 100644 index 000000000..989af2e75 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformChangeInfo.java @@ -0,0 +1,117 @@ +/* + * 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.event.TableId; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.utils.SchemaUtils; +import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; +import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter; + +import java.util.List; + +/** + * PostTransformChangeInfo caches pre-transformed / pre-transformed schema, schema field getters, + * and binary record data generator for post-transform schema. + */ +public class PostTransformChangeInfo { + private TableId tableId; + private Schema preTransformedSchema; + private Schema postTransformedSchema; + private RecordData.FieldGetter[] preTransformedFieldGetters; + private RecordData.FieldGetter[] postTransformedFieldGetters; + private BinaryRecordDataGenerator recordDataGenerator; + + public PostTransformChangeInfo( + TableId tableId, + Schema postTransformedSchema, + RecordData.FieldGetter[] postTransformedFieldGetters, + Schema preTransformedSchema, + RecordData.FieldGetter[] preTransformedFieldGetters, + BinaryRecordDataGenerator recordDataGenerator) { + this.tableId = tableId; + this.postTransformedSchema = postTransformedSchema; + this.postTransformedFieldGetters = postTransformedFieldGetters; + this.preTransformedSchema = preTransformedSchema; + this.preTransformedFieldGetters = preTransformedFieldGetters; + this.recordDataGenerator = recordDataGenerator; + } + + public String getName() { + return tableId.identifier(); + } + + public String getTableName() { + return tableId.getTableName(); + } + + public String getSchemaName() { + return tableId.getSchemaName(); + } + + public String getNamespace() { + return tableId.getNamespace(); + } + + public TableId getTableId() { + return tableId; + } + + public Schema getPostTransformedSchema() { + return postTransformedSchema; + } + + public Schema getPreTransformedSchema() { + return preTransformedSchema; + } + + public RecordData.FieldGetter[] getPostTransformedFieldGetters() { + return postTransformedFieldGetters; + } + + public RecordData.FieldGetter[] getPreTransformedFieldGetters() { + return preTransformedFieldGetters; + } + + public BinaryRecordDataGenerator getRecordDataGenerator() { + return recordDataGenerator; + } + + public static PostTransformChangeInfo of( + TableId tableId, Schema postTransformedSchema, Schema preTransformedSchema) { + + List postTransformedFieldGetters = + SchemaUtils.createFieldGetters(postTransformedSchema.getColumns()); + + List preTransformedFieldGetters = + SchemaUtils.createFieldGetters(preTransformedSchema.getColumns()); + + BinaryRecordDataGenerator postTransformedRecordDataGenerator = + new BinaryRecordDataGenerator( + DataTypeConverter.toRowType(postTransformedSchema.getColumns())); + + return new PostTransformChangeInfo( + tableId, + postTransformedSchema, + postTransformedFieldGetters.toArray(new RecordData.FieldGetter[0]), + preTransformedSchema, + preTransformedFieldGetters.toArray(new RecordData.FieldGetter[0]), + postTransformedRecordDataGenerator); + } +} 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 57% 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 9f3da6a66..b280bf901 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,74 +29,84 @@ import org.apache.flink.cdc.common.pipeline.PipelineOptions; 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; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import javax.annotation.Nullable; +import java.io.Serializable; import java.time.ZoneId; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; 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 - implements OneInputStreamOperator { +/** + * A data process function that performs column filtering, calculated column evaluation & final + * projection. + */ +public class PostTransformOperator extends AbstractStreamOperator + implements OneInputStreamOperator, Serializable { + + private static final long serialVersionUID = 1L; - 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 final Map postTransformChangeInfoMap; private transient Map, TransformProjectionProcessor> 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,35 +115,19 @@ public class TransformDataOperator extends AbstractStreamOperator return this; } - public TransformDataOperator build() { - return new TransformDataOperator(transformRules, schemaOperatorID, timezone); + public PostTransformOperator build() { + return new PostTransformOperator(transformRules, timezone); } } - private TransformDataOperator( - List> transformRules, - OperatorID schemaOperatorID, - String timezone) { + private PostTransformOperator(List transformRules, String timezone) { this.transformRules = transformRules; - this.schemaOperatorID = schemaOperatorID; this.timezone = timezone; - this.tableInfoMap = new ConcurrentHashMap<>(); + this.postTransformChangeInfoMap = new ConcurrentHashMap<>(); this.transformFilterProcessorMap = new ConcurrentHashMap<>(); this.transformProjectionProcessorMap = new ConcurrentHashMap<>(); } - @Override - public void setup( - StreamTask containingTask, - StreamConfig config, - Output> output) { - super.setup(containingTask, config, output); - schemaEvolutionClient = - new SchemaEvolutionClient( - containingTask.getEnvironment().getOperatorCoordinatorEventGateway(), - schemaOperatorID); - } - @Override public void open() throws Exception { super.open(); @@ -142,24 +135,22 @@ public class TransformDataOperator extends AbstractStreamOperator 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 PostTransformer( 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 @@ -174,16 +165,15 @@ public class TransformDataOperator extends AbstractStreamOperator clearOperator(); } - @Override - public void initializeState(StateInitializationContext context) throws Exception { - schemaEvolutionClient.registerSubtask(getRuntimeContext().getIndexOfThisSubtask()); - } - @Override public void processElement(StreamRecord element) throws Exception { Event event = element.getValue(); if (event instanceof SchemaChangeEvent) { - event = cacheSchema((SchemaChangeEvent) event); + SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; + transformProjectionProcessorMap + .keySet() + .removeIf(e -> Objects.equals(e.f0, schemaChangeEvent.tableId())); + event = cacheSchema(schemaChangeEvent); output.collect(new StreamRecord<>(event)); } else if (event instanceof DataChangeEvent) { Optional dataChangeEventOptional = @@ -196,95 +186,78 @@ public class TransformDataOperator extends AbstractStreamOperator 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); + getPostTransformChangeInfo(tableId).getPreTransformedSchema(), event); + } + + Schema projectedSchema = transformSchema(tableId, schema); + postTransformChangeInfoMap.put( + tableId, PostTransformChangeInfo.of(tableId, projectedSchema, schema)); + + if (event instanceof CreateTableEvent) { + return new CreateTableEvent(event.tableId(), projectedSchema); } - transformSchema(tableId, newSchema); - tableInfoMap.put(tableId, TableInfo.of(tableId, newSchema)); return event; } - private TableInfo getTableInfoFromSchemaEvolutionClient(TableId tableId) throws Exception { - TableInfo tableInfo = tableInfoMap.get(tableId); + private PostTransformChangeInfo getPostTransformChangeInfo(TableId tableId) { + PostTransformChangeInfo tableInfo = postTransformChangeInfoMap.get(tableId); if (tableInfo == null) { - Optional schemaOptional = - schemaEvolutionClient.getLatestOriginalSchema(tableId); - if (schemaOptional.isPresent()) { - tableInfo = TableInfo.of(tableId, schemaOptional.get()); - } else { - throw new RuntimeException( - "Could not find schema message from SchemaRegistry for " + tableId); - } + throw new RuntimeException( + "Schema for " + tableId + " not found. This shouldn't happen."); } 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 (PostTransformer transform : transforms) { + Selectors selectors = transform.getSelectors(); + if (selectors.isMatch(tableId) && transform.getProjection().isPresent()) { + TransformProjection transformProjection = transform.getProjection().get(); if (transformProjection.isValid()) { if (!transformProjectionProcessorMap.containsKey( Tuple2.of(tableId, transformProjection))) { transformProjectionProcessorMap.put( Tuple2.of(tableId, transformProjection), - TransformProjectionProcessor.of(transformProjection)); + TransformProjectionProcessor.of(transformProjection, timezone)); } - TransformProjectionProcessor transformProjectionProcessor = + TransformProjectionProcessor 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.inferWiderSchema(newSchemas); } private Optional processDataChangeEvent(DataChangeEvent dataChangeEvent) throws Exception { TableId tableId = dataChangeEvent.tableId(); + PostTransformChangeInfo tableInfo = getPostTransformChangeInfo(tableId); List> transformedDataChangeEventOptionalList = new ArrayList<>(); long epochTime = System.currentTimeMillis(); - for (Tuple4, Optional, Boolean> - transform : transforms) { - Selectors selectors = transform.f0; - Boolean isPreProjection = transform.f3; + for (PostTransformer 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(); @@ -292,10 +265,7 @@ public class TransformDataOperator extends AbstractStreamOperator 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)); @@ -305,8 +275,7 @@ public class TransformDataOperator extends AbstractStreamOperator dataChangeEventOptional.get(), epochTime); } - if (!isPreProjection - && dataChangeEventOptional.isPresent() + if (dataChangeEventOptional.isPresent() && transformProjectionOptional.isPresent() && transformProjectionOptional.get().isValid()) { TransformProjection transformProjection = transformProjectionOptional.get(); @@ -318,29 +287,28 @@ public class TransformDataOperator extends AbstractStreamOperator transformProjectionProcessorMap.put( Tuple2.of(tableId, transformProjection), TransformProjectionProcessor.of( - getTableInfoFromSchemaEvolutionClient(tableId), - transformProjection, - timezone)); + tableInfo, transformProjection, timezone)); } - TransformProjectionProcessor transformProjectionProcessor = + TransformProjectionProcessor 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(); @@ -372,39 +340,50 @@ public class TransformDataOperator extends AbstractStreamOperator } private Optional processProjection( - TransformProjectionProcessor transformProjectionProcessor, + TransformProjectionProcessor 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( + PostTransformChangeInfo 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( + PostTransformChangeInfo tableInfo, BinaryRecordData recordData) { + List valueList = new ArrayList<>(); + RecordData.FieldGetter[] fieldGetters = tableInfo.getPostTransformedFieldGetters(); + + 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/PostTransformer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformer.java new file mode 100644 index 000000000..33d4395c1 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformer.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; + +/** Post-Transformation rule used by {@link PostTransformOperator}. */ +public class PostTransformer { + private final Selectors selectors; + + private final Optional projection; + private final Optional filter; + + public PostTransformer( + 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/TableChangeInfo.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformChangeInfo.java similarity index 63% rename from flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TableChangeInfo.java rename to flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformChangeInfo.java index ca9781eb1..850eb4cc4 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TableChangeInfo.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformChangeInfo.java @@ -36,27 +36,31 @@ import java.io.DataOutputStream; import java.io.IOException; import java.util.List; -/** The TableInfo applies to cache schema change and fieldGetters. */ -public class TableChangeInfo { +/** + * PreTransformChangeInfo caches source / pre-transformed schema, source schema field getters, and + * binary record data generator for pre-transform schema. + */ +public class PreTransformChangeInfo { private TableId tableId; - private Schema originalSchema; - private Schema transformedSchema; - private RecordData.FieldGetter[] fieldGetters; - private BinaryRecordDataGenerator recordDataGenerator; + private Schema sourceSchema; + private Schema preTransformedSchema; + private RecordData.FieldGetter[] sourceFieldGetters; + private BinaryRecordDataGenerator preTransformedRecordDataGenerator; - public static final TableChangeInfo.Serializer SERIALIZER = new TableChangeInfo.Serializer(); + public static final PreTransformChangeInfo.Serializer SERIALIZER = + new PreTransformChangeInfo.Serializer(); - public TableChangeInfo( + public PreTransformChangeInfo( TableId tableId, - Schema originalSchema, - Schema transformedSchema, - RecordData.FieldGetter[] fieldGetters, - BinaryRecordDataGenerator recordDataGenerator) { + Schema sourceSchema, + Schema preTransformedSchema, + RecordData.FieldGetter[] sourceFieldGetters, + BinaryRecordDataGenerator preTransformedRecordDataGenerator) { this.tableId = tableId; - this.originalSchema = originalSchema; - this.transformedSchema = transformedSchema; - this.fieldGetters = fieldGetters; - this.recordDataGenerator = recordDataGenerator; + this.sourceSchema = sourceSchema; + this.preTransformedSchema = preTransformedSchema; + this.sourceFieldGetters = sourceFieldGetters; + this.preTransformedRecordDataGenerator = preTransformedRecordDataGenerator; } public String getName() { @@ -75,39 +79,39 @@ public class TableChangeInfo { return tableId; } - public Schema getOriginalSchema() { - return originalSchema; + public Schema getSourceSchema() { + return sourceSchema; } - public Schema getTransformedSchema() { - return transformedSchema; + public Schema getPreTransformedSchema() { + return preTransformedSchema; } - public RecordData.FieldGetter[] getFieldGetters() { - return fieldGetters; + public RecordData.FieldGetter[] getSourceFieldGetters() { + return sourceFieldGetters; } - public BinaryRecordDataGenerator getRecordDataGenerator() { - return recordDataGenerator; + public BinaryRecordDataGenerator getPreTransformedRecordDataGenerator() { + return preTransformedRecordDataGenerator; } - public static TableChangeInfo of( - TableId tableId, Schema originalSchema, Schema transformedSchema) { - List fieldGetters = - SchemaUtils.createFieldGetters(originalSchema.getColumns()); - BinaryRecordDataGenerator recordDataGenerator = + public static PreTransformChangeInfo of( + TableId tableId, Schema sourceSchema, Schema preTransformedSchema) { + List sourceFieldGetters = + SchemaUtils.createFieldGetters(sourceSchema.getColumns()); + BinaryRecordDataGenerator preTransformedDataGenerator = new BinaryRecordDataGenerator( - DataTypeConverter.toRowType(transformedSchema.getColumns())); - return new TableChangeInfo( + DataTypeConverter.toRowType(preTransformedSchema.getColumns())); + return new PreTransformChangeInfo( tableId, - originalSchema, - transformedSchema, - fieldGetters.toArray(new RecordData.FieldGetter[0]), - recordDataGenerator); + sourceSchema, + preTransformedSchema, + sourceFieldGetters.toArray(new RecordData.FieldGetter[0]), + preTransformedDataGenerator); } - /** Serializer for {@link TableChangeInfo}. */ - public static class Serializer implements SimpleVersionedSerializer { + /** Serializer for {@link PreTransformChangeInfo}. */ + public static class Serializer implements SimpleVersionedSerializer { /** The latest version before change of state compatibility. */ public static final int VERSION_BEFORE_STATE_COMPATIBILITY = 1; @@ -123,7 +127,7 @@ public class TableChangeInfo { } @Override - public byte[] serialize(TableChangeInfo tableChangeInfo) throws IOException { + public byte[] serialize(PreTransformChangeInfo tableChangeInfo) throws IOException { TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE; SchemaSerializer schemaSerializer = SchemaSerializer.INSTANCE; try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); @@ -133,15 +137,16 @@ public class TableChangeInfo { tableIdSerializer.serialize( tableChangeInfo.getTableId(), new DataOutputViewStreamWrapper(out)); schemaSerializer.serialize( - tableChangeInfo.originalSchema, new DataOutputViewStreamWrapper(out)); + tableChangeInfo.sourceSchema, new DataOutputViewStreamWrapper(out)); schemaSerializer.serialize( - tableChangeInfo.transformedSchema, new DataOutputViewStreamWrapper(out)); + tableChangeInfo.preTransformedSchema, new DataOutputViewStreamWrapper(out)); return baos.toByteArray(); } } @Override - public TableChangeInfo deserialize(int version, byte[] serialized) throws IOException { + public PreTransformChangeInfo deserialize(int version, byte[] serialized) + throws IOException { TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE; SchemaSerializer schemaSerializer = SchemaSerializer.INSTANCE; try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); @@ -157,7 +162,7 @@ public class TableChangeInfo { schemaSerializer.deserialize(version, new DataInputViewStreamWrapper(in)); Schema transformedSchema = schemaSerializer.deserialize(version, new DataInputViewStreamWrapper(in)); - return TableChangeInfo.of(tableId, originalSchema, transformedSchema); + return PreTransformChangeInfo.of(tableId, originalSchema, transformedSchema); } } } 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 55% 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 ca8949ed6..7dbba91d1 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.ListState; 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; @@ -41,54 +40,69 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import javax.annotation.Nullable; import java.io.IOException; +import java.io.Serializable; 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 - implements OneInputStreamOperator { +/** + * A data process function that filters out columns which aren't (directly & indirectly) referenced. + */ +public class PreTransformOperator extends AbstractStreamOperator + implements OneInputStreamOperator, Serializable { + + private static final long serialVersionUID = 1L; - private final List> transformRules; - private transient List>> transforms; - private final Map tableChangeInfoMap; - private transient Map processorMap; + private final List transformRules; + private transient List transforms; + private final Map preTransformChangeInfoMap; 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.preTransformChangeInfoMap = new ConcurrentHashMap<>(); + this.preTransformProcessorMap = new ConcurrentHashMap<>(); this.schemaMetadataTransformers = new ArrayList<>(); this.chainingStrategy = ChainingStrategy.ALWAYS; } @@ -97,21 +111,26 @@ public class TransformSchemaOperator extends AbstractStreamOperator 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 PreTransformer( + 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 @@ -123,10 +142,19 @@ public class TransformSchemaOperator extends AbstractStreamOperator state = stateStore.getUnionListState(descriptor); if (context.isRestored()) { for (byte[] serializedTableInfo : state.get()) { - TableChangeInfo stateTableChangeInfo = - TableChangeInfo.SERIALIZER.deserialize( - TableChangeInfo.SERIALIZER.getVersion(), serializedTableInfo); - tableChangeInfoMap.put(stateTableChangeInfo.getTableId(), stateTableChangeInfo); + PreTransformChangeInfo stateTableChangeInfo = + PreTransformChangeInfo.SERIALIZER.deserialize( + PreTransformChangeInfo.SERIALIZER.getVersion(), + serializedTableInfo); + preTransformChangeInfoMap.put( + stateTableChangeInfo.getTableId(), stateTableChangeInfo); + // Since PostTransformOperator doesn't preserve state, pre-transformed schema + // information needs to be passed by PreTransformOperator. + output.collect( + new StreamRecord<>( + new CreateTableEvent( + stateTableChangeInfo.getTableId(), + stateTableChangeInfo.getPreTransformedSchema()))); } } } @@ -136,11 +164,11 @@ public class TransformSchemaOperator extends AbstractStreamOperator super.snapshotState(context); state.update( new ArrayList<>( - tableChangeInfoMap.values().stream() + preTransformChangeInfoMap.values().stream() .map( tableChangeInfo -> { try { - return TableChangeInfo.SERIALIZER.serialize( + return PreTransformChangeInfo.SERIALIZER.serialize( tableChangeInfo); } catch (IOException e) { throw new RuntimeException(e); @@ -166,13 +194,18 @@ public class TransformSchemaOperator extends AbstractStreamOperator public void processElement(StreamRecord element) throws Exception { Event event = element.getValue(); if (event instanceof CreateTableEvent) { - event = cacheCreateTable((CreateTableEvent) event); + CreateTableEvent createTableEvent = (CreateTableEvent) event; + preTransformProcessorMap.remove(createTableEvent.tableId()); + event = cacheCreateTable(createTableEvent); output.collect(new StreamRecord<>(event)); } else if (event instanceof SchemaChangeEvent) { - event = cacheChangeSchema((SchemaChangeEvent) event); + SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; + preTransformProcessorMap.remove(schemaChangeEvent.tableId()); + event = cacheChangeSchema(schemaChangeEvent); output.collect(new StreamRecord<>(event)); } else if (event instanceof DataChangeEvent) { - output.collect(new StreamRecord<>(processDataChangeEvent(((DataChangeEvent) event)))); + DataChangeEvent dataChangeEvent = processDataChangeEvent(((DataChangeEvent) event)); + output.collect(new StreamRecord<>(dataChangeEvent)); } } @@ -181,23 +214,27 @@ public class TransformSchemaOperator extends AbstractStreamOperator Schema originalSchema = event.getSchema(); event = transformCreateTableEvent(event); Schema newSchema = (event).getSchema(); - tableChangeInfoMap.put(tableId, TableChangeInfo.of(tableId, originalSchema, newSchema)); + preTransformChangeInfoMap.put( + tableId, PreTransformChangeInfo.of(tableId, originalSchema, newSchema)); return event; } private SchemaChangeEvent cacheChangeSchema(SchemaChangeEvent event) { TableId tableId = event.tableId(); - TableChangeInfo tableChangeInfo = tableChangeInfoMap.get(tableId); + PreTransformChangeInfo tableChangeInfo = preTransformChangeInfoMap.get(tableId); Schema originalSchema = - SchemaUtils.applySchemaChangeEvent(tableChangeInfo.getOriginalSchema(), event); + SchemaUtils.applySchemaChangeEvent(tableChangeInfo.getSourceSchema(), event); Schema newSchema = - SchemaUtils.applySchemaChangeEvent(tableChangeInfo.getTransformedSchema(), event); - tableChangeInfoMap.put(tableId, TableChangeInfo.of(tableId, originalSchema, newSchema)); + SchemaUtils.applySchemaChangeEvent( + tableChangeInfo.getPreTransformedSchema(), event); + preTransformChangeInfoMap.put( + tableId, PreTransformChangeInfo.of(tableId, originalSchema, newSchema)); return event; } private CreateTableEvent transformCreateTableEvent(CreateTableEvent createTableEvent) { TableId tableId = createTableEvent.tableId(); + PreTransformChangeInfo tableChangeInfo = preTransformChangeInfoMap.get(tableId); for (Tuple2 transform : schemaMetadataTransformers) { Selectors selectors = transform.f0; @@ -210,19 +247,22 @@ public class TransformSchemaOperator extends AbstractStreamOperator } } - for (Tuple2> transform : transforms) { - Selectors selectors = transform.f0; - if (selectors.isMatch(tableId) && transform.f1.isPresent()) { - TransformProjection transformProjection = transform.f1.get(); + for (PreTransformer 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); } } } @@ -253,12 +293,14 @@ public class TransformSchemaOperator extends AbstractStreamOperator 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 (PreTransformer 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); } } } @@ -266,25 +308,27 @@ public class TransformSchemaOperator extends AbstractStreamOperator } 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)); + PreTransformChangeInfo tableChangeInfo = preTransformChangeInfoMap.get(tableId); + 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; @@ -292,6 +336,6 @@ public class TransformSchemaOperator extends AbstractStreamOperator private void clearOperator() { this.transforms = null; - this.processorMap = null; + this.preTransformProcessorMap = 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 000000000..1ede3ce77 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformProcessor.java @@ -0,0 +1,138 @@ +/* + * 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 PreTransformChangeInfo tableChangeInfo; + private TransformProjection transformProjection; + private @Nullable TransformFilter transformFilter; + private List cachedProjectionColumnsState; + + public PreTransformProcessor( + PreTransformChangeInfo tableChangeInfo, + TransformProjection transformProjection, + @Nullable TransformFilter transformFilter) { + this.tableChangeInfo = tableChangeInfo; + this.transformProjection = transformProjection; + this.transformFilter = transformFilter; + this.cachedProjectionColumnsState = + cacheIsProjectionColumnMap(tableChangeInfo, transformProjection); + } + + public boolean hasTableChangeInfo() { + return this.tableChangeInfo != null; + } + + /** + * This method analyses (directly and indirectly) referenced columns, and peels unused columns + * from schema. For example, given original schema with columns (A, B, C, D, E) with projection + * rule (A, B + 1 as newB) and filtering rule (C > 0), a peeled schema containing (A, B, C) only + * will be sent to downstream, and (D, E) column along with corresponding data will be trimmed. + */ + 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<>(); + List columns = tableChangeInfo.getPreTransformedSchema().getColumns(); + + for (int i = 0; i < columns.size(); i++) { + if (cachedProjectionColumnsState.get(i)) { + valueList.add(null); + } else { + valueList.add( + getValueFromBinaryRecordData( + columns.get(i).getName(), + data, + tableChangeInfo.getSourceSchema().getColumns(), + tableChangeInfo.getSourceFieldGetters())); + } + } + + return tableChangeInfo + .getPreTransformedRecordDataGenerator() + .generate(valueList.toArray(new Object[0])); + } + + 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; + } + + private List cacheIsProjectionColumnMap( + PreTransformChangeInfo tableChangeInfo, TransformProjection transformProjection) { + List cachedMap = new ArrayList<>(); + if (!hasTableChangeInfo()) { + return cachedMap; + } + + for (Column column : tableChangeInfo.getPreTransformedSchema().getColumns()) { + boolean isProjectionColumn = false; + for (ProjectionColumn projectionColumn : transformProjection.getProjectionColumns()) { + if (column.getName().equals(projectionColumn.getColumnName()) + && projectionColumn.isValidTransformedProjectionColumn()) { + isProjectionColumn = true; + break; + } + } + cachedMap.add(isProjectionColumn); + } + + return cachedMap; + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformer.java new file mode 100644 index 000000000..305d7f387 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformer.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; + +/** Pre-Transformation rule used by {@link PreTransformOperator}. */ +public class PreTransformer { + private final Selectors selectors; + + private final Optional projection; + private final Optional filter; + + public PreTransformer( + 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 500e36c99..4bbe3d558 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.types.DataType; 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 class ProjectionColumn implements Serializable { 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 class ProjectionColumn implements Serializable { 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 6c5202342..0d7df06ab 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 @@ -39,13 +39,13 @@ import java.util.List; public class ProjectionColumnProcessor { private static final Logger LOG = LoggerFactory.getLogger(ProjectionColumnProcessor.class); - private TableInfo tableInfo; + private PostTransformChangeInfo tableInfo; private ProjectionColumn projectionColumn; private String timezone; private TransformExpressionKey transformExpressionKey; public ProjectionColumnProcessor( - TableInfo tableInfo, ProjectionColumn projectionColumn, String timezone) { + PostTransformChangeInfo tableInfo, ProjectionColumn projectionColumn, String timezone) { this.tableInfo = tableInfo; this.projectionColumn = projectionColumn; this.timezone = timezone; @@ -53,10 +53,14 @@ public class ProjectionColumnProcessor { } public static ProjectionColumnProcessor of( - TableInfo tableInfo, ProjectionColumn projectionColumn, String timezone) { + PostTransformChangeInfo tableInfo, ProjectionColumn projectionColumn, String timezone) { return new ProjectionColumnProcessor(tableInfo, projectionColumn, timezone); } + public ProjectionColumn getProjectionColumn() { + return projectionColumn; + } + public Object evaluate(BinaryRecordData after, long epochTime) { ExpressionEvaluator expressionEvaluator = TransformExpressionCompiler.compileExpression(transformExpressionKey); @@ -75,30 +79,36 @@ public class ProjectionColumnProcessor { private Object[] generateParams(BinaryRecordData after, long epochTime) { List params = new ArrayList<>(); - List columns = tableInfo.getSchema().getColumns(); - RecordData.FieldGetter[] fieldGetters = tableInfo.getFieldGetters(); + List columns = tableInfo.getPreTransformedSchema().getColumns(); + RecordData.FieldGetter[] fieldGetters = tableInfo.getPreTransformedFieldGetters(); 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 +118,11 @@ public class ProjectionColumnProcessor { private TransformExpressionKey generateTransformExpressionKey() { List argumentNames = new ArrayList<>(); List> paramTypes = new ArrayList<>(); - List columns = tableInfo.getSchema().getColumns(); + List columns = tableInfo.getPreTransformedSchema().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 deleted file mode 100644 index 93bfcafec..000000000 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TableInfo.java +++ /dev/null @@ -1,90 +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.RecordData; -import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.utils.SchemaUtils; -import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; -import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter; - -import java.util.List; - -/** The TableInfo applies to cache schema and fieldGetters. */ -public class TableInfo { - private TableId tableId; - private Schema schema; - private RecordData.FieldGetter[] fieldGetters; - private BinaryRecordDataGenerator recordDataGenerator; - - public TableInfo( - TableId tableId, - Schema schema, - RecordData.FieldGetter[] fieldGetters, - BinaryRecordDataGenerator recordDataGenerator) { - this.tableId = tableId; - this.schema = schema; - this.fieldGetters = fieldGetters; - this.recordDataGenerator = recordDataGenerator; - } - - public String getName() { - return tableId.identifier(); - } - - public String getTableName() { - return tableId.getTableName(); - } - - public String getSchemaName() { - return tableId.getSchemaName(); - } - - public String getNamespace() { - return tableId.getNamespace(); - } - - public TableId getTableId() { - return tableId; - } - - public Schema getSchema() { - return schema; - } - - public RecordData.FieldGetter[] getFieldGetters() { - return fieldGetters; - } - - 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())); - return new TableInfo( - tableId, - schema, - fieldGetters.toArray(new RecordData.FieldGetter[0]), - recordDataGenerator); - } -} 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 7eaefc3c8..9bae6a573 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 @@ -17,11 +17,11 @@ package org.apache.flink.cdc.runtime.operators.transform; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.runtime.parser.JaninoCompiler; -import org.apache.flink.cdc.runtime.parser.TransformParser; import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter; import org.codehaus.janino.ExpressionEvaluator; @@ -31,17 +31,22 @@ import org.slf4j.LoggerFactory; import java.lang.reflect.InvocationTargetException; import java.util.ArrayList; import java.util.List; +import java.util.stream.Stream; + +import static org.apache.flink.cdc.runtime.parser.TransformParser.DEFAULT_NAMESPACE_NAME; +import static org.apache.flink.cdc.runtime.parser.TransformParser.DEFAULT_SCHEMA_NAME; +import static org.apache.flink.cdc.runtime.parser.TransformParser.DEFAULT_TABLE_NAME; /** The processor of the transform filter. It processes the data change event of matched table. */ public class TransformFilterProcessor { private static final Logger LOG = LoggerFactory.getLogger(TransformFilterProcessor.class); - private TableInfo tableInfo; + private PostTransformChangeInfo tableInfo; private TransformFilter transformFilter; private String timezone; private TransformExpressionKey transformExpressionKey; public TransformFilterProcessor( - TableInfo tableInfo, TransformFilter transformFilter, String timezone) { + PostTransformChangeInfo tableInfo, TransformFilter transformFilter, String timezone) { this.tableInfo = tableInfo; this.transformFilter = transformFilter; this.timezone = timezone; @@ -49,7 +54,7 @@ public class TransformFilterProcessor { } public static TransformFilterProcessor of( - TableInfo tableInfo, TransformFilter transformFilter, String timezone) { + PostTransformChangeInfo tableInfo, TransformFilter transformFilter, String timezone) { return new TransformFilterProcessor(tableInfo, transformFilter, timezone); } @@ -68,22 +73,52 @@ public class TransformFilterProcessor { } } + private Tuple2, List>> generateArguments() { + List argNames = new ArrayList<>(); + List> argTypes = new ArrayList<>(); + String scriptExpression = transformFilter.getScriptExpression(); + List columns = tableInfo.getPreTransformedSchema().getColumns(); + List columnNames = transformFilter.getColumnNames(); + for (String columnName : columnNames) { + for (Column column : columns) { + if (column.getName().equals(columnName)) { + if (!argNames.contains(columnName)) { + argNames.add(columnName); + argTypes.add(DataTypeConverter.convertOriginalClass(column.getType())); + } + break; + } + } + } + Stream.of(DEFAULT_NAMESPACE_NAME, DEFAULT_SCHEMA_NAME, DEFAULT_TABLE_NAME) + .forEach( + metadataColumn -> { + if (scriptExpression.contains(metadataColumn) + && !argNames.contains(metadataColumn)) { + argNames.add(metadataColumn); + argTypes.add(String.class); + } + }); + return Tuple2.of(argNames, argTypes); + } + private Object[] generateParams(BinaryRecordData after, long epochTime) { List params = new ArrayList<>(); - List columns = tableInfo.getSchema().getColumns(); - RecordData.FieldGetter[] fieldGetters = tableInfo.getFieldGetters(); - for (String columnName : transformFilter.getColumnNames()) { - if (columnName.equals(TransformParser.DEFAULT_NAMESPACE_NAME)) { - params.add(tableInfo.getNamespace()); - continue; - } - if (columnName.equals(TransformParser.DEFAULT_SCHEMA_NAME)) { - params.add(tableInfo.getSchemaName()); - continue; - } - if (columnName.equals(TransformParser.DEFAULT_TABLE_NAME)) { - params.add(tableInfo.getTableName()); - continue; + List columns = tableInfo.getPreTransformedSchema().getColumns(); + + Tuple2, List>> args = generateArguments(); + RecordData.FieldGetter[] fieldGetters = tableInfo.getPreTransformedFieldGetters(); + for (String columnName : args.f0) { + switch (columnName) { + case DEFAULT_NAMESPACE_NAME: + params.add(tableInfo.getNamespace()); + continue; + case DEFAULT_SCHEMA_NAME: + params.add(tableInfo.getSchemaName()); + continue; + case DEFAULT_TABLE_NAME: + params.add(tableInfo.getTableName()); + continue; } for (int i = 0; i < columns.size(); i++) { Column column = columns.get(i); @@ -101,48 +136,17 @@ public class TransformFilterProcessor { } private TransformExpressionKey generateTransformExpressionKey() { - List argumentNames = new ArrayList<>(); - List> paramTypes = new ArrayList<>(); - List columns = tableInfo.getSchema().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); - if (column.getName().equals(columnName)) { - argumentNames.add(columnName); - paramTypes.add(DataTypeConverter.convertOriginalClass(column.getType())); - break; - } - } - } - if (scriptExpression.contains(TransformParser.DEFAULT_NAMESPACE_NAME) - && !argumentNames.contains(TransformParser.DEFAULT_NAMESPACE_NAME)) { - argumentNames.add(TransformParser.DEFAULT_NAMESPACE_NAME); - paramTypes.add(String.class); - } - - if (scriptExpression.contains(TransformParser.DEFAULT_SCHEMA_NAME) - && !argumentNames.contains(TransformParser.DEFAULT_SCHEMA_NAME)) { - argumentNames.add(TransformParser.DEFAULT_SCHEMA_NAME); - paramTypes.add(String.class); - } - - if (scriptExpression.contains(TransformParser.DEFAULT_TABLE_NAME) - && !argumentNames.contains(TransformParser.DEFAULT_TABLE_NAME)) { - argumentNames.add(TransformParser.DEFAULT_TABLE_NAME); - paramTypes.add(String.class); - } + Tuple2, List>> args = generateArguments(); - argumentNames.add(JaninoCompiler.DEFAULT_TIME_ZONE); - paramTypes.add(String.class); - argumentNames.add(JaninoCompiler.DEFAULT_EPOCH_TIME); - paramTypes.add(Long.class); + args.f0.add(JaninoCompiler.DEFAULT_TIME_ZONE); + args.f1.add(String.class); + args.f0.add(JaninoCompiler.DEFAULT_EPOCH_TIME); + args.f1.add(Long.class); return TransformExpressionKey.of( - JaninoCompiler.loadSystemFunction(scriptExpression), - argumentNames, - paramTypes, + JaninoCompiler.loadSystemFunction(transformFilter.getScriptExpression()), + args.f0, + args.f1, Boolean.class); } } 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/TransformProjectionProcessor.java index 7049bbdfd..307c890fd 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/TransformProjectionProcessor.java @@ -19,9 +19,9 @@ 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.common.types.DataType; import org.apache.flink.cdc.runtime.parser.TransformParser; import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter; @@ -30,8 +30,8 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; +import java.util.Optional; +import java.util.stream.Collectors; /** * The processor of transform projection applies to process a row of filtering tables. @@ -41,146 +41,127 @@ import java.util.concurrent.ConcurrentHashMap; *
    *
  • 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); - private TableInfo tableInfo; - private TableChangeInfo tableChangeInfo; - private TransformProjection transformProjection; - private String timezone; - private Map projectionColumnProcessorMap; + private final PostTransformChangeInfo postTransformChangeInfo; + private final TransformProjection transformProjection; + private final String timezone; + private final List cachedProjectionColumnProcessors; public TransformProjectionProcessor( - TableInfo tableInfo, - TableChangeInfo tableChangeInfo, + PostTransformChangeInfo postTransformChangeInfo, TransformProjection transformProjection, String timezone) { - this.tableInfo = tableInfo; - this.tableChangeInfo = tableChangeInfo; + this.postTransformChangeInfo = postTransformChangeInfo; this.transformProjection = transformProjection; this.timezone = timezone; - this.projectionColumnProcessorMap = new ConcurrentHashMap<>(); - } - - public boolean hasTableChangeInfo() { - return this.tableChangeInfo != null; + this.cachedProjectionColumnProcessors = + cacheProjectionColumnProcessors(postTransformChangeInfo, transformProjection); } public boolean hasTableInfo() { - return this.tableInfo != null; + return this.postTransformChangeInfo != null; } public static TransformProjectionProcessor of( - TableInfo tableInfo, TransformProjection transformProjection, String timezone) { - return new TransformProjectionProcessor(tableInfo, null, transformProjection, timezone); + PostTransformChangeInfo tableInfo, + TransformProjection transformProjection, + String timezone) { + return new TransformProjectionProcessor(tableInfo, transformProjection, timezone); } public static TransformProjectionProcessor of( - TableChangeInfo tableChangeInfo, TransformProjection transformProjection) { - return new TransformProjectionProcessor(null, tableChangeInfo, transformProjection, null); + TransformProjection transformProjection, String timezone) { + return new TransformProjectionProcessor(null, transformProjection, timezone); } public static TransformProjectionProcessor of(TransformProjection transformProjection) { - return new TransformProjectionProcessor(null, null, transformProjection, 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); + return new TransformProjectionProcessor(null, transformProjection, null); } - 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 payload, long epochTime) { 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()])); - } + List columns = postTransformChangeInfo.getPostTransformedSchema().getColumns(); - public BinaryRecordData processData(BinaryRecordData after, long epochTime) { - List valueList = new ArrayList<>(); - for (Column column : tableInfo.getSchema().getColumns()) { - boolean isProjectionColumn = false; - for (ProjectionColumn projectionColumn : transformProjection.getProjectionColumns()) { - if (column.getName().equals(projectionColumn.getColumnName()) - && projectionColumn.isValidTransformedProjectionColumn()) { - 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; - } - } - if (!isProjectionColumn) { + for (int i = 0; i < columns.size(); i++) { + ProjectionColumnProcessor projectionColumnProcessor = + cachedProjectionColumnProcessors.get(i); + if (projectionColumnProcessor != null) { + ProjectionColumn projectionColumn = projectionColumnProcessor.getProjectionColumn(); + valueList.add( + DataTypeConverter.convert( + projectionColumnProcessor.evaluate(payload, epochTime), + projectionColumn.getDataType())); + } else { + Column column = columns.get(i); valueList.add( getValueFromBinaryRecordData( column.getName(), - after, - tableInfo.getSchema().getColumns(), - tableInfo.getFieldGetters())); + column.getType(), + payload, + postTransformChangeInfo.getPreTransformedSchema().getColumns(), + postTransformChangeInfo.getPreTransformedFieldGetters())); } } - return tableInfo + + return postTransformChangeInfo .getRecordDataGenerator() - .generate(valueList.toArray(new Object[valueList.size()])); + .generate(valueList.toArray(new Object[0])); } 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; } + + private List cacheProjectionColumnProcessors( + PostTransformChangeInfo tableInfo, TransformProjection transformProjection) { + List cachedProjectionColumnProcessors = new ArrayList<>(); + if (!hasTableInfo()) { + return cachedProjectionColumnProcessors; + } + + for (Column column : tableInfo.getPostTransformedSchema().getColumns()) { + ProjectionColumn matchedProjectionColumn = null; + for (ProjectionColumn projectionColumn : transformProjection.getProjectionColumns()) { + if (column.getName().equals(projectionColumn.getColumnName()) + && projectionColumn.isValidTransformedProjectionColumn()) { + matchedProjectionColumn = projectionColumn; + break; + } + } + + cachedProjectionColumnProcessors.add( + Optional.ofNullable(matchedProjectionColumn) + .map(col -> ProjectionColumnProcessor.of(tableInfo, col, timezone)) + .orElse(null)); + } + + return cachedProjectionColumnProcessors; + } } 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 000000000..27b265226 --- /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 23cf4b376..4db3f354f 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.api.common.io.ParseException; 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.RelDataTypeField; 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.apache.calcite.sql2rel.StandardConvertletTable; 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 class TransformParser { } } - // 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 class TransformParser { 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 class TransformParser { 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 class TransformParser { 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 class TransformParser { } public static String translateFilterExpressionToJaninoExpression(String filterExpression) { - if (StringUtils.isNullOrWhitespaceOnly(filterExpression)) { + if (isNullOrWhitespaceOnly(filterExpression)) { return ""; } SqlSelect sqlSelect = TransformParser.parseFilterExpression(filterExpression); @@ -257,7 +354,7 @@ public class TransformParser { 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 class TransformParser { } public static List parseFilterColumnNameList(String filterExpression) { - if (StringUtils.isNullOrWhitespaceOnly(filterExpression)) { + if (isNullOrWhitespaceOnly(filterExpression)) { return new ArrayList<>(); } SqlSelect sqlSelect = parseFilterExpression(filterExpression); @@ -315,12 +412,12 @@ public class TransformParser { 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 @@ public class TransformParser { 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 class TransformParser { 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/main/java/org/apache/flink/cdc/runtime/partitioning/PrePartitionOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/PrePartitionOperator.java index 13ddbb6b4..6d66fa878 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/PrePartitionOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/PrePartitionOperator.java @@ -39,13 +39,14 @@ import org.apache.flink.shaded.guava31.com.google.common.cache.CacheBuilder; import org.apache.flink.shaded.guava31.com.google.common.cache.CacheLoader; import org.apache.flink.shaded.guava31.com.google.common.cache.LoadingCache; +import java.io.Serializable; import java.time.Duration; import java.util.Optional; /** Operator for processing events from {@link SchemaOperator} before {@link EventPartitioner}. */ @Internal public class PrePartitionOperator extends AbstractStreamOperator - implements OneInputStreamOperator { + implements OneInputStreamOperator, Serializable { private static final long serialVersionUID = 1L; private static final Duration CACHE_EXPIRE_DURATION = Duration.ofDays(1); 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 83% 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 a19722ef1..34b710374 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.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.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(); @@ -126,8 +132,8 @@ public class TransformDataOperatorTest { .physicalColumn("nullBigint", DataTypes.BIGINT()) .physicalColumn("nullFloat", DataTypes.FLOAT()) .physicalColumn("nullDouble", DataTypes.DOUBLE()) - .physicalColumn("nullChar", DataTypes.CHAR(1)) - .physicalColumn("nullVarchar", DataTypes.VARCHAR(1)) + .physicalColumn("nullChar", DataTypes.STRING()) + .physicalColumn("nullVarchar", DataTypes.STRING()) .physicalColumn("nullDecimal", DataTypes.DECIMAL(4, 2)) .physicalColumn("nullTimestamp", DataTypes.TIMESTAMP(3)) .primaryKey("col1") @@ -145,8 +151,8 @@ public class TransformDataOperatorTest { .physicalColumn("castBigint", DataTypes.BIGINT()) .physicalColumn("castFloat", DataTypes.FLOAT()) .physicalColumn("castDouble", DataTypes.DOUBLE()) - .physicalColumn("castChar", DataTypes.CHAR(1)) - .physicalColumn("castVarchar", DataTypes.VARCHAR(1)) + .physicalColumn("castChar", DataTypes.STRING()) + .physicalColumn("castVarchar", DataTypes.STRING()) .physicalColumn("castDecimal", DataTypes.DECIMAL(4, 2)) .physicalColumn("castTimestamp", DataTypes.TIMESTAMP(3)) .primaryKey("col1") @@ -170,16 +176,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 @@ -252,7 +308,6 @@ public class TransformDataOperatorTest { Assertions.assertThat( transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) .isEqualTo(new StreamRecord<>(insertEventExpect)); - transform.processElement(new StreamRecord<>(insertEventIgnored)); transform.processElement(new StreamRecord<>(updateEvent)); Assertions.assertThat( transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll()) @@ -261,8 +316,8 @@ public class TransformDataOperatorTest { @Test void testDataChangeEventTransformTwice() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( CUSTOMERS_TABLEID.identifier(), "*, concat(col1, '1') col12", @@ -272,7 +327,7 @@ public class TransformDataOperatorTest { "*, concat(col1, '2') col12", "col1 = '2'") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -366,11 +421,11 @@ public class TransformDataOperatorTest { @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 @@ -412,14 +467,14 @@ public class TransformDataOperatorTest { @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 @@ -428,16 +483,17 @@ public class TransformDataOperatorTest { 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"), @@ -450,7 +506,7 @@ public class TransformDataOperatorTest { 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()) @@ -459,14 +515,14 @@ public class TransformDataOperatorTest { @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 @@ -506,8 +562,8 @@ public class TransformDataOperatorTest { @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," @@ -516,7 +572,7 @@ public class TransformDataOperatorTest { "LOCALTIMESTAMP = CAST(CURRENT_TIMESTAMP AS TIMESTAMP)") .addTimezone("UTC") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -551,8 +607,8 @@ public class TransformDataOperatorTest { @Test void testTimestampDiffTransform() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( TIMESTAMPDIFF_TABLEID.identifier(), "col1, TIMESTAMP_DIFF('SECOND', LOCALTIMESTAMP, CAST(CURRENT_TIMESTAMP AS TIMESTAMP)) as second_diff," @@ -569,7 +625,7 @@ public class TransformDataOperatorTest { "col1='2'") .addTimezone("Asia/Shanghai") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -620,15 +676,15 @@ public class TransformDataOperatorTest { @Test void testTimezoneTransform() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( TIMEZONE_TABLEID.identifier(), "col1, DATE_FORMAT(TO_TIMESTAMP('2024-08-01 00:00:00'), 'yyyy-MM-dd HH:mm:ss') as datetime", null) .addTimezone("UTC") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -665,8 +721,8 @@ public class TransformDataOperatorTest { @Test void testNullCastTransform() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( NULL_TABLEID.identifier(), "col1" @@ -684,7 +740,7 @@ public class TransformDataOperatorTest { + ",cast(colString as TIMESTAMP(3)) as nullTimestamp", null) .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -725,8 +781,8 @@ public class TransformDataOperatorTest { @Test void testCastTransform() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( CAST_TABLEID.identifier(), "col1" @@ -878,7 +934,7 @@ public class TransformDataOperatorTest { + ",cast('1970-01-01T00:00:01.234' as TIMESTAMP(3)) as castTimestamp", "col1 = '10'") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -1297,8 +1353,8 @@ public class TransformDataOperatorTest { @Test void testCastErrorTransform() throws Exception { - TransformDataOperator transform = - TransformDataOperator.newBuilder() + PostTransformOperator transform = + PostTransformOperator.newBuilder() .addTransform( CAST_TABLEID.identifier(), "col1" @@ -1312,10 +1368,10 @@ public class TransformDataOperatorTest { + ",cast(castFloat as char) as castChar" + ",cast(castFloat as varchar) as castVarchar" + ",cast(castFloat as DECIMAL(4,2)) as castDecimal" - + ",cast(castFloat as TIMESTAMP) as castTimestamp", + + ",cast(castFloat as TIMESTAMP(3)) as castTimestamp", "col1 = '1'") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -1417,15 +1473,15 @@ public class TransformDataOperatorTest { } 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("UTC") .build(); - EventOperatorTestHarness + EventOperatorTestHarness transformFunctionEventEventOperatorTestHarness = new EventOperatorTestHarness<>(transform, 1); // Initialization @@ -1457,4 +1513,202 @@ public class TransformDataOperatorTest { 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 000000000..c447d51c2 --- /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 82992a64e..000000000 --- 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 000000000..009d700fe --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/UnifiedTransformOperatorTest.java @@ -0,0 +1,985 @@ +/* + * 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(); + } + + @Test + public void testTransformWithCast() throws Exception { + TableId tableId = TableId.tableId("my_company", "my_branch", "transform_with_cast"); + UnifiedTransformTestCase.of( + tableId, + "id, age + 1 as newage, CAST(CAST(id AS INT) + age AS BIGINT) as longevity, CAST(age AS VARCHAR) as string_age", + "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("ref2", DataTypes.INT()) + .primaryKey("id") + .build(), + Schema.newBuilder() + .physicalColumn("id", DataTypes.STRING().notNull()) + .physicalColumn("newage", DataTypes.INT()) + .physicalColumn("longevity", DataTypes.BIGINT()) + .physicalColumn("string_age", DataTypes.STRING()) + .primaryKey("id") + .build()) + .initializeHarness() + .insertSource("1001", "Alice", 17, "Reference001", 2021) + .insertPreTransformed("1001", 17, 2021) + .insertPostTransformed("1001", 18, 1018L, "17") + // this data record is filtered out since newage <= 17 + .insertSource("1002", "Bob", 15, "Reference002", 2017) + .insertPreTransformed("1002", 15, 2017) + .insertPostTransformed() + // this data record is filtered out since ref2 <= 17 + .insertSource("1003", "Bill", 18, "Reference003", 0) + .insertPreTransformed("1003", 18, 0) + .insertPostTransformed() + .insertSource("1004", "Carol", 18, "Reference004", 2018) + .insertPreTransformed("1004", 18, 2018) + .insertPostTransformed("1004", 19, 1022L, "18") + // test update event transform + .updateSource( + new Object[] {"1004", "Carol", 18, "Reference004", 2018}, + new Object[] {"1004", "Colin", 19, "NeoReference004", 2018}) + .updatePreTransformed( + new Object[] {"1004", 18, 2018}, new Object[] {"1004", 19, 2018}) + .updatePostTransformed( + new Object[] {"1004", 19, 1022L, "18"}, + new Object[] {"1004", 20, 1023L, "19"}) + // updated value to a filtered out condition + .updateSource( + new Object[] {"1004", "Colin", 19, "NeoReference004", 2018}, + new Object[] {"1004", "Colin", 10, "NeoReference004", 2018}) + .updatePreTransformed( + new Object[] {"1004", 19, 2018}, new Object[] {"1004", 10, 2018}) + .updatePostTransformed() + .deleteSource("1001", "Alice", 17, "Reference001", 2021) + .deletePreTransformed("1001", 17, 2021) + .deletePostTransformed("1001", 18, 1018L, "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 66a405a97..270e77534 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; @@ -43,9 +46,10 @@ import org.apache.calcite.sql2rel.RelDecorrelator; import org.apache.calcite.sql2rel.SqlToRelConverter; import org.apache.calcite.sql2rel.StandardConvertletTable; import org.apache.calcite.tools.RelBuilder; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -66,9 +70,11 @@ public class TransformParserTest { SqlSelect parse = TransformParser.parseSelect( "select CONCAT(id, order_id) as uniq_id, * from tb where uniq_id > 10 and id is not null"); - Assert.assertEquals( - "`CONCAT`(`id`, `order_id`) AS `uniq_id`, *", parse.getSelectList().toString()); - Assert.assertEquals("`uniq_id` > 10 AND `id` IS NOT NULL", parse.getWhere().toString()); + Assertions.assertThat(parse.getSelectList().toString()) + .isEqualTo("`CONCAT`(`id`, `order_id`) AS `uniq_id`, *"); + + Assertions.assertThat(parse.getWhere().toString()) + .isEqualTo("`uniq_id` > 10 AND `id` IS NOT NULL"); } @Test @@ -101,15 +107,17 @@ public class TransformParserTest { factory, SqlValidator.Config.DEFAULT.withIdentifierExpansion(true)); SqlNode validateSqlNode = validator.validate(parse); - Assert.assertEquals( - "SUBSTR(`tb`.`id`, 1) AS `uniq_id`, `tb`.`id`, `tb`.`order_id`", - parse.getSelectList().toString()); - Assert.assertEquals("`tb`.`id` IS NOT NULL", parse.getWhere().toString()); - Assert.assertEquals( - "SELECT SUBSTR(`tb`.`id`, 1) AS `uniq_id`, `tb`.`id`, `tb`.`order_id`\n" - + "FROM `default_schema`.`tb` AS `tb`\n" - + "WHERE `tb`.`id` IS NOT NULL", - validateSqlNode.toString().replaceAll("\r\n", "\n")); + + Assertions.assertThat(parse.getSelectList().toString()) + .isEqualTo("SUBSTR(`tb`.`id`, 1) AS `uniq_id`, `tb`.`id`, `tb`.`order_id`"); + + Assertions.assertThat(parse.getWhere().toString()).isEqualTo("`tb`.`id` IS NOT NULL"); + + Assertions.assertThat(validateSqlNode.toString().replaceAll("\r\n", "\n")) + .isEqualTo( + "SELECT SUBSTR(`tb`.`id`, 1) AS `uniq_id`, `tb`.`id`, `tb`.`order_id`\n" + + "FROM `default_schema`.`tb` AS `tb`\n" + + "WHERE `tb`.`id` IS NOT NULL"); } @Test @@ -160,29 +168,33 @@ public class TransformParserTest { RelBuilder relBuilder = config.getRelBuilderFactory().create(cluster, null); relRoot = relRoot.withRel(RelDecorrelator.decorrelateQuery(relRoot.rel, relBuilder)); RelNode relNode = relRoot.rel; - Assert.assertEquals( - "SUBSTR(`tb`.`id`, 1) AS `uniq_id`, `tb`.`id`, `tb`.`order_id`", - parse.getSelectList().toString()); - Assert.assertEquals("`tb`.`id` IS NOT NULL", parse.getWhere().toString()); - Assert.assertEquals( - "SELECT SUBSTR(`tb`.`id`, 1) AS `uniq_id`, `tb`.`id`, `tb`.`order_id`\n" - + "FROM `default_schema`.`tb` AS `tb`\n" - + "WHERE `tb`.`id` IS NOT NULL", - validateSqlNode.toString().replaceAll("\r\n", "\n")); + + Assertions.assertThat(parse.getSelectList().toString()) + .isEqualTo("SUBSTR(`tb`.`id`, 1) AS `uniq_id`, `tb`.`id`, `tb`.`order_id`"); + + Assertions.assertThat(parse.getWhere().toString()).isEqualTo("`tb`.`id` IS NOT NULL"); + + Assertions.assertThat(validateSqlNode.toString().replaceAll("\r\n", "\n")) + .isEqualTo( + "SELECT SUBSTR(`tb`.`id`, 1) AS `uniq_id`, `tb`.`id`, `tb`.`order_id`\n" + + "FROM `default_schema`.`tb` AS `tb`\n" + + "WHERE `tb`.`id` IS NOT NULL"); } @Test public void testParseComputedColumnNames() { List computedColumnNames = TransformParser.parseComputedColumnNames("CONCAT(id, order_id) as uniq_id, *"); - Assert.assertEquals(new String[] {"uniq_id"}, computedColumnNames.toArray()); + + Assertions.assertThat(computedColumnNames.toArray()).isEqualTo(new String[] {"uniq_id"}); } @Test public void testParseFilterColumnNameList() { List computedColumnNames = TransformParser.parseFilterColumnNameList(" uniq_id > 10 and id is not null"); - Assert.assertEquals(new String[] {"uniq_id", "id"}, computedColumnNames.toArray()); + Assertions.assertThat(computedColumnNames.toArray()) + .isEqualTo(new String[] {"uniq_id", "id"}); } @Test @@ -298,9 +310,112 @@ public class TransformParserTest { testFilterExpression("cast(dt as TIMESTAMP)", "castToTimestamp(dt, __time_zone__)"); } + @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}"); + Assertions.assertThat(result.toString()).isEqualTo("[" + 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}"); + Assertions.assertThat(metadataResult.toString()) + .isEqualTo("[" + String.join(", ", metadataExpected) + "]"); + + // calculated columns must use AS to provide an alias name + Assertions.assertThatThrownBy( + () -> TransformParser.generateProjectionColumns("id, 1 + 1", testColumns)) + .isExactlyInstanceOf(ParseException.class); + } + + @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'"); + Assertions.assertThat(result.toString()).isEqualTo("[" + String.join(", ", expected) + "]"); + + // calculated columns must use AS to provide an alias name + Assertions.assertThatThrownBy( + () -> + TransformParser.generateReferencedColumns( + "id, 1 + 1", null, testColumns)) + .isExactlyInstanceOf(ParseException.class); + } + + @Test + public void testNormalizeFilter() { + Assertions.assertThat(TransformParser.normalizeFilter("a, b, c, d", "a > 0 and b > 0")) + .isEqualTo("`a` > 0 AND `b` > 0"); + Assertions.assertThat(TransformParser.normalizeFilter("a, b, c, d", null)).isEqualTo(null); + Assertions.assertThat( + 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")) + .isEqualTo("`a` > 4 AND ABS(`a`) > 8 AND CHAR_LENGTH(`b`) < 17 AND `c` <> `d`"); + + Assertions.assertThat( + 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")) + .isEqualTo( + "`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); - Assert.assertEquals(expressionExpect, janinoExpression); + Assertions.assertThat(janinoExpression).isEqualTo(expressionExpect); } }