From b937db25d5431d8cd5894b393ec7d4d7531c6fc2 Mon Sep 17 00:00:00 2001 From: yuxiqian <34335406+yuxiqian@users.noreply.github.com> Date: Fri, 9 Aug 2024 10:00:21 +0800 Subject: [PATCH 1/6] [hotfix][cdc-connector][mongodb] Fix unstable test cases for snapshot back-filling (#3506) --- .../mongodb/source/MongoDBParallelSourceITCase.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBParallelSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBParallelSourceITCase.java index 25918fd8bb..0595f6b5aa 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBParallelSourceITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBParallelSourceITCase.java @@ -468,6 +468,15 @@ private List testBackfillWhenWritingEvents( mongoCollection.updateOne( Filters.eq("cid", 2000L), Updates.set("address", "Pittsburgh")); mongoCollection.deleteOne(Filters.eq("cid", 1019L)); + + // Rarely happens, but if there's no operation or heartbeat events between + // watermark #a (the ChangeStream opLog caused by the last event in this hook) + // and watermark #b (the calculated high watermark that limits the bounded + // back-filling stream fetch task), the last event of hook will be missed since + // back-filling task reads between [loW, hiW) (high watermark not included). + // Workaround: insert a dummy event in another collection to forcefully push + // opLog forward. + database.getCollection("customers_1").insertOne(new Document()); }; switch (hookType) { From 09f36a4a05797b7e319653db3b90658cd7c31659 Mon Sep 17 00:00:00 2001 From: North Lin <37775475+qg-lin@users.noreply.github.com> Date: Fri, 9 Aug 2024 13:30:46 +0800 Subject: [PATCH 2/6] [FLINK-35143][pipeline-connector][mysql] Expose newly added tables capture in mysql pipeline connector. (#3411) Co-authored-by: Muhammet Orazov <916295+morazow@users.noreply.github.com> Co-authored-by: north.lin --- .../connectors/pipeline-connectors/mysql.md | 7 + .../connectors/pipeline-connectors/mysql.md | 7 + .../mysql/factory/MySqlDataSourceFactory.java | 7 +- .../mysql/source/MySqlDataSourceOptions.java | 8 + .../MysqlPipelineNewlyAddedTableITCase.java | 582 ++++++++++++++++++ .../src/test/resources/ddl/customer.sql | 328 ++++++++++ 6 files changed, 937 insertions(+), 2 deletions(-) create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MysqlPipelineNewlyAddedTableITCase.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/customer.sql diff --git a/docs/content.zh/docs/connectors/pipeline-connectors/mysql.md b/docs/content.zh/docs/connectors/pipeline-connectors/mysql.md index 40c774ffcd..636ed975fb 100644 --- a/docs/content.zh/docs/connectors/pipeline-connectors/mysql.md +++ b/docs/content.zh/docs/connectors/pipeline-connectors/mysql.md @@ -268,6 +268,13 @@ pipeline: 是否在快照结束后关闭空闲的 Reader。 此特性需要 flink 版本大于等于 1.14 并且 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' 需要设置为 true。
若 flink 版本大于等于 1.15,'execution.checkpointing.checkpoints-after-tasks-finish.enabled' 默认值变更为 true,可以不用显式配置 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' = true。 + + scan.newly-added-table.enabled + optional + false + Boolean + 是否启用动态加表特性,默认关闭。 此配置项只有作业从savepoint/checkpoint启动时才生效。 + diff --git a/docs/content/docs/connectors/pipeline-connectors/mysql.md b/docs/content/docs/connectors/pipeline-connectors/mysql.md index 41eccf9398..879701614e 100644 --- a/docs/content/docs/connectors/pipeline-connectors/mysql.md +++ b/docs/content/docs/connectors/pipeline-connectors/mysql.md @@ -275,6 +275,13 @@ pipeline: so it does not need to be explicitly configured 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' = 'true' + + scan.newly-added-table.enabled + optional + false + Boolean + Whether to enable scan the newly added tables feature or not, by default is false. This option is only useful when we start the job from a savepoint/checkpoint. + diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java index b1d3e5966f..e8c39ce3df 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java @@ -61,6 +61,7 @@ import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.PORT; import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED; import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; +import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SCAN_NEWLY_ADDED_TABLE_ENABLED; import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SCAN_SNAPSHOT_FETCH_SIZE; import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SCAN_STARTUP_MODE; import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_FILE; @@ -123,6 +124,7 @@ public DataSource createDataSource(Context context) { Duration connectTimeout = config.get(CONNECT_TIMEOUT); int connectMaxRetries = config.get(CONNECT_MAX_RETRIES); int connectionPoolSize = config.get(CONNECTION_POOL_SIZE); + boolean scanNewlyAddedTableEnabled = config.get(SCAN_NEWLY_ADDED_TABLE_ENABLED); validateIntegerOption(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE, splitSize, 1); validateIntegerOption(CHUNK_META_GROUP_SIZE, splitMetaGroupSize, 1); @@ -158,7 +160,8 @@ public DataSource createDataSource(Context context) { .closeIdleReaders(closeIdleReaders) .includeSchemaChanges(includeSchemaChanges) .debeziumProperties(getDebeziumProperties(configMap)) - .jdbcProperties(getJdbcProperties(configMap)); + .jdbcProperties(getJdbcProperties(configMap)) + .scanNewlyAddedTableEnabled(scanNewlyAddedTableEnabled); Selectors selectors = new Selectors.SelectorsBuilder().includeTables(tables).build(); List capturedTables = getTableList(configFactory.createConfig(0), selectors); @@ -216,7 +219,7 @@ public Set> optionalOptions() { options.add(CONNECTION_POOL_SIZE); options.add(HEARTBEAT_INTERVAL); options.add(SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED); - + options.add(SCAN_NEWLY_ADDED_TABLE_ENABLED); options.add(CHUNK_META_GROUP_SIZE); options.add(CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND); options.add(CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/MySqlDataSourceOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/MySqlDataSourceOptions.java index e852eb3d7a..f6f1a671a0 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/MySqlDataSourceOptions.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/MySqlDataSourceOptions.java @@ -224,6 +224,14 @@ public class MySqlDataSourceOptions { + "FLIP-147: Support Checkpoints After Tasks Finished. The flink version is required to be " + "greater than or equal to 1.14 when enabling this feature."); + @Experimental + public static final ConfigOption SCAN_NEWLY_ADDED_TABLE_ENABLED = + ConfigOptions.key("scan.newly-added-table.enabled") + .booleanType() + .defaultValue(false) + .withDescription( + "Whether to scan the newly added tables or not, by default is false. This option is only useful when we start the job from a savepoint/checkpoint."); + @Experimental public static final ConfigOption SCHEMA_CHANGE_ENABLED = ConfigOptions.key("schema-change.enabled") diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MysqlPipelineNewlyAddedTableITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MysqlPipelineNewlyAddedTableITCase.java new file mode 100644 index 0000000000..7187e64473 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MysqlPipelineNewlyAddedTableITCase.java @@ -0,0 +1,582 @@ +/* + * 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.connectors.mysql.source; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.cdc.common.data.binary.BinaryStringData; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.factories.Factory; +import org.apache.flink.cdc.common.factories.FactoryHelper; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.source.FlinkSourceProvider; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.RowType; +import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; +import org.apache.flink.cdc.connectors.mysql.factory.MySqlDataSourceFactory; +import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; +import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; +import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.checkpoint.CheckpointException; +import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.collect.AbstractCollectResultBuffer; +import org.apache.flink.streaming.api.operators.collect.CheckpointedCollectResultBuffer; +import org.apache.flink.streaming.api.operators.collect.CollectResultIterator; +import org.apache.flink.streaming.api.operators.collect.CollectSinkOperator; +import org.apache.flink.streaming.api.operators.collect.CollectSinkOperatorFactory; +import org.apache.flink.streaming.api.operators.collect.CollectStreamSink; +import org.apache.flink.table.planner.factories.TestValuesTableFactory; +import org.apache.flink.util.ExceptionUtils; + +import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.jdbc.JdbcConnection; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.sql.SQLException; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static java.lang.String.format; +import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.HOSTNAME; +import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.PASSWORD; +import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.PORT; +import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SCAN_NEWLY_ADDED_TABLE_ENABLED; +import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SERVER_ID; +import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SERVER_TIME_ZONE; +import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.TABLES; +import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.USERNAME; +import static org.apache.flink.cdc.connectors.mysql.testutils.MySqSourceTestUtils.TEST_PASSWORD; +import static org.apache.flink.cdc.connectors.mysql.testutils.MySqSourceTestUtils.TEST_USER; +import static org.apache.flink.cdc.connectors.mysql.testutils.MySqSourceTestUtils.fetchResults; +import static org.apache.flink.cdc.connectors.mysql.testutils.MySqSourceTestUtils.getServerId; +import static org.assertj.core.api.Assertions.assertThat; + +/** IT tests to cover various newly added tables during capture process in pipeline mode. */ +public class MysqlPipelineNewlyAddedTableITCase extends MySqlSourceTestBase { + private final UniqueDatabase customDatabase = + new UniqueDatabase(MYSQL_CONTAINER, "customer", "mysqluser", "mysqlpw"); + + private final ScheduledExecutorService mockBinlogExecutor = Executors.newScheduledThreadPool(1); + + @Before + public void before() throws SQLException { + TestValuesTableFactory.clearAllData(); + customDatabase.createAndInitialize(); + + try (MySqlConnection connection = getConnection()) { + connection.setAutoCommit(false); + // prepare initial data for given table + String tableId = customDatabase.getDatabaseName() + ".produce_binlog_table"; + connection.execute( + format("CREATE TABLE %s ( id BIGINT PRIMARY KEY, cnt BIGINT);", tableId)); + connection.execute( + format("INSERT INTO %s VALUES (0, 100), (1, 101), (2, 102);", tableId)); + connection.commit(); + + // mock continuous binlog during the newly added table capturing process + mockBinlogExecutor.schedule( + () -> { + try { + connection.execute( + format("UPDATE %s SET cnt = cnt +1 WHERE id < 2;", tableId)); + connection.commit(); + } catch (SQLException e) { + e.printStackTrace(); + } + }, + 500, + TimeUnit.MICROSECONDS); + } + } + + @After + public void after() { + mockBinlogExecutor.shutdown(); + } + + private MySqlConnection getConnection() { + Map properties = new HashMap<>(); + properties.put("database.hostname", MYSQL_CONTAINER.getHost()); + properties.put("database.port", String.valueOf(MYSQL_CONTAINER.getDatabasePort())); + properties.put("database.user", customDatabase.getUsername()); + properties.put("database.password", customDatabase.getPassword()); + properties.put("database.serverTimezone", ZoneId.of("UTC").toString()); + io.debezium.config.Configuration configuration = + io.debezium.config.Configuration.from(properties); + return DebeziumUtils.createMySqlConnection(configuration, new Properties()); + } + + @Test + public void testAddNewTableOneByOneSingleParallelism() throws Exception { + TestParam testParam = + TestParam.newBuilder( + Collections.singletonList("address_hangzhou"), + 4, + Arrays.asList("address_hangzhou", "address_beijing"), + 4) + .setFirstRoundInitTables( + Arrays.asList("address_hangzhou", "address_beijing")) + .build(); + + testAddNewTable(testParam, 1); + } + + @Test + public void testAddNewTableOneByOne() throws Exception { + TestParam testParam = + TestParam.newBuilder( + Collections.singletonList("address_hangzhou"), + 4, + Arrays.asList("address_hangzhou", "address_beijing"), + 4) + .setFirstRoundInitTables( + Arrays.asList("address_hangzhou", "address_beijing")) + .build(); + + testAddNewTable(testParam, DEFAULT_PARALLELISM); + } + + @Test + public void testAddNewTableByPatternSingleParallelism() throws Exception { + TestParam testParam = + TestParam.newBuilder( + Collections.singletonList("address_\\.*"), + 8, + Collections.singletonList("address_\\.*"), + 8) + .setFirstRoundInitTables( + Arrays.asList("address_hangzhou", "address_beijing")) + .setSecondRoundInitTables( + Arrays.asList("address_shanghai", "address_suzhou")) + .build(); + + testAddNewTable(testParam, 1); + } + + @Test + public void testAddNewTableByPattern() throws Exception { + TestParam testParam = + TestParam.newBuilder( + Collections.singletonList("address_\\.*"), + 8, + Collections.singletonList("address_\\.*"), + 12) + .setFirstRoundInitTables( + Arrays.asList("address_hangzhou", "address_beijing")) + .setSecondRoundInitTables( + Arrays.asList( + "address_shanghai", "address_suzhou", "address_shenzhen")) + .build(); + + testAddNewTable(testParam, DEFAULT_PARALLELISM); + } + + private void testAddNewTable(TestParam testParam, int parallelism) throws Exception { + // step 1: create mysql tables + if (CollectionUtils.isNotEmpty(testParam.getFirstRoundInitTables())) { + initialAddressTables(getConnection(), testParam.getFirstRoundInitTables()); + } + Path savepointDir = Files.createTempDirectory("add-new-table-test"); + final String savepointDirectory = savepointDir.toAbsolutePath().toString(); + String finishedSavePointPath = null; + StreamExecutionEnvironment env = + getStreamExecutionEnvironment(finishedSavePointPath, parallelism); + // step 2: listen tables first time + List listenTablesFirstRound = testParam.getFirstRoundListenTables(); + + FlinkSourceProvider sourceProvider = + getFlinkSourceProvider(listenTablesFirstRound, parallelism); + DataStreamSource source = + env.fromSource( + sourceProvider.getSource(), + WatermarkStrategy.noWatermarks(), + MySqlDataSourceFactory.IDENTIFIER, + new EventTypeInfo()); + + TypeSerializer serializer = + source.getTransformation().getOutputType().createSerializer(env.getConfig()); + CheckpointedCollectResultBuffer resultBuffer = + new CheckpointedCollectResultBuffer<>(serializer); + String accumulatorName = "dataStreamCollect_" + UUID.randomUUID(); + CollectResultIterator iterator = + addCollector(env, source, resultBuffer, serializer, accumulatorName); + JobClient jobClient = env.executeAsync("beforeAddNewTable"); + iterator.setJobClient(jobClient); + + List actual = fetchResults(iterator, testParam.getFirstRoundFetchSize()); + Optional listenByPattern = + listenTablesFirstRound.stream() + .filter(table -> StringUtils.contains(table, "\\.*")) + .findAny(); + multiAssert( + actual, + listenByPattern.isPresent() + ? testParam.getFirstRoundInitTables() + : listenTablesFirstRound); + + // step 3: create new tables if needed + if (CollectionUtils.isNotEmpty(testParam.getSecondRoundInitTables())) { + initialAddressTables(getConnection(), testParam.getSecondRoundInitTables()); + } + + // step 4: trigger a savepoint and cancel the job + finishedSavePointPath = triggerSavepointWithRetry(jobClient, savepointDirectory); + jobClient.cancel().get(); + iterator.close(); + + // step 5: restore from savepoint + StreamExecutionEnvironment restoredEnv = + getStreamExecutionEnvironment(finishedSavePointPath, parallelism); + List listenTablesSecondRound = testParam.getSecondRoundListenTables(); + FlinkSourceProvider restoredSourceProvider = + getFlinkSourceProvider(listenTablesSecondRound, parallelism); + DataStreamSource restoreSource = + restoredEnv.fromSource( + restoredSourceProvider.getSource(), + WatermarkStrategy.noWatermarks(), + MySqlDataSourceFactory.IDENTIFIER, + new EventTypeInfo()); + CollectResultIterator restoredIterator = + addCollector(restoredEnv, restoreSource, resultBuffer, serializer, accumulatorName); + JobClient restoreClient = restoredEnv.executeAsync("AfterAddNewTable"); + + List newlyAddTables = + listenTablesSecondRound.stream() + .filter(table -> !listenTablesFirstRound.contains(table)) + .collect(Collectors.toList()); + // it means listen by pattern when newlyAddTables is empty + if (CollectionUtils.isEmpty(newlyAddTables)) { + newlyAddTables = testParam.getSecondRoundInitTables(); + } + List newlyTableEvent = + fetchResults(restoredIterator, testParam.getSecondRoundFetchSize()); + multiAssert(newlyTableEvent, newlyAddTables); + restoreClient.cancel().get(); + restoredIterator.close(); + } + + private void multiAssert(List actualEvents, List listenTables) { + List expectedCreateTableEvents = new ArrayList<>(); + List expectedDataChangeEvents = new ArrayList<>(); + for (String table : listenTables) { + expectedCreateTableEvents.add( + getCreateTableEvent(TableId.tableId(customDatabase.getDatabaseName(), table))); + expectedDataChangeEvents.addAll( + getSnapshotExpected(TableId.tableId(customDatabase.getDatabaseName(), table))); + } + // compare create table events + List actualCreateTableEvents = + actualEvents.stream() + .filter(event -> event instanceof CreateTableEvent) + .collect(Collectors.toList()); + assertThat(actualCreateTableEvents) + .containsExactlyInAnyOrder(expectedCreateTableEvents.toArray(new Event[0])); + + // compare data change events + List actualDataChangeEvents = + actualEvents.stream() + .filter(event -> event instanceof DataChangeEvent) + .collect(Collectors.toList()); + assertThat(actualDataChangeEvents) + .containsExactlyInAnyOrder(expectedDataChangeEvents.toArray(new Event[0])); + } + + private CreateTableEvent getCreateTableEvent(TableId tableId) { + Schema schema = + Schema.newBuilder() + .physicalColumn("id", DataTypes.BIGINT().notNull()) + .physicalColumn("country", DataTypes.VARCHAR(255).notNull()) + .physicalColumn("city", DataTypes.VARCHAR(255).notNull()) + .physicalColumn("detail_address", DataTypes.VARCHAR(1024)) + .primaryKey(Collections.singletonList("id")) + .build(); + return new CreateTableEvent(tableId, schema); + } + + private List getSnapshotExpected(TableId tableId) { + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.BIGINT().notNull(), + DataTypes.VARCHAR(255).notNull(), + DataTypes.VARCHAR(255).notNull(), + DataTypes.VARCHAR(1024) + }, + new String[] {"id", "country", "city", "detail_address"}); + BinaryRecordDataGenerator generator = new BinaryRecordDataGenerator(rowType); + String cityName = tableId.getTableName().split("_")[1]; + return Arrays.asList( + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 416874195632735147L, + BinaryStringData.fromString("China"), + BinaryStringData.fromString(cityName), + BinaryStringData.fromString(cityName + " West Town address 1") + })), + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 416927583791428523L, + BinaryStringData.fromString("China"), + BinaryStringData.fromString(cityName), + BinaryStringData.fromString(cityName + " West Town address 2") + })), + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 417022095255614379L, + BinaryStringData.fromString("China"), + BinaryStringData.fromString(cityName), + BinaryStringData.fromString(cityName + " West Town address 3") + }))); + } + + private String triggerSavepointWithRetry(JobClient jobClient, String savepointDirectory) + throws ExecutionException, InterruptedException { + int retryTimes = 0; + // retry 600 times, it takes 100 milliseconds per time, at most retry 1 minute + while (retryTimes < 600) { + try { + return jobClient.triggerSavepoint(savepointDirectory).get(); + } catch (Exception e) { + Optional exception = + ExceptionUtils.findThrowable(e, CheckpointException.class); + if (exception.isPresent() + && exception.get().getMessage().contains("Checkpoint triggering task")) { + Thread.sleep(100); + retryTimes++; + } else { + throw e; + } + } + } + return null; + } + + private void initialAddressTables(JdbcConnection connection, List addressTables) + throws SQLException { + try { + connection.setAutoCommit(false); + for (String tableName : addressTables) { + // make initial data for given table + String tableId = customDatabase.getDatabaseName() + "." + tableName; + String cityName = tableName.split("_")[1]; + connection.execute( + "CREATE TABLE IF NOT EXISTS " + + tableId + + "(" + + " id BIGINT NOT NULL PRIMARY KEY," + + " country VARCHAR(255) NOT NULL," + + " city VARCHAR(255) NOT NULL," + + " detail_address VARCHAR(1024)" + + ");"); + connection.execute( + format( + "INSERT INTO %s " + + "VALUES (416874195632735147, 'China', '%s', '%s West Town address 1')," + + " (416927583791428523, 'China', '%s', '%s West Town address 2')," + + " (417022095255614379, 'China', '%s', '%s West Town address 3');", + tableId, cityName, cityName, cityName, cityName, cityName, + cityName)); + } + connection.commit(); + } finally { + connection.close(); + } + } + + private FlinkSourceProvider getFlinkSourceProvider(List tables, int parallelism) { + List fullTableNames = + tables.stream() + .map(table -> customDatabase.getDatabaseName() + "." + table) + .collect(Collectors.toList()); + Map options = new HashMap<>(); + options.put(HOSTNAME.key(), MYSQL_CONTAINER.getHost()); + options.put(PORT.key(), String.valueOf(MYSQL_CONTAINER.getDatabasePort())); + options.put(USERNAME.key(), TEST_USER); + options.put(PASSWORD.key(), TEST_PASSWORD); + options.put(SERVER_TIME_ZONE.key(), "UTC"); + options.put(TABLES.key(), StringUtils.join(fullTableNames, ",")); + options.put(SERVER_ID.key(), getServerId(parallelism)); + options.put(SCAN_NEWLY_ADDED_TABLE_ENABLED.key(), "true"); + Factory.Context context = + new FactoryHelper.DefaultContext( + org.apache.flink.cdc.common.configuration.Configuration.fromMap(options), + null, + this.getClass().getClassLoader()); + + MySqlDataSourceFactory factory = new MySqlDataSourceFactory(); + MySqlDataSource dataSource = (MySqlDataSource) factory.createDataSource(context); + + return (FlinkSourceProvider) dataSource.getEventSourceProvider(); + } + + private CollectResultIterator addCollector( + StreamExecutionEnvironment env, + DataStreamSource source, + AbstractCollectResultBuffer buffer, + TypeSerializer serializer, + String accumulatorName) { + CollectSinkOperatorFactory sinkFactory = + new CollectSinkOperatorFactory<>(serializer, accumulatorName); + CollectSinkOperator operator = (CollectSinkOperator) sinkFactory.getOperator(); + CollectResultIterator iterator = + new CollectResultIterator<>( + buffer, operator.getOperatorIdFuture(), accumulatorName, 0); + CollectStreamSink sink = new CollectStreamSink<>(source, sinkFactory); + sink.name("Data stream collect sink"); + env.addOperator(sink.getTransformation()); + env.registerCollectIterator(iterator); + return iterator; + } + + private StreamExecutionEnvironment getStreamExecutionEnvironment( + String finishedSavePointPath, int parallelism) { + Configuration configuration = new Configuration(); + if (finishedSavePointPath != null) { + configuration.setString(SavepointConfigOptions.SAVEPOINT_PATH, finishedSavePointPath); + } + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); + env.setParallelism(parallelism); + env.enableCheckpointing(500L); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 1000L)); + return env; + } + + private static class TestParam { + private final List firstRoundInitTables; + private final List firstRoundListenTables; + private final Integer firstRoundFetchSize; + private final List secondRoundInitTables; + private final List secondRoundListenTables; + private final Integer secondRoundFetchSize; + + private TestParam(Builder builder) { + this.firstRoundInitTables = builder.firstRoundInitTables; + this.firstRoundListenTables = builder.firstRoundListenTables; + this.firstRoundFetchSize = builder.firstRoundFetchSize; + this.secondRoundInitTables = builder.secondRoundInitTables; + this.secondRoundListenTables = builder.secondRoundListenTables; + this.secondRoundFetchSize = builder.secondRoundFetchSize; + } + + public static Builder newBuilder( + List firstRoundListenTables, + Integer firstRoundFetchSize, + List secondRoundListenTables, + Integer secondRoundFetchSize) { + return new Builder( + firstRoundListenTables, + firstRoundFetchSize, + secondRoundListenTables, + secondRoundFetchSize); + } + + public static class Builder { + private List firstRoundInitTables; + private final List firstRoundListenTables; + private final Integer firstRoundFetchSize; + + private List secondRoundInitTables; + private final List secondRoundListenTables; + private final Integer secondRoundFetchSize; + + public Builder( + List firstRoundListenTables, + Integer firstRoundFetchSize, + List secondRoundListenTables, + Integer secondRoundFetchSize) { + this.firstRoundListenTables = firstRoundListenTables; + this.firstRoundFetchSize = firstRoundFetchSize; + this.secondRoundListenTables = secondRoundListenTables; + this.secondRoundFetchSize = secondRoundFetchSize; + } + + public TestParam build() { + return new TestParam(this); + } + + public Builder setFirstRoundInitTables(List firstRoundInitTables) { + this.firstRoundInitTables = firstRoundInitTables; + return this; + } + + public Builder setSecondRoundInitTables(List secondRoundInitTables) { + this.secondRoundInitTables = secondRoundInitTables; + return this; + } + } + + public List getFirstRoundInitTables() { + return firstRoundInitTables; + } + + public List getFirstRoundListenTables() { + return firstRoundListenTables; + } + + public Integer getFirstRoundFetchSize() { + return firstRoundFetchSize; + } + + public List getSecondRoundInitTables() { + return secondRoundInitTables; + } + + public List getSecondRoundListenTables() { + return secondRoundListenTables; + } + + public Integer getSecondRoundFetchSize() { + return secondRoundFetchSize; + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/customer.sql b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/customer.sql new file mode 100644 index 0000000000..e4df63f1a3 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/customer.sql @@ -0,0 +1,328 @@ +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: customer +-- ---------------------------------------------------------------------------------------------------------------- + +-- Create and populate our users using a single insert with many rows +CREATE TABLE customers ( + id INTEGER NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + address VARCHAR(1024), + phone_number VARCHAR(512) +); + +INSERT INTO customers +VALUES (101,"user_1","Shanghai","123567891234"), + (102,"user_2","Shanghai","123567891234"), + (103,"user_3","Shanghai","123567891234"), + (109,"user_4","Shanghai","123567891234"), + (110,"user_5","Shanghai","123567891234"), + (111,"user_6","Shanghai","123567891234"), + (118,"user_7","Shanghai","123567891234"), + (121,"user_8","Shanghai","123567891234"), + (123,"user_9","Shanghai","123567891234"), + (1009,"user_10","Shanghai","123567891234"), + (1010,"user_11","Shanghai","123567891234"), + (1011,"user_12","Shanghai","123567891234"), + (1012,"user_13","Shanghai","123567891234"), + (1013,"user_14","Shanghai","123567891234"), + (1014,"user_15","Shanghai","123567891234"), + (1015,"user_16","Shanghai","123567891234"), + (1016,"user_17","Shanghai","123567891234"), + (1017,"user_18","Shanghai","123567891234"), + (1018,"user_19","Shanghai","123567891234"), + (1019,"user_20","Shanghai","123567891234"), + (2000,"user_21","Shanghai","123567891234"); + +-- Create a table will not be read +CREATE TABLE prefix_customers ( + id INTEGER NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + address VARCHAR(1024), + phone_number VARCHAR(512) +); + +INSERT INTO prefix_customers +VALUES (101,"user_1","Shanghai","123567891234"), + (102,"user_2","Shanghai","123567891234"), + (103,"user_3","Shanghai","123567891234"), + (109,"user_4","Shanghai","123567891234"); + +-- table has same name prefix with 'customers.*' +CREATE TABLE customers_1 ( + id INTEGER NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + address VARCHAR(1024), + phone_number VARCHAR(512) +); + +INSERT INTO customers_1 +VALUES (101,"user_1","Shanghai","123567891234"), + (102,"user_2","Shanghai","123567891234"), + (103,"user_3","Shanghai","123567891234"), + (109,"user_4","Shanghai","123567891234"), + (110,"user_5","Shanghai","123567891234"), + (111,"user_6","Shanghai","123567891234"), + (118,"user_7","Shanghai","123567891234"), + (121,"user_8","Shanghai","123567891234"), + (123,"user_9","Shanghai","123567891234"), + (1009,"user_10","Shanghai","123567891234"), + (1010,"user_11","Shanghai","123567891234"), + (1011,"user_12","Shanghai","123567891234"), + (1012,"user_13","Shanghai","123567891234"), + (1013,"user_14","Shanghai","123567891234"), + (1014,"user_15","Shanghai","123567891234"), + (1015,"user_16","Shanghai","123567891234"), + (1016,"user_17","Shanghai","123567891234"), + (1017,"user_18","Shanghai","123567891234"), + (1018,"user_19","Shanghai","123567891234"), + (1019,"user_20","Shanghai","123567891234"), + (2000,"user_21","Shanghai","123567891234"); + +-- create table whose split key is evenly distributed +CREATE TABLE customers_even_dist ( + id INTEGER NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL , + address VARCHAR(1024), + phone_number VARCHAR(512) +); +INSERT INTO customers_even_dist +VALUES (101,'user_1','Shanghai','123567891234'), + (102,'user_2','Shanghai','123567891234'), + (103,'user_3','Shanghai','123567891234'), + (104,'user_4','Shanghai','123567891234'), + (105,'user_5','Shanghai','123567891234'), + (106,'user_6','Shanghai','123567891234'), + (107,'user_7','Shanghai','123567891234'), + (108,'user_8','Shanghai','123567891234'), + (109,'user_9','Shanghai','123567891234'), + (110,'user_10','Shanghai','123567891234'); + +-- create table whose split key is evenly distributed and sparse +CREATE TABLE customers_sparse_dist ( + id INTEGER NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL , + address VARCHAR(1024), + phone_number VARCHAR(512) +); +INSERT INTO customers_sparse_dist +VALUES (2,'user_1','Shanghai','123567891234'), + (4,'user_2','Shanghai','123567891234'), + (6,'user_3','Shanghai','123567891234'), + (8,'user_4','Shanghai','123567891234'), + (10,'user_5','Shanghai','123567891234'), + (16,'user_6','Shanghai','123567891234'), + (17,'user_7','Shanghai','123567891234'), + (18,'user_8','Shanghai','123567891234'), + (20,'user_9','Shanghai','123567891234'), + (22,'user_10','Shanghai','123567891234'); + +-- create table whose split key is evenly distributed and dense +CREATE TABLE customers_dense_dist ( + id1 INTEGER NOT NULL, + id2 VARCHAR(255) NOT NULL , + address VARCHAR(1024), + phone_number VARCHAR(512), + PRIMARY KEY(id1, id2) +); +INSERT INTO customers_dense_dist +VALUES (1,'user_1','Shanghai','123567891234'), + (1,'user_2','Shanghai','123567891234'), + (1,'user_3','Shanghai','123567891234'), + (1,'user_4','Shanghai','123567891234'), + (2,'user_5','Shanghai','123567891234'), + (2,'user_6','Shanghai','123567891234'), + (2,'user_7','Shanghai','123567891234'), + (3,'user_8','Shanghai','123567891234'), + (3,'user_9','Shanghai','123567891234'), + (3,'user_10','Shanghai','123567891234'); + +CREATE TABLE customers_no_pk ( + id INTEGER, + name VARCHAR(255) DEFAULT 'flink', + address VARCHAR(1024), + phone_number VARCHAR(512) +); + +INSERT INTO customers_no_pk +VALUES (101,"user_1","Shanghai","123567891234"), + (102,"user_2","Shanghai","123567891234"), + (103,"user_3","Shanghai","123567891234"), + (109,"user_4","Shanghai","123567891234"), + (110,"user_5","Shanghai","123567891234"), + (111,"user_6","Shanghai","123567891234"), + (118,"user_7","Shanghai","123567891234"), + (121,"user_8","Shanghai","123567891234"), + (123,"user_9","Shanghai","123567891234"), + (1009,"user_10","Shanghai","123567891234"), + (1010,"user_11","Shanghai","123567891234"), + (1011,"user_12","Shanghai","123567891234"), + (1012,"user_13","Shanghai","123567891234"), + (1013,"user_14","Shanghai","123567891234"), + (1014,"user_15","Shanghai","123567891234"), + (1015,"user_16","Shanghai","123567891234"), + (1016,"user_17","Shanghai","123567891234"), + (1017,"user_18","Shanghai","123567891234"), + (1018,"user_19","Shanghai","123567891234"), + (1019,"user_20","Shanghai","123567891234"), + (2000,"user_21","Shanghai","123567891234"); + +-- table has combined primary key +CREATE TABLE customer_card ( + card_no BIGINT NOT NULL, + level VARCHAR(10) NOT NULL, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + note VARCHAR(1024), + PRIMARY KEY(card_no, level) +); + +insert into customer_card +VALUES (20001, 'LEVEL_4', 'user_1', 'user with level 4'), + (20002, 'LEVEL_4', 'user_2', 'user with level 4'), + (20003, 'LEVEL_4', 'user_3', 'user with level 4'), + (20004, 'LEVEL_4', 'user_4', 'user with level 4'), + (20004, 'LEVEL_1', 'user_4', 'user with level 4'), + (20004, 'LEVEL_2', 'user_4', 'user with level 4'), + (20004, 'LEVEL_3', 'user_4', 'user with level 4'), + (30006, 'LEVEL_3', 'user_5', 'user with level 3'), + (30007, 'LEVEL_3', 'user_6', 'user with level 3'), + (30008, 'LEVEL_3', 'user_7', 'user with level 3'), + (30009, 'LEVEL_3', 'user_8', 'user with level 3'), + (30009, 'LEVEL_2', 'user_8', 'user with level 3'), + (30009, 'LEVEL_1', 'user_8', 'user with level 3'), + (40001, 'LEVEL_2', 'user_9', 'user with level 2'), + (40002, 'LEVEL_2', 'user_10', 'user with level 2'), + (40003, 'LEVEL_2', 'user_11', 'user with level 2'), + (50001, 'LEVEL_1', 'user_12', 'user with level 1'), + (50002, 'LEVEL_1', 'user_13', 'user with level 1'), + (50003, 'LEVEL_1', 'user_14', 'user with level 1'); + +-- table has single line +CREATE TABLE customer_card_single_line ( + card_no BIGINT NOT NULL, + level VARCHAR(10) NOT NULL, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + note VARCHAR(1024), + PRIMARY KEY(card_no, level) +); + +insert into customer_card_single_line +VALUES (20001, 'LEVEL_1', 'user_1', 'user with level 1'); + + +-- table has combined primary key +CREATE TABLE shopping_cart ( + product_no INT NOT NULL, + product_kind VARCHAR(255), + user_id VARCHAR(255) NOT NULL, + description VARCHAR(255) NOT NULL, + PRIMARY KEY(user_id, product_no, product_kind) +); + +insert into shopping_cart +VALUES (101, 'KIND_001', 'user_1', 'my shopping cart'), + (101, 'KIND_002', 'user_1', 'my shopping cart'), + (102, 'KIND_007', 'user_1', 'my shopping cart'), + (102, 'KIND_008', 'user_1', 'my shopping cart'), + (501, 'KIND_100', 'user_2', 'my shopping list'), + (701, 'KIND_999', 'user_3', 'my shopping list'), + (801, 'KIND_010', 'user_4', 'my shopping list'), + (600, 'KIND_009', 'user_4', 'my shopping list'), + (401, 'KIND_002', 'user_5', 'leo list'), + (401, 'KIND_007', 'user_5', 'leo list'), + (404, 'KIND_008', 'user_5', 'leo list'), + (600, 'KIND_009', 'user_6', 'my shopping cart'); + +-- table has combined primary key and one of the primary key is evenly +CREATE TABLE evenly_shopping_cart ( + product_no INT NOT NULL, + product_kind VARCHAR(255), + user_id VARCHAR(255) NOT NULL, + description VARCHAR(255) NOT NULL, + PRIMARY KEY(product_kind, product_no, user_id) +); + +insert into evenly_shopping_cart +VALUES (101, 'KIND_001', 'user_1', 'my shopping cart'), + (102, 'KIND_002', 'user_1', 'my shopping cart'), + (103, 'KIND_007', 'user_1', 'my shopping cart'), + (104, 'KIND_008', 'user_1', 'my shopping cart'), + (105, 'KIND_100', 'user_2', 'my shopping list'), + (105, 'KIND_999', 'user_3', 'my shopping list'), + (107, 'KIND_010', 'user_4', 'my shopping list'), + (108, 'KIND_009', 'user_4', 'my shopping list'), + (109, 'KIND_002', 'user_5', 'leo list'), + (111, 'KIND_007', 'user_5', 'leo list'), + (111, 'KIND_008', 'user_5', 'leo list'), + (112, 'KIND_009', 'user_6', 'my shopping cart'); + +-- table has bigint unsigned auto increment primary key +CREATE TABLE shopping_cart_big ( + product_no BIGINT UNSIGNED AUTO_INCREMENT NOT NULL, + product_kind VARCHAR(255), + user_id VARCHAR(255) NOT NULL, + description VARCHAR(255) NOT NULL, + PRIMARY KEY(product_no) +); + +insert into shopping_cart_big +VALUES (default, 'KIND_001', 'user_1', 'my shopping cart'), + (default, 'KIND_002', 'user_1', 'my shopping cart'), + (default, 'KIND_003', 'user_1', 'my shopping cart'); + +-- table has decimal primary key +CREATE TABLE shopping_cart_dec ( + product_no DECIMAL(10, 4) NOT NULL, + product_kind VARCHAR(255), + user_id VARCHAR(255) NOT NULL, + description VARCHAR(255) DEFAULT 'flink', + PRIMARY KEY(product_no) +); + +insert into shopping_cart_dec +VALUES (123456.123, 'KIND_001', 'user_1', 'my shopping cart'), + (123457.456, 'KIND_002', 'user_2', 'my shopping cart'), + (123458.6789, 'KIND_003', 'user_3', 'my shopping cart'), + (123459.1234, 'KIND_004', 'user_4', null); + +-- create table whose primary key are produced by snowflake algorithm +CREATE TABLE address ( + id BIGINT UNSIGNED NOT NULL PRIMARY KEY, + country VARCHAR(255) NOT NULL, + city VARCHAR(255) NOT NULL, + detail_address VARCHAR(1024) +); + +INSERT INTO address +VALUES (416874195632735147, 'China', 'Beijing', 'West Town address 1'), + (416927583791428523, 'China', 'Beijing', 'West Town address 2'), + (417022095255614379, 'China', 'Beijing', 'West Town address 3'), + (417111867899200427, 'America', 'New York', 'East Town address 1'), + (417271541558096811, 'America', 'New York', 'East Town address 2'), + (417272886855938987, 'America', 'New York', 'East Town address 3'), + (417420106184475563, 'Germany', 'Berlin', 'West Town address 1'), + (418161258277847979, 'Germany', 'Berlin', 'West Town address 2'); + +CREATE TABLE default_value_test ( + id INTEGER NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + address VARCHAR(1024), + phone_number INTEGER DEFAULT ' 123 ' +); +INSERT INTO default_value_test +VALUES (1,'user1','Shanghai',123567), + (2,'user2','Shanghai',123567); From 07446d1f9de23b6d5e7b50de1b00ba901ffd31db Mon Sep 17 00:00:00 2001 From: Hang Ruan Date: Fri, 9 Aug 2024 14:28:43 +0800 Subject: [PATCH 3/6] [FLINK-35715][common] Ignore the compact optimize for mysql timestamp type in BinaryRecordData (#3511) --- .../common/data/binary/BinaryRecordData.java | 54 ----------- .../mysql/source/MySqlFullTypesITCase.java | 89 +++++++++++++++++++ .../source/MySqlMetadataAccessorITCase.java | 2 +- .../test/resources/ddl/column_type_test.sql | 35 +++++++- .../resources/ddl/column_type_test_mysql8.sql | 35 +++++++- .../data/writer/AbstractBinaryWriter.java | 56 +++++------- 6 files changed, 182 insertions(+), 89 deletions(-) diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/binary/BinaryRecordData.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/binary/BinaryRecordData.java index a2d91bbd8c..0b02ad7e13 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/binary/BinaryRecordData.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/binary/BinaryRecordData.java @@ -26,12 +26,6 @@ import org.apache.flink.cdc.common.data.StringData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; -import org.apache.flink.cdc.common.types.DataType; -import org.apache.flink.cdc.common.types.DataTypeRoot; -import org.apache.flink.cdc.common.types.DecimalType; -import org.apache.flink.cdc.common.types.LocalZonedTimestampType; -import org.apache.flink.cdc.common.types.TimestampType; -import org.apache.flink.cdc.common.types.ZonedTimestampType; import org.apache.flink.cdc.common.utils.Preconditions; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; @@ -72,45 +66,6 @@ public static int calculateBitSetWidthInBytes(int arity) { return ((arity + 63 + HEADER_SIZE_IN_BITS) / 64) * 8; } - public static int calculateFixPartSizeInBytes(int arity) { - return calculateBitSetWidthInBytes(arity) + 8 * arity; - } - - /** - * If it is a fixed-length field, we can call this BinaryRecordData's setXX method for in-place - * updates. If it is variable-length field, can't use this method, because the underlying data - * is stored continuously. - */ - public static boolean isInFixedLengthPart(DataType type) { - switch (type.getTypeRoot()) { - case BOOLEAN: - case TINYINT: - case SMALLINT: - case INTEGER: - case DATE: - case TIME_WITHOUT_TIME_ZONE: - case BIGINT: - case FLOAT: - case DOUBLE: - return true; - case DECIMAL: - return DecimalData.isCompact(((DecimalType) type).getPrecision()); - case TIMESTAMP_WITHOUT_TIME_ZONE: - return TimestampData.isCompact(((TimestampType) type).getPrecision()); - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return LocalZonedTimestampData.isCompact( - ((LocalZonedTimestampType) type).getPrecision()); - case TIMESTAMP_WITH_TIME_ZONE: - return ZonedTimestampData.isCompact(((ZonedTimestampType) type).getPrecision()); - default: - return false; - } - } - - public static boolean isMutable(DataType type) { - return isInFixedLengthPart(type) || type.getTypeRoot() == DataTypeRoot.DECIMAL; - } - private final int arity; private final int nullBitsSizeInBytes; @@ -213,10 +168,6 @@ public DecimalData getDecimal(int pos, int precision, int scale) { public TimestampData getTimestamp(int pos, int precision) { assertIndexIsValid(pos); - if (TimestampData.isCompact(precision)) { - return TimestampData.fromMillis(segments[0].getLong(getFieldOffset(pos))); - } - int fieldOffset = getFieldOffset(pos); final long offsetAndNanoOfMilli = segments[0].getLong(fieldOffset); return BinarySegmentUtils.readTimestampData(segments, offset, offsetAndNanoOfMilli); @@ -233,11 +184,6 @@ public ZonedTimestampData getZonedTimestamp(int pos, int precision) { public LocalZonedTimestampData getLocalZonedTimestampData(int pos, int precision) { assertIndexIsValid(pos); - if (LocalZonedTimestampData.isCompact(precision)) { - return LocalZonedTimestampData.fromEpochMillis( - segments[0].getLong(getFieldOffset(pos))); - } - int fieldOffset = getFieldOffset(pos); final long offsetAndNanoOfMilli = segments[0].getLong(fieldOffset); return BinarySegmentUtils.readLocalZonedTimestampData( diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlFullTypesITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlFullTypesITCase.java index 41613a86e9..91351dabf4 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlFullTypesITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlFullTypesITCase.java @@ -223,6 +223,95 @@ public void testMysql8TimeDataTypes() throws Throwable { fullTypesMySql8Database, recordType, expectedSnapshot, expectedStreamRecord); } + @Test + public void testMysql57PrecisionTypes() throws Throwable { + testMysqlPrecisionTypes(fullTypesMySql57Database); + } + + @Test + public void testMysql8PrecisionTypes() throws Throwable { + testMysqlPrecisionTypes(fullTypesMySql8Database); + } + + public void testMysqlPrecisionTypes(UniqueDatabase database) throws Throwable { + RowType recordType = + RowType.of( + DataTypes.DECIMAL(20, 0).notNull(), + DataTypes.DECIMAL(6, 2), + DataTypes.DECIMAL(9, 4), + DataTypes.DECIMAL(20, 4), + DataTypes.TIME(0), + DataTypes.TIME(3), + DataTypes.TIME(6), + DataTypes.TIMESTAMP(0), + DataTypes.TIMESTAMP(3), + DataTypes.TIMESTAMP(6), + DataTypes.TIMESTAMP_LTZ(0), + DataTypes.TIMESTAMP_LTZ(3), + DataTypes.TIMESTAMP_LTZ(6), + DataTypes.TIMESTAMP_LTZ(0)); + + Object[] expectedSnapshot = + new Object[] { + DecimalData.fromBigDecimal(new BigDecimal("1"), 20, 0), + DecimalData.fromBigDecimal(new BigDecimal("123.4"), 6, 2), + DecimalData.fromBigDecimal(new BigDecimal("1234.5"), 9, 4), + DecimalData.fromBigDecimal(new BigDecimal("1234.56"), 20, 4), + 64800000, + 64822100, + 64822100, + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:00")), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:00")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:22")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:22")) + }; + + Object[] expectedStreamRecord = + new Object[] { + DecimalData.fromBigDecimal(new BigDecimal("1"), 20, 0), + DecimalData.fromBigDecimal(new BigDecimal("123.4"), 6, 2), + DecimalData.fromBigDecimal(new BigDecimal("1234.5"), 9, 4), + DecimalData.fromBigDecimal(new BigDecimal("1234.56"), 20, 4), + 64800000, + 64822100, + null, + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:00")), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:00")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:22")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:22")) + }; + + database.createAndInitialize(); + CloseableIterator iterator = + env.fromSource( + getFlinkSourceProvider(new String[] {"precision_types"}, database) + .getSource(), + WatermarkStrategy.noWatermarks(), + "Event-Source") + .executeAndCollect(); + + // skip CreateTableEvent + List snapshotResults = MySqSourceTestUtils.fetchResults(iterator, 2); + RecordData snapshotRecord = ((DataChangeEvent) snapshotResults.get(1)).after(); + + Assertions.assertThat(RecordDataTestUtils.recordFields(snapshotRecord, recordType)) + .isEqualTo(expectedSnapshot); + + try (Connection connection = database.getJdbcConnection(); + Statement statement = connection.createStatement()) { + statement.execute("UPDATE precision_types SET time_6_c = null WHERE id = 1;"); + } + + List streamResults = MySqSourceTestUtils.fetchResults(iterator, 1); + RecordData streamRecord = ((DataChangeEvent) streamResults.get(0)).after(); + Assertions.assertThat(RecordDataTestUtils.recordFields(streamRecord, recordType)) + .isEqualTo(expectedStreamRecord); + } + private void testCommonDataTypes(UniqueDatabase database) throws Exception { database.createAndInitialize(); CloseableIterator iterator = diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlMetadataAccessorITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlMetadataAccessorITCase.java index 4b45e1079a..3d3f0276b6 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlMetadataAccessorITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlMetadataAccessorITCase.java @@ -210,7 +210,7 @@ public void testMysql8AccessTimeTypesSchema() { private void testAccessDatabaseAndTable(UniqueDatabase database) { database.createAndInitialize(); - String[] tables = new String[] {"common_types", "time_types"}; + String[] tables = new String[] {"common_types", "time_types", "precision_types"}; MySqlMetadataAccessor metadataAccessor = getMetadataAccessor(tables, database); assertThatThrownBy(metadataAccessor::listNamespaces) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test.sql b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test.sql index a83e8710ec..08d25a714d 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test.sql +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test.sql @@ -123,4 +123,37 @@ VALUES (DEFAULT, '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', '2020-07-17 18:00:22', - NULL); \ No newline at end of file + NULL); + +CREATE TABLE precision_types +( + id SERIAL, + decimal_c0 DECIMAL(6, 2), + decimal_c1 DECIMAL(9, 4), + decimal_c2 DECIMAL(20, 4), + time_c TIME(0), + time_3_c TIME(3), + time_6_c TIME(6), + datetime_c DATETIME(0), + datetime3_c DATETIME(3), + datetime6_c DATETIME(6), + timestamp_c TIMESTAMP(0) NULL, + timestamp3_c TIMESTAMP(3) NULL, + timestamp6_c TIMESTAMP(6) NULL, + PRIMARY KEY (id) +) DEFAULT CHARSET=utf8; + +INSERT INTO precision_types +VALUES (DEFAULT, + 123.4, + 1234.5, + 1234.56, + '18:00', + '18:00:22.1', + '18:00:22.1', + '2020-07-17 18:00', + '2020-07-17 18:00:22', + '2020-07-17 18:00:22', + '2020-07-17 18:00', + '2020-07-17 18:00:22', + '2020-07-17 18:00:22'); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test_mysql8.sql b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test_mysql8.sql index 66b75bfefd..54c6c71705 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test_mysql8.sql +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test_mysql8.sql @@ -127,4 +127,37 @@ VALUES (DEFAULT, '2020-07-17 18:00:22', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', - NULL); \ No newline at end of file + NULL); + +CREATE TABLE precision_types +( + id SERIAL, + decimal_c0 DECIMAL(6, 2), + decimal_c1 DECIMAL(9, 4), + decimal_c2 DECIMAL(20, 4), + time_c TIME(0), + time_3_c TIME(3), + time_6_c TIME(6), + datetime_c DATETIME(0), + datetime3_c DATETIME(3), + datetime6_c DATETIME(6), + timestamp_c TIMESTAMP(0), + timestamp3_c TIMESTAMP(3), + timestamp6_c TIMESTAMP(6), + PRIMARY KEY (id) +) DEFAULT CHARSET=utf8; + +INSERT INTO precision_types +VALUES (DEFAULT, + 123.4, + 1234.5, + 1234.56, + '18:00', + '18:00:22.1', + '18:00:22.1', + '2020-07-17 18:00', + '2020-07-17 18:00:22', + '2020-07-17 18:00:22', + '2020-07-17 18:00', + '2020-07-17 18:00:22', + '2020-07-17 18:00:22'); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/writer/AbstractBinaryWriter.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/writer/AbstractBinaryWriter.java index 1422935ea9..44636fd7e1 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/writer/AbstractBinaryWriter.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/writer/AbstractBinaryWriter.java @@ -173,46 +173,38 @@ public void writeDecimal(int pos, DecimalData value, int precision) { @Override public void writeTimestamp(int pos, TimestampData value, int precision) { - if (TimestampData.isCompact(precision)) { - writeLong(pos, value.getMillisecond()); - } else { - // store the nanoOfMillisecond in fixed-length part as offset and nanoOfMillisecond - ensureCapacity(8); + // store the nanoOfMillisecond in fixed-length part as offset and nanoOfMillisecond + ensureCapacity(8); - if (value == null) { - setNullBit(pos); - // zero-out the bytes - segment.putLong(cursor, 0L); - setOffsetAndSize(pos, cursor, 0); - } else { - segment.putLong(cursor, value.getMillisecond()); - setOffsetAndSize(pos, cursor, value.getNanoOfMillisecond()); - } - - cursor += 8; + if (value == null) { + setNullBit(pos); + // zero-out the bytes + segment.putLong(cursor, 0L); + setOffsetAndSize(pos, cursor, 0); + } else { + segment.putLong(cursor, value.getMillisecond()); + setOffsetAndSize(pos, cursor, value.getNanoOfMillisecond()); } + + cursor += 8; } @Override public void writeLocalZonedTimestamp(int pos, LocalZonedTimestampData value, int precision) { - if (LocalZonedTimestampData.isCompact(precision)) { - writeLong(pos, value.getEpochMillisecond()); - } else { - // store the nanoOfMillisecond in fixed-length part as offset and nanoOfMillisecond - ensureCapacity(8); + // store the nanoOfMillisecond in fixed-length part as offset and nanoOfMillisecond + ensureCapacity(8); - if (value == null) { - setNullBit(pos); - // zero-out the bytes - segment.putLong(cursor, 0L); - setOffsetAndSize(pos, cursor, 0); - } else { - segment.putLong(cursor, value.getEpochMillisecond()); - setOffsetAndSize(pos, cursor, value.getEpochNanoOfMillisecond()); - } - - cursor += 8; + if (value == null) { + setNullBit(pos); + // zero-out the bytes + segment.putLong(cursor, 0L); + setOffsetAndSize(pos, cursor, 0); + } else { + segment.putLong(cursor, value.getEpochMillisecond()); + setOffsetAndSize(pos, cursor, value.getEpochNanoOfMillisecond()); } + + cursor += 8; } @Override From 4bf5a395a5a8f83a5e309e5e985ad7c839b953db Mon Sep 17 00:00:00 2001 From: Hongshun Wang <125648852+loserwang1024@users.noreply.github.com> Date: Fri, 9 Aug 2024 15:38:58 +0800 Subject: [PATCH 4/6] [FLINK-34688][cdc-connector][mysql] Make scan newly table trigger condition strictly This closes #3519. --- .../assigner/SnapshotSplitAssigner.java | 3 ++- .../assigners/MySqlSnapshotSplitAssigner.java | 3 ++- .../MySqlSnapshotSplitAssignerTest.java | 25 ++++++++++++++++--- 3 files changed, 25 insertions(+), 6 deletions(-) diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java index cd0e772004..a3a234b732 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java @@ -203,7 +203,8 @@ else if (!isRemainingTablesCheckpointed && !isSnapshotAssigningFinished(assigner private void captureNewlyAddedTables() { if (sourceConfig.isScanNewlyAddedTableEnabled() - && !sourceConfig.getStartupOptions().isSnapshotOnly()) { + && !sourceConfig.getStartupOptions().isSnapshotOnly() + && AssignerStatus.isAssigningFinished(assignerStatus)) { try { // check whether we got newly added tables final List currentCapturedTables = diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssigner.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssigner.java index e209921b55..9ea69b11ad 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssigner.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssigner.java @@ -215,7 +215,8 @@ else if (!isRemainingTablesCheckpointed private void captureNewlyAddedTables() { // Don't scan newly added table in snapshot mode. if (sourceConfig.isScanNewlyAddedTableEnabled() - && !sourceConfig.getStartupOptions().isSnapshotOnly()) { + && !sourceConfig.getStartupOptions().isSnapshotOnly() + && AssignerStatus.isAssigningFinished(assignerStatus)) { // check whether we got newly added tables try (JdbcConnection jdbc = DebeziumUtils.openJdbcConnection(sourceConfig)) { final List currentCapturedTables = diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssignerTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssignerTest.java index 759827d9df..de875a0ed7 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssignerTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssignerTest.java @@ -484,7 +484,7 @@ public void testEnumerateTablesLazily() { } @Test - public void testScanNewlyAddedTableStartFromCheckpoint() { + public void testScanNewlyAddedTableStartFromInitialAssigningFinishedCheckpoint() { List expected = Arrays.asList( "customers_sparse_dist [109] null", @@ -492,7 +492,24 @@ public void testScanNewlyAddedTableStartFromCheckpoint() { "customers_even_dist [10] [18]", "customers_even_dist [18] null", "customer_card_single_line null null"); - assertEquals(expected, getTestAssignSnapshotSplitsFromCheckpoint()); + assertEquals( + expected, + getTestAssignSnapshotSplitsFromCheckpoint( + AssignerStatus.INITIAL_ASSIGNING_FINISHED)); + } + + @Test + public void testScanNewlyAddedTableStartFromNewlyAddedAssigningSnapshotFinishedCheckpoint() { + List expected = + Arrays.asList( + "customers_sparse_dist [109] null", + "customers_even_dist null [10]", + "customers_even_dist [10] [18]", + "customers_even_dist [18] null"); + assertEquals( + expected, + getTestAssignSnapshotSplitsFromCheckpoint( + AssignerStatus.NEWLY_ADDED_ASSIGNING_SNAPSHOT_FINISHED)); } private List getTestAssignSnapshotSplits( @@ -536,7 +553,7 @@ private List getTestAssignSnapshotSplits( return getSplitsFromAssigner(assigner); } - private List getTestAssignSnapshotSplitsFromCheckpoint() { + private List getTestAssignSnapshotSplitsFromCheckpoint(AssignerStatus assignerStatus) { TableId newTable = TableId.parse(customerDatabase.getDatabaseName() + ".customer_card_single_line"); TableId processedTable = @@ -619,7 +636,7 @@ private List getTestAssignSnapshotSplitsFromCheckpoint() { assignedSplits, new HashMap<>(), splitFinishedOffsets, - AssignerStatus.INITIAL_ASSIGNING, + assignerStatus, remainingTables, false, true, From e0d6d1d1a8ebf5878479c8453ba070875268611b Mon Sep 17 00:00:00 2001 From: Kunni Date: Fri, 9 Aug 2024 17:29:14 +0800 Subject: [PATCH 5/6] [FLINK-35442][cdc-connect][kafka] add key.format and partition.strategy option to make sure the same record sending to the same partition. (#3522) --- .../connectors/pipeline-connectors/kafka.md | 14 ++ .../connectors/pipeline-connectors/kafka.md | 14 ++ .../pom.xml | 8 + .../kafka/json/TableSchemaInfo.java | 46 +++++- .../canal/CanalJsonSerializationSchema.java | 13 +- .../DebeziumJsonSerializationSchema.java | 11 +- .../serialization/CsvSerializationSchema.java | 117 +++++++++++++++ .../JsonSerializationSchema.java | 141 ++++++++++++++++++ .../connectors/kafka/sink/KafkaDataSink.java | 14 +- .../kafka/sink/KafkaDataSinkFactory.java | 13 +- .../kafka/sink/KafkaDataSinkOptions.java | 16 ++ .../cdc/connectors/kafka/sink/KeyFormat.java | 40 +++++ .../kafka/sink/KeySerializationFactory.java | 74 +++++++++ .../kafka/sink/PartitionStrategy.java | 41 +++++ ...ipelineKafkaRecordSerializationSchema.java | 42 ++---- .../kafka/json/TableSchemaInfoTest.java | 7 +- .../CsvSerializationSchemaTest.java | 126 ++++++++++++++++ .../JsonSerializationSchemaTest.java | 135 +++++++++++++++++ .../kafka/sink/KafkaDataSinkITCase.java | 127 +++++++++++++++- .../cdc/connectors/kafka/sink/KafkaUtil.java | 11 +- 20 files changed, 946 insertions(+), 64 deletions(-) create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/serialization/CsvSerializationSchema.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/serialization/JsonSerializationSchema.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KeyFormat.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KeySerializationFactory.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/PartitionStrategy.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/serialization/CsvSerializationSchemaTest.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/serialization/JsonSerializationSchemaTest.java diff --git a/docs/content.zh/docs/connectors/pipeline-connectors/kafka.md b/docs/content.zh/docs/connectors/pipeline-connectors/kafka.md index aa9dc8879b..66467b4d5f 100644 --- a/docs/content.zh/docs/connectors/pipeline-connectors/kafka.md +++ b/docs/content.zh/docs/connectors/pipeline-connectors/kafka.md @@ -87,6 +87,20 @@ Pipeline 连接器配置项 String Sink 的名称。 + + partition.strategy + optional + (none) + String + 定义发送数据到 Kafka 分区的策略, 可以设置的选项有 `all-to-zero`(将所有数据发送到 0 号分区) 以及 `hash-by-key`(所有数据根据主键的哈希值分发),默认值为 `all-to-zero`。 + + + key.format + optional + (none) + String + 用于序列化 Kafka 消息的键部分数据的格式。可以设置的选项有 `csv` 以及 `json`, 默认值为 `json`。 + value.format optional diff --git a/docs/content/docs/connectors/pipeline-connectors/kafka.md b/docs/content/docs/connectors/pipeline-connectors/kafka.md index 57f690666d..6bb94bc370 100644 --- a/docs/content/docs/connectors/pipeline-connectors/kafka.md +++ b/docs/content/docs/connectors/pipeline-connectors/kafka.md @@ -85,6 +85,20 @@ Pipeline Connector Options String The name of the sink. + + partition.strategy + optional + (none) + String + Defines the strategy for sending record to kafka topic, available options are `all-to-zero`(sending all records to 0 partition) and `hash-by-key`(distributing all records by hash of primary keys), default option is `all-to-zero`. + + + key.format + optional + (none) + String + Defines the format identifier for encoding key data, available options are `csv` and `json`, default option is `json`. + value.format optional diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/pom.xml index 5be032a76b..2614f594f3 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/pom.xml @@ -45,6 +45,14 @@ limitations under the License. provided + + org.apache.flink + flink-csv + ${flink.version} + provided + + + org.apache.flink diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfo.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfo.java index a8480d77df..fe764ea85c 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfo.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfo.java @@ -25,6 +25,7 @@ import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.data.binary.BinaryStringData; @@ -39,26 +40,59 @@ /** maintain the {@link SerializationSchema} of a specific {@link TableId}. */ public class TableSchemaInfo { + private final TableId tableId; + private final Schema schema; + private final List primaryKeyColumnIndexes; + private final List fieldGetters; private final SerializationSchema serializationSchema; public TableSchemaInfo( - Schema schema, SerializationSchema serializationSchema, ZoneId zoneId) { + TableId tableId, + Schema schema, + SerializationSchema serializationSchema, + ZoneId zoneId) { + this.tableId = tableId; this.schema = schema; this.serializationSchema = serializationSchema; this.fieldGetters = createFieldGetters(schema, zoneId); + primaryKeyColumnIndexes = new ArrayList<>(); + for (int keyIndex = 0; keyIndex < schema.primaryKeys().size(); keyIndex++) { + for (int columnIndex = 0; columnIndex < schema.getColumnCount(); columnIndex++) { + if (schema.getColumns() + .get(columnIndex) + .getName() + .equals(schema.primaryKeys().get(keyIndex))) { + primaryKeyColumnIndexes.add(columnIndex); + break; + } + } + } } /** convert to {@link RowData}, which will be pass to serializationSchema. */ - public RowData getRowDataFromRecordData(RecordData recordData) { - GenericRowData genericRowData = new GenericRowData(recordData.getArity()); - for (int i = 0; i < recordData.getArity(); i++) { - genericRowData.setField(i, fieldGetters.get(i).getFieldOrNull(recordData)); + public RowData getRowDataFromRecordData(RecordData recordData, boolean primaryKeyOnly) { + if (primaryKeyOnly) { + GenericRowData genericRowData = new GenericRowData(primaryKeyColumnIndexes.size() + 1); + genericRowData.setField(0, StringData.fromString(tableId.toString())); + for (int i = 0; i < primaryKeyColumnIndexes.size(); i++) { + genericRowData.setField( + i + 1, + fieldGetters + .get(primaryKeyColumnIndexes.get(i)) + .getFieldOrNull(recordData)); + } + return genericRowData; + } else { + GenericRowData genericRowData = new GenericRowData(recordData.getArity()); + for (int i = 0; i < recordData.getArity(); i++) { + genericRowData.setField(i, fieldGetters.get(i).getFieldOrNull(recordData)); + } + return genericRowData; } - return genericRowData; } private static List createFieldGetters(Schema schema, ZoneId zoneId) { diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/canal/CanalJsonSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/canal/CanalJsonSerializationSchema.java index 0a145cab73..d0c6179752 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/canal/CanalJsonSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/canal/CanalJsonSerializationSchema.java @@ -127,7 +127,8 @@ public byte[] serialize(Event event) { } jsonSerializers.put( schemaChangeEvent.tableId(), - new TableSchemaInfo(schema, jsonSerializer, zoneId)); + new TableSchemaInfo( + schemaChangeEvent.tableId(), schema, jsonSerializer, zoneId)); return null; } @@ -153,7 +154,8 @@ public byte[] serialize(Event event) { new RowData[] { jsonSerializers .get(dataChangeEvent.tableId()) - .getRowDataFromRecordData((dataChangeEvent.after())) + .getRowDataFromRecordData( + dataChangeEvent.after(), false) })); reuseGenericRowData.setField(2, OP_INSERT); return jsonSerializers @@ -168,7 +170,7 @@ public byte[] serialize(Event event) { jsonSerializers .get(dataChangeEvent.tableId()) .getRowDataFromRecordData( - (dataChangeEvent.before())) + dataChangeEvent.before(), false) })); reuseGenericRowData.setField(1, null); reuseGenericRowData.setField(2, OP_DELETE); @@ -185,7 +187,7 @@ public byte[] serialize(Event event) { jsonSerializers .get(dataChangeEvent.tableId()) .getRowDataFromRecordData( - (dataChangeEvent.before())) + dataChangeEvent.before(), false) })); reuseGenericRowData.setField( 1, @@ -193,7 +195,8 @@ public byte[] serialize(Event event) { new RowData[] { jsonSerializers .get(dataChangeEvent.tableId()) - .getRowDataFromRecordData((dataChangeEvent.after())) + .getRowDataFromRecordData( + dataChangeEvent.after(), false) })); reuseGenericRowData.setField(2, OP_UPDATE); return jsonSerializers diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java index ce8afc0dbe..15cecbc4f8 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java @@ -126,7 +126,8 @@ public byte[] serialize(Event event) { } jsonSerializers.put( schemaChangeEvent.tableId(), - new TableSchemaInfo(schema, jsonSerializer, zoneId)); + new TableSchemaInfo( + schemaChangeEvent.tableId(), schema, jsonSerializer, zoneId)); return null; } @@ -144,7 +145,7 @@ public byte[] serialize(Event event) { 1, jsonSerializers .get(dataChangeEvent.tableId()) - .getRowDataFromRecordData(dataChangeEvent.after())); + .getRowDataFromRecordData(dataChangeEvent.after(), false)); reuseGenericRowData.setField(2, OP_INSERT); return jsonSerializers .get(dataChangeEvent.tableId()) @@ -155,7 +156,7 @@ public byte[] serialize(Event event) { 0, jsonSerializers .get(dataChangeEvent.tableId()) - .getRowDataFromRecordData(dataChangeEvent.before())); + .getRowDataFromRecordData(dataChangeEvent.before(), false)); reuseGenericRowData.setField(1, null); reuseGenericRowData.setField(2, OP_DELETE); return jsonSerializers @@ -168,12 +169,12 @@ public byte[] serialize(Event event) { 0, jsonSerializers .get(dataChangeEvent.tableId()) - .getRowDataFromRecordData(dataChangeEvent.before())); + .getRowDataFromRecordData(dataChangeEvent.before(), false)); reuseGenericRowData.setField( 1, jsonSerializers .get(dataChangeEvent.tableId()) - .getRowDataFromRecordData(dataChangeEvent.after())); + .getRowDataFromRecordData(dataChangeEvent.after(), false)); reuseGenericRowData.setField(2, OP_UPDATE); return jsonSerializers .get(dataChangeEvent.tableId()) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/serialization/CsvSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/serialization/CsvSerializationSchema.java new file mode 100644 index 0000000000..5e08d92ee1 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/serialization/CsvSerializationSchema.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.kafka.serialization; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.OperationType; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.DataField; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.utils.DataTypeUtils; +import org.apache.flink.cdc.common.utils.SchemaUtils; +import org.apache.flink.cdc.connectors.kafka.json.TableSchemaInfo; +import org.apache.flink.formats.csv.CsvRowDataSerializationSchema; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; + +import java.time.ZoneId; +import java.util.HashMap; +import java.util.Map; + +/** A {@link SerializationSchema} to convert {@link Event} into byte of csv format. */ +public class CsvSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = 1L; + + /** + * A map of {@link TableId} and its {@link SerializationSchema} to serialize Debezium JSON data. + */ + private final Map csvSerializers; + + private final ZoneId zoneId; + + private InitializationContext context; + + public CsvSerializationSchema(ZoneId zoneId) { + this.zoneId = zoneId; + csvSerializers = new HashMap<>(); + } + + @Override + public void open(InitializationContext context) { + this.context = context; + } + + @Override + public byte[] serialize(Event event) { + if (event instanceof SchemaChangeEvent) { + Schema schema; + SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; + if (event instanceof CreateTableEvent) { + CreateTableEvent createTableEvent = (CreateTableEvent) event; + schema = createTableEvent.getSchema(); + } else { + schema = + SchemaUtils.applySchemaChangeEvent( + csvSerializers.get(schemaChangeEvent.tableId()).getSchema(), + schemaChangeEvent); + } + CsvRowDataSerializationSchema csvSerializer = buildSerializationForPrimaryKey(schema); + try { + csvSerializer.open(context); + } catch (Exception e) { + throw new RuntimeException(e); + } + csvSerializers.put( + schemaChangeEvent.tableId(), + new TableSchemaInfo( + schemaChangeEvent.tableId(), schema, csvSerializer, zoneId)); + return null; + } + DataChangeEvent dataChangeEvent = (DataChangeEvent) event; + RecordData recordData = + dataChangeEvent.op().equals(OperationType.DELETE) + ? dataChangeEvent.before() + : dataChangeEvent.after(); + TableSchemaInfo tableSchemaInfo = csvSerializers.get(dataChangeEvent.tableId()); + return tableSchemaInfo + .getSerializationSchema() + .serialize(tableSchemaInfo.getRowDataFromRecordData(recordData, true)); + } + + private CsvRowDataSerializationSchema buildSerializationForPrimaryKey(Schema schema) { + DataField[] fields = new DataField[schema.primaryKeys().size() + 1]; + fields[0] = DataTypes.FIELD("TableId", DataTypes.STRING()); + for (int i = 0; i < schema.primaryKeys().size(); i++) { + Column column = schema.getColumn(schema.primaryKeys().get(i)).get(); + fields[i + 1] = DataTypes.FIELD(column.getName(), column.getType()); + } + // the row should never be null + DataType dataType = DataTypes.ROW(fields).notNull(); + LogicalType rowType = DataTypeUtils.toFlinkDataType(dataType).getLogicalType(); + return new CsvRowDataSerializationSchema.Builder((RowType) rowType).build(); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/serialization/JsonSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/serialization/JsonSerializationSchema.java new file mode 100644 index 0000000000..5425d444ea --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/serialization/JsonSerializationSchema.java @@ -0,0 +1,141 @@ +/* + * 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.connectors.kafka.serialization; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.OperationType; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.DataField; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.utils.DataTypeUtils; +import org.apache.flink.cdc.common.utils.SchemaUtils; +import org.apache.flink.cdc.connectors.kafka.json.TableSchemaInfo; +import org.apache.flink.formats.common.TimestampFormat; +import org.apache.flink.formats.json.JsonFormatOptions; +import org.apache.flink.formats.json.JsonRowDataSerializationSchema; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; + +import java.time.ZoneId; +import java.util.HashMap; +import java.util.Map; + +/** A {@link SerializationSchema} to convert {@link Event} into byte of json format. */ +public class JsonSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = 1L; + + /** + * A map of {@link TableId} and its {@link SerializationSchema} to serialize Debezium JSON data. + */ + private final Map jsonSerializers; + + private final TimestampFormat timestampFormat; + + private final JsonFormatOptions.MapNullKeyMode mapNullKeyMode; + + private final String mapNullKeyLiteral; + + private final boolean encodeDecimalAsPlainNumber; + + private final ZoneId zoneId; + + private InitializationContext context; + + public JsonSerializationSchema( + TimestampFormat timestampFormat, + JsonFormatOptions.MapNullKeyMode mapNullKeyMode, + String mapNullKeyLiteral, + ZoneId zoneId, + boolean encodeDecimalAsPlainNumber) { + this.timestampFormat = timestampFormat; + this.mapNullKeyMode = mapNullKeyMode; + this.mapNullKeyLiteral = mapNullKeyLiteral; + this.encodeDecimalAsPlainNumber = encodeDecimalAsPlainNumber; + this.zoneId = zoneId; + jsonSerializers = new HashMap<>(); + } + + @Override + public void open(InitializationContext context) { + this.context = context; + } + + @Override + public byte[] serialize(Event event) { + if (event instanceof SchemaChangeEvent) { + Schema schema; + SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; + if (event instanceof CreateTableEvent) { + CreateTableEvent createTableEvent = (CreateTableEvent) event; + schema = createTableEvent.getSchema(); + } else { + schema = + SchemaUtils.applySchemaChangeEvent( + jsonSerializers.get(schemaChangeEvent.tableId()).getSchema(), + schemaChangeEvent); + } + JsonRowDataSerializationSchema jsonSerializer = buildSerializationForPrimaryKey(schema); + try { + jsonSerializer.open(context); + } catch (Exception e) { + throw new RuntimeException(e); + } + jsonSerializers.put( + schemaChangeEvent.tableId(), + new TableSchemaInfo( + schemaChangeEvent.tableId(), schema, jsonSerializer, zoneId)); + return null; + } + DataChangeEvent dataChangeEvent = (DataChangeEvent) event; + RecordData recordData = + dataChangeEvent.op().equals(OperationType.DELETE) + ? dataChangeEvent.before() + : dataChangeEvent.after(); + TableSchemaInfo tableSchemaInfo = jsonSerializers.get(dataChangeEvent.tableId()); + return tableSchemaInfo + .getSerializationSchema() + .serialize(tableSchemaInfo.getRowDataFromRecordData(recordData, true)); + } + + private JsonRowDataSerializationSchema buildSerializationForPrimaryKey(Schema schema) { + DataField[] fields = new DataField[schema.primaryKeys().size() + 1]; + fields[0] = DataTypes.FIELD("TableId", DataTypes.STRING()); + for (int i = 0; i < schema.primaryKeys().size(); i++) { + Column column = schema.getColumn(schema.primaryKeys().get(i)).get(); + fields[i + 1] = DataTypes.FIELD(column.getName(), column.getType()); + } + // the row should never be null + DataType dataType = DataTypes.ROW(fields).notNull(); + LogicalType rowType = DataTypeUtils.toFlinkDataType(dataType).getLogicalType(); + return new JsonRowDataSerializationSchema( + (RowType) rowType, + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSink.java index 2ab264c5a9..2dfc021b3b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSink.java @@ -29,7 +29,6 @@ import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.sink.KafkaSink; import org.apache.flink.connector.kafka.sink.KafkaSinkBuilder; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.kafka.clients.producer.ProducerConfig; @@ -46,10 +45,12 @@ public class KafkaDataSink implements DataSink { final DeliveryGuarantee deliveryGuarantee; - final FlinkKafkaPartitioner partitioner; + private final PartitionStrategy partitionStrategy; final ZoneId zoneId; + final SerializationSchema keySerialization; + final SerializationSchema valueSerialization; final String topic; @@ -61,16 +62,18 @@ public class KafkaDataSink implements DataSink { public KafkaDataSink( DeliveryGuarantee deliveryGuarantee, Properties kafkaProperties, - FlinkKafkaPartitioner partitioner, + PartitionStrategy partitionStrategy, ZoneId zoneId, + SerializationSchema keySerialization, SerializationSchema valueSerialization, String topic, boolean addTableToHeaderEnabled, String customHeaders) { this.deliveryGuarantee = deliveryGuarantee; this.kafkaProperties = kafkaProperties; - this.partitioner = partitioner; + this.partitionStrategy = partitionStrategy; this.zoneId = zoneId; + this.keySerialization = keySerialization; this.valueSerialization = valueSerialization; this.topic = topic; this.addTableToHeaderEnabled = addTableToHeaderEnabled; @@ -90,7 +93,8 @@ public EventSinkProvider getEventSinkProvider() { .setKafkaProducerConfig(kafkaProperties) .setRecordSerializer( new PipelineKafkaRecordSerializationSchema( - partitioner, + partitionStrategy, + keySerialization, valueSerialization, topic, addTableToHeaderEnabled, diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkFactory.java index 243991d1ae..d898129139 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkFactory.java @@ -28,7 +28,6 @@ import org.apache.flink.cdc.connectors.kafka.json.JsonSerializationType; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.DeliveryGuarantee; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import java.time.ZoneId; import java.util.HashSet; @@ -37,6 +36,8 @@ import java.util.Properties; import java.util.Set; +import static org.apache.flink.cdc.connectors.kafka.sink.KafkaDataSinkOptions.KEY_FORMAT; +import static org.apache.flink.cdc.connectors.kafka.sink.KafkaDataSinkOptions.PARTITION_STRATEGY; import static org.apache.flink.cdc.connectors.kafka.sink.KafkaDataSinkOptions.PROPERTIES_PREFIX; import static org.apache.flink.cdc.connectors.kafka.sink.KafkaDataSinkOptions.SINK_ADD_TABLEID_TO_HEADER_ENABLED; import static org.apache.flink.cdc.connectors.kafka.sink.KafkaDataSinkOptions.SINK_CUSTOM_HEADER; @@ -65,6 +66,9 @@ public DataSink createDataSink(Context context) { context.getPipelineConfiguration() .get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE)); } + KeyFormat keyFormat = context.getFactoryConfiguration().get(KEY_FORMAT); + SerializationSchema keySerialization = + KeySerializationFactory.createSerializationSchema(configuration, keyFormat, zoneId); JsonSerializationType jsonSerializationType = context.getFactoryConfiguration().get(KafkaDataSinkOptions.VALUE_FORMAT); SerializationSchema valueSerialization = @@ -86,11 +90,14 @@ public DataSink createDataSink(Context context) { .get(KafkaDataSinkOptions.SINK_ADD_TABLEID_TO_HEADER_ENABLED); String customHeaders = context.getFactoryConfiguration().get(KafkaDataSinkOptions.SINK_CUSTOM_HEADER); + PartitionStrategy partitionStrategy = + context.getFactoryConfiguration().get(KafkaDataSinkOptions.PARTITION_STRATEGY); return new KafkaDataSink( deliveryGuarantee, kafkaProperties, - new FlinkFixedPartitioner<>(), + partitionStrategy, zoneId, + keySerialization, valueSerialization, topic, addTableToHeaderEnabled, @@ -110,7 +117,9 @@ public Set> requiredOptions() { @Override public Set> optionalOptions() { Set> options = new HashSet<>(); + options.add(KEY_FORMAT); options.add(VALUE_FORMAT); + options.add(PARTITION_STRATEGY); options.add(TOPIC); options.add(SINK_ADD_TABLEID_TO_HEADER_ENABLED); options.add(SINK_CUSTOM_HEADER); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkOptions.java index e55e149a9c..ca82f5c807 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkOptions.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkOptions.java @@ -35,6 +35,22 @@ public class KafkaDataSinkOptions { .defaultValue(DeliveryGuarantee.AT_LEAST_ONCE) .withDescription("Optional delivery guarantee when committing."); + public static final ConfigOption PARTITION_STRATEGY = + key("partition.strategy") + .enumType(PartitionStrategy.class) + .defaultValue(PartitionStrategy.ALL_TO_ZERO) + .withDescription( + "Defines the strategy for sending record to kafka topic, " + + "available options are `all-to-zero` and `hash-by-key`, default option is `all-to-zero`."); + + public static final ConfigOption KEY_FORMAT = + key("key.format") + .enumType(KeyFormat.class) + .defaultValue(KeyFormat.JSON) + .withDescription( + "Defines the format identifier for encoding key data, " + + "available options are `csv` and `json`, default option is `json`."); + public static final ConfigOption VALUE_FORMAT = key("value.format") .enumType(JsonSerializationType.class) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KeyFormat.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KeyFormat.java new file mode 100644 index 0000000000..7c065d47fd --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KeyFormat.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.connectors.kafka.sink; + +import org.apache.flink.api.common.serialization.SerializationSchema; + +import org.apache.kafka.clients.producer.ProducerRecord; + +/** Enum class for building {@link SerializationSchema} for {@link ProducerRecord}. */ +public enum KeyFormat { + JSON("json"), + + CSV("csv"); + + private final String value; + + KeyFormat(String value) { + this.value = value; + } + + @Override + public String toString() { + return value; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KeySerializationFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KeySerializationFactory.java new file mode 100644 index 0000000000..76132d8e54 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KeySerializationFactory.java @@ -0,0 +1,74 @@ +/* + * 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.connectors.kafka.sink; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.connectors.kafka.serialization.CsvSerializationSchema; +import org.apache.flink.cdc.connectors.kafka.serialization.JsonSerializationSchema; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.formats.common.TimestampFormat; +import org.apache.flink.formats.json.JsonFormatOptions; +import org.apache.flink.formats.json.JsonFormatOptionsUtil; + +import java.time.ZoneId; + +import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; +import static org.apache.flink.formats.json.debezium.DebeziumJsonFormatOptions.JSON_MAP_NULL_KEY_LITERAL; + +/** + * Format factory for providing configured instances of {@link SerializationSchema} to convert + * {@link Event} to byte. + */ +public class KeySerializationFactory { + + /** + * Creates a configured instance of {@link SerializationSchema} to convert {@link Event} to + * byte. + */ + public static SerializationSchema createSerializationSchema( + ReadableConfig formatOptions, KeyFormat keyFormat, ZoneId zoneId) { + switch (keyFormat) { + case JSON: + { + TimestampFormat timestampFormat = + JsonFormatOptionsUtil.getTimestampFormat(formatOptions); + JsonFormatOptions.MapNullKeyMode mapNullKeyMode = + JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions); + String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL); + + final boolean encodeDecimalAsPlainNumber = + formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER); + return new JsonSerializationSchema( + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral, + zoneId, + encodeDecimalAsPlainNumber); + } + case CSV: + { + return new CsvSerializationSchema(zoneId); + } + default: + { + throw new IllegalArgumentException("UnSupport key format of " + keyFormat); + } + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/PartitionStrategy.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/PartitionStrategy.java new file mode 100644 index 0000000000..7638c66356 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/PartitionStrategy.java @@ -0,0 +1,41 @@ +/* + * 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.connectors.kafka.sink; + +import org.apache.kafka.clients.producer.ProducerRecord; + +/** Partition Strategy for sending {@link ProducerRecord} to kafka partition. */ +public enum PartitionStrategy { + + /** All {@link ProducerRecord} will be sent to partition 0. */ + ALL_TO_ZERO("all-to-zero"), + + /** {@link ProducerRecord} will be sent to specific partition by primary key. */ + HASH_BY_KEY("hash-by-key"); + + private final String value; + + PartitionStrategy(String value) { + this.value = value; + } + + @Override + public String toString() { + return value; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/PipelineKafkaRecordSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/PipelineKafkaRecordSerializationSchema.java index bb27753d96..85e5e3f199 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/PipelineKafkaRecordSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/PipelineKafkaRecordSerializationSchema.java @@ -23,14 +23,11 @@ import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.header.internals.RecordHeaders; -import javax.annotation.Nullable; - import java.util.HashMap; import java.util.Map; @@ -46,7 +43,11 @@ */ public class PipelineKafkaRecordSerializationSchema implements KafkaRecordSerializationSchema { - private final FlinkKafkaPartitioner partitioner; + + private final Integer partition; + + private final SerializationSchema keySerialization; + private final SerializationSchema valueSerialization; private final String unifiedTopic; @@ -63,12 +64,13 @@ public class PipelineKafkaRecordSerializationSchema public static final String TABLE_NAME_HEADER_KEY = "tableName"; PipelineKafkaRecordSerializationSchema( - @Nullable FlinkKafkaPartitioner partitioner, + PartitionStrategy partitionStrategy, + SerializationSchema keySerialization, SerializationSchema valueSerialization, String unifiedTopic, boolean addTableToHeaderEnabled, String customHeaderString) { - this.partitioner = partitioner; + this.keySerialization = keySerialization; this.valueSerialization = checkNotNull(valueSerialization); this.unifiedTopic = unifiedTopic; this.addTableToHeaderEnabled = addTableToHeaderEnabled; @@ -87,12 +89,14 @@ public class PipelineKafkaRecordSerializationSchema } } } + partition = partitionStrategy.equals(PartitionStrategy.ALL_TO_ZERO) ? 0 : null; } @Override public ProducerRecord serialize( Event event, KafkaSinkContext context, Long timestamp) { ChangeEvent changeEvent = (ChangeEvent) event; + final byte[] keySerialized = keySerialization.serialize(event); final byte[] valueSerialized = valueSerialization.serialize(event); if (event instanceof SchemaChangeEvent) { // skip sending SchemaChangeEvent. @@ -121,37 +125,13 @@ public ProducerRecord serialize( } } return new ProducerRecord<>( - topic, - extractPartition( - changeEvent, valueSerialized, context.getPartitionsForTopic(topic)), - null, - null, - valueSerialized, - recordHeaders); + topic, partition, null, keySerialized, valueSerialized, recordHeaders); } @Override public void open( SerializationSchema.InitializationContext context, KafkaSinkContext sinkContext) throws Exception { - if (partitioner != null) { - partitioner.open( - sinkContext.getParallelInstanceId(), - sinkContext.getNumberOfParallelInstances()); - } valueSerialization.open(context); } - - private Integer extractPartition( - ChangeEvent changeEvent, byte[] valueSerialized, int[] partitions) { - if (partitioner != null) { - return partitioner.partition( - changeEvent, - null, - valueSerialized, - changeEvent.tableId().toString(), - partitions); - } - return null; - } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfoTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfoTest.java index 73f8ad64df..2c2feaae75 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfoTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfoTest.java @@ -22,6 +22,7 @@ import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; +import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.types.DataType; import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; @@ -95,7 +96,9 @@ public void testGetRowDataFromRecordData() { "null_string", org.apache.flink.cdc.common.types.DataTypes.STRING()) .primaryKey("col1") .build(); - TableSchemaInfo tableSchemaInfo = new TableSchemaInfo(schema, null, ZoneId.of("UTC+8")); + TableSchemaInfo tableSchemaInfo = + new TableSchemaInfo( + TableId.parse("testDatabase.testTable"), schema, null, ZoneId.of("UTC+8")); Object[] testData = new Object[] { BinaryStringData.fromString("pk"), @@ -159,6 +162,6 @@ public void testGetRowDataFromRecordData() { org.apache.flink.table.data.TimestampData.fromInstant( Instant.parse("2023-01-01T08:00:00.000Z")), null), - tableSchemaInfo.getRowDataFromRecordData(recordData)); + tableSchemaInfo.getRowDataFromRecordData(recordData, false)); } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/serialization/CsvSerializationSchemaTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/serialization/CsvSerializationSchemaTest.java new file mode 100644 index 0000000000..92b302f7eb --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/serialization/CsvSerializationSchemaTest.java @@ -0,0 +1,126 @@ +/* + * 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.connectors.kafka.serialization; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.cdc.common.data.binary.BinaryStringData; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.RowType; +import org.apache.flink.cdc.connectors.kafka.json.MockInitializationContext; +import org.apache.flink.cdc.connectors.kafka.sink.KeyFormat; +import org.apache.flink.cdc.connectors.kafka.sink.KeySerializationFactory; +import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.jackson.JacksonMapperFactory; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.time.ZoneId; + +/** Tests for {@link CsvSerializationSchema}. */ +public class CsvSerializationSchemaTest { + + public static final TableId TABLE_1 = + TableId.tableId("default_namespace", "default_schema", "table1"); + + @Test + public void testSerialize() throws Exception { + ObjectMapper mapper = + JacksonMapperFactory.createObjectMapper() + .configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, false); + SerializationSchema serializationSchema = + KeySerializationFactory.createSerializationSchema( + new Configuration(), KeyFormat.CSV, ZoneId.systemDefault()); + serializationSchema.open(new MockInitializationContext()); + + // create table + Schema schema = + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .primaryKey("col1") + .build(); + CreateTableEvent createTableEvent = new CreateTableEvent(TABLE_1, schema); + Assertions.assertNull(serializationSchema.serialize(createTableEvent)); + + // insert + BinaryRecordDataGenerator generator = + new BinaryRecordDataGenerator(RowType.of(DataTypes.STRING(), DataTypes.STRING())); + DataChangeEvent insertEvent1 = + DataChangeEvent.insertEvent( + TABLE_1, + generator.generate( + new Object[] { + BinaryStringData.fromString("1"), + BinaryStringData.fromString("1") + })); + String expected = "\"default_namespace.default_schema.table1\",1"; + String actual = new String(serializationSchema.serialize(insertEvent1)); + Assertions.assertEquals(expected, actual); + + DataChangeEvent insertEvent2 = + DataChangeEvent.insertEvent( + TABLE_1, + generator.generate( + new Object[] { + BinaryStringData.fromString("2"), + BinaryStringData.fromString("2") + })); + expected = "\"default_namespace.default_schema.table1\",2"; + actual = new String(serializationSchema.serialize(insertEvent2)); + Assertions.assertEquals(expected, actual); + + DataChangeEvent deleteEvent = + DataChangeEvent.deleteEvent( + TABLE_1, + generator.generate( + new Object[] { + BinaryStringData.fromString("2"), + BinaryStringData.fromString("2") + })); + expected = "\"default_namespace.default_schema.table1\",2"; + actual = new String(serializationSchema.serialize(deleteEvent)); + Assertions.assertEquals(expected, actual); + + DataChangeEvent updateEvent = + DataChangeEvent.updateEvent( + TABLE_1, + generator.generate( + new Object[] { + BinaryStringData.fromString("1"), + BinaryStringData.fromString("1") + }), + generator.generate( + new Object[] { + BinaryStringData.fromString("1"), + BinaryStringData.fromString("x") + })); + expected = "\"default_namespace.default_schema.table1\",1"; + actual = new String(serializationSchema.serialize(updateEvent)); + Assertions.assertEquals(expected, actual); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/serialization/JsonSerializationSchemaTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/serialization/JsonSerializationSchemaTest.java new file mode 100644 index 0000000000..d21f1fb5f1 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/serialization/JsonSerializationSchemaTest.java @@ -0,0 +1,135 @@ +/* + * 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.connectors.kafka.serialization; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.cdc.common.data.binary.BinaryStringData; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.RowType; +import org.apache.flink.cdc.connectors.kafka.json.MockInitializationContext; +import org.apache.flink.cdc.connectors.kafka.sink.KeyFormat; +import org.apache.flink.cdc.connectors.kafka.sink.KeySerializationFactory; +import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.jackson.JacksonMapperFactory; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.time.ZoneId; + +/** Tests for {@link JsonSerializationSchema}. */ +public class JsonSerializationSchemaTest { + + public static final TableId TABLE_1 = + TableId.tableId("default_namespace", "default_schema", "table1"); + + @Test + public void testSerialize() throws Exception { + ObjectMapper mapper = + JacksonMapperFactory.createObjectMapper() + .configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, false); + SerializationSchema serializationSchema = + KeySerializationFactory.createSerializationSchema( + new Configuration(), KeyFormat.JSON, ZoneId.systemDefault()); + serializationSchema.open(new MockInitializationContext()); + + // create table + Schema schema = + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .primaryKey("col1") + .build(); + CreateTableEvent createTableEvent = new CreateTableEvent(TABLE_1, schema); + Assertions.assertNull(serializationSchema.serialize(createTableEvent)); + + // insert + BinaryRecordDataGenerator generator = + new BinaryRecordDataGenerator(RowType.of(DataTypes.STRING(), DataTypes.STRING())); + DataChangeEvent insertEvent1 = + DataChangeEvent.insertEvent( + TABLE_1, + generator.generate( + new Object[] { + BinaryStringData.fromString("1"), + BinaryStringData.fromString("1") + })); + JsonNode expected = + mapper.readTree( + "{\"TableId\":\"default_namespace.default_schema.table1\",\"col1\":\"1\"}"); + JsonNode actual = mapper.readTree(serializationSchema.serialize(insertEvent1)); + Assertions.assertEquals(expected, actual); + + DataChangeEvent insertEvent2 = + DataChangeEvent.insertEvent( + TABLE_1, + generator.generate( + new Object[] { + BinaryStringData.fromString("2"), + BinaryStringData.fromString("2") + })); + expected = + mapper.readTree( + "{\"TableId\":\"default_namespace.default_schema.table1\",\"col1\":\"2\"}"); + actual = mapper.readTree(serializationSchema.serialize(insertEvent2)); + Assertions.assertEquals(expected, actual); + + DataChangeEvent deleteEvent = + DataChangeEvent.deleteEvent( + TABLE_1, + generator.generate( + new Object[] { + BinaryStringData.fromString("2"), + BinaryStringData.fromString("2") + })); + expected = + mapper.readTree( + "{\"TableId\":\"default_namespace.default_schema.table1\",\"col1\":\"2\"}"); + actual = mapper.readTree(serializationSchema.serialize(deleteEvent)); + Assertions.assertEquals(expected, actual); + + DataChangeEvent updateEvent = + DataChangeEvent.updateEvent( + TABLE_1, + generator.generate( + new Object[] { + BinaryStringData.fromString("1"), + BinaryStringData.fromString("1") + }), + generator.generate( + new Object[] { + BinaryStringData.fromString("1"), + BinaryStringData.fromString("x") + })); + expected = + mapper.readTree( + "{\"TableId\":\"default_namespace.default_schema.table1\",\"col1\":\"1\"}"); + actual = mapper.readTree(serializationSchema.serialize(updateEvent)); + Assertions.assertEquals(expected, actual); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkITCase.java index 1ef10603b7..7936c0ef3e 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.connectors.kafka.sink; import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.data.binary.BinaryStringData; import org.apache.flink.cdc.common.event.AddColumnEvent; @@ -67,9 +68,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.UUID; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; @@ -259,7 +262,7 @@ void testDebeziumJsonFormat() throws Exception { env.execute(); final List> collectedRecords = - drainAllRecordsFromTopic(topic, false); + drainAllRecordsFromTopic(topic, false, 0); final long recordsCount = 5; assertThat(recordsCount).isEqualTo(collectedRecords.size()); ObjectMapper mapper = @@ -321,7 +324,7 @@ void testCanalJsonFormat() throws Exception { env.execute(); final List> collectedRecords = - drainAllRecordsFromTopic(topic, false); + drainAllRecordsFromTopic(topic, false, 0); final long recordsCount = 5; assertThat(recordsCount).isEqualTo(collectedRecords.size()); for (ConsumerRecord consumerRecord : collectedRecords) { @@ -363,6 +366,104 @@ void testCanalJsonFormat() throws Exception { checkProducerLeak(); } + @Test + void testHashByKeyPartitionStrategyUsingJson() throws Exception { + final StreamExecutionEnvironment env = new LocalStreamEnvironment(); + env.enableCheckpointing(1000L); + env.setRestartStrategy(RestartStrategies.noRestart()); + final DataStream source = + env.fromCollection(createSourceEvents(), new EventTypeInfo()); + Map config = new HashMap<>(); + Properties properties = getKafkaClientConfiguration(); + properties.forEach( + (key, value) -> + config.put( + KafkaDataSinkOptions.PROPERTIES_PREFIX + key.toString(), + value.toString())); + config.put(KafkaDataSinkOptions.KEY_FORMAT.key(), KeyFormat.JSON.toString()); + config.put( + KafkaDataSinkOptions.VALUE_FORMAT.key(), + JsonSerializationType.CANAL_JSON.toString()); + source.sinkTo( + ((FlinkSinkProvider) + (new KafkaDataSinkFactory() + .createDataSink( + new FactoryHelper.DefaultContext( + Configuration.fromMap(config), + Configuration.fromMap(new HashMap<>()), + this.getClass().getClassLoader())) + .getEventSinkProvider())) + .getSink()); + env.execute(); + + final List> collectedRecords = + drainAllRecordsFromTopic(topic, false); + final long recordsCount = 5; + assertThat(recordsCount).isEqualTo(collectedRecords.size()); + for (ConsumerRecord consumerRecord : collectedRecords) { + assertThat( + consumerRecord + .headers() + .headers( + PipelineKafkaRecordSerializationSchema + .TABLE_NAME_HEADER_KEY) + .iterator()) + .isExhausted(); + } + ObjectMapper mapper = + JacksonMapperFactory.createObjectMapper() + .configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, false); + List> expected = + Arrays.asList( + Tuple2.of( + mapper.readTree( + String.format( + "{\"TableId\":\"%s\",\"col1\":\"1\"}", + table1.toString())), + mapper.readTree( + String.format( + "{\"old\":null,\"data\":[{\"col1\":\"1\",\"col2\":\"1\"}],\"type\":\"INSERT\",\"database\":\"default_schema\",\"table\":\"%s\",\"pkNames\":[\"col1\"]})", + table1.getTableName()))), + Tuple2.of( + mapper.readTree( + String.format( + "{\"TableId\":\"%s\",\"col1\":\"2\"}", + table1.toString())), + mapper.readTree( + String.format( + "{\"old\":null,\"data\":[{\"col1\":\"2\",\"col2\":\"2\"}],\"type\":\"INSERT\",\"database\":\"default_schema\",\"table\":\"%s\",\"pkNames\":[\"col1\"]})", + table1.getTableName()))), + Tuple2.of( + mapper.readTree( + String.format( + "{\"TableId\":\"%s\",\"col1\":\"3\"}", + table1.toString())), + mapper.readTree( + String.format( + "{\"old\":null,\"data\":[{\"col1\":\"3\",\"col2\":\"3\"}],\"type\":\"INSERT\",\"database\":\"default_schema\",\"table\":\"%s\",\"pkNames\":[\"col1\"]})", + table1.getTableName()))), + Tuple2.of( + mapper.readTree( + String.format( + "{\"TableId\":\"%s\",\"col1\":\"1\"}", + table1.toString())), + mapper.readTree( + String.format( + "{\"old\":[{\"col1\":\"1\",\"newCol3\":\"1\"}],\"data\":null,\"type\":\"DELETE\",\"database\":\"default_schema\",\"table\":\"%s\",\"pkNames\":[\"col1\"]})", + table1.getTableName()))), + Tuple2.of( + mapper.readTree( + String.format( + "{\"TableId\":\"%s\",\"col1\":\"2\"}", + table1.toString())), + mapper.readTree( + String.format( + "{\"old\":[{\"col1\":\"2\",\"newCol3\":\"\"}],\"data\":[{\"col1\":\"2\",\"newCol3\":\"x\"}],\"type\":\"UPDATE\",\"database\":\"default_schema\",\"table\":\"%s\",\"pkNames\":[\"col1\"]}", + table1.getTableName())))); + assertThat(deserializeKeyValues(collectedRecords)).containsAll(expected); + checkProducerLeak(); + } + @Test void testTopicAndHeaderOption() throws Exception { final StreamExecutionEnvironment env = new LocalStreamEnvironment(); @@ -392,7 +493,7 @@ void testTopicAndHeaderOption() throws Exception { env.execute(); final List> collectedRecords = - drainAllRecordsFromTopic("test_topic", false); + drainAllRecordsFromTopic("test_topic", false, 0); final long recordsCount = 5; assertThat(recordsCount).isEqualTo(collectedRecords.size()); for (ConsumerRecord consumerRecord : collectedRecords) { @@ -460,9 +561,13 @@ void testTopicAndHeaderOption() throws Exception { } private List> drainAllRecordsFromTopic( - String topic, boolean committed) { + String topic, boolean committed, int... partitionArr) { Properties properties = getKafkaClientConfiguration(); - return KafkaUtil.drainAllRecordsFromTopic(topic, properties, committed); + Set partitions = new HashSet<>(); + for (int partition : partitionArr) { + partitions.add(partition); + } + return KafkaUtil.drainAllRecordsFromTopic(topic, properties, committed, partitions); } private void checkProducerLeak() throws InterruptedException { @@ -486,6 +591,18 @@ private void checkProducerLeak() throws InterruptedException { + leaks.stream().map(this::format).collect(Collectors.joining("\n\n"))); } + private static List> deserializeKeyValues( + List> records) throws IOException { + ObjectMapper mapper = + JacksonMapperFactory.createObjectMapper() + .configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, false); + List> result = new ArrayList<>(); + for (ConsumerRecord record : records) { + result.add(Tuple2.of(mapper.readTree(record.key()), mapper.readTree(record.value()))); + } + return result; + } + private static List deserializeValues(List> records) throws IOException { ObjectMapper mapper = diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaUtil.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaUtil.java index d1f36f657b..775b5b8a02 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaUtil.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaUtil.java @@ -115,13 +115,14 @@ public static KafkaContainer createKafkaContainer( * @throws KafkaException */ public static List> drainAllRecordsFromTopic( - String topic, Properties properties, boolean committed) throws KafkaException { + String topic, Properties properties, boolean committed, Set partitions) + throws KafkaException { final Properties consumerConfig = new Properties(); consumerConfig.putAll(properties); consumerConfig.put( ConsumerConfig.ISOLATION_LEVEL_CONFIG, committed ? "read_committed" : "read_uncommitted"); - return drainAllRecordsFromTopic(topic, consumerConfig); + return drainAllRecordsFromTopic(topic, consumerConfig, partitions); } /** @@ -137,13 +138,17 @@ public static List> drainAllRecordsFromTopic( * @throws KafkaException */ public static List> drainAllRecordsFromTopic( - String topic, Properties properties) throws KafkaException { + String topic, Properties properties, Set partitions) throws KafkaException { final Properties consumerConfig = new Properties(); consumerConfig.putAll(properties); consumerConfig.put("key.deserializer", ByteArrayDeserializer.class.getName()); consumerConfig.put("value.deserializer", ByteArrayDeserializer.class.getName()); try (KafkaConsumer consumer = new KafkaConsumer<>(consumerConfig)) { Set topicPartitions = getAllPartitions(consumer, topic); + if (!partitions.isEmpty()) { + topicPartitions.removeIf( + topicPartition -> !partitions.contains(topicPartition.partition())); + } Map endOffsets = consumer.endOffsets(topicPartitions); consumer.assign(topicPartitions); consumer.seekToBeginning(topicPartitions); From e2bb917ad02c5fd8ee3ec898842d9f0c1aac5f5c Mon Sep 17 00:00:00 2001 From: yuxiqian <34335406+yuxiqian@users.noreply.github.com> Date: Fri, 9 Aug 2024 17:35:29 +0800 Subject: [PATCH 6/6] [FLINK-34876][transform] Support UDF functions in transform (#3465) --- 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 | 31 +- .../YamlPipelineDefinitionParserTest.java | 40 + .../pipeline-definition-with-udf.yaml | 34 + .../cdc/common/udf/UserDefinedFunction.java | 38 + flink-cdc-composer/pom.xml | 14 + .../cdc/composer/definition/PipelineDef.java | 12 +- .../flink/cdc/composer/definition/UdfDef.java | 71 ++ .../composer/flink/FlinkPipelineComposer.java | 8 +- .../flink/translator/TransformTranslator.java | 19 +- .../flink/FlinkPipelineComposerITCase.java | 35 +- .../flink/FlinkPipelineTransformITCase.java | 1 + .../flink/FlinkPipelineUdfITCase.java | 811 ++++++++++++++++++ .../flink-cdc-pipeline-e2e-tests/pom.xml | 34 + .../cdc/pipeline/tests/UdfE2eITCase.java | 403 +++++++++ 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/PostTransformOperator.java | 133 ++- .../transform/PreTransformOperator.java | 33 +- .../transform/ProjectionColumnProcessor.java | 32 +- .../TransformExpressionCompiler.java | 19 +- .../operators/transform/TransformFilter.java | 6 +- .../transform/TransformFilterProcessor.java | 31 +- .../TransformProjectionProcessor.java | 44 +- .../UserDefinedFunctionDescriptor.java | 138 +++ .../cdc/runtime/parser/JaninoCompiler.java | 92 +- .../cdc/runtime/parser/TransformParser.java | 67 +- .../UserDefinedFunctionDescriptorTest.java | 97 +++ .../runtime/parser/TransformParserTest.java | 75 +- pom.xml | 3 + 50 files changed, 3072 insertions(+), 134 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-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/operators/transform/UserDefinedFunctionDescriptor.java create mode 100644 flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/UserDefinedFunctionDescriptorTest.java diff --git a/README.md b/README.md index 426aa96dae..07182ccfb3 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 4de5df3fb8..7b286bfbf5 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 73db8196a8..62f8f210eb 100644 --- a/docs/content.zh/docs/core-concept/transform.md +++ b/docs/content.zh/docs/core-concept/transform.md @@ -268,6 +268,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` which has a constructor 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 8914b721d7..759a47b245 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 73db8196a8..62f8f210eb 100644 --- a/docs/content/docs/core-concept/transform.md +++ b/docs/content/docs/core-concept/transform.md @@ -268,6 +268,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` which has a constructor 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 907462f9fb..d5df8eda14 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 @@ -25,6 +25,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.jackson2.com.fasterxml.jackson.core.type.TypeReference; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; @@ -70,6 +71,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"; @@ -122,6 +128,11 @@ private PipelineDef parse(JsonNode pipelineDefJsonNode, Configuration globalPipe Optional.ofNullable(pipelineDefJsonNode.get(ROUTE_KEY)) .ifPresent(node -> node.forEach(route -> routeDefs.add(toRouteDef(route)))); + // UDFs are optional + List udfDefs = new ArrayList<>(); + Optional.ofNullable(((ObjectNode) pipelineDefJsonNode.get(PIPELINE_KEY)).remove(UDF_KEY)) + .ifPresent(node -> node.forEach(udf -> udfDefs.add(toUdfDef(udf)))); + // Pipeline configs are optional Configuration userPipelineConfig = toPipelineConfig(pipelineDefJsonNode.get(PIPELINE_KEY)); @@ -130,7 +141,8 @@ private PipelineDef parse(JsonNode pipelineDefJsonNode, Configuration globalPipe 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) { @@ -208,6 +220,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 a26dfa8cd7..f57dd62c7a 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") @@ -344,6 +354,7 @@ void testParsingFullDefinitionFromString() throws Exception { null, null, "add new uniq_id for each row")), + Collections.emptyList(), Configuration.fromMap( ImmutableMap.builder() .put("name", "source-database-sync-pipe") @@ -381,6 +392,7 @@ void testParsingFullDefinitionFromString() throws Exception { null, null)), Collections.emptyList(), + Collections.emptyList(), Configuration.fromMap( ImmutableMap.builder() .put("parallelism", "4") @@ -392,6 +404,7 @@ void testParsingFullDefinitionFromString() throws Exception { new SinkDef("kafka", null, new Configuration()), Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), Configuration.fromMap(Collections.singletonMap("parallelism", "1"))); private final PipelineDef fullDefWithRouteRepSym = @@ -449,6 +462,7 @@ void testParsingFullDefinitionFromString() throws Exception { null, null, "add new uniq_id for each row")), + Collections.emptyList(), Configuration.fromMap( ImmutableMap.builder() .put("name", "source-database-sync-pipe") @@ -456,4 +470,30 @@ void testParsingFullDefinitionFromString() 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 0000000000..66a23f9cea --- /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 0000000000..a408785870 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/udf/UserDefinedFunction.java @@ -0,0 +1,38 @@ +/* + * 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.annotation.PublicEvolving; +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. + */ +@PublicEvolving +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-composer/pom.xml b/flink-cdc-composer/pom.xml index f9f57fd3c7..646f1c8673 100644 --- a/flink-cdc-composer/pom.xml +++ b/flink-cdc-composer/pom.xml @@ -61,6 +61,20 @@ limitations under the License. flink-kubernetes ${flink.version} + + 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 49e6a4a0c6..6353c4e746 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 0000000000..6dbc580fb3 --- /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 user-defined function. + * + *

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 f00941b124..ffb328f1fe 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 @@ -106,7 +106,9 @@ public PipelineExecution compose(PipelineDef pipelineDef) { // Build PreTransformOperator for processing Schema Event TransformTranslator transformTranslator = new TransformTranslator(); - stream = transformTranslator.translatePreTransform(stream, pipelineDef.getTransforms()); + stream = + transformTranslator.translatePreTransform( + stream, pipelineDef.getTransforms(), pipelineDef.getUdfs()); // Schema operator SchemaOperatorTranslator schemaOperatorTranslator = @@ -124,8 +126,8 @@ public PipelineExecution compose(PipelineDef pipelineDef) { transformTranslator.translatePostTransform( stream, pipelineDef.getTransforms(), - schemaOperatorIDGenerator.generate(), - pipelineDef.getConfig().get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE)); + pipelineDef.getConfig().get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE), + pipelineDef.getUdfs()); // 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 6feec31d92..7235f2b67b 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 @@ -17,15 +17,17 @@ package org.apache.flink.cdc.composer.flink.translator; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.composer.definition.TransformDef; +import org.apache.flink.cdc.composer.definition.UdfDef; import org.apache.flink.cdc.runtime.operators.transform.PostTransformOperator; import org.apache.flink.cdc.runtime.operators.transform.PreTransformOperator; import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo; -import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.datastream.DataStream; import java.util.List; +import java.util.stream.Collectors; /** * Translator used to build {@link PreTransformOperator} and {@link PostTransformOperator} for event @@ -34,7 +36,7 @@ public class TransformTranslator { public DataStream translatePreTransform( - DataStream input, List transforms) { + DataStream input, List transforms, List udfFunctions) { if (transforms.isEmpty()) { return input; } @@ -52,6 +54,10 @@ public DataStream translatePreTransform( transform.getTableOptions()); } } + preTransformFunctionBuilder.addUdfFunctions( + udfFunctions.stream() + .map(udf -> Tuple2.of(udf.getName(), udf.getClasspath())) + .collect(Collectors.toList())); return input.transform( "Transform:Schema", new EventTypeInfo(), preTransformFunctionBuilder.build()); } @@ -59,8 +65,8 @@ public DataStream translatePreTransform( public DataStream translatePostTransform( DataStream input, List transforms, - OperatorID schemaOperatorID, - String timezone) { + String timezone, + List udfFunctions) { if (transforms.isEmpty()) { return input; } @@ -78,8 +84,11 @@ public DataStream translatePostTransform( transform.getTableOptions()); } } - postTransformFunctionBuilder.addSchemaOperatorID(schemaOperatorID); postTransformFunctionBuilder.addTimezone(timezone); + postTransformFunctionBuilder.addUdfFunctions( + udfFunctions.stream() + .map(udf -> Tuple2.of(udf.getName(), udf.getClasspath())) + .collect(Collectors.toList())); return input.transform( "Transform:Data", new EventTypeInfo(), postTransformFunctionBuilder.build()); } diff --git a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposerITCase.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposerITCase.java index 39c55a6f86..eafb4035e6 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 @@ -140,6 +140,7 @@ void testSingleSplitSingleTable(ValuesDataSink.SinkApi sinkApi) throws Exception sinkDef, Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), pipelineConfig); // Execute the pipeline @@ -196,6 +197,7 @@ void testSingleSplitMultipleTables(ValuesDataSink.SinkApi sinkApi) throws Except sinkDef, Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), pipelineConfig); // Execute the pipeline @@ -262,6 +264,7 @@ void testMultiSplitsSingleTable(ValuesDataSink.SinkApi sinkApi) throws Exception sinkDef, Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), pipelineConfig); // Execute the pipeline @@ -316,6 +319,7 @@ void testTransform(ValuesDataSink.SinkApi sinkApi) throws Exception { sinkDef, Collections.emptyList(), new ArrayList<>(Arrays.asList(transformDef)), + Collections.emptyList(), pipelineConfig); // Execute the pipeline @@ -383,6 +387,7 @@ void testTransformTwice(ValuesDataSink.SinkApi sinkApi) throws Exception { sinkDef, Collections.emptyList(), new ArrayList<>(Arrays.asList(transformDef1, transformDef2)), + Collections.emptyList(), pipelineConfig); // Execute the pipeline @@ -433,7 +438,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); @@ -501,7 +511,12 @@ void testIdenticalOneToOneRouting() 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); @@ -693,7 +708,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); @@ -893,7 +913,13 @@ void testTransformMergingWithRoute() throws Exception { Configuration pipelineConfig = new Configuration(); pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); PipelineDef pipelineDef = - new PipelineDef(sourceDef, sinkDef, routeDef, transformDef, pipelineConfig); + new PipelineDef( + sourceDef, + sinkDef, + routeDef, + transformDef, + Collections.emptyList(), + pipelineConfig); // Execute the pipeline PipelineExecution execution = composer.compose(pipelineDef); @@ -960,6 +986,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/FlinkPipelineTransformITCase.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineTransformITCase.java index b50dbae47c..c23b8624d4 100644 --- a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineTransformITCase.java +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineTransformITCase.java @@ -220,6 +220,7 @@ void testTransformWithTemporalFunction() throws Exception { null, null, null)), + 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 0000000000..c85cf8050e --- /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 557ecafa48..c9850f29d8 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 @@ -106,6 +106,12 @@ limitations under the License. ${project.version} test + + org.apache.flink + flink-cdc-pipeline-udf-examples + ${project.version} + test + @@ -114,6 +120,14 @@ limitations under the License. ${testcontainers.version} test + + + + org.scala-lang + scala-library + ${scala.version} + test + @@ -232,6 +246,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 0000000000..c965e4dec7 --- /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,403 @@ +/* + * 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.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 User-defined functions. */ +@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` VARCHAR(17),`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` VARCHAR(17),`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, 0, 0);"); + 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` VARCHAR(17),`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` VARCHAR(17),`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, 0, 0);"); + 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 0000000000..2466482bb2 --- /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 + + + + + + + + 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 0000000000..32b6ff2aa5 --- /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 0000000000..606362bb36 --- /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 0000000000..d0d700ccd9 --- /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 0000000000..f0329cb6fa --- /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 0000000000..2d0f0069da --- /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 0000000000..2d40a8a5be --- /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 0000000000..8a4310a767 --- /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 0000000000..161fdd2d6a --- /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 0000000000..cbb958e5e1 --- /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 0000000000..f44c472736 --- /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 0000000000..60b5a69566 --- /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 = 0 + + 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 0000000000..ce3049393d --- /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 0000000000..1da5e4175e --- /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 0000000000..85e02f8c93 --- /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 0000000000..34ddff91d3 --- /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 0000000000..6be332e98b --- /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 77c1e6014f..e278fd669a 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/PostTransformOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperator.java index b280bf9016..a4f938d158 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperator.java @@ -29,14 +29,17 @@ import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.schema.Selectors; import org.apache.flink.cdc.common.utils.SchemaUtils; -import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; import javax.annotation.Nullable; import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; import java.time.ZoneId; import java.util.ArrayList; import java.util.List; @@ -62,6 +65,10 @@ public class PostTransformOperator extends AbstractStreamOperator /** keep the relationship of TableId and table information. */ private final Map postTransformChangeInfoMap; + private final List> udfFunctions; + private List udfDescriptors; + private transient Map udfFunctionInstances; + private transient Map, TransformProjectionProcessor> transformProjectionProcessorMap; private transient Map, TransformFilterProcessor> @@ -74,8 +81,8 @@ public static PostTransformOperator.Builder newBuilder() { /** Builder of {@link PostTransformOperator}. */ public static class Builder { private final List transformRules = new ArrayList<>(); - private OperatorID schemaOperatorID; private String timezone; + private final List> udfFunctions = new ArrayList<>(); public PostTransformOperator.Builder addTransform( String tableInclusions, @@ -101,11 +108,6 @@ public PostTransformOperator.Builder addTransform( return this; } - public PostTransformOperator.Builder addSchemaOperatorID(OperatorID schemaOperatorID) { - this.schemaOperatorID = schemaOperatorID; - return this; - } - public PostTransformOperator.Builder addTimezone(String timezone) { if (PipelineOptions.PIPELINE_LOCAL_TIME_ZONE.defaultValue().equals(timezone)) { this.timezone = ZoneId.systemDefault().toString(); @@ -115,17 +117,43 @@ public PostTransformOperator.Builder addTimezone(String timezone) { return this; } + public PostTransformOperator.Builder addUdfFunctions( + List> udfFunctions) { + this.udfFunctions.addAll(udfFunctions); + return this; + } + public PostTransformOperator build() { - return new PostTransformOperator(transformRules, timezone); + return new PostTransformOperator(transformRules, timezone, udfFunctions); } } - private PostTransformOperator(List transformRules, String timezone) { + private PostTransformOperator( + List transformRules, + String timezone, + List> udfFunctions) { this.transformRules = transformRules; this.timezone = timezone; this.postTransformChangeInfoMap = new ConcurrentHashMap<>(); this.transformFilterProcessorMap = new ConcurrentHashMap<>(); this.transformProjectionProcessorMap = new ConcurrentHashMap<>(); + this.udfFunctions = udfFunctions; + this.udfFunctionInstances = new ConcurrentHashMap<>(); + } + + @Override + public void setup( + StreamTask containingTask, + StreamConfig config, + Output> output) { + super.setup(containingTask, config, output); + udfDescriptors = + udfFunctions.stream() + .map( + udf -> { + return new UserDefinedFunctionDescriptor(udf.f0, udf.f1); + }) + .collect(Collectors.toList()); } @Override @@ -146,11 +174,25 @@ public void open() throws Exception { return new PostTransformer( selectors, TransformProjection.of(projection).orElse(null), - TransformFilter.of(filterExpression).orElse(null)); + TransformFilter.of(filterExpression, udfDescriptors) + .orElse(null)); }) .collect(Collectors.toList()); this.transformProjectionProcessorMap = new ConcurrentHashMap<>(); this.transformFilterProcessorMap = new ConcurrentHashMap<>(); + this.udfFunctionInstances = new ConcurrentHashMap<>(); + udfDescriptors.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 @@ -163,6 +205,10 @@ public void finish() throws Exception { public void close() throws Exception { super.close(); clearOperator(); + + // Clean up UDF instances + destroyUdf(); + udfFunctionInstances.clear(); } @Override @@ -226,7 +272,11 @@ private Schema transformSchema(TableId tableId, Schema schema) throws Exception Tuple2.of(tableId, transformProjection))) { transformProjectionProcessorMap.put( Tuple2.of(tableId, transformProjection), - TransformProjectionProcessor.of(transformProjection, timezone)); + TransformProjectionProcessor.of( + transformProjection, + timezone, + udfDescriptors, + getUdfFunctionInstances())); } TransformProjectionProcessor postTransformProcessor = transformProjectionProcessorMap.get( @@ -243,6 +293,12 @@ private Schema transformSchema(TableId tableId, Schema schema) throws Exception return SchemaUtils.inferWiderSchema(newSchemas); } + private List getUdfFunctionInstances() { + return udfDescriptors.stream() + .map(e -> udfFunctionInstances.get(e.getName())) + .collect(Collectors.toList()); + } + private Optional processDataChangeEvent(DataChangeEvent dataChangeEvent) throws Exception { TableId tableId = dataChangeEvent.tableId(); @@ -265,7 +321,12 @@ private Optional processDataChangeEvent(DataChangeEvent dataCha Tuple2.of(tableId, transformFilter))) { transformFilterProcessorMap.put( Tuple2.of(tableId, transformFilter), - TransformFilterProcessor.of(tableInfo, transformFilter, timezone)); + TransformFilterProcessor.of( + tableInfo, + transformFilter, + timezone, + udfDescriptors, + getUdfFunctionInstances())); } TransformFilterProcessor transformFilterProcessor = transformFilterProcessorMap.get(Tuple2.of(tableId, transformFilter)); @@ -287,7 +348,11 @@ private Optional processDataChangeEvent(DataChangeEvent dataCha transformProjectionProcessorMap.put( Tuple2.of(tableId, transformProjection), TransformProjectionProcessor.of( - tableInfo, transformProjection, timezone)); + tableInfo, + transformProjection, + timezone, + udfDescriptors, + getUdfFunctionInstances())); } TransformProjectionProcessor postTransformProcessor = transformProjectionProcessorMap.get( @@ -392,4 +457,46 @@ private void clearOperator() { this.transformFilterProcessorMap = null; TransformExpressionCompiler.cleanUp(); } + + private void initializeUdf() { + udfDescriptors.forEach( + udf -> { + try { + if (udf.isCdcPipelineUdf()) { + // We use reflection to invoke UDF methods since we may add more methods + // into UserDefinedFunction interface, thus the provided UDF classes + // might not be compatible with the interface definition in CDC common. + Object udfInstance = udfFunctionInstances.get(udf.getName()); + udfInstance.getClass().getMethod("open").invoke(udfInstance); + } else { + // Do nothing, Flink-style UDF lifecycle hooks are not supported + } + } catch (InvocationTargetException + | NoSuchMethodException + | IllegalAccessException ex) { + throw new RuntimeException("Failed to initialize UDF " + udf, ex); + } + }); + } + + private void destroyUdf() { + udfDescriptors.forEach( + udf -> { + try { + if (udf.isCdcPipelineUdf()) { + // We use reflection to invoke UDF methods since we may add more methods + // into UserDefinedFunction interface, thus the provided UDF classes + // might not be compatible with the interface definition in CDC common. + Object udfInstance = udfFunctionInstances.get(udf.getName()); + udfInstance.getClass().getMethod("close").invoke(udfInstance); + } else { + // Do nothing, Flink-style UDF lifecycle hooks are not supported + } + } catch (InvocationTargetException + | NoSuchMethodException + | IllegalAccessException 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/PreTransformOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java index 7dbba91d1b..b8dc2448e3 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java @@ -32,10 +32,13 @@ import org.apache.flink.cdc.common.utils.SchemaUtils; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; import javax.annotation.Nullable; @@ -60,6 +63,8 @@ public class PreTransformOperator extends AbstractStreamOperator private final Map preTransformChangeInfoMap; private final List> schemaMetadataTransformers; private transient ListState state; + private final List> udfFunctions; + private List udfDescriptors; private Map preTransformProcessorMap; public static PreTransformOperator.Builder newBuilder() { @@ -70,6 +75,8 @@ public static PreTransformOperator.Builder newBuilder() { public static class Builder { private final List transformRules = new ArrayList<>(); + private final List> udfFunctions = new ArrayList<>(); + public PreTransformOperator.Builder addTransform( String tableInclusions, @Nullable String projection, @Nullable String filter) { transformRules.add(new TransformRule(tableInclusions, projection, filter, "", "", "")); @@ -94,17 +101,37 @@ public PreTransformOperator.Builder addTransform( return this; } + public PreTransformOperator.Builder addUdfFunctions( + List> udfFunctions) { + this.udfFunctions.addAll(udfFunctions); + return this; + } + public PreTransformOperator build() { - return new PreTransformOperator(transformRules); + return new PreTransformOperator(transformRules, udfFunctions); } } - private PreTransformOperator(List transformRules) { + private PreTransformOperator( + List transformRules, List> udfFunctions) { this.transformRules = transformRules; this.preTransformChangeInfoMap = new ConcurrentHashMap<>(); this.preTransformProcessorMap = new ConcurrentHashMap<>(); this.schemaMetadataTransformers = new ArrayList<>(); this.chainingStrategy = ChainingStrategy.ALWAYS; + this.udfFunctions = udfFunctions; + } + + @Override + public void setup( + StreamTask containingTask, + StreamConfig config, + Output> output) { + super.setup(containingTask, config, output); + this.udfDescriptors = + this.udfFunctions.stream() + .map(udf -> new UserDefinedFunctionDescriptor(udf.f0, udf.f1)) + .collect(Collectors.toList()); } @Override @@ -124,7 +151,7 @@ public void open() throws Exception { new PreTransformer( selectors, TransformProjection.of(projection).orElse(null), - TransformFilter.of(filter).orElse(null))); + TransformFilter.of(filter, udfDescriptors).orElse(null))); schemaMetadataTransformers.add( new Tuple2<>( selectors, 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 0d7df06ab0..3dde9b20c5 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 @@ -43,18 +43,35 @@ public class ProjectionColumnProcessor { private ProjectionColumn projectionColumn; private String timezone; private TransformExpressionKey transformExpressionKey; + private final List udfDescriptors; + private final transient List udfFunctionInstances; + private transient ExpressionEvaluator expressionEvaluator; public ProjectionColumnProcessor( - PostTransformChangeInfo tableInfo, ProjectionColumn projectionColumn, String timezone) { + PostTransformChangeInfo tableInfo, + ProjectionColumn projectionColumn, + String timezone, + List udfDescriptors, + final List udfFunctionInstances) { this.tableInfo = tableInfo; this.projectionColumn = projectionColumn; this.timezone = timezone; + this.udfDescriptors = udfDescriptors; this.transformExpressionKey = generateTransformExpressionKey(); + this.expressionEvaluator = + TransformExpressionCompiler.compileExpression( + transformExpressionKey, udfDescriptors); + this.udfFunctionInstances = udfFunctionInstances; } public static ProjectionColumnProcessor of( - PostTransformChangeInfo tableInfo, ProjectionColumn projectionColumn, String timezone) { - return new ProjectionColumnProcessor(tableInfo, projectionColumn, timezone); + PostTransformChangeInfo tableInfo, + ProjectionColumn projectionColumn, + String timezone, + List udfDescriptors, + List udfFunctionInstances) { + return new ProjectionColumnProcessor( + tableInfo, projectionColumn, timezone, udfDescriptors, udfFunctionInstances); } public ProjectionColumn getProjectionColumn() { @@ -62,8 +79,6 @@ public ProjectionColumn getProjectionColumn() { } public Object evaluate(BinaryRecordData after, long epochTime) { - ExpressionEvaluator expressionEvaluator = - TransformExpressionCompiler.compileExpression(transformExpressionKey); try { return expressionEvaluator.evaluate(generateParams(after, epochTime)); } catch (InvocationTargetException e) { @@ -80,6 +95,8 @@ public Object evaluate(BinaryRecordData after, long epochTime) { private Object[] generateParams(BinaryRecordData after, long epochTime) { List params = new ArrayList<>(); List columns = tableInfo.getPreTransformedSchema().getColumns(); + + // 1 - Add referenced columns RecordData.FieldGetter[] fieldGetters = tableInfo.getPreTransformedFieldGetters(); for (String originalColumnName : projectionColumn.getOriginalColumnNames()) { switch (originalColumnName) { @@ -110,8 +127,13 @@ private Object[] generateParams(BinaryRecordData after, long epochTime) { "Failed to evaluate argument " + originalColumnName); } } + + // 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/TransformExpressionCompiler.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformExpressionCompiler.java index 510afdf6ec..e22d7943d9 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 @@ -26,6 +26,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 +43,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 udfDescriptors) { try { return COMPILED_EXPRESSION_CACHE.get( key, () -> { ExpressionEvaluator expressionEvaluator = new ExpressionEvaluator(); + + List argumentNames = key.getArgumentNames(); + List> argumentClasses = key.getArgumentClasses(); + + for (UserDefinedFunctionDescriptor udfFunction : udfDescriptors) { + 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 422623af09..4f1aee57e0 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 @@ -60,13 +60,15 @@ public List getColumnNames() { return columnNames; } - public static Optional of(String filterExpression) { + public static Optional of( + String filterExpression, List udfDescriptors) { if (StringUtils.isNullOrWhitespaceOnly(filterExpression)) { return Optional.empty(); } List columnNames = TransformParser.parseFilterColumnNameList(filterExpression); String scriptExpression = - TransformParser.translateFilterExpressionToJaninoExpression(filterExpression); + TransformParser.translateFilterExpressionToJaninoExpression( + filterExpression, udfDescriptors); 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 9bae6a5730..84d4830357 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 @@ -44,23 +44,36 @@ public class TransformFilterProcessor { private TransformFilter transformFilter; private String timezone; private TransformExpressionKey transformExpressionKey; + private final transient List udfFunctionInstances; + private transient ExpressionEvaluator expressionEvaluator; public TransformFilterProcessor( - PostTransformChangeInfo tableInfo, TransformFilter transformFilter, String timezone) { + PostTransformChangeInfo tableInfo, + TransformFilter transformFilter, + String timezone, + List udfDescriptors, + List udfFunctionInstances) { this.tableInfo = tableInfo; this.transformFilter = transformFilter; this.timezone = timezone; - transformExpressionKey = generateTransformExpressionKey(); + this.transformExpressionKey = generateTransformExpressionKey(); + this.udfFunctionInstances = udfFunctionInstances; + this.expressionEvaluator = + TransformExpressionCompiler.compileExpression( + transformExpressionKey, udfDescriptors); } public static TransformFilterProcessor of( - PostTransformChangeInfo tableInfo, TransformFilter transformFilter, String timezone) { - return new TransformFilterProcessor(tableInfo, transformFilter, timezone); + PostTransformChangeInfo tableInfo, + TransformFilter transformFilter, + String timezone, + List udfDescriptors, + List udfFunctionInstances) { + return new TransformFilterProcessor( + tableInfo, transformFilter, timezone, udfDescriptors, udfFunctionInstances); } public boolean process(BinaryRecordData after, long epochTime) { - ExpressionEvaluator expressionEvaluator = - TransformExpressionCompiler.compileExpression(transformExpressionKey); try { return (Boolean) expressionEvaluator.evaluate(generateParams(after, epochTime)); } catch (InvocationTargetException e) { @@ -106,6 +119,7 @@ private Object[] generateParams(BinaryRecordData after, long epochTime) { List params = new ArrayList<>(); List columns = tableInfo.getPreTransformedSchema().getColumns(); + // 1 - Add referenced columns Tuple2, List>> args = generateArguments(); RecordData.FieldGetter[] fieldGetters = tableInfo.getPreTransformedFieldGetters(); for (String columnName : args.f0) { @@ -130,8 +144,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 307c890fd6..452ac2238c 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 @@ -51,14 +51,22 @@ public class TransformProjectionProcessor { private final TransformProjection transformProjection; private final String timezone; private final List cachedProjectionColumnProcessors; + private final List udfDescriptors; + private final transient List udfFunctionInstances; public TransformProjectionProcessor( PostTransformChangeInfo postTransformChangeInfo, TransformProjection transformProjection, - String timezone) { + String timezone, + List udfDescriptors, + final List udfFunctionInstances) { this.postTransformChangeInfo = postTransformChangeInfo; this.transformProjection = transformProjection; this.timezone = timezone; + this.udfDescriptors = udfDescriptors; + this.udfFunctionInstances = udfFunctionInstances; + + // Create cached projection column processors after setting all other fields. this.cachedProjectionColumnProcessors = cacheProjectionColumnProcessors(postTransformChangeInfo, transformProjection); } @@ -70,23 +78,34 @@ public boolean hasTableInfo() { public static TransformProjectionProcessor of( PostTransformChangeInfo tableInfo, TransformProjection transformProjection, - String timezone) { - return new TransformProjectionProcessor(tableInfo, transformProjection, timezone); + String timezone, + List udfDescriptors, + List udfFunctionInstances) { + return new TransformProjectionProcessor( + tableInfo, transformProjection, timezone, udfDescriptors, udfFunctionInstances); } public static TransformProjectionProcessor of( - TransformProjection transformProjection, String timezone) { - return new TransformProjectionProcessor(null, transformProjection, timezone); + TransformProjection transformProjection, + String timezone, + List udfDescriptors, + List udfFunctionInstances) { + return new TransformProjectionProcessor( + null, transformProjection, timezone, udfDescriptors, udfFunctionInstances); } - public static TransformProjectionProcessor of(TransformProjection transformProjection) { - return new TransformProjectionProcessor(null, transformProjection, null); + public static TransformProjectionProcessor of( + TransformProjection transformProjection, + List udfDescriptors, + List udfFunctionInstances) { + return new TransformProjectionProcessor( + null, transformProjection, null, udfDescriptors, udfFunctionInstances); } public Schema processSchemaChangeEvent(Schema schema) { List projectionColumns = TransformParser.generateProjectionColumns( - transformProjection.getProjection(), schema.getColumns()); + transformProjection.getProjection(), schema.getColumns(), udfDescriptors); transformProjection.setProjectionColumns(projectionColumns); return schema.copy( projectionColumns.stream() @@ -158,7 +177,14 @@ private List cacheProjectionColumnProcessors( cachedProjectionColumnProcessors.add( Optional.ofNullable(matchedProjectionColumn) - .map(col -> ProjectionColumnProcessor.of(tableInfo, col, timezone)) + .map( + col -> + ProjectionColumnProcessor.of( + tableInfo, + col, + timezone, + udfDescriptors, + udfFunctionInstances)) .orElse(null)); } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/UserDefinedFunctionDescriptor.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/UserDefinedFunctionDescriptor.java new file mode 100644 index 0000000000..e12e8417c3 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/UserDefinedFunctionDescriptor.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.runtime.operators.transform; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.udf.UserDefinedFunction; + +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. */ +@Internal +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) { + // We use reflection to invoke UDF methods since we may add more methods + // into UserDefinedFunction interface, thus the provided UDF classes + // might not be compatible with the interface definition in CDC common. + 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-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 4877d01754..9a5c5680c6 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 @@ -20,6 +20,8 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.io.ParseException; import org.apache.flink.cdc.common.utils.StringUtils; +import org.apache.flink.cdc.runtime.operators.transform.UserDefinedFunctionDescriptor; +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 @@ -89,21 +92,23 @@ public static ExpressionEvaluator compileExpression( } } - public static String translateSqlNodeToJaninoExpression(SqlNode transform) { - Java.Rvalue rvalue = translateSqlNodeToJaninoRvalue(transform); + public static String translateSqlNodeToJaninoExpression( + SqlNode transform, List udfDescriptors) { + Java.Rvalue rvalue = translateSqlNodeToJaninoRvalue(transform, udfDescriptors); if (rvalue != null) { return rvalue.toString(); } return ""; } - public static Java.Rvalue translateSqlNodeToJaninoRvalue(SqlNode transform) { + public static Java.Rvalue translateSqlNodeToJaninoRvalue( + SqlNode transform, List udfDescriptors) { if (transform instanceof SqlIdentifier) { return translateSqlIdentifier((SqlIdentifier) transform); } else if (transform instanceof SqlBasicCall) { - return translateSqlBasicCall((SqlBasicCall) transform); + return translateSqlBasicCall((SqlBasicCall) transform, udfDescriptors); } else if (transform instanceof SqlCase) { - return translateSqlCase((SqlCase) transform); + return translateSqlCase((SqlCase) transform, udfDescriptors); } else if (transform instanceof SqlLiteral) { return translateSqlSqlLiteral((SqlLiteral) transform); } @@ -140,11 +145,12 @@ 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 udfDescriptors) { List operandList = sqlBasicCall.getOperandList(); List atoms = new ArrayList<>(); for (SqlNode sqlNode : operandList) { - translateSqlNodeToAtoms(sqlNode, atoms); + translateSqlNodeToAtoms(sqlNode, atoms, udfDescriptors); } if (TIMEZONE_FREE_TEMPORAL_FUNCTIONS.contains(sqlBasicCall.getOperator().getName())) { atoms.add(new Java.AmbiguousName(Location.NOWHERE, new String[] {DEFAULT_EPOCH_TIME})); @@ -156,22 +162,24 @@ private static Java.Rvalue translateSqlBasicCall(SqlBasicCall sqlBasicCall) { sqlBasicCall.getOperator().getName())) { 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]), udfDescriptors); } - private static Java.Rvalue translateSqlCase(SqlCase sqlCase) { + private static Java.Rvalue translateSqlCase( + SqlCase sqlCase, List udfDescriptors) { 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, udfDescriptors); } List thenAtoms = new ArrayList<>(); for (SqlNode sqlNode : thenOperands) { - translateSqlNodeToAtoms(sqlNode, thenAtoms); + translateSqlNodeToAtoms(sqlNode, thenAtoms, udfDescriptors); } - Java.Rvalue elseAtoms = translateSqlNodeToJaninoRvalue(elseOperand); + Java.Rvalue elseAtoms = translateSqlNodeToJaninoRvalue(elseOperand, udfDescriptors); Java.Rvalue sqlCaseRvalueTemp = elseAtoms; for (int i = whenAtoms.size() - 1; i >= 0; i--) { sqlCaseRvalueTemp = @@ -184,24 +192,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 udfDescriptors) { 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, udfDescriptors)); } else if (sqlNode instanceof SqlNodeList) { for (SqlNode node : (SqlNodeList) sqlNode) { - translateSqlNodeToAtoms(node, atoms); + translateSqlNodeToAtoms(node, atoms, udfDescriptors); } } else if (sqlNode instanceof SqlCase) { - atoms.add(translateSqlCase((SqlCase) sqlNode)); + atoms.add(translateSqlCase((SqlCase) sqlNode, udfDescriptors)); } } private static Java.Rvalue sqlBasicCallToJaninoRvalue( - SqlBasicCall sqlBasicCall, Java.Rvalue[] atoms) { + SqlBasicCall sqlBasicCall, + Java.Rvalue[] atoms, + List udfDescriptors) { switch (sqlBasicCall.getKind()) { case AND: return generateBinaryOperation(sqlBasicCall, atoms, "&&"); @@ -231,7 +244,7 @@ private static Java.Rvalue sqlBasicCallToJaninoRvalue( case FLOOR: case TRIM: case OTHER_FUNCTION: - return generateOtherFunctionOperation(sqlBasicCall, atoms); + return generateOtherFunctionOperation(sqlBasicCall, atoms, udfDescriptors); case PLUS: return generateBinaryOperation(sqlBasicCall, atoms, "+"); case MINUS: @@ -297,21 +310,38 @@ private static Java.Rvalue generateOtherOperation( } private static Java.Rvalue generateOtherFunctionOperation( - SqlBasicCall sqlBasicCall, Java.Rvalue[] atoms) { + SqlBasicCall sqlBasicCall, + Java.Rvalue[] atoms, + List udfDescriptors) { 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 { - return new Java.MethodInvocation( - Location.NOWHERE, - null, - StringUtils.convertToCamelCase(sqlBasicCall.getOperator().getName()), - atoms); + Optional udfFunctionOptional = + udfDescriptors.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)); } } @@ -419,4 +449,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 4db3f354fc..6260bf7b0a 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 @@ -22,6 +22,7 @@ import org.apache.flink.cdc.common.types.DataType; import org.apache.flink.cdc.common.types.DataTypes; import org.apache.flink.cdc.runtime.operators.transform.ProjectionColumn; +import org.apache.flink.cdc.runtime.operators.transform.UserDefinedFunctionDescriptor; import org.apache.flink.cdc.runtime.parser.metadata.TransformSchemaFactory; import org.apache.flink.cdc.runtime.parser.metadata.TransformSqlOperatorTable; import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter; @@ -39,8 +40,13 @@ 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.SqlCall; +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.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.sql.parser.SqlParserPos; +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; @@ -75,6 +85,7 @@ import java.util.stream.IntStream; import static org.apache.flink.cdc.common.utils.StringUtils.isNullOrWhitespaceOnly; +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 { @@ -94,15 +105,49 @@ 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 udfDescriptors) { 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 : udfDescriptors) { + 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( @@ -112,9 +157,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)); @@ -219,7 +267,9 @@ private static void expandWildcard(SqlSelect sqlSelect, List columns) { // For example, given projection expression "a, b, c, upper(a) as d, b as e" and columns array // [a, b, c, x, y, z], returns projection column array [a, b, c, d, e]. public static List generateProjectionColumns( - String projectionExpression, List columns) { + String projectionExpression, + List columns, + List udfDescriptors) { if (isNullOrWhitespaceOnly(projectionExpression)) { return new ArrayList<>(); } @@ -228,7 +278,7 @@ public static List generateProjectionColumns( return new ArrayList<>(); } expandWildcard(sqlSelect, columns); - RelNode relNode = sqlToRel(columns, sqlSelect); + RelNode relNode = sqlToRel(columns, sqlSelect, udfDescriptors); Map relDataTypeMap = relNode.getRowType().getFieldList().stream() .collect( @@ -276,7 +326,7 @@ public static List generateProjectionColumns( relDataTypeMap.get(columnName)), transformOptional.get().toString(), JaninoCompiler.translateSqlNodeToJaninoExpression( - transformOptional.get()), + transformOptional.get(), udfDescriptors), parseColumnNameList(transformOptional.get())) : ProjectionColumn.of( columnName, @@ -340,7 +390,8 @@ public static List generateProjectionColumns( return projectionColumns; } - public static String translateFilterExpressionToJaninoExpression(String filterExpression) { + public static String translateFilterExpressionToJaninoExpression( + String filterExpression, List udfDescriptors) { if (isNullOrWhitespaceOnly(filterExpression)) { return ""; } @@ -349,7 +400,7 @@ public static String translateFilterExpressionToJaninoExpression(String filterEx return ""; } SqlNode where = sqlSelect.getWhere(); - return JaninoCompiler.translateSqlNodeToJaninoExpression(where); + return JaninoCompiler.translateSqlNodeToJaninoExpression(where, udfDescriptors); } public static List parseComputedColumnNames(String projection) { diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/UserDefinedFunctionDescriptorTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/UserDefinedFunctionDescriptorTest.java new file mode 100644 index 0000000000..3f4bc366a8 --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/UserDefinedFunctionDescriptorTest.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.runtime.operators.transform; + +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.udf.UserDefinedFunction; +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.runtime.operators.transform.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.runtime.operators.transform.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.runtime.operators.transform.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.runtime.operators.transform.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-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 270e775345..5eee2663e9 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 @@ -22,6 +22,7 @@ import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.types.DataTypes; import org.apache.flink.cdc.runtime.operators.transform.ProjectionColumn; +import org.apache.flink.cdc.runtime.operators.transform.UserDefinedFunctionDescriptor; import org.apache.flink.cdc.runtime.parser.metadata.TransformSchemaFactory; import org.apache.flink.cdc.runtime.parser.metadata.TransformSqlOperatorTable; @@ -50,6 +51,7 @@ import org.junit.jupiter.api.Test; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -324,7 +326,8 @@ public void testGenerateProjectionColumns() { List result = TransformParser.generateProjectionColumns( "id, upper(name) as name, age + 1 as newage, weight / (height * height) as bmi", - testColumns); + testColumns, + Collections.emptyList()); List expected = Arrays.asList( @@ -336,7 +339,9 @@ public void testGenerateProjectionColumns() { List metadataResult = TransformParser.generateProjectionColumns( - "*, __namespace_name__, __schema_name__, __table_name__", testColumns); + "*, __namespace_name__, __schema_name__, __table_name__", + testColumns, + Collections.emptyList()); List metadataExpected = Arrays.asList( @@ -354,7 +359,9 @@ public void testGenerateProjectionColumns() { // calculated columns must use AS to provide an alias name Assertions.assertThatThrownBy( - () -> TransformParser.generateProjectionColumns("id, 1 + 1", testColumns)) + () -> + TransformParser.generateProjectionColumns( + "id, 1 + 1", testColumns, Collections.emptyList())) .isExactlyInstanceOf(ParseException.class); } @@ -413,9 +420,69 @@ public void testNormalizeFilter() { "`concat`(1 - `x`, `concat`(1 - `y`, `concat`(1 - `z`, `x`), `y`), `z`) <> 10"); } + @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()); + Assertions.assertThat(janinoExpression).isEqualTo(expressionExpect); + } + + 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"))); Assertions.assertThat(janinoExpression).isEqualTo(expressionExpect); } } diff --git a/pom.xml b/pom.xml index 4b01c18c60..9955a6c56c 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