From 2ca3e9732659a7953496d66e2060322785a774f2 Mon Sep 17 00:00:00 2001 From: yuxiqian <34335406+yuxiqian@users.noreply.github.com> Date: Wed, 10 Jul 2024 16:47:51 +0800 Subject: [PATCH] [FLINK-34876][transform] Support UDF functions in transform --- README.md | 75 +- .../docs/core-concept/data-pipeline.md | 17 +- .../content.zh/docs/core-concept/transform.md | 69 ++ .../docs/core-concept/data-pipeline.md | 18 +- docs/content/docs/core-concept/transform.md | 69 ++ .../parser/YamlPipelineDefinitionParser.java | 32 +- .../YamlPipelineDefinitionParserTest.java | 40 + .../pipeline-definition-with-udf.yaml | 34 + .../cdc/common/udf/UserDefinedFunction.java | 36 + .../udf/UserDefinedFunctionDescriptor.java | 132 +++ .../UserDefinedFunctionDescriptorTest.java | 96 +++ flink-cdc-composer/pom.xml | 14 + .../cdc/composer/definition/PipelineDef.java | 12 +- .../flink/cdc/composer/definition/UdfDef.java | 71 ++ .../composer/flink/FlinkPipelineComposer.java | 17 +- .../flink/translator/TransformTranslator.java | 10 +- .../flink/FlinkPipelineComposerITCase.java | 20 +- .../flink/FlinkPipelineUdfITCase.java | 811 ++++++++++++++++++ .../flink-cdc-pipeline-e2e-tests/pom.xml | 34 + .../cdc/pipeline/tests/UdfE2eITCase.java | 404 +++++++++ flink-cdc-pipeline-udf-examples/pom.xml | 104 +++ .../examples/java/AddOneFunctionClass.java | 31 + .../examples/java/FormatFunctionClass.java | 27 + .../examples/java/LifecycleFunctionClass.java | 40 + .../examples/java/TypeHintFunctionClass.java | 37 + .../examples/java/TypeOfFunctionClass.java | 43 + .../examples/java/AddOneFunctionClass.java | 31 + .../examples/java/FormatFunctionClass.java | 27 + .../examples/java/TypeOfFunctionClass.java | 43 + .../examples/scala/AddOneFunctionClass.scala | 27 + .../examples/scala/FormatFunctionClass.scala | 27 + .../scala/LifecycleFunctionClass.scala | 40 + .../scala/TypeHintFunctionClass.scala | 34 + .../examples/scala/TypeOfFunctionClass.scala | 33 + .../examples/scala/AddOneFunctionClass.scala | 27 + .../examples/scala/FormatFunctionClass.scala | 27 + .../examples/scala/TypeOfFunctionClass.scala | 33 + flink-cdc-runtime/pom.xml | 6 + .../transform/ProjectionColumnProcessor.java | 32 +- .../transform/TransformDataOperator.java | 92 +- .../TransformExpressionCompiler.java | 20 +- .../operators/transform/TransformFilter.java | 7 +- .../transform/TransformFilterProcessor.java | 32 +- .../TransformProjectionProcessor.java | 50 +- .../transform/TransformSchemaOperator.java | 30 +- .../cdc/runtime/parser/JaninoCompiler.java | 92 +- .../cdc/runtime/parser/TransformParser.java | 66 +- .../flink/cdc/runtime/typeutils/UdfUtils.java | 23 + .../runtime/parser/TransformParserTest.java | 65 +- pom.xml | 3 + 50 files changed, 3039 insertions(+), 121 deletions(-) create mode 100644 flink-cdc-cli/src/test/resources/definitions/pipeline-definition-with-udf.yaml create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/udf/UserDefinedFunction.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/udf/UserDefinedFunctionDescriptor.java create mode 100644 flink-cdc-common/src/test/java/org/apache/flink/cdc/common/udf/UserDefinedFunctionDescriptorTest.java create mode 100644 flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/UdfDef.java create mode 100644 flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineUdfITCase.java create mode 100644 flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/UdfE2eITCase.java create mode 100644 flink-cdc-pipeline-udf-examples/pom.xml create mode 100644 flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/AddOneFunctionClass.java create mode 100644 flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/FormatFunctionClass.java create mode 100644 flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/LifecycleFunctionClass.java create mode 100644 flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/TypeHintFunctionClass.java create mode 100644 flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/TypeOfFunctionClass.java create mode 100644 flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/udf/examples/java/AddOneFunctionClass.java create mode 100644 flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/udf/examples/java/FormatFunctionClass.java create mode 100644 flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/udf/examples/java/TypeOfFunctionClass.java create mode 100644 flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/AddOneFunctionClass.scala create mode 100644 flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/FormatFunctionClass.scala create mode 100644 flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/LifecycleFunctionClass.scala create mode 100644 flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/TypeHintFunctionClass.scala create mode 100644 flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/TypeOfFunctionClass.scala create mode 100644 flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/udf/examples/scala/AddOneFunctionClass.scala create mode 100644 flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/udf/examples/scala/FormatFunctionClass.scala create mode 100644 flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/udf/examples/scala/TypeOfFunctionClass.scala create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/UdfUtils.java diff --git a/README.md b/README.md index 426aa96daea..07182ccfb36 100644 --- a/README.md +++ b/README.md @@ -35,41 +35,46 @@ full database synchronization, sharding table synchronization, schema evolution 2. [Download](https://github.com/apache/flink-cdc/releases) Flink CDC tar, unzip it and put jars of pipeline connector to Flink `lib` directory. 3. Create a **YAML** file to describe the data source and data sink, the following example synchronizes all tables under MySQL app_db database to Doris : ```yaml - source: - type: mysql - name: MySQL Source - hostname: 127.0.0.1 - port: 3306 - username: admin - password: pass - tables: adb.\.* - server-id: 5401-5404 - - sink: - type: doris - name: Doris Sink - fenodes: 127.0.0.1:8030 - username: root - password: pass - - transform: - - source-table: adb.web_order01 - projection: \*, UPPER(product_name) as product_name - filter: id > 10 AND order_id > 100 - description: project fields and filter - - source-table: adb.web_order02 - projection: \*, UPPER(product_name) as product_name - filter: id > 20 AND order_id > 200 - description: project fields and filter - - route: - - source-table: adb.web_order\.* - sink-table: adb.ods_web_orders - description: sync sharding tables to one destination table - - pipeline: - name: MySQL to Doris Pipeline - parallelism: 4 + source: + type: mysql + hostname: localhost + port: 3306 + username: root + password: 123456 + tables: app_db.\.* + + sink: + type: doris + fenodes: 127.0.0.1:8030 + username: root + password: "" + + transform: + - source-table: adb.web_order01 + projection: \*, format('%S', product_name) as product_name + filter: addone(id) > 10 AND order_id > 100 + description: project fields and filter + - source-table: adb.web_order02 + projection: \*, format('%S', product_name) as product_name + filter: addone(id) > 20 AND order_id > 200 + description: project fields and filter + + route: + - source-table: app_db.orders + sink-table: ods_db.ods_orders + - source-table: app_db.shipments + sink-table: ods_db.ods_shipments + - source-table: app_db.products + sink-table: ods_db.ods_products + + pipeline: + name: Sync MySQL Database to Doris + parallelism: 2 + user-defined-function: + - name: addone + classpath: com.example.functions.AddOneFunctionClass + - name: format + classpath: com.example.functions.FormatFunctionClass ``` 4. Submit pipeline job using `flink-cdc.sh` script. ```shell diff --git a/docs/content.zh/docs/core-concept/data-pipeline.md b/docs/content.zh/docs/core-concept/data-pipeline.md index 4de5df3fb82..7b286bfbf5d 100644 --- a/docs/content.zh/docs/core-concept/data-pipeline.md +++ b/docs/content.zh/docs/core-concept/data-pipeline.md @@ -79,15 +79,15 @@ We could use following yaml file to define a complicated Data Pipeline describin fenodes: 127.0.0.1:8030 username: root password: "" - + transform: - source-table: adb.web_order01 - projection: \*, UPPER(product_name) as product_name - filter: id > 10 AND order_id > 100 + projection: \*, format('%S', product_name) as product_name + filter: addone(id) > 10 AND order_id > 100 description: project fields and filter - source-table: adb.web_order02 - projection: \*, UPPER(product_name) as product_name - filter: id > 20 AND order_id > 200 + projection: \*, format('%S', product_name) as product_name + filter: addone(id) > 20 AND order_id > 200 description: project fields and filter route: @@ -96,11 +96,16 @@ We could use following yaml file to define a complicated Data Pipeline describin - source-table: app_db.shipments sink-table: ods_db.ods_shipments - source-table: app_db.products - sink-table: ods_db.ods_products + sink-table: ods_db.ods_products pipeline: name: Sync MySQL Database to Doris parallelism: 2 + user-defined-function: + - name: addone + classpath: com.example.functions.AddOneFunctionClass + - name: format + classpath: com.example.functions.FormatFunctionClass ``` # Pipeline Configurations diff --git a/docs/content.zh/docs/core-concept/transform.md b/docs/content.zh/docs/core-concept/transform.md index a34db338769..e18de2c9383 100644 --- a/docs/content.zh/docs/core-concept/transform.md +++ b/docs/content.zh/docs/core-concept/transform.md @@ -266,6 +266,75 @@ transform: description: classification mapping example ``` +## User-defined Functions + +User-defined functions (UDFs) can be used in transform rules. + +Classes could be used as a UDF if: + +* implements `org.apache.flink.cdc.common.udf.UserDefinedFunction` interface +* has a public constructor with no parameters +* has at least one public method named `eval` + +It may also: + +* overrides `getReturnType` method to indicate its return CDC type +* overrides `open` and `close` method to do some initialization and cleanup work + +For example, this is a valid UDF class: + +```java +public class AddOneFunctionClass implements UserDefinedFunction { + + public Object eval(Integer num) { + return num + 1; + } + + @Override + public DataType getReturnType() { + return DataTypes.INT(); + } + + @Override + public void open() throws Exception { + // ... + } + + @Override + public void close() throws Exception { + // ... + } +} +``` + +To ease the migration from Flink SQL to Flink CDC, a Flink `ScalarFunction` could also be used as a transform UDF, with some limitations: + +* `ScalarFunction` with parameters is not supported. +* Flink-style type hint in `ScalarFunction` will be ignored. +* `open` / `close` lifecycle hooks will not be invoked. + +UDF classes could be registered by adding a `user-defined-function` block: + +```yaml +pipeline: + user-defined-function: + - name: addone + classpath: org.apache.flink.cdc.udf.examples.java.AddOneFunctionClass + - name: format + classpath: org.apache.flink.cdc.udf.examples.java.FormatFunctionClass +``` + +Notice that given classpath must be fully-qualified, and corresponding `jar` files must be included in Flink `/lib` folder, or be passed with `flink-cdc.sh --jar` option. + +After being correctly registered, UDFs could be used in both `projection` and `filter` expressions, just like built-in functions: + +```yaml +transform: + - source-table: db.\.* + projection: "*, inc(inc(inc(id))) as inc_id, format(id, 'id -> %d') as formatted_id" + filter: inc(id) < 100 +``` + # Known limitations * Currently, transform doesn't work with route rules. It will be supported in future versions. * Computed columns cannot reference trimmed columns that do not present in final projection results. This will be fixed in future versions. diff --git a/docs/content/docs/core-concept/data-pipeline.md b/docs/content/docs/core-concept/data-pipeline.md index 8914b721d78..759a47b245d 100644 --- a/docs/content/docs/core-concept/data-pipeline.md +++ b/docs/content/docs/core-concept/data-pipeline.md @@ -97,17 +97,33 @@ We could use following yaml file to define a complicated Data Pipeline describin fenodes: 127.0.0.1:8030 username: root password: "" + + transform: + - source-table: adb.web_order01 + projection: \*, format('%S', product_name) as product_name + filter: addone(id) > 10 AND order_id > 100 + description: project fields and filter + - source-table: adb.web_order02 + projection: \*, format('%S', product_name) as product_name + filter: addone(id) > 20 AND order_id > 200 + description: project fields and filter + route: - source-table: app_db.orders sink-table: ods_db.ods_orders - source-table: app_db.shipments sink-table: ods_db.ods_shipments - source-table: app_db.products - sink-table: ods_db.ods_products + sink-table: ods_db.ods_products pipeline: name: Sync MySQL Database to Doris parallelism: 2 + user-defined-function: + - name: addone + classpath: com.example.functions.AddOneFunctionClass + - name: format + classpath: com.example.functions.FormatFunctionClass ``` # Pipeline Configurations diff --git a/docs/content/docs/core-concept/transform.md b/docs/content/docs/core-concept/transform.md index a34db338769..85095fe2189 100644 --- a/docs/content/docs/core-concept/transform.md +++ b/docs/content/docs/core-concept/transform.md @@ -266,6 +266,75 @@ transform: description: classification mapping example ``` +## User-defined Functions + +User-defined functions (UDFs) can be used in transform rules. + +Classes could be used as a UDF if: + +* implements `org.apache.flink.cdc.common.udf.UserDefinedFunction` interface +* has a public constructor with no parameters +* has at least one public method named `eval` + +It may also: + +* overrides `getReturnType` method to indicate its return CDC type +* overrides `open` and `close` method to do some initialization and cleanup work + +For example, this is a valid UDF class: + +```java +public class AddOneFunctionClass implements UserDefinedFunction { + + public Object eval(Integer num) { + return num + 1; + } + + @Override + public DataType getReturnType() { + return DataTypes.INT(); + } + + @Override + public void open() throws Exception { + // ... + } + + @Override + public void close() throws Exception { + // ... + } +} +``` + +To ease the migration from Flink SQL to Flink CDC, a Flink `ScalarFunction` could also be used as a transform UDF, with some limitations: + +* `ScalarFunction` with parameters is not supported. +* Flink-style type hint in `ScalarFunction` will be ignored. +* `open` / `close` lifecycle hooks will not be invoked. + +UDF classes could be registered by adding a `user-defined-function` block: + +```yaml +pipeline: + user-defined-function: + - name: addone + classpath: org.apache.flink.cdc.udf.examples.java.AddOneFunctionClass + - name: format + classpath: org.apache.flink.cdc.udf.examples.java.FormatFunctionClass +``` + +Notice that given classpath must be fully-qualified, and corresponding `jar` files must be included in Flink `/lib` folder, or be passed with `flink-cdc.sh --jar` option. + +After being correctly registered, UDFs could be used in both `projection` and `filter` expressions, just like built-in functions: + +```yaml +transform: + - source-table: db.\.* + projection: "*, inc(inc(inc(id))) as inc_id, format(id, 'id -> %d') as formatted_id" + filter: inc(id) < 100 +``` + # Known limitations * Currently, transform doesn't work with route rules. It will be supported in future versions. * Computed columns cannot reference trimmed columns that do not present in final projection results. This will be fixed in future versions. diff --git a/flink-cdc-cli/src/main/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParser.java b/flink-cdc-cli/src/main/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParser.java index 7ad07af1a44..3aed413b12b 100644 --- a/flink-cdc-cli/src/main/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParser.java +++ b/flink-cdc-cli/src/main/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParser.java @@ -24,10 +24,12 @@ import org.apache.flink.cdc.composer.definition.SinkDef; import org.apache.flink.cdc.composer.definition.SourceDef; import org.apache.flink.cdc.composer.definition.TransformDef; +import org.apache.flink.cdc.composer.definition.UdfDef; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.type.TypeReference; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import java.nio.file.Path; @@ -64,6 +66,11 @@ public class YamlPipelineDefinitionParser implements PipelineDefinitionParser { private static final String TRANSFORM_FILTER_KEY = "filter"; private static final String TRANSFORM_DESCRIPTION_KEY = "description"; + // UDF related keys + private static final String UDF_KEY = "user-defined-function"; + private static final String UDF_FUNCTION_NAME_KEY = "name"; + private static final String UDF_CLASSPATH_KEY = "classpath"; + public static final String TRANSFORM_PRIMARY_KEY_KEY = "primary-keys"; public static final String TRANSFORM_PARTITION_KEY_KEY = "partition-keys"; @@ -107,6 +114,11 @@ public PipelineDef parse(Path pipelineDefPath, Configuration globalPipelineConfi Optional.ofNullable(root.get(ROUTE_KEY)) .ifPresent(node -> node.forEach(route -> routeDefs.add(toRouteDef(route)))); + // UDFs are optional + List udfDefs = new ArrayList<>(); + Optional.ofNullable(((ObjectNode) root.get(PIPELINE_KEY)).remove(UDF_KEY)) + .ifPresent(node -> node.forEach(udf -> udfDefs.add(toUdfDef(udf)))); + // Pipeline configs are optional Configuration userPipelineConfig = toPipelineConfig(root.get(PIPELINE_KEY)); @@ -115,7 +127,8 @@ public PipelineDef parse(Path pipelineDefPath, Configuration globalPipelineConfi pipelineConfig.addAll(globalPipelineConfig); pipelineConfig.addAll(userPipelineConfig); - return new PipelineDef(sourceDef, sinkDef, routeDefs, transformDefs, pipelineConfig); + return new PipelineDef( + sourceDef, sinkDef, routeDefs, transformDefs, udfDefs, pipelineConfig); } private SourceDef toSourceDef(JsonNode sourceNode) { @@ -176,6 +189,23 @@ private RouteDef toRouteDef(JsonNode routeNode) { return new RouteDef(sourceTable, sinkTable, replaceSymbol, description); } + private UdfDef toUdfDef(JsonNode udfNode) { + String functionName = + checkNotNull( + udfNode.get(UDF_FUNCTION_NAME_KEY), + "Missing required field \"%s\" in UDF configuration", + UDF_FUNCTION_NAME_KEY) + .asText(); + String classPath = + checkNotNull( + udfNode.get(UDF_CLASSPATH_KEY), + "Missing required field \"%s\" in UDF configuration", + UDF_CLASSPATH_KEY) + .asText(); + + return new UdfDef(functionName, classPath); + } + private TransformDef toTransformDef(JsonNode transformNode) { String sourceTable = checkNotNull( diff --git a/flink-cdc-cli/src/test/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParserTest.java b/flink-cdc-cli/src/test/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParserTest.java index 75dc5bd628b..b48feaa8e15 100644 --- a/flink-cdc-cli/src/test/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParserTest.java +++ b/flink-cdc-cli/src/test/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParserTest.java @@ -24,6 +24,7 @@ import org.apache.flink.cdc.composer.definition.SinkDef; import org.apache.flink.cdc.composer.definition.SourceDef; import org.apache.flink.cdc.composer.definition.TransformDef; +import org.apache.flink.cdc.composer.definition.UdfDef; import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; import org.apache.flink.shaded.guava31.com.google.common.io.Resources; @@ -175,6 +176,14 @@ void testRouteWithReplacementSymbol() throws Exception { assertThat(pipelineDef).isEqualTo(fullDefWithRouteRepSym); } + @Test + void testUdfDefinition() throws Exception { + URL resource = Resources.getResource("definitions/pipeline-definition-with-udf.yaml"); + YamlPipelineDefinitionParser parser = new YamlPipelineDefinitionParser(); + PipelineDef pipelineDef = parser.parse(Paths.get(resource.toURI()), new Configuration()); + assertThat(pipelineDef).isEqualTo(pipelineDefWithUdf); + } + private final PipelineDef fullDef = new PipelineDef( new SourceDef( @@ -230,6 +239,7 @@ void testRouteWithReplacementSymbol() throws Exception { null, null, "add new uniq_id for each row")), + Collections.emptyList(), Configuration.fromMap( ImmutableMap.builder() .put("name", "source-database-sync-pipe") @@ -293,6 +303,7 @@ void testRouteWithReplacementSymbol() throws Exception { null, null, "add new uniq_id for each row")), + Collections.emptyList(), Configuration.fromMap( ImmutableMap.builder() .put("name", "source-database-sync-pipe") @@ -330,6 +341,7 @@ void testRouteWithReplacementSymbol() throws Exception { null, null)), Collections.emptyList(), + Collections.emptyList(), Configuration.fromMap( ImmutableMap.builder() .put("parallelism", "4") @@ -341,6 +353,7 @@ void testRouteWithReplacementSymbol() throws Exception { new SinkDef("kafka", null, new Configuration()), Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), Configuration.fromMap(Collections.singletonMap("parallelism", "1"))); private final PipelineDef fullDefWithRouteRepSym = @@ -398,6 +411,7 @@ void testRouteWithReplacementSymbol() throws Exception { null, null, "add new uniq_id for each row")), + Collections.emptyList(), Configuration.fromMap( ImmutableMap.builder() .put("name", "source-database-sync-pipe") @@ -405,4 +419,30 @@ void testRouteWithReplacementSymbol() throws Exception { .put("schema.change.behavior", "evolve") .put("schema-operator.rpc-timeout", "1 h") .build())); + + private final PipelineDef pipelineDefWithUdf = + new PipelineDef( + new SourceDef("values", null, new Configuration()), + new SinkDef("values", null, new Configuration()), + Collections.emptyList(), + Collections.singletonList( + new TransformDef( + "mydb.web_order", + "*, inc(inc(inc(id))) as inc_id, format(id, 'id -> %d') as formatted_id", + "inc(id) < 100", + null, + null, + null, + null)), + Arrays.asList( + new UdfDef( + "inc", + "org.apache.flink.cdc.udf.examples.java.AddOneFunctionClass"), + new UdfDef( + "format", + "org.apache.flink.cdc.udf.examples.java.FormatFunctionClass")), + Configuration.fromMap( + ImmutableMap.builder() + .put("parallelism", "1") + .build())); } diff --git a/flink-cdc-cli/src/test/resources/definitions/pipeline-definition-with-udf.yaml b/flink-cdc-cli/src/test/resources/definitions/pipeline-definition-with-udf.yaml new file mode 100644 index 00000000000..66a23f9ceaf --- /dev/null +++ b/flink-cdc-cli/src/test/resources/definitions/pipeline-definition-with-udf.yaml @@ -0,0 +1,34 @@ +################################################################################ +# 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. +################################################################################ +source: + type: values + +sink: + type: values + +transform: + - source-table: mydb.web_order + projection: "*, inc(inc(inc(id))) as inc_id, format(id, 'id -> %d') as formatted_id" + filter: inc(id) < 100 + +pipeline: + parallelism: 1 + user-defined-function: + - name: inc + classpath: org.apache.flink.cdc.udf.examples.java.AddOneFunctionClass + - name: format + classpath: org.apache.flink.cdc.udf.examples.java.FormatFunctionClass diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/udf/UserDefinedFunction.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/udf/UserDefinedFunction.java new file mode 100644 index 00000000000..c50b00c3229 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/udf/UserDefinedFunction.java @@ -0,0 +1,36 @@ +/* + * 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.common.udf; + +import org.apache.flink.cdc.common.types.DataType; + +/** + * Base interface for creating a UDF in transform projection and filtering expressions. You should + * define at least one {@code eval} method. + */ +public interface UserDefinedFunction { + default DataType getReturnType() { + return null; + } + + /** This will be invoked every time when a UDF got created. */ + default void open() throws Exception {} + + /** This will be invoked before a UDF got destroyed. */ + default void close() throws Exception {} +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/udf/UserDefinedFunctionDescriptor.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/udf/UserDefinedFunctionDescriptor.java new file mode 100644 index 00000000000..9defa5f4c0a --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/udf/UserDefinedFunctionDescriptor.java @@ -0,0 +1,132 @@ +/* + * 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.common.udf; + +import org.apache.flink.cdc.common.types.DataType; + +import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; +import java.util.Arrays; +import java.util.Objects; +import java.util.stream.Collectors; + +/** Descriptor of a UDF function. */ +public class UserDefinedFunctionDescriptor implements Serializable { + + private static final long serialVersionUID = 1L; + + private final String name; + private final String classPath; + private final String className; + private final DataType returnTypeHint; + private final boolean isCdcPipelineUdf; + + public UserDefinedFunctionDescriptor(String name, String classPath) { + this.name = name; + this.classPath = classPath; + this.className = classPath.substring(classPath.lastIndexOf('.') + 1); + try { + Class clazz = Class.forName(classPath); + isCdcPipelineUdf = isCdcPipelineUdf(clazz); + if (isCdcPipelineUdf) { + returnTypeHint = + (DataType) clazz.getMethod("getReturnType").invoke(clazz.newInstance()); + } else { + returnTypeHint = null; + } + } catch (ClassNotFoundException + | InvocationTargetException + | IllegalAccessException + | NoSuchMethodException + | InstantiationException e) { + throw new IllegalArgumentException( + "Failed to instantiate UDF " + name + "@" + classPath, e); + } + } + + private boolean isCdcPipelineUdf(Class clazz) { + Class cdcPipelineUdfClazz = UserDefinedFunction.class; + Class flinkScalarFunctionClazz = org.apache.flink.table.functions.ScalarFunction.class; + + if (Arrays.stream(clazz.getInterfaces()) + .map(Class::getName) + .collect(Collectors.toList()) + .contains(cdcPipelineUdfClazz.getName())) { + return true; + } else if (clazz.getSuperclass().getName().equals(flinkScalarFunctionClazz.getName())) { + return false; + } else { + throw new IllegalArgumentException( + String.format( + "Failed to detect UDF class " + + clazz + + " since it never implements %s or extends Flink %s.", + cdcPipelineUdfClazz, + flinkScalarFunctionClazz)); + } + } + + public DataType getReturnTypeHint() { + return returnTypeHint; + } + + public boolean isCdcPipelineUdf() { + return isCdcPipelineUdf; + } + + public String getName() { + return name; + } + + public String getClassPath() { + return classPath; + } + + public String getClassName() { + return className; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + UserDefinedFunctionDescriptor context = (UserDefinedFunctionDescriptor) o; + return Objects.equals(name, context.name) && Objects.equals(classPath, context.classPath); + } + + @Override + public int hashCode() { + return Objects.hash(name, classPath); + } + + @Override + public String toString() { + return "UserDefinedFunctionDescriptor{" + + "name='" + + name + + '\'' + + ", classPath='" + + classPath + + '\'' + + '}'; + } +} diff --git a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/udf/UserDefinedFunctionDescriptorTest.java b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/udf/UserDefinedFunctionDescriptorTest.java new file mode 100644 index 00000000000..dcda5dc86cf --- /dev/null +++ b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/udf/UserDefinedFunctionDescriptorTest.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.udf; + +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.table.functions.ScalarFunction; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Testcases for {@link UserDefinedFunctionDescriptor}. */ +public class UserDefinedFunctionDescriptorTest { + + /** This is a plain Flink CDC UDF. */ + public static class CdcUdf implements UserDefinedFunction {} + + /** This is a Flink CDC UDF with type hint. */ + public static class CdcUdfWithTypeHint implements UserDefinedFunction { + @Override + public DataType getReturnType() { + return DataTypes.TIMESTAMP_LTZ(9); + } + } + + /** This is a Flink ScalarFunction. */ + public static class FlinkUdf extends ScalarFunction {} + + /** This is not a valid UDF class. */ + public static class NotUDF {} + + @Test + void testUserDefinedFunctionDescriptor() { + + assertThat(new UserDefinedFunctionDescriptor("cdc_udf", CdcUdf.class.getName())) + .extracting("name", "className", "classPath", "returnTypeHint", "isCdcPipelineUdf") + .containsExactly( + "cdc_udf", + "UserDefinedFunctionDescriptorTest$CdcUdf", + "org.apache.flink.cdc.common.udf.UserDefinedFunctionDescriptorTest$CdcUdf", + null, + true); + + assertThat( + new UserDefinedFunctionDescriptor( + "cdc_udf_with_type_hint", CdcUdfWithTypeHint.class.getName())) + .extracting("name", "className", "classPath", "returnTypeHint", "isCdcPipelineUdf") + .containsExactly( + "cdc_udf_with_type_hint", + "UserDefinedFunctionDescriptorTest$CdcUdfWithTypeHint", + "org.apache.flink.cdc.common.udf.UserDefinedFunctionDescriptorTest$CdcUdfWithTypeHint", + DataTypes.TIMESTAMP_LTZ(9), + true); + + assertThat(new UserDefinedFunctionDescriptor("flink_udf", FlinkUdf.class.getName())) + .extracting("name", "className", "classPath", "returnTypeHint", "isCdcPipelineUdf") + .containsExactly( + "flink_udf", + "UserDefinedFunctionDescriptorTest$FlinkUdf", + "org.apache.flink.cdc.common.udf.UserDefinedFunctionDescriptorTest$FlinkUdf", + null, + false); + + assertThatThrownBy( + () -> new UserDefinedFunctionDescriptor("not_udf", NotUDF.class.getName())) + .isExactlyInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Failed to detect UDF class class org.apache.flink.cdc.common.udf.UserDefinedFunctionDescriptorTest$NotUDF " + + "since it never implements interface org.apache.flink.cdc.common.udf.UserDefinedFunction or " + + "extends Flink class org.apache.flink.table.functions.ScalarFunction."); + + assertThatThrownBy( + () -> + new UserDefinedFunctionDescriptor( + "not_even_exist", "not.a.valid.class.path")) + .isExactlyInstanceOf(IllegalArgumentException.class) + .hasMessage("Failed to instantiate UDF not_even_exist@not.a.valid.class.path"); + } +} diff --git a/flink-cdc-composer/pom.xml b/flink-cdc-composer/pom.xml index 12471f03299..b72b7069a3b 100644 --- a/flink-cdc-composer/pom.xml +++ b/flink-cdc-composer/pom.xml @@ -56,6 +56,20 @@ limitations under the License. ${flink.version} test + + org.apache.flink + flink-cdc-pipeline-udf-examples + ${project.version} + test + + + + + org.scala-lang + scala-library + ${scala.version} + test + \ No newline at end of file diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/PipelineDef.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/PipelineDef.java index 49e6a4a0c6d..6353c4e746d 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/PipelineDef.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/PipelineDef.java @@ -54,6 +54,7 @@ public class PipelineDef { private final SinkDef sink; private final List routes; private final List transforms; + private final List udfs; private final Configuration config; public PipelineDef( @@ -61,11 +62,13 @@ public PipelineDef( SinkDef sink, List routes, List transforms, + List udfs, Configuration config) { this.source = source; this.sink = sink; this.routes = routes; this.transforms = transforms; + this.udfs = udfs; this.config = evaluatePipelineTimeZone(config); } @@ -85,6 +88,10 @@ public List getTransforms() { return transforms; } + public List getUdfs() { + return udfs; + } + public Configuration getConfig() { return config; } @@ -100,6 +107,8 @@ public String toString() { + routes + ", transforms=" + transforms + + ", udfs=" + + udfs + ", config=" + config + '}'; @@ -118,12 +127,13 @@ public boolean equals(Object o) { && Objects.equals(sink, that.sink) && Objects.equals(routes, that.routes) && Objects.equals(transforms, that.transforms) + && Objects.equals(udfs, that.udfs) && Objects.equals(config, that.config); } @Override public int hashCode() { - return Objects.hash(source, sink, routes, transforms, config); + return Objects.hash(source, sink, routes, transforms, udfs, config); } // ------------------------------------------------------------------------ diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/UdfDef.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/UdfDef.java new file mode 100644 index 00000000000..a1f21792fbf --- /dev/null +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/UdfDef.java @@ -0,0 +1,71 @@ +/* + * 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.composer.definition; + +import java.util.Objects; + +/** + * Definition of a transformation. + * + *

A transformation definition contains: + * + *

    + *
  • name: Static method name of user-defined functions. + *
  • classpath: Fully-qualified class path of package containing given function. + *
+ */ +public class UdfDef { + private final String name; + private final String classPath; + + public UdfDef(String name, String classPath) { + this.name = name; + this.classPath = classPath; + } + + public String getName() { + return name; + } + + public String getClassPath() { + return classPath; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + UdfDef udfDef = (UdfDef) o; + return Objects.equals(name, udfDef.name) && Objects.equals(classPath, udfDef.classPath); + } + + @Override + public int hashCode() { + return Objects.hash(name, classPath); + } + + @Override + public String toString() { + return "UdfDef{" + "name='" + name + '\'' + ", classPath='" + classPath + '\'' + '}'; + } +} 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 0735b90cf32..8d4d01cc7e7 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 @@ -24,6 +24,7 @@ import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.pipeline.PipelineOptions; import org.apache.flink.cdc.common.sink.DataSink; +import org.apache.flink.cdc.common.udf.UserDefinedFunctionDescriptor; import org.apache.flink.cdc.composer.PipelineComposer; import org.apache.flink.cdc.composer.PipelineExecution; import org.apache.flink.cdc.composer.definition.PipelineDef; @@ -49,6 +50,7 @@ import java.util.List; import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; /** Composer for translating data pipeline to a Flink DataStream job. */ @Internal @@ -95,6 +97,14 @@ public PipelineExecution compose(PipelineDef pipelineDef) { int parallelism = pipelineDef.getConfig().get(PipelineOptions.PIPELINE_PARALLELISM); env.getConfig().setParallelism(parallelism); + List udfFunctions = + pipelineDef.getUdfs().stream() + .map( + udf -> + new UserDefinedFunctionDescriptor( + udf.getName(), udf.getClassPath())) + .collect(Collectors.toList()); + // Build Source Operator DataSourceTranslator sourceTranslator = new DataSourceTranslator(); DataStream stream = @@ -102,7 +112,9 @@ public PipelineExecution compose(PipelineDef pipelineDef) { // Build TransformSchemaOperator for processing Schema Event TransformTranslator transformTranslator = new TransformTranslator(); - stream = transformTranslator.translateSchema(stream, pipelineDef.getTransforms()); + stream = + transformTranslator.translateSchema( + stream, pipelineDef.getTransforms(), udfFunctions); // Schema operator SchemaOperatorTranslator schemaOperatorTranslator = @@ -123,7 +135,8 @@ public PipelineExecution compose(PipelineDef pipelineDef) { stream, pipelineDef.getTransforms(), schemaOperatorIDGenerator.generate(), - pipelineDef.getConfig().get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE)); + pipelineDef.getConfig().get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE), + udfFunctions); // Build DataSink in advance as schema operator requires MetadataApplier DataSink dataSink = createDataSink(pipelineDef.getSink(), pipelineDef.getConfig()); diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/TransformTranslator.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/TransformTranslator.java index 53400f628f3..f4a65e9a932 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 @@ -18,6 +18,7 @@ package org.apache.flink.cdc.composer.flink.translator; import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.udf.UserDefinedFunctionDescriptor; 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; @@ -34,7 +35,9 @@ public class TransformTranslator { public DataStream translateSchema( - DataStream input, List transforms) { + DataStream input, + List transforms, + List udfFunctions) { if (transforms.isEmpty()) { return input; } @@ -51,6 +54,7 @@ public DataStream translateSchema( transform.getTableOptions()); } } + transformSchemaFunctionBuilder.addUdfFunctions(udfFunctions); return input.transform( "Transform:Schema", new EventTypeInfo(), transformSchemaFunctionBuilder.build()); } @@ -59,7 +63,8 @@ public DataStream translateData( DataStream input, List transforms, OperatorID schemaOperatorID, - String timezone) { + String timezone, + List udfFunctions) { if (transforms.isEmpty()) { return input; } @@ -76,6 +81,7 @@ public DataStream translateData( } transformDataFunctionBuilder.addSchemaOperatorID(schemaOperatorID); transformDataFunctionBuilder.addTimezone(timezone); + transformDataFunctionBuilder.addUdfFunctions(udfFunctions); return input.transform( "Transform:Data", new EventTypeInfo(), transformDataFunctionBuilder.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 26c9c918751..83bd1fd56b2 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 @@ -139,6 +139,7 @@ void testSingleSplitSingleTable(ValuesDataSink.SinkApi sinkApi) throws Exception sinkDef, Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), pipelineConfig); // Execute the pipeline @@ -195,6 +196,7 @@ void testSingleSplitMultipleTables(ValuesDataSink.SinkApi sinkApi) throws Except sinkDef, Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), pipelineConfig); // Execute the pipeline @@ -261,6 +263,7 @@ void testMultiSplitsSingleTable(ValuesDataSink.SinkApi sinkApi) throws Exception sinkDef, Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), pipelineConfig); // Execute the pipeline @@ -315,6 +318,7 @@ void testTransform(ValuesDataSink.SinkApi sinkApi) throws Exception { sinkDef, Collections.emptyList(), new ArrayList<>(Arrays.asList(transformDef)), + Collections.emptyList(), pipelineConfig); // Execute the pipeline @@ -382,6 +386,7 @@ void testTransformTwice(ValuesDataSink.SinkApi sinkApi) throws Exception { sinkDef, Collections.emptyList(), new ArrayList<>(Arrays.asList(transformDef1, transformDef2)), + Collections.emptyList(), pipelineConfig); // Execute the pipeline @@ -432,7 +437,12 @@ void testOneToOneRouting() throws Exception { pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); PipelineDef pipelineDef = new PipelineDef( - sourceDef, sinkDef, routeDef, Collections.emptyList(), pipelineConfig); + sourceDef, + sinkDef, + routeDef, + Collections.emptyList(), + Collections.emptyList(), + pipelineConfig); // Execute the pipeline PipelineExecution execution = composer.compose(pipelineDef); @@ -624,7 +634,12 @@ void testMergingWithRoute() throws Exception { pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); PipelineDef pipelineDef = new PipelineDef( - sourceDef, sinkDef, routeDef, Collections.emptyList(), pipelineConfig); + sourceDef, + sinkDef, + routeDef, + Collections.emptyList(), + Collections.emptyList(), + pipelineConfig); // Execute the pipeline PipelineExecution execution = composer.compose(pipelineDef); @@ -692,6 +707,7 @@ void testRouteWithReplaceSymbol(ValuesDataSink.SinkApi sinkApi) throws Exception "__$__", null)), Collections.emptyList(), + Collections.emptyList(), pipelineConfig); // Execute the pipeline diff --git a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineUdfITCase.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineUdfITCase.java new file mode 100644 index 00000000000..c85cf8050e4 --- /dev/null +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineUdfITCase.java @@ -0,0 +1,811 @@ +/* + * 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.composer.flink; + +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.pipeline.PipelineOptions; +import org.apache.flink.cdc.composer.PipelineExecution; +import org.apache.flink.cdc.composer.definition.PipelineDef; +import org.apache.flink.cdc.composer.definition.SinkDef; +import org.apache.flink.cdc.composer.definition.SourceDef; +import org.apache.flink.cdc.composer.definition.TransformDef; +import org.apache.flink.cdc.composer.definition.UdfDef; +import org.apache.flink.cdc.connectors.values.ValuesDatabase; +import org.apache.flink.cdc.connectors.values.factory.ValuesDataFactory; +import org.apache.flink.cdc.connectors.values.sink.ValuesDataSink; +import org.apache.flink.cdc.connectors.values.sink.ValuesDataSinkOptions; +import org.apache.flink.cdc.connectors.values.source.ValuesDataSourceHelper; +import org.apache.flink.cdc.connectors.values.source.ValuesDataSourceOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.MiniClusterExtension; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.Arrays; +import java.util.Collections; +import java.util.stream.Stream; + +import static org.apache.flink.configuration.CoreOptions.ALWAYS_PARENT_FIRST_LOADER_PATTERNS_ADDITIONAL; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.params.provider.Arguments.arguments; + +/** Integration test for UDFs. */ +public class FlinkPipelineUdfITCase { + private static final int MAX_PARALLELISM = 4; + + // Always use parent-first classloader for CDC classes. + // The reason is that ValuesDatabase uses static field for holding data, we need to make sure + // the class is loaded by AppClassloader so that we can verify data in the test case. + private static final org.apache.flink.configuration.Configuration MINI_CLUSTER_CONFIG = + new org.apache.flink.configuration.Configuration(); + + static { + MINI_CLUSTER_CONFIG.set( + ALWAYS_PARENT_FIRST_LOADER_PATTERNS_ADDITIONAL, + Collections.singletonList("org.apache.flink.cdc")); + } + + /** + * Use {@link MiniClusterExtension} to reduce the overhead of restarting the MiniCluster for + * every test case. + */ + @RegisterExtension + static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(MAX_PARALLELISM) + .setConfiguration(MINI_CLUSTER_CONFIG) + .build()); + + private final PrintStream standardOut = System.out; + private final ByteArrayOutputStream outCaptor = new ByteArrayOutputStream(); + + @BeforeEach + void init() { + // Take over STDOUT as we need to check the output of values sink + System.setOut(new PrintStream(outCaptor)); + // Initialize in-memory database + ValuesDatabase.clear(); + } + + @AfterEach + void cleanup() { + System.setOut(standardOut); + } + + // ---------------------- + // CDC pipeline UDF tests + // ---------------------- + @ParameterizedTest + @MethodSource("testParams") + void testTransformWithUdf(ValuesDataSink.SinkApi sinkApi, String language) throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.TRANSFORM_TABLE); + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + // Setup value sink + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + sinkConfig.set(ValuesDataSinkOptions.SINK_API, sinkApi); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + // Setup transform + TransformDef transformDef = + new TransformDef( + "default_namespace.default_schema.table1", + "*,format('from %s to %s is %s', col1, 'z', 'lie') AS fmt", + null, + "col1", + null, + "key1=value1", + ""); + + UdfDef udfDef = + new UdfDef( + "format", + String.format( + "org.apache.flink.cdc.udf.examples.%s.FormatFunctionClass", + language)); + + // Setup pipeline + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef( + sourceDef, + sinkDef, + Collections.emptyList(), + Collections.singletonList(transformDef), + Collections.singletonList(udfDef), + pipelineConfig); + + // Execute the pipeline + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + + // Check the order and content of all received events + String[] outputEvents = outCaptor.toString().trim().split("\n"); + assertThat(outputEvents) + .containsExactly( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`fmt` STRING}, primaryKeys=col1, options=({key1=value1})}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, from 1 to z is lie], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, from 2 to z is lie], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3, from 3 to z is lie], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1, from 1 to z is lie], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , from 2 to z is lie], after=[2, x, from 2 to z is lie], op=UPDATE, meta=()}"); + } + + @ParameterizedTest + @MethodSource("testParams") + void testFilterWithUdf(ValuesDataSink.SinkApi sinkApi, String language) throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.TRANSFORM_TABLE); + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + // Setup value sink + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + sinkConfig.set(ValuesDataSinkOptions.SINK_API, sinkApi); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + // Setup transform + TransformDef transformDef = + new TransformDef( + "default_namespace.default_schema.table1", + "*,addone(col1) as collen", + "addone(col1) <> '2'", + "col1", + null, + "key1=value1", + ""); + + UdfDef udfDef = + new UdfDef( + "addone", + String.format( + "org.apache.flink.cdc.udf.examples.%s.AddOneFunctionClass", + language)); + + // Setup pipeline + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef( + sourceDef, + sinkDef, + Collections.emptyList(), + Collections.singletonList(transformDef), + Collections.singletonList(udfDef), + pipelineConfig); + + // Execute the pipeline + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + + // Check the order and content of all received events + String[] outputEvents = outCaptor.toString().trim().split("\n"); + assertThat(outputEvents) + .containsExactly( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`collen` STRING}, primaryKeys=col1, options=({key1=value1})}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, 3], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3, 4], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , 3], after=[2, x, 3], op=UPDATE, meta=()}"); + } + + @ParameterizedTest + @MethodSource("testParams") + void testOverloadedUdf(ValuesDataSink.SinkApi sinkApi, String language) throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.TRANSFORM_TABLE); + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + // Setup value sink + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + sinkConfig.set(ValuesDataSinkOptions.SINK_API, sinkApi); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + // Setup transform + TransformDef transformDef = + new TransformDef( + "default_namespace.default_schema.table1", + "*, typeof(true) as tob, typeof(1) as toi, typeof(3.14) as tof, typeof('str') as tos", + null, + "col1", + null, + "key1=value1", + ""); + + UdfDef udfDef = + new UdfDef( + "typeof", + String.format( + "org.apache.flink.cdc.udf.examples.%s.TypeOfFunctionClass", + language)); + + // Setup pipeline + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef( + sourceDef, + sinkDef, + Collections.emptyList(), + Collections.singletonList(transformDef), + Collections.singletonList(udfDef), + pipelineConfig); + + // Execute the pipeline + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + + // Check the order and content of all received events + String[] outputEvents = outCaptor.toString().trim().split("\n"); + assertThat(outputEvents) + .containsExactly( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`tob` STRING,`toi` STRING,`tof` STRING,`tos` STRING}, primaryKeys=col1, options=({key1=value1})}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, Boolean: true, Integer: 1, Double: 3.14, String: str], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, Boolean: true, Integer: 1, Double: 3.14, String: str], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3, Boolean: true, Integer: 1, Double: 3.14, String: str], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1, Boolean: true, Integer: 1, Double: 3.14, String: str], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , Boolean: true, Integer: 1, Double: 3.14, String: str], after=[2, x, Boolean: true, Integer: 1, Double: 3.14, String: str], op=UPDATE, meta=()}"); + } + + @ParameterizedTest + @MethodSource("testParams") + void testUdfLifecycle(ValuesDataSink.SinkApi sinkApi, String language) throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.TRANSFORM_TABLE); + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + // Setup value sink + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + sinkConfig.set(ValuesDataSinkOptions.SINK_API, sinkApi); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + // Setup transform + TransformDef transformDef = + new TransformDef( + "default_namespace.default_schema.table1", + "*, lifecycle() as stt", + null, + "col1", + null, + "key1=value1", + ""); + + UdfDef udfDef = + new UdfDef( + "lifecycle", + String.format( + "org.apache.flink.cdc.udf.examples.%s.LifecycleFunctionClass", + language)); + + // Setup pipeline + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef( + sourceDef, + sinkDef, + Collections.emptyList(), + Collections.singletonList(transformDef), + Collections.singletonList(udfDef), + pipelineConfig); + + // Execute the pipeline + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + + // Check the order and content of all received events + String[] outputEvents = outCaptor.toString().trim().split("\n"); + assertThat(outputEvents) + .contains("[ LifecycleFunction ] opened.") + .contains( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`stt` STRING}, primaryKeys=col1, options=({key1=value1})}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, #0], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, #1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3, #2], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1, #3], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , #4], after=[2, x, #5], op=UPDATE, meta=()}") + .contains("[ LifecycleFunction ] closed. Called 6 times."); + } + + @ParameterizedTest + @MethodSource("testParams") + void testTypeHintedUdf(ValuesDataSink.SinkApi sinkApi, String language) throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.TRANSFORM_TABLE); + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + // Setup value sink + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + sinkConfig.set(ValuesDataSinkOptions.SINK_API, sinkApi); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + // Setup transform + TransformDef transformDef = + new TransformDef( + "default_namespace.default_schema.table1", + "*, answer() as ans", + null, + "col1", + null, + "key1=value1", + ""); + + UdfDef udfDef = + new UdfDef( + "answer", + String.format( + "org.apache.flink.cdc.udf.examples.%s.TypeHintFunctionClass", + language)); + + // Setup pipeline + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef( + sourceDef, + sinkDef, + Collections.emptyList(), + Collections.singletonList(transformDef), + Collections.singletonList(udfDef), + pipelineConfig); + + // Execute the pipeline + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + + // Check the order and content of all received events + String[] outputEvents = outCaptor.toString().trim().split("\n"); + assertThat(outputEvents) + .containsExactly( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`ans` STRING}, primaryKeys=col1, options=({key1=value1})}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, Forty-two], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, Forty-two], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3, Forty-two], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1, Forty-two], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , Forty-two], after=[2, x, Forty-two], op=UPDATE, meta=()}"); + } + + @ParameterizedTest + @MethodSource("testParams") + void testComplicatedUdf(ValuesDataSink.SinkApi sinkApi, String language) throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.TRANSFORM_TABLE); + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + // Setup value sink + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + sinkConfig.set(ValuesDataSinkOptions.SINK_API, sinkApi); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + // Setup transform + TransformDef transformDef = + new TransformDef( + "default_namespace.default_schema.table1", + "*, addone(addone(col1)) as inccol, typeof(42) as typ, format('%s-%d', col1, 42) as fmt, lifecycle() as stt", + null, + "col1", + null, + "key1=value1", + ""); + + // Setup pipeline + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef( + sourceDef, + sinkDef, + Collections.emptyList(), + Collections.singletonList(transformDef), + Arrays.asList( + new UdfDef( + "lifecycle", + String.format( + "org.apache.flink.cdc.udf.examples.%s.LifecycleFunctionClass", + language)), + new UdfDef( + "addone", + String.format( + "org.apache.flink.cdc.udf.examples.%s.AddOneFunctionClass", + language)), + new UdfDef( + "typeof", + String.format( + "org.apache.flink.cdc.udf.examples.%s.TypeOfFunctionClass", + language)), + new UdfDef( + "format", + String.format( + "org.apache.flink.cdc.udf.examples.%s.FormatFunctionClass", + language))), + pipelineConfig); + + // Execute the pipeline + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + + // Check the order and content of all received events + String[] outputEvents = outCaptor.toString().trim().split("\n"); + assertThat(outputEvents) + .contains("[ LifecycleFunction ] opened.") + .contains( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`inccol` STRING,`typ` STRING,`fmt` STRING,`stt` STRING}, primaryKeys=col1, options=({key1=value1})}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, 3, Integer: 42, 1-42, #0], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, 4, Integer: 42, 2-42, #1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3, 5, Integer: 42, 3-42, #2], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1, 3, Integer: 42, 1-42, #3], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , 4, Integer: 42, 2-42, #4], after=[2, x, 4, Integer: 42, 2-42, #5], op=UPDATE, meta=()}") + .contains("[ LifecycleFunction ] closed. Called 6 times."); + } + + // -------------------------- + // Flink-compatible UDF tests + // -------------------------- + @ParameterizedTest + @MethodSource("testParams") + void testTransformWithFlinkUdf(ValuesDataSink.SinkApi sinkApi, String language) + throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.TRANSFORM_TABLE); + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + // Setup value sink + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + sinkConfig.set(ValuesDataSinkOptions.SINK_API, sinkApi); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + // Setup transform + TransformDef transformDef = + new TransformDef( + "default_namespace.default_schema.table1", + "*,format('from %s to %s is %s', col1, 'z', 'lie') AS fmt", + null, + "col1", + null, + "key1=value1", + ""); + + UdfDef udfDef = + new UdfDef( + "format", + String.format( + "org.apache.flink.udf.examples.%s.FormatFunctionClass", language)); + + // Setup pipeline + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef( + sourceDef, + sinkDef, + Collections.emptyList(), + Collections.singletonList(transformDef), + Collections.singletonList(udfDef), + pipelineConfig); + + // Execute the pipeline + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + + // Check the order and content of all received events + String[] outputEvents = outCaptor.toString().trim().split("\n"); + assertThat(outputEvents) + .containsExactly( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`fmt` STRING}, primaryKeys=col1, options=({key1=value1})}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, from 1 to z is lie], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, from 2 to z is lie], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3, from 3 to z is lie], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1, from 1 to z is lie], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , from 2 to z is lie], after=[2, x, from 2 to z is lie], op=UPDATE, meta=()}"); + } + + @ParameterizedTest + @MethodSource("testParams") + void testFilterWithFlinkUdf(ValuesDataSink.SinkApi sinkApi, String language) throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.TRANSFORM_TABLE); + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + // Setup value sink + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + sinkConfig.set(ValuesDataSinkOptions.SINK_API, sinkApi); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + // Setup transform + TransformDef transformDef = + new TransformDef( + "default_namespace.default_schema.table1", + "*,addone(col1) as collen", + "addone(col1) <> '2'", + "col1", + null, + "key1=value1", + ""); + + UdfDef udfDef = + new UdfDef( + "addone", + String.format( + "org.apache.flink.udf.examples.%s.AddOneFunctionClass", language)); + + // Setup pipeline + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef( + sourceDef, + sinkDef, + Collections.emptyList(), + Collections.singletonList(transformDef), + Collections.singletonList(udfDef), + pipelineConfig); + + // Execute the pipeline + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + + // Check the order and content of all received events + String[] outputEvents = outCaptor.toString().trim().split("\n"); + assertThat(outputEvents) + .containsExactly( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`collen` STRING}, primaryKeys=col1, options=({key1=value1})}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, 3], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3, 4], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , 3], after=[2, x, 3], op=UPDATE, meta=()}"); + } + + @ParameterizedTest + @MethodSource("testParams") + void testOverloadedFlinkUdf(ValuesDataSink.SinkApi sinkApi, String language) throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.TRANSFORM_TABLE); + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + // Setup value sink + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + sinkConfig.set(ValuesDataSinkOptions.SINK_API, sinkApi); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + // Setup transform + TransformDef transformDef = + new TransformDef( + "default_namespace.default_schema.table1", + "*, typeof(true) as tob, typeof(1) as toi, typeof(3.14) as tof, typeof('str') as tos", + null, + "col1", + null, + "key1=value1", + ""); + + UdfDef udfDef = + new UdfDef( + "typeof", + String.format( + "org.apache.flink.udf.examples.%s.TypeOfFunctionClass", language)); + + // Setup pipeline + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef( + sourceDef, + sinkDef, + Collections.emptyList(), + Collections.singletonList(transformDef), + Collections.singletonList(udfDef), + pipelineConfig); + + // Execute the pipeline + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + + // Check the order and content of all received events + String[] outputEvents = outCaptor.toString().trim().split("\n"); + assertThat(outputEvents) + .containsExactly( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`tob` STRING,`toi` STRING,`tof` STRING,`tos` STRING}, primaryKeys=col1, options=({key1=value1})}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, Boolean: true, Integer: 1, Double: 3.14, String: str], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, Boolean: true, Integer: 1, Double: 3.14, String: str], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3, Boolean: true, Integer: 1, Double: 3.14, String: str], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1, Boolean: true, Integer: 1, Double: 3.14, String: str], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , Boolean: true, Integer: 1, Double: 3.14, String: str], after=[2, x, Boolean: true, Integer: 1, Double: 3.14, String: str], op=UPDATE, meta=()}"); + } + + @ParameterizedTest + @MethodSource("testParams") + void testComplicatedFlinkUdf(ValuesDataSink.SinkApi sinkApi, String language) throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.TRANSFORM_TABLE); + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + // Setup value sink + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + sinkConfig.set(ValuesDataSinkOptions.SINK_API, sinkApi); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + // Setup transform + TransformDef transformDef = + new TransformDef( + "default_namespace.default_schema.table1", + "*, addone(addone(col1)) as inccol, typeof(42) as typ, format('%s-%d', col1, 42) as fmt", + null, + "col1", + null, + "key1=value1", + ""); + + // Setup pipeline + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef( + sourceDef, + sinkDef, + Collections.emptyList(), + Collections.singletonList(transformDef), + Arrays.asList( + new UdfDef( + "addone", + String.format( + "org.apache.flink.udf.examples.%s.AddOneFunctionClass", + language)), + new UdfDef( + "typeof", + String.format( + "org.apache.flink.udf.examples.%s.TypeOfFunctionClass", + language)), + new UdfDef( + "format", + String.format( + "org.apache.flink.udf.examples.%s.FormatFunctionClass", + language))), + pipelineConfig); + + // Execute the pipeline + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + + // Check the order and content of all received events + String[] outputEvents = outCaptor.toString().trim().split("\n"); + assertThat(outputEvents) + .contains( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`inccol` STRING,`typ` STRING,`fmt` STRING}, primaryKeys=col1, options=({key1=value1})}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, 3, Integer: 42, 1-42], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, 4, Integer: 42, 2-42], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3, 5, Integer: 42, 3-42], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1, 3, Integer: 42, 1-42], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , 4, Integer: 42, 2-42], after=[2, x, 4, Integer: 42, 2-42], op=UPDATE, meta=()}"); + } + + private static Stream testParams() { + return Stream.of( + arguments(ValuesDataSink.SinkApi.SINK_FUNCTION, "java"), + arguments(ValuesDataSink.SinkApi.SINK_V2, "java"), + arguments(ValuesDataSink.SinkApi.SINK_FUNCTION, "scala"), + arguments(ValuesDataSink.SinkApi.SINK_V2, "scala")); + } +} diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml index 2326240b6d4..0f7c59bccb1 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml @@ -105,6 +105,12 @@ limitations under the License. ${project.version} test + + org.apache.flink + flink-cdc-pipeline-udf-examples + ${project.version} + test + @@ -113,6 +119,14 @@ limitations under the License. ${testcontainers.version} test + + + + org.scala-lang + scala-library + ${scala.version} + test + @@ -231,6 +245,26 @@ limitations under the License. ${project.build.directory}/dependencies + + + org.apache.flink + flink-cdc-pipeline-udf-examples + ${project.version} + udf-examples.jar + jar + ${project.build.directory}/dependencies + + + + + org.scala-lang + scala-library + ${scala.version} + scala-library.jar + jar + ${project.build.directory}/dependencies + + diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/UdfE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/UdfE2eITCase.java new file mode 100644 index 00000000000..7270fc5dae4 --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/UdfE2eITCase.java @@ -0,0 +1,404 @@ +/* + * 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.pipeline.tests; + +import org.apache.flink.cdc.common.test.utils.TestUtils; +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.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.output.Slf4jLogConsumer; + +import java.nio.file.Path; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeoutException; + +/** E2e tests for the {@link TransformSchemaOperator}. */ +@RunWith(Parameterized.class) +public class UdfE2eITCase extends PipelineTestEnvironment { + private static final Logger LOG = LoggerFactory.getLogger(TransformE2eITCase.class); + + // ------------------------------------------------------------------------------------------ + // MySQL Variables (we always use MySQL as the data source for easier verifying) + // ------------------------------------------------------------------------------------------ + protected static final String MYSQL_TEST_USER = "mysqluser"; + 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"; + + @ClassRule + public static final MySqlContainer MYSQL = + (MySqlContainer) + new MySqlContainer( + MySqlVersion.V8_0) // v8 support both ARM and AMD architectures + .withConfigurationOverride("docker/mysql/my.cnf") + .withSetupSQL("docker/mysql/setup.sql") + .withDatabaseName("flink-test") + .withUsername("flinkuser") + .withPassword("flinkpw") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_MYSQL_ALIAS) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + protected final UniqueDatabase transformRenameDatabase = + new UniqueDatabase(MYSQL, "transform_test", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + + @Before + public void before() throws Exception { + super.before(); + transformRenameDatabase.createAndInitialize(); + } + + @After + public void after() { + super.after(); + transformRenameDatabase.dropDatabase(); + } + + @Test + public void testUserDefinedFunctionsInJava() throws Exception { + testUserDefinedFunctions("java"); + } + + @Test + public void testUserDefinedFunctionsInScala() throws Exception { + testUserDefinedFunctions("scala"); + } + + @Test + public void testFlinkCompatibleScalarFunctionsInJava() throws Exception { + testFlinkCompatibleScalarFunctions("java"); + } + + @Test + public void testFlinkCompatibleScalarFunctionsInScala() throws Exception { + testFlinkCompatibleScalarFunctions("scala"); + } + + private void testUserDefinedFunctions(String language) throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.\\.*\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "transform:\n" + + " - source-table: %s.TABLEALPHA\n" + + " projection: ID, VERSION, addone(addone(ID)) AS INC_ID, format('<%%s>', VERSION) AS FMT_VER\n" + + " filter: addone(ID) <> '1009'\n" + + " - source-table: %s.TABLEBETA\n" + + " projection: ID, VERSION, answer() AS ANS, typeof(ID) AS TYP\n" + + "\n" + + "pipeline:\n" + + " parallelism: 1\n" + + " user-defined-function:\n" + + " - name: addone\n" + + " classpath: org.apache.flink.cdc.udf.examples.%s.AddOneFunctionClass\n" + + " - name: format\n" + + " classpath: org.apache.flink.cdc.udf.examples.%s.FormatFunctionClass\n" + + " - name: lifecycle\n" + + " classpath: org.apache.flink.cdc.udf.examples.%s.LifecycleFunctionClass\n" + + " - name: typeof\n" + + " classpath: org.apache.flink.cdc.udf.examples.%s.TypeOfFunctionClass\n" + + " - name: answer\n" + + " classpath: org.apache.flink.cdc.udf.examples.%s.TypeHintFunctionClass\n", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName(), + language, + language, + language, + language, + language); + 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"); + Path udfJar = TestUtils.getResource("udf-examples.jar"); + Path scalaLibJar = TestUtils.getResource("scala-library.jar"); + submitPipelineJob( + pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar, udfJar, scalaLibJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + waitUntilSpecificEvent("[ LifecycleFunction ] opened.", 60000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 1013, <11>], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 60000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Forty-two, Integer: 2014], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 60000L); + + List expectedEvents = + Arrays.asList( + String.format( + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`INC_ID` STRING,`FMT_VER` STRING}, primaryKeys=ID, options=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 1011, <8.1>], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 1012, <10>], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 1013, <11>], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`ANS` STRING,`TYP` STRING}, primaryKeys=ID, options=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Forty-two, Integer: 2011], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Forty-two, Integer: 2012], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Forty-two, Integer: 2013], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Forty-two, Integer: 2014], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName())); + validateResult(expectedEvents); + LOG.info("Begin incremental reading stage."); + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + transformRenameDatabase.getDatabaseName()); + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stat = conn.createStatement()) { + stat.execute("UPDATE TABLEALPHA SET VERSION='100' WHERE id=1009;"); + stat.execute("INSERT INTO TABLEALPHA VALUES (3007, '7', 79);"); + stat.execute("DELETE FROM TABLEBETA WHERE id=2011;"); + } catch (SQLException e) { + LOG.error("Update table for CDC failed.", e); + throw e; + } + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 1011, <8.1>], after=[1009, 100, 1011, <100>], op=UPDATE, meta=()}", + transformRenameDatabase.getDatabaseName()), + 20000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 3009, <7>], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 20000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Forty-two, Integer: 2011], after=[], op=DELETE, meta=()}", + transformRenameDatabase.getDatabaseName()), + 20000L); + } + + private void testFlinkCompatibleScalarFunctions(String language) throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.\\.*\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "transform:\n" + + " - source-table: %s.TABLEALPHA\n" + + " projection: ID, VERSION, addone(addone(ID)) AS INC_ID, format('<%%s>', VERSION) AS FMT_VER\n" + + " filter: addone(ID) <> '1009'\n" + + " - source-table: %s.TABLEBETA\n" + + " projection: ID, VERSION, typeof(ID) AS TYP\n" + + "\n" + + "pipeline:\n" + + " parallelism: 1\n" + + " user-defined-function:\n" + + " - name: addone\n" + + " classpath: org.apache.flink.udf.examples.%s.AddOneFunctionClass\n" + + " - name: format\n" + + " classpath: org.apache.flink.udf.examples.%s.FormatFunctionClass\n" + + " - name: typeof\n" + + " classpath: org.apache.flink.udf.examples.%s.TypeOfFunctionClass\n", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName(), + transformRenameDatabase.getDatabaseName(), + language, + language, + language); + 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"); + Path udfJar = TestUtils.getResource("udf-examples.jar"); + Path scalaLibJar = TestUtils.getResource("scala-library.jar"); + submitPipelineJob( + pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar, udfJar, scalaLibJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 1013, <11>], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 60000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Integer: 2014], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 60000L); + + List expectedEvents = + Arrays.asList( + String.format( + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`INC_ID` STRING,`FMT_VER` STRING}, primaryKeys=ID, options=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 1011, <8.1>], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 1012, <10>], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 1013, <11>], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`TYP` STRING}, primaryKeys=ID, options=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Integer: 2011], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Integer: 2012], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Integer: 2013], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Integer: 2014], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName())); + validateResult(expectedEvents); + LOG.info("Begin incremental reading stage."); + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + transformRenameDatabase.getDatabaseName()); + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stat = conn.createStatement()) { + stat.execute("UPDATE TABLEALPHA SET VERSION='100' WHERE id=1009;"); + stat.execute("INSERT INTO TABLEALPHA VALUES (3007, '7', 79);"); + stat.execute("DELETE FROM TABLEBETA WHERE id=2011;"); + } catch (SQLException e) { + LOG.error("Update table for CDC failed.", e); + throw e; + } + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 1011, <8.1>], after=[1009, 100, 1011, <100>], op=UPDATE, meta=()}", + transformRenameDatabase.getDatabaseName()), + 20000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 3009, <7>], op=INSERT, meta=()}", + transformRenameDatabase.getDatabaseName()), + 20000L); + + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Integer: 2011], after=[], op=DELETE, meta=()}", + transformRenameDatabase.getDatabaseName()), + 20000L); + } + + private void validateResult(List expectedEvents) throws Exception { + for (String event : expectedEvents) { + waitUntilSpecificEvent(event, 6000L); + } + } + + private void waitUntilSpecificEvent(String event, long timeout) throws Exception { + boolean result = false; + long endTimeout = System.currentTimeMillis() + timeout; + while (System.currentTimeMillis() < endTimeout) { + String stdout = taskManagerConsumer.toUtf8String(); + if (stdout.contains(event)) { + result = true; + break; + } + Thread.sleep(1000); + } + if (!result) { + throw new TimeoutException( + "failed to get specific event: " + + event + + " from stdout: " + + taskManagerConsumer.toUtf8String()); + } + } +} diff --git a/flink-cdc-pipeline-udf-examples/pom.xml b/flink-cdc-pipeline-udf-examples/pom.xml new file mode 100644 index 00000000000..d103ed5b459 --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/pom.xml @@ -0,0 +1,104 @@ + + + + 4.0.0 + + org.apache.flink + flink-cdc + ${revision} + + + flink-cdc-pipeline-udf-examples + + + 8 + 8 + UTF-8 + 4.9.2 + UTF-8 + + + + org.apache.flink + flink-cdc-common + ${project.version} + compile + + + org.scala-lang + scala-library + ${scala.version} + + + + + + + + + + + net.alchim31.maven + scala-maven-plugin + ${scala.plugin.version} + + incremental + + -Xlint:unchecked + -Xlint:deprecation + -encoding + ${compiler.encoding} + + + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + compile + + compile + + + + + + + + \ No newline at end of file diff --git a/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/AddOneFunctionClass.java b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/AddOneFunctionClass.java new file mode 100644 index 00000000000..32b6ff2aa5e --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/AddOneFunctionClass.java @@ -0,0 +1,31 @@ +/* + * 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.udf.examples.java; + +import org.apache.flink.cdc.common.udf.UserDefinedFunction; + +/** This is an example UDF class for testing purposes only. */ +public class AddOneFunctionClass implements UserDefinedFunction { + public String eval(Integer num) { + return String.valueOf(num + 1); + } + + public String eval(String num) { + return String.valueOf(Integer.parseInt(num) + 1); + } +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/FormatFunctionClass.java b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/FormatFunctionClass.java new file mode 100644 index 00000000000..606362bb364 --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/FormatFunctionClass.java @@ -0,0 +1,27 @@ +/* + * 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.udf.examples.java; + +import org.apache.flink.cdc.common.udf.UserDefinedFunction; + +/** This is an example UDF class for testing purposes only. */ +public class FormatFunctionClass implements UserDefinedFunction { + public String eval(String format, Object... args) { + return String.format(format, args); + } +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/LifecycleFunctionClass.java b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/LifecycleFunctionClass.java new file mode 100644 index 00000000000..d0d700ccd92 --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/LifecycleFunctionClass.java @@ -0,0 +1,40 @@ +/* + * 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.udf.examples.java; + +import org.apache.flink.cdc.common.udf.UserDefinedFunction; + +/** This is an example UDF class for testing purposes only. */ +public class LifecycleFunctionClass implements UserDefinedFunction { + private Integer counter; + + public String eval() { + return "#" + (counter++); + } + + @Override + public void open() { + counter = 0; + System.out.println("[ LifecycleFunction ] opened."); + } + + @Override + public void close() { + System.out.println("[ LifecycleFunction ] closed. Called " + counter + " times."); + } +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/TypeHintFunctionClass.java b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/TypeHintFunctionClass.java new file mode 100644 index 00000000000..f0329cb6fa6 --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/TypeHintFunctionClass.java @@ -0,0 +1,37 @@ +/* + * 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.udf.examples.java; + +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.udf.UserDefinedFunction; + +/** This is an example UDF class for testing purposes only. */ +public class TypeHintFunctionClass implements UserDefinedFunction { + + @Override + public DataType getReturnType() { + return DataTypes.STRING(); + } + + public Object eval() { + // Return type could not be inferred from function signature + // So we can test if type hint works as expected + return "Forty-two"; + } +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/TypeOfFunctionClass.java b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/TypeOfFunctionClass.java new file mode 100644 index 00000000000..2d0f0069da7 --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/cdc/udf/examples/java/TypeOfFunctionClass.java @@ -0,0 +1,43 @@ +/* + * 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.udf.examples.java; + +import org.apache.flink.cdc.common.udf.UserDefinedFunction; + +/** This is an example UDF class for testing purposes only. */ +public class TypeOfFunctionClass implements UserDefinedFunction { + public String eval(Boolean b) { + return "Boolean: " + b; + } + + public String eval(Integer i) { + return "Integer: " + i; + } + + public String eval(Float f) { + return "Float: " + f; + } + + public String eval(Double d) { + return "Double: " + d; + } + + public String eval(String s) { + return "String: " + s; + } +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/udf/examples/java/AddOneFunctionClass.java b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/udf/examples/java/AddOneFunctionClass.java new file mode 100644 index 00000000000..2d40a8a5be9 --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/udf/examples/java/AddOneFunctionClass.java @@ -0,0 +1,31 @@ +/* + * 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.udf.examples.java; + +import org.apache.flink.table.functions.ScalarFunction; + +/** This is an example Flink UDF class for testing purposes only. */ +public class AddOneFunctionClass extends ScalarFunction { + public String eval(Integer num) { + return String.valueOf(num + 1); + } + + public String eval(String num) { + return String.valueOf(Integer.parseInt(num) + 1); + } +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/udf/examples/java/FormatFunctionClass.java b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/udf/examples/java/FormatFunctionClass.java new file mode 100644 index 00000000000..8a4310a767d --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/udf/examples/java/FormatFunctionClass.java @@ -0,0 +1,27 @@ +/* + * 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.udf.examples.java; + +import org.apache.flink.table.functions.ScalarFunction; + +/** This is an example UDF class for testing purposes only. */ +public class FormatFunctionClass extends ScalarFunction { + public String eval(String format, Object... args) { + return String.format(format, args); + } +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/udf/examples/java/TypeOfFunctionClass.java b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/udf/examples/java/TypeOfFunctionClass.java new file mode 100644 index 00000000000..161fdd2d6a5 --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/java/org/apache/flink/udf/examples/java/TypeOfFunctionClass.java @@ -0,0 +1,43 @@ +/* + * 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.udf.examples.java; + +import org.apache.flink.table.functions.ScalarFunction; + +/** This is an example UDF class for testing purposes only. */ +public class TypeOfFunctionClass extends ScalarFunction { + public String eval(Boolean b) { + return "Boolean: " + b; + } + + public String eval(Integer i) { + return "Integer: " + i; + } + + public String eval(Float f) { + return "Float: " + f; + } + + public String eval(Double d) { + return "Double: " + d; + } + + public String eval(String s) { + return "String: " + s; + } +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/AddOneFunctionClass.scala b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/AddOneFunctionClass.scala new file mode 100644 index 00000000000..cbb958e5e1c --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/AddOneFunctionClass.scala @@ -0,0 +1,27 @@ +/* + * 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.udf.examples.scala + +import org.apache.flink.cdc.common.udf.UserDefinedFunction + +/** This is an example UDF class for testing purposes only. */ +class AddOneFunctionClass extends UserDefinedFunction { + def eval(num: Integer): String = (num + 1).toString + + def eval(num: String): String = (num.toInt + 1).toString +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/FormatFunctionClass.scala b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/FormatFunctionClass.scala new file mode 100644 index 00000000000..49cbffff9df --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/FormatFunctionClass.scala @@ -0,0 +1,27 @@ +/* + * 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.udf.examples.scala + +import org.apache.flink.cdc.common.udf.UserDefinedFunction + +import scala.annotation.varargs + +/** This is an example UDF class for testing purposes only. */ +class FormatFunctionClass extends UserDefinedFunction { + @varargs def eval(format: String, args: Object*): String = String.format(format, args:_*) +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/LifecycleFunctionClass.scala b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/LifecycleFunctionClass.scala new file mode 100644 index 00000000000..7cd9c2ee653 --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/LifecycleFunctionClass.scala @@ -0,0 +1,40 @@ +/* + * 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.udf.examples.scala + +import org.apache.flink.cdc.common.udf.UserDefinedFunction + +/** This is an example UDF class for testing purposes only. */ +class LifecycleFunctionClass extends UserDefinedFunction { + private var counter: Integer = null + + def eval: String = { + "#" + { + counter += 1; counter - 1 + } + } + + override def open(): Unit = { + counter = 0 + System.out.println("[ LifecycleFunction ] opened.") + } + + override def close(): Unit = { + System.out.println("[ LifecycleFunction ] closed. Called " + counter + " times.") + } +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/TypeHintFunctionClass.scala b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/TypeHintFunctionClass.scala new file mode 100644 index 00000000000..8c039e545db --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/TypeHintFunctionClass.scala @@ -0,0 +1,34 @@ +/* + * 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.udf.examples.scala + +import org.apache.flink.cdc.common.types.DataType +import org.apache.flink.cdc.common.types.DataTypes +import org.apache.flink.cdc.common.udf.UserDefinedFunction + + +/** This is an example UDF class for testing purposes only. */ +class TypeHintFunctionClass extends UserDefinedFunction { + override def getReturnType: DataType = DataTypes.STRING() + + def eval: Object = { + // Return type could not be inferred from function signature + // So we can test if type hint works as expected + "Forty-two" + } +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/TypeOfFunctionClass.scala b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/TypeOfFunctionClass.scala new file mode 100644 index 00000000000..1da5e4175e6 --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/cdc/udf/examples/scala/TypeOfFunctionClass.scala @@ -0,0 +1,33 @@ +/* + * 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.udf.examples.scala + +import org.apache.flink.cdc.common.udf.UserDefinedFunction + +/** This is an example UDF class for testing purposes only. */ +class TypeOfFunctionClass extends UserDefinedFunction { + def eval(b: Boolean): String = "Boolean: " + b + + def eval(i: Int): String = "Integer: " + i + + def eval(f: Float): String = "Float: " + f + + def eval(d: Double): String = "Double: " + d + + def eval(s: String): String = "String: " + s +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/udf/examples/scala/AddOneFunctionClass.scala b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/udf/examples/scala/AddOneFunctionClass.scala new file mode 100644 index 00000000000..85e02f8c935 --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/udf/examples/scala/AddOneFunctionClass.scala @@ -0,0 +1,27 @@ +/* + * 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.udf.examples.scala + +import org.apache.flink.table.functions.ScalarFunction + +/** This is an example Flink UDF class for testing purposes only. */ +class AddOneFunctionClass extends ScalarFunction { + def eval(num: Integer): String = (num + 1).toString + + def eval(num: String): String = (num.toInt + 1).toString +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/udf/examples/scala/FormatFunctionClass.scala b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/udf/examples/scala/FormatFunctionClass.scala new file mode 100644 index 00000000000..7c01f972623 --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/udf/examples/scala/FormatFunctionClass.scala @@ -0,0 +1,27 @@ +/* + * 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.udf.examples.scala + +import org.apache.flink.table.functions.ScalarFunction + +import scala.annotation.varargs + +/** This is an example UDF class for testing purposes only. */ +class FormatFunctionClass extends ScalarFunction { + @varargs def eval(format: String, args: Object*): String = String.format(format, args:_*) +} diff --git a/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/udf/examples/scala/TypeOfFunctionClass.scala b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/udf/examples/scala/TypeOfFunctionClass.scala new file mode 100644 index 00000000000..6be332e98bb --- /dev/null +++ b/flink-cdc-pipeline-udf-examples/src/main/scala/org/apache/flink/udf/examples/scala/TypeOfFunctionClass.scala @@ -0,0 +1,33 @@ +/* + * 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.udf.examples.scala + +import org.apache.flink.table.functions.ScalarFunction + +/** This is an example UDF class for testing purposes only. */ +class TypeOfFunctionClass extends ScalarFunction { + def eval(b: Boolean): String = "Boolean: " + b + + def eval(i: Int): String = "Integer: " + i + + def eval(f: Float): String = "Float: " + f + + def eval(d: Double): String = "Double: " + d + + def eval(s: String): String = "String: " + s +} diff --git a/flink-cdc-runtime/pom.xml b/flink-cdc-runtime/pom.xml index 77c1e6014f2..e278fd669ad 100644 --- a/flink-cdc-runtime/pom.xml +++ b/flink-cdc-runtime/pom.xml @@ -111,6 +111,12 @@ limitations under the License. test test-jar + + org.apache.flink + flink-cdc-pipeline-udf-examples + ${project.version} + test + \ No newline at end of file diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/ProjectionColumnProcessor.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/ProjectionColumnProcessor.java index 6c5202342d8..87098df2e10 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 @@ -20,6 +20,7 @@ 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.common.udf.UserDefinedFunctionDescriptor; import org.apache.flink.cdc.runtime.parser.JaninoCompiler; import org.apache.flink.cdc.runtime.parser.TransformParser; import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter; @@ -43,23 +44,37 @@ public class ProjectionColumnProcessor { private ProjectionColumn projectionColumn; private String timezone; private TransformExpressionKey transformExpressionKey; + private List udfFunctions; + private final transient List udfFunctionInstances; + private transient ExpressionEvaluator expressionEvaluator; public ProjectionColumnProcessor( - TableInfo tableInfo, ProjectionColumn projectionColumn, String timezone) { + TableInfo tableInfo, + ProjectionColumn projectionColumn, + String timezone, + List udfFunctions, + final List udfFunctionInstances) { this.tableInfo = tableInfo; this.projectionColumn = projectionColumn; this.timezone = timezone; + this.udfFunctions = udfFunctions; this.transformExpressionKey = generateTransformExpressionKey(); + this.expressionEvaluator = + TransformExpressionCompiler.compileExpression(transformExpressionKey, udfFunctions); + this.udfFunctionInstances = udfFunctionInstances; } public static ProjectionColumnProcessor of( - TableInfo tableInfo, ProjectionColumn projectionColumn, String timezone) { - return new ProjectionColumnProcessor(tableInfo, projectionColumn, timezone); + TableInfo tableInfo, + ProjectionColumn projectionColumn, + String timezone, + List udfFunctions, + List udfFunctionInstances) { + return new ProjectionColumnProcessor( + tableInfo, projectionColumn, timezone, udfFunctions, udfFunctionInstances); } public Object evaluate(BinaryRecordData after, long epochTime) { - ExpressionEvaluator expressionEvaluator = - TransformExpressionCompiler.compileExpression(transformExpressionKey); try { return expressionEvaluator.evaluate(generateParams(after, epochTime)); } catch (InvocationTargetException e) { @@ -76,6 +91,8 @@ public Object evaluate(BinaryRecordData after, long epochTime) { private Object[] generateParams(BinaryRecordData after, long epochTime) { List params = new ArrayList<>(); List columns = tableInfo.getSchema().getColumns(); + + // 1 - Add referenced columns RecordData.FieldGetter[] fieldGetters = tableInfo.getFieldGetters(); for (String originalColumnName : projectionColumn.getOriginalColumnNames()) { if (originalColumnName.equals(TransformParser.DEFAULT_NAMESPACE_NAME)) { @@ -100,8 +117,13 @@ private Object[] generateParams(BinaryRecordData after, long epochTime) { } } } + + // 2 - Add time-sensitive function arguments params.add(timezone); params.add(epochTime); + + // 3 - Add UDF function instances + params.addAll(udfFunctionInstances); return params.toArray(); } 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/TransformDataOperator.java index 20479c87322..ce5375ec9d1 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/TransformDataOperator.java @@ -29,6 +29,8 @@ 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.udf.UserDefinedFunction; +import org.apache.flink.cdc.common.udf.UserDefinedFunctionDescriptor; 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; @@ -71,6 +73,9 @@ public class TransformDataOperator extends AbstractStreamOperator /** keep the relationship of TableId and table information. */ private final Map tableInfoMap; + private final List udfFunctions; + private transient Map udfFunctionInstances; + private transient Map, TransformProjectionProcessor> transformProjectionProcessorMap; private transient Map, TransformFilterProcessor> @@ -85,6 +90,7 @@ public static class Builder { private final List> transformRules = new ArrayList<>(); private OperatorID schemaOperatorID; private String timezone; + private List udfFunctions = new ArrayList<>(); public TransformDataOperator.Builder addTransform( String tableInclusions, @Nullable String projection, @Nullable String filter) { @@ -106,21 +112,31 @@ public TransformDataOperator.Builder addTimezone(String timezone) { return this; } + public TransformDataOperator.Builder addUdfFunctions( + List udfFunctions) { + this.udfFunctions.addAll(udfFunctions); + return this; + } + public TransformDataOperator build() { - return new TransformDataOperator(transformRules, schemaOperatorID, timezone); + return new TransformDataOperator( + transformRules, schemaOperatorID, timezone, udfFunctions); } } private TransformDataOperator( List> transformRules, OperatorID schemaOperatorID, - String timezone) { + String timezone, + List udfFunctions) { this.transformRules = transformRules; this.schemaOperatorID = schemaOperatorID; this.timezone = timezone; this.tableInfoMap = new ConcurrentHashMap<>(); this.transformFilterProcessorMap = new ConcurrentHashMap<>(); this.transformProjectionProcessorMap = new ConcurrentHashMap<>(); + this.udfFunctions = udfFunctions; + this.udfFunctionInstances = new ConcurrentHashMap<>(); } @Override @@ -153,13 +169,26 @@ public void open() throws Exception { return new Tuple4<>( selectors, TransformProjection.of(projection), - TransformFilter.of(filterExpression), + TransformFilter.of(filterExpression, udfFunctions), containFilteredComputedColumn( projection, filterExpression)); }) .collect(Collectors.toList()); this.transformFilterProcessorMap = new ConcurrentHashMap<>(); this.transformProjectionProcessorMap = new ConcurrentHashMap<>(); + this.udfFunctionInstances = new ConcurrentHashMap<>(); + udfFunctions.forEach( + udf -> { + try { + Class clazz = Class.forName(udf.getClassPath()); + udfFunctionInstances.put(udf.getName(), clazz.newInstance()); + } catch (ClassNotFoundException + | InstantiationException + | IllegalAccessException e) { + throw new RuntimeException("Failed to instantiate UDF function " + udf); + } + }); + initializeUdf(); } @Override @@ -172,6 +201,10 @@ public void finish() throws Exception { public void close() throws Exception { super.close(); clearOperator(); + + // Clean up UDF instances + destroyUdf(); + udfFunctionInstances.clear(); } @Override @@ -234,7 +267,10 @@ private void transformSchema(TableId tableId, Schema schema) throws Exception { Tuple2.of(tableId, transformProjection))) { transformProjectionProcessorMap.put( Tuple2.of(tableId, transformProjection), - TransformProjectionProcessor.of(transformProjection)); + TransformProjectionProcessor.of( + transformProjection, + udfFunctions, + getUdfFunctionInstances())); } TransformProjectionProcessor transformProjectionProcessor = transformProjectionProcessorMap.get( @@ -246,6 +282,12 @@ private void transformSchema(TableId tableId, Schema schema) throws Exception { } } + private List getUdfFunctionInstances() { + return udfFunctions.stream() + .map(e -> udfFunctionInstances.get(e.getName())) + .collect(Collectors.toList()); + } + private Optional processDataChangeEvent(DataChangeEvent dataChangeEvent) throws Exception { TableId tableId = dataChangeEvent.tableId(); @@ -272,7 +314,9 @@ private Optional processDataChangeEvent(DataChangeEvent dataCha TransformProjectionProcessor.of( getTableInfoFromSchemaEvolutionClient(tableId), transformProjection, - timezone)); + timezone, + udfFunctions, + getUdfFunctionInstances())); } TransformProjectionProcessor transformProjectionProcessor = transformProjectionProcessorMap.get( @@ -294,7 +338,9 @@ private Optional processDataChangeEvent(DataChangeEvent dataCha TransformFilterProcessor.of( getTableInfoFromSchemaEvolutionClient(tableId), transformFilter, - timezone)); + timezone, + udfFunctions, + getUdfFunctionInstances())); } TransformFilterProcessor transformFilterProcessor = transformFilterProcessorMap.get(Tuple2.of(tableId, transformFilter)); @@ -319,7 +365,9 @@ private Optional processDataChangeEvent(DataChangeEvent dataCha TransformProjectionProcessor.of( getTableInfoFromSchemaEvolutionClient(tableId), transformProjection, - timezone)); + timezone, + udfFunctions, + getUdfFunctionInstances())); } TransformProjectionProcessor transformProjectionProcessor = transformProjectionProcessorMap.get( @@ -412,4 +460,34 @@ private void clearOperator() { this.transformFilterProcessorMap = null; TransformExpressionCompiler.cleanUp(); } + + private void initializeUdf() { + udfFunctions.forEach( + udf -> { + try { + if (udf.isCdcPipelineUdf()) { + ((UserDefinedFunction) udfFunctionInstances.get(udf.getName())).open(); + } else { + // Do nothing, Flink-style UDF lifecycle hooks are not supported + } + } catch (Exception ex) { + throw new RuntimeException("Failed to initialize UDF " + udf, ex); + } + }); + } + + private void destroyUdf() { + udfFunctions.forEach( + udf -> { + try { + if (udf.isCdcPipelineUdf()) { + ((UserDefinedFunction) udfFunctionInstances.get(udf.getName())).close(); + } else { + // Do nothing, Flink-style UDF lifecycle hooks are not supported + } + } catch (Exception ex) { + throw new RuntimeException("Failed to destroy UDF " + udf, ex); + } + }); + } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformExpressionCompiler.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformExpressionCompiler.java index 510afdf6ec6..c549935d32b 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformExpressionCompiler.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformExpressionCompiler.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.runtime.operators.transform; import org.apache.flink.api.common.InvalidProgramException; +import org.apache.flink.cdc.common.udf.UserDefinedFunctionDescriptor; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.shaded.guava31.com.google.common.cache.Cache; @@ -26,6 +27,8 @@ import org.codehaus.commons.compiler.CompileException; import org.codehaus.janino.ExpressionEvaluator; +import java.util.List; + /** * The processor of the transform expression. It processes the expression of projections and * filters. @@ -41,16 +44,27 @@ public static void cleanUp() { } /** Compiles an expression code to a janino {@link ExpressionEvaluator}. */ - public static ExpressionEvaluator compileExpression(TransformExpressionKey key) { + public static ExpressionEvaluator compileExpression( + TransformExpressionKey key, List udfFunctions) { try { return COMPILED_EXPRESSION_CACHE.get( key, () -> { ExpressionEvaluator expressionEvaluator = new ExpressionEvaluator(); + + List argumentNames = key.getArgumentNames(); + List> argumentClasses = key.getArgumentClasses(); + + for (UserDefinedFunctionDescriptor udfFunction : udfFunctions) { + argumentNames.add("__instanceOf" + udfFunction.getClassName()); + argumentClasses.add(Class.forName(udfFunction.getClassPath())); + } + // Input args expressionEvaluator.setParameters( - key.getArgumentNames().toArray(new String[0]), - key.getArgumentClasses().toArray(new Class[0])); + argumentNames.toArray(new String[0]), + argumentClasses.toArray(new Class[0])); + // Result type expressionEvaluator.setExpressionType(key.getReturnClass()); try { diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformFilter.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformFilter.java index 422623af091..bbfe99a56c5 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformFilter.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformFilter.java @@ -17,6 +17,7 @@ package org.apache.flink.cdc.runtime.operators.transform; +import org.apache.flink.cdc.common.udf.UserDefinedFunctionDescriptor; import org.apache.flink.cdc.common.utils.StringUtils; import org.apache.flink.cdc.runtime.parser.TransformParser; @@ -60,13 +61,15 @@ public List getColumnNames() { return columnNames; } - public static Optional of(String filterExpression) { + public static Optional of( + String filterExpression, List udfFunctions) { if (StringUtils.isNullOrWhitespaceOnly(filterExpression)) { return Optional.empty(); } List columnNames = TransformParser.parseFilterColumnNameList(filterExpression); String scriptExpression = - TransformParser.translateFilterExpressionToJaninoExpression(filterExpression); + TransformParser.translateFilterExpressionToJaninoExpression( + filterExpression, udfFunctions); return Optional.of(new TransformFilter(filterExpression, scriptExpression, columnNames)); } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformFilterProcessor.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformFilterProcessor.java index 7eaefc3c84b..0681d467833 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 @@ -20,6 +20,7 @@ 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.common.udf.UserDefinedFunctionDescriptor; import org.apache.flink.cdc.runtime.parser.JaninoCompiler; import org.apache.flink.cdc.runtime.parser.TransformParser; import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter; @@ -39,23 +40,35 @@ public class TransformFilterProcessor { private TransformFilter transformFilter; private String timezone; private TransformExpressionKey transformExpressionKey; + private final transient List udfFunctionInstances; + private transient ExpressionEvaluator expressionEvaluator; public TransformFilterProcessor( - TableInfo tableInfo, TransformFilter transformFilter, String timezone) { + TableInfo tableInfo, + TransformFilter transformFilter, + String timezone, + List udfFunctions, + List udfFunctionInstances) { this.tableInfo = tableInfo; this.transformFilter = transformFilter; this.timezone = timezone; - transformExpressionKey = generateTransformExpressionKey(); + this.transformExpressionKey = generateTransformExpressionKey(); + this.udfFunctionInstances = udfFunctionInstances; + this.expressionEvaluator = + TransformExpressionCompiler.compileExpression(transformExpressionKey, udfFunctions); } public static TransformFilterProcessor of( - TableInfo tableInfo, TransformFilter transformFilter, String timezone) { - return new TransformFilterProcessor(tableInfo, transformFilter, timezone); + TableInfo tableInfo, + TransformFilter transformFilter, + String timezone, + List udfFunctions, + List udfFunctionInstances) { + return new TransformFilterProcessor( + tableInfo, transformFilter, timezone, udfFunctions, udfFunctionInstances); } public boolean process(BinaryRecordData after, long epochTime) { - ExpressionEvaluator expressionEvaluator = - TransformExpressionCompiler.compileExpression(transformExpressionKey); try { return (Boolean) expressionEvaluator.evaluate(generateParams(after, epochTime)); } catch (InvocationTargetException e) { @@ -71,6 +84,8 @@ public boolean process(BinaryRecordData after, long epochTime) { private Object[] generateParams(BinaryRecordData after, long epochTime) { List params = new ArrayList<>(); List columns = tableInfo.getSchema().getColumns(); + + // 1 - Add referenced columns RecordData.FieldGetter[] fieldGetters = tableInfo.getFieldGetters(); for (String columnName : transformFilter.getColumnNames()) { if (columnName.equals(TransformParser.DEFAULT_NAMESPACE_NAME)) { @@ -95,8 +110,13 @@ private Object[] generateParams(BinaryRecordData after, long epochTime) { } } } + + // 2 - Add time-sensitive function arguments params.add(timezone); params.add(epochTime); + + // 3 - Add UDF function instances + params.addAll(udfFunctionInstances); return params.toArray(); } 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 7049bbdfda5..418fc0df994 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 @@ -22,6 +22,7 @@ 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.udf.UserDefinedFunctionDescriptor; import org.apache.flink.cdc.runtime.parser.TransformParser; import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter; @@ -52,17 +53,23 @@ public class TransformProjectionProcessor { private TransformProjection transformProjection; private String timezone; private Map projectionColumnProcessorMap; + private List udfFunctions; + private final transient List udfFunctionInstances; public TransformProjectionProcessor( TableInfo tableInfo, TableChangeInfo tableChangeInfo, TransformProjection transformProjection, - String timezone) { + String timezone, + List udfFunctions, + final List udfFunctionInstances) { this.tableInfo = tableInfo; this.tableChangeInfo = tableChangeInfo; this.transformProjection = transformProjection; this.timezone = timezone; this.projectionColumnProcessorMap = new ConcurrentHashMap<>(); + this.udfFunctions = udfFunctions; + this.udfFunctionInstances = udfFunctionInstances; } public boolean hasTableChangeInfo() { @@ -74,24 +81,43 @@ public boolean hasTableInfo() { } public static TransformProjectionProcessor of( - TableInfo tableInfo, TransformProjection transformProjection, String timezone) { - return new TransformProjectionProcessor(tableInfo, null, transformProjection, timezone); + TableInfo tableInfo, + TransformProjection transformProjection, + String timezone, + List udfFunctions, + List udfFunctionInstances) { + return new TransformProjectionProcessor( + tableInfo, null, transformProjection, timezone, udfFunctions, udfFunctionInstances); } public static TransformProjectionProcessor of( - TableChangeInfo tableChangeInfo, TransformProjection transformProjection) { - return new TransformProjectionProcessor(null, tableChangeInfo, transformProjection, null); + TableChangeInfo tableChangeInfo, + TransformProjection transformProjection, + List udfFunctions, + List udfFunctionInstances) { + return new TransformProjectionProcessor( + null, + tableChangeInfo, + transformProjection, + null, + udfFunctions, + udfFunctionInstances); } - public static TransformProjectionProcessor of(TransformProjection transformProjection) { - return new TransformProjectionProcessor(null, null, transformProjection, null); + public static TransformProjectionProcessor of( + TransformProjection transformProjection, + List udfFunctions, + List udfFunctionInstances) { + return new TransformProjectionProcessor( + null, null, transformProjection, null, udfFunctions, udfFunctionInstances); } public CreateTableEvent processCreateTableEvent(CreateTableEvent createTableEvent) { List projectionColumns = TransformParser.generateProjectionColumns( transformProjection.getProjection(), - createTableEvent.getSchema().getColumns()); + createTableEvent.getSchema().getColumns(), + udfFunctions); transformProjection.setProjectionColumns(projectionColumns); List allColumnList = transformProjection.getAllColumnList(); // add the column of projection into Schema @@ -102,7 +128,7 @@ public CreateTableEvent processCreateTableEvent(CreateTableEvent createTableEven public void processSchemaChangeEvent(Schema schema) { List projectionColumns = TransformParser.generateProjectionColumns( - transformProjection.getProjection(), schema.getColumns()); + transformProjection.getProjection(), schema.getColumns(), udfFunctions); transformProjection.setProjectionColumns(projectionColumns); } @@ -144,7 +170,11 @@ public BinaryRecordData processData(BinaryRecordData after, long epochTime) { projectionColumnProcessorMap.put( projectionColumn.getColumnName(), ProjectionColumnProcessor.of( - tableInfo, projectionColumn, timezone)); + tableInfo, + projectionColumn, + timezone, + udfFunctions, + udfFunctionInstances)); } ProjectionColumnProcessor projectionColumnProcessor = projectionColumnProcessorMap.get(projectionColumn.getColumnName()); 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/TransformSchemaOperator.java index 230fc1a6ffd..59673a2de92 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/TransformSchemaOperator.java @@ -30,6 +30,7 @@ import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.schema.Selectors; +import org.apache.flink.cdc.common.udf.UserDefinedFunctionDescriptor; import org.apache.flink.cdc.common.utils.SchemaUtils; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; @@ -42,6 +43,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -58,6 +60,7 @@ public class TransformSchemaOperator extends AbstractStreamOperator private transient Map processorMap; private final List> schemaMetadataTransformers; private transient ListState state; + private final List udfFunctions; public static TransformSchemaOperator.Builder newBuilder() { return new TransformSchemaOperator.Builder(); @@ -68,6 +71,8 @@ public static class Builder { private final List> transformRules = new ArrayList<>(); + private final List udfFunctions = new ArrayList<>(); + public TransformSchemaOperator.Builder addTransform( String tableInclusions, @Nullable String projection, @@ -79,18 +84,26 @@ public TransformSchemaOperator.Builder addTransform( return this; } + public TransformSchemaOperator.Builder addUdfFunctions( + List udfFunctions) { + this.udfFunctions.addAll(udfFunctions); + return this; + } + public TransformSchemaOperator build() { - return new TransformSchemaOperator(transformRules); + return new TransformSchemaOperator(transformRules, udfFunctions); } } private TransformSchemaOperator( - List> transformRules) { + List> transformRules, + List udfFunctions) { this.transformRules = transformRules; this.tableChangeInfoMap = new ConcurrentHashMap<>(); this.processorMap = new ConcurrentHashMap<>(); this.schemaMetadataTransformers = new ArrayList<>(); this.chainingStrategy = ChainingStrategy.ALWAYS; + this.udfFunctions = udfFunctions; } @Override @@ -216,7 +229,11 @@ private CreateTableEvent transformCreateTableEvent(CreateTableEvent createTableE if (transformProjection.isValid()) { if (!processorMap.containsKey(tableId)) { processorMap.put( - tableId, TransformProjectionProcessor.of(transformProjection)); + tableId, + TransformProjectionProcessor.of( + transformProjection, + udfFunctions, + Collections.emptyList())); } TransformProjectionProcessor transformProjectionProcessor = processorMap.get(tableId); @@ -271,7 +288,12 @@ private DataChangeEvent processProjection( TableChangeInfo tableChangeInfo = tableChangeInfoMap.get(tableId); if (!processorMap.containsKey(tableId) || !processorMap.get(tableId).hasTableChangeInfo()) { processorMap.put( - tableId, TransformProjectionProcessor.of(tableChangeInfo, transformProjection)); + tableId, + TransformProjectionProcessor.of( + tableChangeInfo, + transformProjection, + udfFunctions, + Collections.emptyList())); } TransformProjectionProcessor transformProjectionProcessor = processorMap.get(tableId); BinaryRecordData before = (BinaryRecordData) dataChangeEvent.before(); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/JaninoCompiler.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/JaninoCompiler.java index 2dd1b8402e4..521d88a47f2 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/JaninoCompiler.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/JaninoCompiler.java @@ -19,7 +19,9 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.io.ParseException; +import org.apache.flink.cdc.common.udf.UserDefinedFunctionDescriptor; import org.apache.flink.cdc.common.utils.StringUtils; +import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlBasicTypeNameSpec; @@ -39,6 +41,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Optional; /** * Use Janino compiler to compiler the statement of flink cdc pipeline transform into the executable @@ -84,21 +87,23 @@ public static ExpressionEvaluator compileExpression( } } - public static String translateSqlNodeToJaninoExpression(SqlNode transform) { - Java.Rvalue rvalue = translateSqlNodeToJaninoRvalue(transform); + public static String translateSqlNodeToJaninoExpression( + SqlNode transform, List udfFunctions) { + Java.Rvalue rvalue = translateSqlNodeToJaninoRvalue(transform, udfFunctions); if (rvalue != null) { return rvalue.toString(); } return ""; } - public static Java.Rvalue translateSqlNodeToJaninoRvalue(SqlNode transform) { + public static Java.Rvalue translateSqlNodeToJaninoRvalue( + SqlNode transform, List udfFunctions) { if (transform instanceof SqlIdentifier) { return translateSqlIdentifier((SqlIdentifier) transform); } else if (transform instanceof SqlBasicCall) { - return translateSqlBasicCall((SqlBasicCall) transform); + return translateSqlBasicCall((SqlBasicCall) transform, udfFunctions); } else if (transform instanceof SqlCase) { - return translateSqlCase((SqlCase) transform); + return translateSqlCase((SqlCase) transform, udfFunctions); } else if (transform instanceof SqlLiteral) { return translateSqlSqlLiteral((SqlLiteral) transform); } @@ -129,32 +134,35 @@ private static Java.Rvalue translateSqlSqlLiteral(SqlLiteral sqlLiteral) { return new Java.AmbiguousName(Location.NOWHERE, new String[] {value}); } - private static Java.Rvalue translateSqlBasicCall(SqlBasicCall sqlBasicCall) { + private static Java.Rvalue translateSqlBasicCall( + SqlBasicCall sqlBasicCall, List udfFunctions) { List operandList = sqlBasicCall.getOperandList(); List atoms = new ArrayList<>(); for (SqlNode sqlNode : operandList) { - translateSqlNodeToAtoms(sqlNode, atoms); + translateSqlNodeToAtoms(sqlNode, atoms, udfFunctions); } if (NO_OPERAND_TIMESTAMP_FUNCTIONS.contains(sqlBasicCall.getOperator().getName())) { atoms.add(new Java.AmbiguousName(Location.NOWHERE, new String[] {DEFAULT_EPOCH_TIME})); atoms.add(new Java.AmbiguousName(Location.NOWHERE, new String[] {DEFAULT_TIME_ZONE})); } - return sqlBasicCallToJaninoRvalue(sqlBasicCall, atoms.toArray(new Java.Rvalue[0])); + return sqlBasicCallToJaninoRvalue( + sqlBasicCall, atoms.toArray(new Java.Rvalue[0]), udfFunctions); } - private static Java.Rvalue translateSqlCase(SqlCase sqlCase) { + private static Java.Rvalue translateSqlCase( + SqlCase sqlCase, List udfFunctions) { SqlNodeList whenOperands = sqlCase.getWhenOperands(); SqlNodeList thenOperands = sqlCase.getThenOperands(); SqlNode elseOperand = sqlCase.getElseOperand(); List whenAtoms = new ArrayList<>(); for (SqlNode sqlNode : whenOperands) { - translateSqlNodeToAtoms(sqlNode, whenAtoms); + translateSqlNodeToAtoms(sqlNode, whenAtoms, udfFunctions); } List thenAtoms = new ArrayList<>(); for (SqlNode sqlNode : thenOperands) { - translateSqlNodeToAtoms(sqlNode, thenAtoms); + translateSqlNodeToAtoms(sqlNode, thenAtoms, udfFunctions); } - Java.Rvalue elseAtoms = translateSqlNodeToJaninoRvalue(elseOperand); + Java.Rvalue elseAtoms = translateSqlNodeToJaninoRvalue(elseOperand, udfFunctions); Java.Rvalue sqlCaseRvalueTemp = elseAtoms; for (int i = whenAtoms.size() - 1; i >= 0; i--) { sqlCaseRvalueTemp = @@ -167,24 +175,29 @@ private static Java.Rvalue translateSqlCase(SqlCase sqlCase) { return new Java.ParenthesizedExpression(Location.NOWHERE, sqlCaseRvalueTemp); } - private static void translateSqlNodeToAtoms(SqlNode sqlNode, List atoms) { + private static void translateSqlNodeToAtoms( + SqlNode sqlNode, + List atoms, + List udfFunctions) { if (sqlNode instanceof SqlIdentifier) { atoms.add(translateSqlIdentifier((SqlIdentifier) sqlNode)); } else if (sqlNode instanceof SqlLiteral) { atoms.add(translateSqlSqlLiteral((SqlLiteral) sqlNode)); } else if (sqlNode instanceof SqlBasicCall) { - atoms.add(translateSqlBasicCall((SqlBasicCall) sqlNode)); + atoms.add(translateSqlBasicCall((SqlBasicCall) sqlNode, udfFunctions)); } else if (sqlNode instanceof SqlNodeList) { for (SqlNode node : (SqlNodeList) sqlNode) { - translateSqlNodeToAtoms(node, atoms); + translateSqlNodeToAtoms(node, atoms, udfFunctions); } } else if (sqlNode instanceof SqlCase) { - atoms.add(translateSqlCase((SqlCase) sqlNode)); + atoms.add(translateSqlCase((SqlCase) sqlNode, udfFunctions)); } } private static Java.Rvalue sqlBasicCallToJaninoRvalue( - SqlBasicCall sqlBasicCall, Java.Rvalue[] atoms) { + SqlBasicCall sqlBasicCall, + Java.Rvalue[] atoms, + List udfFunctions) { switch (sqlBasicCall.getKind()) { case AND: return generateBinaryOperation(sqlBasicCall, atoms, "&&"); @@ -214,7 +227,7 @@ private static Java.Rvalue sqlBasicCallToJaninoRvalue( case FLOOR: case TRIM: case OTHER_FUNCTION: - return generateOtherFunctionOperation(sqlBasicCall, atoms); + return generateOtherFunctionOperation(sqlBasicCall, atoms, udfFunctions); case PLUS: return generateBinaryOperation(sqlBasicCall, atoms, "+"); case MINUS: @@ -280,23 +293,40 @@ private static Java.Rvalue generateOtherOperation( } private static Java.Rvalue generateOtherFunctionOperation( - SqlBasicCall sqlBasicCall, Java.Rvalue[] atoms) { + SqlBasicCall sqlBasicCall, + Java.Rvalue[] atoms, + List udfFunctions) { String operationName = sqlBasicCall.getOperator().getName().toUpperCase(); if (operationName.equals("IF")) { if (atoms.length == 3) { return new Java.ConditionalExpression( Location.NOWHERE, atoms[0], atoms[1], atoms[2]); } else { - throw new ParseException("Unrecognized expression: " + sqlBasicCall.toString()); + throw new ParseException("Unrecognized expression: " + sqlBasicCall); } } else if (operationName.equals("NOW")) { return generateNoOperandTimestampFunctionOperation(operationName); } else { - return new Java.MethodInvocation( - Location.NOWHERE, - null, - StringUtils.convertToCamelCase(sqlBasicCall.getOperator().getName()), - atoms); + Optional udfFunctionOptional = + udfFunctions.stream() + .filter(e -> e.getName().equalsIgnoreCase(operationName)) + .findFirst(); + return udfFunctionOptional + .map( + udfFunction -> + new Java.MethodInvocation( + Location.NOWHERE, + null, + generateInvokeExpression(udfFunction), + atoms)) + .orElseGet( + () -> + new Java.MethodInvocation( + Location.NOWHERE, + null, + StringUtils.convertToCamelCase( + sqlBasicCall.getOperator().getName()), + atoms)); } } @@ -364,4 +394,16 @@ private static Java.Rvalue generateTypeConvertMethod( "Unsupported data type cast: " + sqlDataTypeSpec.toString()); } } + + private static String generateInvokeExpression(UserDefinedFunctionDescriptor udfFunction) { + if (udfFunction.getReturnTypeHint() != null) { + return String.format( + "(%s) __instanceOf%s.eval", + DataTypeConverter.convertOriginalClass(udfFunction.getReturnTypeHint()) + .getName(), + udfFunction.getClassName()); + } else { + return String.format("__instanceOf%s.eval", udfFunction.getClassName()); + } + } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/TransformParser.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/TransformParser.java index 23cf4b376d4..79ff53418e0 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,6 +21,7 @@ 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.udf.UserDefinedFunctionDescriptor; 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; @@ -40,7 +41,12 @@ import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rel.type.RelDataTypeSystem; import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.schema.ScalarFunction; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.schema.impl.ScalarFunctionImpl; import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlFunction; +import org.apache.calcite.sql.SqlFunctionCategory; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; @@ -50,7 +56,11 @@ 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.type.InferTypes; +import org.apache.calcite.sql.type.OperandTypes; +import org.apache.calcite.sql.type.SqlReturnTypeInference; import org.apache.calcite.sql.type.SqlTypeFactoryImpl; +import org.apache.calcite.sql.util.ListSqlOperatorTable; import org.apache.calcite.sql.util.SqlOperatorTables; import org.apache.calcite.sql.validate.SqlConformanceEnum; import org.apache.calcite.sql.validate.SqlValidator; @@ -69,6 +79,8 @@ import java.util.Properties; import java.util.stream.Collectors; +import static org.apache.flink.cdc.runtime.typeutils.DataTypeConverter.convertCalciteType; + /** Use Flink's calcite parser to parse the statement of flink cdc pipeline transform. */ public class TransformParser { private static final Logger LOG = LoggerFactory.getLogger(TransformParser.class); @@ -87,15 +99,47 @@ private static SqlParser getCalciteParser(String sql) { .withLex(Lex.JAVA)); } - private static RelNode sqlToRel(List columns, SqlNode sqlNode) { + private static RelNode sqlToRel( + List columns, SqlNode sqlNode, List udfs) { List columnsWithMetadata = copyFillMetadataColumn(sqlNode.toString(), columns); CalciteSchema rootSchema = CalciteSchema.createRootSchema(true); + SchemaPlus schema = rootSchema.plus(); Map operand = new HashMap<>(); operand.put("tableName", DEFAULT_TABLE); operand.put("columns", columnsWithMetadata); rootSchema.add( DEFAULT_SCHEMA, - TransformSchemaFactory.INSTANCE.create(rootSchema.plus(), DEFAULT_SCHEMA, operand)); + TransformSchemaFactory.INSTANCE.create(schema, DEFAULT_SCHEMA, operand)); + List udfFunctions = new ArrayList<>(); + for (UserDefinedFunctionDescriptor udf : udfs) { + try { + Class clazz = Class.forName(udf.getClassPath()); + SqlReturnTypeInference returnTypeInference; + ScalarFunction function = ScalarFunctionImpl.create(clazz, "eval"); + if (udf.getReturnTypeHint() != null) { + // This UDF has return type hint annotation + returnTypeInference = + o -> + o.getTypeFactory() + .createSqlType( + convertCalciteType(udf.getReturnTypeHint())); + } else { + // Infer it from eval method return type + returnTypeInference = o -> function.getReturnType(o.getTypeFactory()); + } + schema.add(udf.getName(), function); + udfFunctions.add( + new SqlFunction( + udf.getName(), + SqlKind.OTHER_FUNCTION, + returnTypeInference, + InferTypes.RETURN_TYPE, + OperandTypes.VARIADIC, + SqlFunctionCategory.USER_DEFINED_FUNCTION)); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Failed to resolve UDF: " + udf, e); + } + } SqlTypeFactoryImpl factory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT); CalciteCatalogReader calciteCatalogReader = new CalciteCatalogReader( @@ -105,9 +149,12 @@ private static RelNode sqlToRel(List columns, SqlNode sqlNode) { new CalciteConnectionConfigImpl(new Properties())); TransformSqlOperatorTable transformSqlOperatorTable = TransformSqlOperatorTable.instance(); SqlStdOperatorTable sqlStdOperatorTable = SqlStdOperatorTable.instance(); + ListSqlOperatorTable udfOperatorTable = new ListSqlOperatorTable(); + udfFunctions.forEach(udfOperatorTable::add); SqlValidator validator = SqlValidatorUtil.newValidator( - SqlOperatorTables.chain(sqlStdOperatorTable, transformSqlOperatorTable), + SqlOperatorTables.chain( + sqlStdOperatorTable, transformSqlOperatorTable, udfOperatorTable), calciteCatalogReader, factory, SqlValidator.Config.DEFAULT.withIdentifierExpansion(true)); @@ -143,7 +190,9 @@ public static SqlSelect parseSelect(String statement) { // Parse all columns public static List generateProjectionColumns( - String projectionExpression, List columns) { + String projectionExpression, + List columns, + List udfFunctions) { if (StringUtils.isNullOrWhitespaceOnly(projectionExpression)) { return new ArrayList<>(); } @@ -151,7 +200,7 @@ public static List generateProjectionColumns( if (sqlSelect.getSelectList().isEmpty()) { return new ArrayList<>(); } - RelNode relNode = sqlToRel(columns, sqlSelect); + RelNode relNode = sqlToRel(columns, sqlSelect, udfFunctions); Map relDataTypeMap = relNode.getRowType().getFieldList().stream() .collect( @@ -190,7 +239,7 @@ public static List generateProjectionColumns( relDataTypeMap.get(columnName)), transformOptional.get().toString(), JaninoCompiler.translateSqlNodeToJaninoExpression( - transformOptional.get()), + transformOptional.get(), udfFunctions), parseColumnNameList(transformOptional.get())) : ProjectionColumn.of( columnName, @@ -243,7 +292,8 @@ public static List generateProjectionColumns( return projectionColumns; } - public static String translateFilterExpressionToJaninoExpression(String filterExpression) { + public static String translateFilterExpressionToJaninoExpression( + String filterExpression, List udfFunctions) { if (StringUtils.isNullOrWhitespaceOnly(filterExpression)) { return ""; } @@ -252,7 +302,7 @@ public static String translateFilterExpressionToJaninoExpression(String filterEx return ""; } SqlNode where = sqlSelect.getWhere(); - return JaninoCompiler.translateSqlNodeToJaninoExpression(where); + return JaninoCompiler.translateSqlNodeToJaninoExpression(where, udfFunctions); } public static List parseComputedColumnNames(String projection) { diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/UdfUtils.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/UdfUtils.java new file mode 100644 index 00000000000..f6cc2521b54 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/UdfUtils.java @@ -0,0 +1,23 @@ +/* + * 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.typeutils; + +/** Utility functions for dealing with User-defined functions. */ +public class UdfUtils { + /** Check if given class is a CDC Pipeline UDF instead of a Flink compatible ScalarFunction. */ +} 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 f81c4a92c27..b95b58ab3f9 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 @@ -19,6 +19,7 @@ import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.udf.UserDefinedFunctionDescriptor; import org.apache.flink.cdc.runtime.parser.metadata.TransformSchemaFactory; import org.apache.flink.cdc.runtime.parser.metadata.TransformSqlOperatorTable; @@ -46,6 +47,8 @@ import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -294,9 +297,69 @@ public void testTranslateFilterToJaninoExpression() { testFilterExpression("cast(null as varchar)", "castToString(null)"); } + @Test + public void testTranslateUdfFilterToJaninoExpression() { + testFilterExpressionWithUdf( + "format(upper(id))", "__instanceOfFormatFunctionClass.eval(upper(id))"); + testFilterExpressionWithUdf( + "format(lower(id))", "__instanceOfFormatFunctionClass.eval(lower(id))"); + testFilterExpressionWithUdf( + "format(concat(a,b))", "__instanceOfFormatFunctionClass.eval(concat(a, b))"); + testFilterExpressionWithUdf( + "format(SUBSTR(a,1))", "__instanceOfFormatFunctionClass.eval(substr(a, 1))"); + testFilterExpressionWithUdf( + "typeof(id like '^[a-zA-Z]')", + "__instanceOfTypeOfFunctionClass.eval(like(id, \"^[a-zA-Z]\"))"); + testFilterExpressionWithUdf( + "typeof(id not like '^[a-zA-Z]')", + "__instanceOfTypeOfFunctionClass.eval(notLike(id, \"^[a-zA-Z]\"))"); + testFilterExpressionWithUdf( + "typeof(abs(2))", "__instanceOfTypeOfFunctionClass.eval(abs(2))"); + testFilterExpressionWithUdf( + "typeof(ceil(2))", "__instanceOfTypeOfFunctionClass.eval(ceil(2))"); + testFilterExpressionWithUdf( + "typeof(floor(2))", "__instanceOfTypeOfFunctionClass.eval(floor(2))"); + testFilterExpressionWithUdf( + "typeof(round(2,2))", "__instanceOfTypeOfFunctionClass.eval(round(2, 2))"); + testFilterExpressionWithUdf( + "typeof(id + 2)", "__instanceOfTypeOfFunctionClass.eval(id + 2)"); + testFilterExpressionWithUdf( + "typeof(id - 2)", "__instanceOfTypeOfFunctionClass.eval(id - 2)"); + testFilterExpressionWithUdf( + "typeof(id * 2)", "__instanceOfTypeOfFunctionClass.eval(id * 2)"); + testFilterExpressionWithUdf( + "typeof(id / 2)", "__instanceOfTypeOfFunctionClass.eval(id / 2)"); + testFilterExpressionWithUdf( + "typeof(id % 2)", "__instanceOfTypeOfFunctionClass.eval(id % 2)"); + testFilterExpressionWithUdf( + "addone(addone(id)) > 4 OR typeof(id) <> 'bool' AND format('from %s to %s is %s', 'a', 'z', 'lie') <> ''", + "__instanceOfAddOneFunctionClass.eval(__instanceOfAddOneFunctionClass.eval(id)) > 4 || !valueEquals(__instanceOfTypeOfFunctionClass.eval(id), \"bool\") && !valueEquals(__instanceOfFormatFunctionClass.eval(\"from %s to %s is %s\", \"a\", \"z\", \"lie\"), \"\")"); + testFilterExpressionWithUdf( + "ADDONE(ADDONE(id)) > 4 OR TYPEOF(id) <> 'bool' AND FORMAT('from %s to %s is %s', 'a', 'z', 'lie') <> ''", + "__instanceOfAddOneFunctionClass.eval(__instanceOfAddOneFunctionClass.eval(id)) > 4 || !valueEquals(__instanceOfTypeOfFunctionClass.eval(id), \"bool\") && !valueEquals(__instanceOfFormatFunctionClass.eval(\"from %s to %s is %s\", \"a\", \"z\", \"lie\"), \"\")"); + } + private void testFilterExpression(String expression, String expressionExpect) { String janinoExpression = - TransformParser.translateFilterExpressionToJaninoExpression(expression); + TransformParser.translateFilterExpressionToJaninoExpression( + expression, Collections.emptyList()); + Assert.assertEquals(expressionExpect, janinoExpression); + } + + private void testFilterExpressionWithUdf(String expression, String expressionExpect) { + String janinoExpression = + TransformParser.translateFilterExpressionToJaninoExpression( + expression, + Arrays.asList( + new UserDefinedFunctionDescriptor( + "format", + "org.apache.flink.cdc.udf.examples.java.FormatFunctionClass"), + new UserDefinedFunctionDescriptor( + "addone", + "org.apache.flink.cdc.udf.examples.java.AddOneFunctionClass"), + new UserDefinedFunctionDescriptor( + "typeof", + "org.apache.flink.cdc.udf.examples.java.TypeOfFunctionClass"))); Assert.assertEquals(expressionExpect, janinoExpression); } } diff --git a/pom.xml b/pom.xml index 4b01c18c609..9955a6c56cc 100644 --- a/pom.xml +++ b/pom.xml @@ -40,6 +40,7 @@ limitations under the License. flink-cdc-connect flink-cdc-runtime flink-cdc-e2e-tests + flink-cdc-pipeline-udf-examples @@ -98,6 +99,8 @@ limitations under the License. at the same time minimum 3.1.x Janino version passing Flink tests without WAs is 3.1.10, more details are in FLINK-27995 --> 3.1.10 + + 2.12.16