> options = new HashSet<>();
+ options.add(HudiConfig.TABLE_TYPE);
+ options.add(HudiConfig.PARTITION_PATH_FIELD);
+ options.add(HudiConfig.INDEX_TYPE);
+ options.add(HudiConfig.INDEX_BUCKET_TARGET);
+ options.add(HudiConfig.HIVE_SYNC_ENABLED);
+ options.add(HudiConfig.HIVE_SYNC_METASTORE_URIS);
+ options.add(HudiConfig.HIVE_SYNC_DB);
+ options.add(HudiConfig.HIVE_SYNC_TABLE);
+
+ options.add(HudiConfig.WRITE_TASKS);
+ options.add(HudiConfig.BUCKET_ASSIGN_TASKS);
+ options.add(HudiConfig.SCHEMA_ON_READ_ENABLE);
+
+ // Compaction settings
+ options.add(HudiConfig.COMPACTION_DELTA_COMMITS);
+ return options;
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkOptions.java
new file mode 100644
index 00000000000..9b2c76bea4d
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkOptions.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.connectors.hudi.sink;
+
+/** HudiDataSink Options reference {@link HudiConfig}. */
+public class HudiDataSinkOptions {
+ // prefix for passing properties for table creation.
+ public static final String PREFIX_TABLE_PROPERTIES = "table.properties.";
+
+ // prefix for passing properties for catalog creation.
+ public static final String PREFIX_CATALOG_PROPERTIES = "catalog.properties.";
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiMetadataApplier.java
new file mode 100644
index 00000000000..9d1177440f2
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiMetadataApplier.java
@@ -0,0 +1,650 @@
+/*
+ * 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.hudi.sink;
+
+import org.apache.flink.cdc.common.configuration.Configuration;
+import org.apache.flink.cdc.common.event.AddColumnEvent;
+import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.DropColumnEvent;
+import org.apache.flink.cdc.common.event.RenameColumnEvent;
+import org.apache.flink.cdc.common.event.SchemaChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.event.visitor.SchemaChangeEventVisitor;
+import org.apache.flink.cdc.common.schema.Column;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.sink.MetadataApplier;
+import org.apache.flink.cdc.common.types.utils.DataTypeUtils;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.ResolvedCatalogTable;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.TableChange;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.types.DataType;
+
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.table.catalog.CatalogOptions;
+import org.apache.hudi.table.catalog.HoodieCatalog;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * A {@link MetadataApplier} that applies schema changes to Hudi tables.
+ *
+ * This applier is responsible for initializing the Hudi table metadata in the file system if it
+ * does not already exist.
+ */
+public class HudiMetadataApplier implements MetadataApplier {
+
+ private static final Logger LOG = LoggerFactory.getLogger(HudiMetadataApplier.class);
+
+ private final Configuration config;
+
+ // Catalog is unSerializable, similar to PaimonMetadataApplier
+ private transient HoodieCatalog catalog;
+
+ private final org.apache.flink.configuration.Configuration catalogConfig;
+
+ public HudiMetadataApplier(Configuration config) {
+ this.config = config;
+ this.catalogConfig = convertToCatalogConfig(config);
+ }
+
+ @Override
+ public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) {
+ LOG.info("Applying schema change event: {}", schemaChangeEvent);
+ // Initialize catalog if not already done
+ if (catalog == null) {
+ catalog = new HoodieCatalog("hoodie_catalog", catalogConfig);
+ try {
+ catalog.open();
+ } catch (CatalogException e) {
+ throw new RuntimeException("Failed to open HoodieCatalog", e);
+ }
+ }
+
+ try {
+ SchemaChangeEventVisitor.visit(
+ schemaChangeEvent,
+ addColumnEvent -> {
+ applyAddColumn(addColumnEvent);
+ return null;
+ },
+ alterColumnTypeEvent -> {
+ applyAlterColumnType(alterColumnTypeEvent);
+ return null;
+ },
+ createTableEvent -> {
+ applyCreateTable(createTableEvent);
+ return null;
+ },
+ dropColumnEvent -> {
+ applyDropColumn(dropColumnEvent);
+ return null;
+ },
+ dropTableEvent -> {
+ throw new UnsupportedOperationException("DropTableEvent is not supported");
+ },
+ renameColumnEvent -> {
+ applyRenameColumn(renameColumnEvent);
+ return null;
+ },
+ truncateTableEvent -> {
+ throw new UnsupportedOperationException(
+ "TruncateTableEvent is not supported");
+ });
+ } catch (Exception e) {
+ LOG.error("Failed to apply schema change for table {}", schemaChangeEvent.tableId(), e);
+ throw new RuntimeException("Failed to apply schema change", e);
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ if (catalog != null) {
+ catalog.close();
+ }
+ }
+
+ private void applyCreateTable(CreateTableEvent event) {
+ try {
+ TableId tableId = event.tableId();
+ String databaseName = tableId.getSchemaName();
+
+ // Create database if it doesn't exist
+ if (!catalog.databaseExists(databaseName)) {
+ CatalogDatabase database = new CatalogDatabaseImpl(new HashMap<>(), null);
+ catalog.createDatabase(databaseName, database, true);
+ LOG.info("Created database: {}", databaseName);
+ }
+
+ // Convert CDC Schema to Flink ResolvedCatalogTable
+ ResolvedCatalogTable catalogTable = convertToCatalogTable(event.getSchema());
+ ObjectPath objectPath = new ObjectPath(databaseName, tableId.getTableName());
+
+ // Create table using catalog
+ catalog.createTable(objectPath, catalogTable, true);
+
+ LOG.info("Successfully created Hudi table {} via catalog", tableId);
+ } catch (DatabaseAlreadyExistException e) {
+ // Should not happen because ignoreIfExists=true
+ LOG.warn("Database already exists: {}", e.getMessage());
+ } catch (TableAlreadyExistException e) {
+ // Should not happen because ignoreIfExists=true
+ LOG.warn("Table already exists: {}", e.getMessage());
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to create table via catalog", e);
+ }
+ }
+
+ private void applyAddColumn(AddColumnEvent event) throws Exception {
+ TableId tableId = event.tableId();
+ ObjectPath objectPath = new ObjectPath(tableId.getSchemaName(), tableId.getTableName());
+
+ // Get existing table and ensure it's resolved
+ ResolvedCatalogTable existingTable = getResolvedCatalogTable(objectPath);
+ ResolvedSchema existingSchema = existingTable.getResolvedSchema();
+
+ // Build new columns list with added columns
+ List newColumns =
+ new ArrayList<>(existingSchema.getColumns());
+ List tableChanges = new ArrayList<>();
+ for (AddColumnEvent.ColumnWithPosition columnWithPosition : event.getAddedColumns()) {
+ Column addColumn = columnWithPosition.getAddColumn();
+ DataType flinkType = DataTypeUtils.toFlinkDataType(addColumn.getType());
+ org.apache.flink.table.catalog.Column newColumn =
+ org.apache.flink.table.catalog.Column.physical(addColumn.getName(), flinkType);
+
+ // Handle column position
+ switch (columnWithPosition.getPosition()) {
+ case FIRST:
+ newColumns.add(0, newColumn);
+
+ tableChanges.add(
+ TableChange.add(newColumn, TableChange.ColumnPosition.first()));
+ break;
+ case LAST:
+ newColumns.add(newColumn);
+ tableChanges.add(TableChange.add(newColumn));
+ break;
+ case BEFORE:
+ int beforeIndex =
+ findColumnIndex(newColumns, columnWithPosition.getExistedColumnName());
+ newColumns.add(beforeIndex, newColumn);
+ tableChanges.add(
+ TableChange.add(
+ newColumn,
+ TableChange.ColumnPosition.after(
+ newColumns.get(beforeIndex).getName())));
+ break;
+ case AFTER:
+ int afterIndex =
+ findColumnIndex(newColumns, columnWithPosition.getExistedColumnName());
+ newColumns.add(afterIndex + 1, newColumn);
+ tableChanges.add(
+ TableChange.add(
+ newColumn,
+ TableChange.ColumnPosition.after(
+ newColumns.get(afterIndex + 1).getName())));
+ break;
+ }
+ LOG.info(
+ "Adding column {} to table {} at position {}",
+ addColumn.getName(),
+ tableId,
+ columnWithPosition.getPosition());
+ }
+
+ // Create new resolved schema
+ ResolvedSchema newSchema =
+ new ResolvedSchema(
+ newColumns,
+ existingSchema.getWatermarkSpecs(),
+ existingSchema.getPrimaryKey().orElse(null));
+
+ // Create new catalog table
+ ResolvedCatalogTable newTable = createUpdatedCatalogTable(existingTable, newSchema);
+
+ // Alter table
+ LOG.info("Committing add column changes {} to HoodieCatalog", tableChanges);
+ catalog.alterTable(objectPath, newTable, tableChanges, false);
+
+ // Verify the change was persisted
+ updateAndVerifyTableChange(tableId, objectPath, newSchema);
+ }
+
+ private void applyDropColumn(DropColumnEvent event) throws Exception {
+ TableId tableId = event.tableId();
+ ObjectPath objectPath = new ObjectPath(tableId.getSchemaName(), tableId.getTableName());
+
+ // Get existing table and ensure it's resolved
+ ResolvedCatalogTable existingTable = getResolvedCatalogTable(objectPath);
+ ResolvedSchema existingSchema = existingTable.getResolvedSchema();
+
+ LOG.info(
+ "Before drop - Table {} has columns: {}",
+ tableId,
+ existingSchema.getColumns().stream()
+ .map(org.apache.flink.table.catalog.Column::getName)
+ .collect(Collectors.toList()));
+
+ // Build new columns list without dropped columns
+ List newColumns =
+ new ArrayList<>(existingSchema.getColumns());
+ newColumns.removeIf(col -> event.getDroppedColumnNames().contains(col.getName()));
+
+ LOG.info("Dropping columns {} from table {}", event.getDroppedColumnNames(), tableId);
+
+ // Create new resolved schema
+ ResolvedSchema newSchema =
+ new ResolvedSchema(
+ newColumns,
+ existingSchema.getWatermarkSpecs(),
+ existingSchema.getPrimaryKey().orElse(null));
+
+ LOG.info(
+ "After drop - Table {} should have columns: {}",
+ tableId,
+ newSchema.getColumns().stream()
+ .map(org.apache.flink.table.catalog.Column::getName)
+ .collect(Collectors.toList()));
+
+ // Create new catalog table
+ ResolvedCatalogTable newTable = createUpdatedCatalogTable(existingTable, newSchema);
+
+ // Build table changes
+ List tableChanges =
+ event.getDroppedColumnNames().stream()
+ .map(colName -> TableChange.dropColumn(colName))
+ .collect(Collectors.toList());
+
+ // Alter table in using Hoodie's catalog and commit required metadata changes
+ LOG.info("Committing drop column changes {} to HoodieCatalog", tableChanges);
+ catalog.alterTable(objectPath, newTable, tableChanges, false);
+
+ LOG.info("Successfully dropped columns from table {}", tableId);
+
+ // Verify the change was persisted
+ updateAndVerifyTableChange(tableId, objectPath, newSchema);
+ }
+
+ private void applyRenameColumn(RenameColumnEvent event) throws Exception {
+ TableId tableId = event.tableId();
+ ObjectPath objectPath = new ObjectPath(tableId.getSchemaName(), tableId.getTableName());
+
+ // Get existing table and ensure it's resolved
+ ResolvedCatalogTable existingTable = getResolvedCatalogTable(objectPath);
+ ResolvedSchema existingSchema = existingTable.getResolvedSchema();
+
+ // Build new columns list with renamed columns
+ List newColumns = new ArrayList<>();
+ List tableChanges = new ArrayList<>();
+ for (org.apache.flink.table.catalog.Column oldCol : existingSchema.getColumns()) {
+ String newName =
+ event.getNameMapping().getOrDefault(oldCol.getName(), oldCol.getName());
+ if (!newName.equals(oldCol.getName())) {
+ LOG.info(
+ "Renaming column {} to {} in table {}", oldCol.getName(), newName, tableId);
+ newColumns.add(
+ org.apache.flink.table.catalog.Column.physical(
+ newName, oldCol.getDataType()));
+ tableChanges.add(TableChange.modifyColumnName(oldCol, newName));
+ } else {
+ // No name change
+ newColumns.add(oldCol);
+ }
+ }
+
+ // Create new resolved schema
+ ResolvedSchema newSchema =
+ new ResolvedSchema(
+ newColumns,
+ existingSchema.getWatermarkSpecs(),
+ existingSchema.getPrimaryKey().orElse(null));
+
+ // Create new catalog table
+ ResolvedCatalogTable newTable = createUpdatedCatalogTable(existingTable, newSchema);
+
+ // Alter table in using Hoodie's catalog and commit required metadata changes
+ catalog.alterTable(objectPath, newTable, tableChanges, false);
+ LOG.info("Successfully renamed columns in table {}", tableId);
+
+ // Verify the change was persisted
+ updateAndVerifyTableChange(tableId, objectPath, newSchema);
+ }
+
+ private void applyAlterColumnType(AlterColumnTypeEvent event) throws Exception {
+ TableId tableId = event.tableId();
+ ObjectPath objectPath = new ObjectPath(tableId.getSchemaName(), tableId.getTableName());
+
+ // Get existing table and ensure it's resolved
+ ResolvedCatalogTable existingTable = getResolvedCatalogTable(objectPath);
+ ResolvedSchema existingSchema = existingTable.getResolvedSchema();
+
+ // Build new columns list with altered types
+ List newColumns = new ArrayList<>();
+ List tableChanges = new ArrayList<>();
+ for (org.apache.flink.table.catalog.Column oldCol : existingSchema.getColumns()) {
+ if (event.getTypeMapping().containsKey(oldCol.getName())) {
+ DataType newType =
+ DataTypeUtils.toFlinkDataType(event.getTypeMapping().get(oldCol.getName()));
+ LOG.info(
+ "Altering column {} type from {} to {} in table {}",
+ oldCol.getName(),
+ oldCol.getDataType(),
+ newType,
+ tableId);
+ newColumns.add(
+ org.apache.flink.table.catalog.Column.physical(oldCol.getName(), newType));
+ tableChanges.add(TableChange.modifyPhysicalColumnType(oldCol, newType));
+ } else {
+ // No type change
+ newColumns.add(oldCol);
+ }
+ }
+
+ // Create new resolved schema
+ ResolvedSchema newSchema =
+ new ResolvedSchema(
+ newColumns,
+ existingSchema.getWatermarkSpecs(),
+ existingSchema.getPrimaryKey().orElse(null));
+
+ // Create new catalog table
+ ResolvedCatalogTable newTable = createUpdatedCatalogTable(existingTable, newSchema);
+
+ // Alter table by passing in tableChanges
+ catalog.alterTable(objectPath, newTable, tableChanges, false);
+ LOG.info("Successfully altered column types in table {}", tableId);
+
+ // Verify the change was persisted
+ updateAndVerifyTableChange(tableId, objectPath, newSchema);
+ }
+
+ /**
+ * Gets a table from the catalog and ensures it's returned as a ResolvedCatalogTable. If the
+ * catalog returns a DefaultCatalogTable, it will be converted to ResolvedCatalogTable.
+ */
+ private ResolvedCatalogTable getResolvedCatalogTable(ObjectPath objectPath) throws Exception {
+ CatalogBaseTable table = catalog.getTable(objectPath);
+
+ if (table instanceof ResolvedCatalogTable) {
+ return (ResolvedCatalogTable) table;
+ }
+
+ // If it's a CatalogTable (or DefaultCatalogTable), resolve it
+ if (table instanceof CatalogTable) {
+ CatalogTable catalogTable = (CatalogTable) table;
+ org.apache.flink.table.api.Schema schema = catalogTable.getUnresolvedSchema();
+
+ // Resolve the schema
+ List resolvedColumns = new ArrayList<>();
+ for (org.apache.flink.table.api.Schema.UnresolvedColumn column : schema.getColumns()) {
+ if (column instanceof org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn) {
+ org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn physicalColumn =
+ (org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn) column;
+ // We need to get the data type - this is already resolved in the schema
+ // For now, we'll rebuild from the schema by resolving it
+ // This is a workaround since we don't have access to the type resolver
+ }
+ }
+
+ // Alternative approach: rebuild the ResolvedCatalogTable from scratch
+ // Extract physical columns from the schema
+ ResolvedSchema resolvedSchema = resolveSchema(schema);
+
+ return new ResolvedCatalogTable(catalogTable, resolvedSchema);
+ }
+
+ throw new IllegalStateException(
+ "Unexpected catalog table type: " + table.getClass().getName());
+ }
+
+ /**
+ * Resolves an unresolved schema to a ResolvedSchema. This manually extracts column information
+ * from the schema.
+ */
+ private ResolvedSchema resolveSchema(org.apache.flink.table.api.Schema unresolvedSchema) {
+ List columns = new ArrayList<>();
+
+ for (org.apache.flink.table.api.Schema.UnresolvedColumn unresolvedColumn :
+ unresolvedSchema.getColumns()) {
+ if (unresolvedColumn
+ instanceof org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn) {
+ org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn physicalColumn =
+ (org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn)
+ unresolvedColumn;
+
+ // Get the column name
+ String columnName = physicalColumn.getName();
+
+ // Get the data type - cast from AbstractDataType to DataType
+ // This is safe because unresolved schemas from catalog tables contain DataType
+ DataType dataType = (DataType) physicalColumn.getDataType();
+
+ columns.add(org.apache.flink.table.catalog.Column.physical(columnName, dataType));
+ }
+ }
+
+ // Extract primary key if exists
+ UniqueConstraint primaryKey = null;
+ if (unresolvedSchema.getPrimaryKey().isPresent()) {
+ org.apache.flink.table.api.Schema.UnresolvedPrimaryKey unresolvedPrimaryKey =
+ unresolvedSchema.getPrimaryKey().get();
+ primaryKey =
+ UniqueConstraint.primaryKey(
+ unresolvedPrimaryKey.getConstraintName(),
+ unresolvedPrimaryKey.getColumnNames());
+ }
+
+ return new ResolvedSchema(columns, new ArrayList<>(), primaryKey);
+ }
+
+ private int findColumnIndex(
+ List columns, String columnName) {
+ for (int i = 0; i < columns.size(); i++) {
+ if (columns.get(i).getName().equals(columnName)) {
+ return i;
+ }
+ }
+ throw new IllegalArgumentException("Column not found: " + columnName);
+ }
+
+ private ResolvedCatalogTable createUpdatedCatalogTable(
+ ResolvedCatalogTable existingTable, ResolvedSchema newSchema) {
+ // Build Flink Schema from resolved schema
+ org.apache.flink.table.api.Schema tableSchema =
+ org.apache.flink.table.api.Schema.newBuilder()
+ .fromResolvedSchema(newSchema)
+ .build();
+
+ // Create new CatalogTable with same options and comment
+ CatalogTable catalogTable =
+ CatalogTable.of(
+ tableSchema,
+ existingTable.getComment(),
+ existingTable.getPartitionKeys(),
+ existingTable.getOptions());
+
+ return new ResolvedCatalogTable(catalogTable, newSchema);
+ }
+
+ /** Converts a Flink DataType to an Avro Schema. */
+ private org.apache.avro.Schema convertFlinkTypeToAvro(DataType flinkType) {
+ org.apache.flink.table.types.logical.LogicalType logicalType = flinkType.getLogicalType();
+
+ switch (logicalType.getTypeRoot()) {
+ case BOOLEAN:
+ return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.BOOLEAN);
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT);
+ case BIGINT:
+ return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.LONG);
+ case FLOAT:
+ return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.FLOAT);
+ case DOUBLE:
+ return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.DOUBLE);
+ case VARCHAR:
+ case CHAR:
+ return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.STRING);
+ case VARBINARY:
+ case BINARY:
+ return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.BYTES);
+ case DECIMAL:
+ org.apache.flink.table.types.logical.DecimalType decimalType =
+ (org.apache.flink.table.types.logical.DecimalType) logicalType;
+ return org.apache.avro.LogicalTypes.decimal(
+ decimalType.getPrecision(), decimalType.getScale())
+ .addToSchema(
+ org.apache.avro.Schema.create(org.apache.avro.Schema.Type.BYTES));
+ case DATE:
+ return org.apache.avro.LogicalTypes.date()
+ .addToSchema(
+ org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT));
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ return org.apache.avro.LogicalTypes.timestampMicros()
+ .addToSchema(
+ org.apache.avro.Schema.create(org.apache.avro.Schema.Type.LONG));
+ default:
+ // Default to string for unsupported types
+ LOG.warn("Unsupported Flink type {}, defaulting to STRING", logicalType);
+ return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.STRING);
+ }
+ }
+
+ private void updateAndVerifyTableChange(
+ TableId tableId, ObjectPath objectPath, ResolvedSchema newSchema) throws Exception {
+ ResolvedCatalogTable verifyTable = getResolvedCatalogTable(objectPath);
+ LOG.info(
+ "Verified - Table {} now has columns: {}",
+ tableId,
+ verifyTable.getResolvedSchema().getColumns().stream()
+ .map(org.apache.flink.table.catalog.Column::getName)
+ .collect(Collectors.toList()));
+ }
+
+ /** Converts CDC Configuration to Flink Configuration for HoodieCatalog. */
+ private org.apache.flink.configuration.Configuration convertToCatalogConfig(
+ Configuration cdcConfig) {
+ org.apache.flink.configuration.Configuration flinkConfig =
+ new org.apache.flink.configuration.Configuration();
+
+ // Set catalog path (base path for all tables)
+ String basePath = cdcConfig.get(HudiConfig.PATH);
+ flinkConfig.setString(CatalogOptions.CATALOG_PATH.key(), basePath);
+
+ // Set mode to DFS (filesystem-based)
+ // TODO: make this configurable
+ flinkConfig.setString(CatalogOptions.MODE.key(), "dfs");
+
+ // Set default database
+ flinkConfig.setString(CatalogOptions.DEFAULT_DATABASE.key(), "default");
+
+ return flinkConfig;
+ }
+
+ /** Converts CDC Schema to Flink ResolvedCatalogTable. */
+ private ResolvedCatalogTable convertToCatalogTable(Schema cdcSchema) {
+ // Build resolved columns
+ List resolvedColumns = new ArrayList<>();
+ for (Column column : cdcSchema.getColumns()) {
+ DataType flinkType = DataTypeUtils.toFlinkDataType(column.getType());
+ resolvedColumns.add(
+ org.apache.flink.table.catalog.Column.physical(column.getName(), flinkType));
+ }
+
+ // Build primary key constraint
+ List primaryKeys = cdcSchema.primaryKeys();
+ UniqueConstraint primaryKeyConstraint = null;
+ if (primaryKeys != null && !primaryKeys.isEmpty()) {
+ primaryKeyConstraint = UniqueConstraint.primaryKey("pk", primaryKeys);
+ }
+
+ // Build ResolvedSchema
+ ResolvedSchema resolvedSchema =
+ new ResolvedSchema(
+ resolvedColumns,
+ new ArrayList<>(), // No watermark specs
+ primaryKeyConstraint);
+
+ // Build Flink Schema from resolved schema
+ org.apache.flink.table.api.Schema tableSchema =
+ org.apache.flink.table.api.Schema.newBuilder()
+ .fromResolvedSchema(resolvedSchema)
+ .build();
+
+ // Build table options (Hudi-specific configurations)
+ Map tableOptions = new HashMap<>();
+
+ // Add table type
+ String tableType = config.get(HudiConfig.TABLE_TYPE);
+ if (tableType != null) {
+ tableOptions.put(FlinkOptions.TABLE_TYPE.key(), tableType);
+ }
+
+ // Add record key field
+ // TODO: Support multiple recordKeyFields
+ String recordKeyField = config.get(HudiConfig.RECORD_KEY_FIELD);
+ if (recordKeyField == null && primaryKeys != null && !primaryKeys.isEmpty()) {
+ // Use first primary key as default record key
+ recordKeyField = primaryKeys.get(0);
+ }
+ if (recordKeyField != null) {
+ tableOptions.put(FlinkOptions.RECORD_KEY_FIELD.key(), recordKeyField);
+ }
+
+ // Add ordering fields if specified
+ String orderingFields = config.get(HudiConfig.ORDERING_FIELDS);
+ if (orderingFields != null) {
+ tableOptions.put(FlinkOptions.ORDERING_FIELDS.key(), orderingFields);
+ }
+
+ // Add partition fields if specified
+ List partitionKeys = cdcSchema.partitionKeys();
+ if (partitionKeys != null && !partitionKeys.isEmpty()) {
+ tableOptions.put(
+ FlinkOptions.PARTITION_PATH_FIELD.key(), String.join(",", partitionKeys));
+ }
+
+ // Create CatalogTable
+ CatalogTable catalogTable =
+ CatalogTable.of(
+ tableSchema,
+ cdcSchema.comment(),
+ partitionKeys != null ? partitionKeys : Collections.emptyList(),
+ tableOptions);
+
+ return new ResolvedCatalogTable(catalogTable, resolvedSchema);
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketAssignOperator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketAssignOperator.java
new file mode 100644
index 00000000000..ed0d61fd9f5
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketAssignOperator.java
@@ -0,0 +1,248 @@
+/*
+ * 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.hudi.sink.bucket;
+
+import org.apache.flink.cdc.common.data.RecordData;
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.common.event.FlushEvent;
+import org.apache.flink.cdc.common.event.SchemaChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.utils.SchemaUtils;
+import org.apache.flink.cdc.connectors.hudi.sink.util.RowDataUtils;
+import org.apache.flink.cdc.connectors.hudi.sink.v2.OperatorIDGenerator;
+import org.apache.flink.cdc.runtime.operators.sink.SchemaEvolutionClient;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway;
+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 org.apache.flink.table.data.RowData;
+
+import org.apache.hudi.common.util.Functions;
+import org.apache.hudi.common.util.hash.BucketIndexUtil;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.sink.bulk.RowDataKeyGen;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.ZoneId;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Operator that assigns bucket indices to events and wraps them for downstream partitioning.
+ *
+ * This operator:
+ *
+ *
+ * - Broadcasts schema events (CreateTableEvent, SchemaChangeEvent, FlushEvent) to all
+ * downstream tasks
+ *
- Calculates bucket for DataChangeEvents and routes to specific task
+ *
- Wraps events in BucketWrapper for downstream partitioning
+ *
+ */
+public class BucketAssignOperator extends AbstractStreamOperator
+ implements OneInputStreamOperator {
+
+ private static final Logger LOG = LoggerFactory.getLogger(BucketAssignOperator.class);
+
+ private final int numBuckets;
+ private final String schemaOperatorUid;
+ private int totalTasksNumber;
+ private int currentTaskNumber;
+
+ /** Function for calculating the task partition to dispatch based on bucket. */
+ private transient Functions.Function3 partitionIndexFunc;
+
+ /** Schema evolution client to query schemas from SchemaOperator coordinator. */
+ private transient SchemaEvolutionClient schemaEvolutionClient;
+
+ /** Cache of schemas per table for bucket calculation. */
+ private final Map schemaCache = new HashMap<>();
+
+ /** RowDataKeyGen cache per table for key and partition extraction. */
+ private final Map keyGenCache = new HashMap<>();
+
+ /** Field getter cache per table - lazily created and invalidated on schema changes. */
+ private final Map> fieldGetterCache = new HashMap<>();
+
+ public BucketAssignOperator(Configuration conf, String schemaOperatorUid) {
+ this.numBuckets = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+ this.schemaOperatorUid = schemaOperatorUid;
+ this.chainingStrategy = ChainingStrategy.ALWAYS;
+ }
+
+ @Override
+ public void setup(
+ StreamTask, ?> containingTask,
+ StreamConfig config,
+ Output> output) {
+ super.setup(containingTask, config, output);
+ TaskOperatorEventGateway toCoordinator =
+ getContainingTask().getEnvironment().getOperatorCoordinatorEventGateway();
+ schemaEvolutionClient =
+ new SchemaEvolutionClient(
+ toCoordinator, new OperatorIDGenerator(schemaOperatorUid).generate());
+ }
+
+ @Override
+ public void open() throws Exception {
+ super.open();
+ this.totalTasksNumber = getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks();
+ this.currentTaskNumber = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask();
+ this.partitionIndexFunc = BucketIndexUtil.getPartitionIndexFunc(totalTasksNumber);
+ LOG.info(
+ "BucketAssignOperator opened with {} buckets and {} tasks",
+ numBuckets,
+ totalTasksNumber);
+ }
+
+ @Override
+ public void processElement(StreamRecord streamRecord) throws Exception {
+ Event event = streamRecord.getValue();
+
+ // Broadcast SchemaChangeEvent (includes CreateTableEvent) to all tasks
+ if (event instanceof SchemaChangeEvent) {
+ SchemaChangeEvent schemaEvent = (SchemaChangeEvent) event;
+ Schema existingSchema = schemaCache.get(schemaEvent.tableId());
+ Schema newSchema = SchemaUtils.applySchemaChangeEvent(existingSchema, schemaEvent);
+ schemaCache.put(schemaEvent.tableId(), newSchema);
+
+ // Clear caches when schema changes
+ keyGenCache.remove(schemaEvent.tableId());
+ fieldGetterCache.remove(schemaEvent.tableId());
+
+ // Broadcast to all tasks
+ for (int i = 0; i < totalTasksNumber; i++) {
+ output.collect(new StreamRecord<>(new BucketWrapper(i, event)));
+ }
+ return;
+ }
+
+ // Broadcast FlushEvent to all tasks wrapped with task metadata
+ if (event instanceof FlushEvent) {
+ FlushEvent flushEvent = (FlushEvent) event;
+ for (int i = 0; i < totalTasksNumber; i++) {
+ output.collect(
+ new StreamRecord<>(
+ new BucketWrapper(
+ i,
+ new BucketWrapperFlushEvent(
+ i,
+ flushEvent.getSourceSubTaskId(),
+ currentTaskNumber,
+ flushEvent.getTableIds(),
+ flushEvent.getSchemaChangeEventType()))));
+ }
+ return;
+ }
+
+ // Calculate bucket for DataChangeEvent and route to specific task
+ if (event instanceof DataChangeEvent) {
+ DataChangeEvent dataEvent = (DataChangeEvent) event;
+ int taskIndex = calculateTaskIndex(dataEvent);
+ output.collect(new StreamRecord<>(new BucketWrapper(taskIndex, event)));
+ return;
+ }
+
+ // Default: broadcast unknown event types to all tasks
+ for (int i = 0; i < totalTasksNumber; i++) {
+ output.collect(new StreamRecord<>(new BucketWrapper(i, event)));
+ }
+ }
+
+ /**
+ * Calculate which task index should handle this event by: 1. Calculating the bucket number (0
+ * to numBuckets-1) based on record key 2. Using partitionIndexFunc to map bucket -> task index
+ * for balanced distribution
+ *
+ * @param event The DataChangeEvent to calculate task index for
+ * @return The task index (0 to parallelism-1) that should handle this event
+ */
+ private int calculateTaskIndex(DataChangeEvent event) {
+ TableId tableId = event.tableId();
+
+ // Get or cache schema, query from SchemaOperator coordinator if not cached
+ Schema schema = schemaCache.get(tableId);
+ if (schema == null) {
+ try {
+ Optional optSchema = schemaEvolutionClient.getLatestEvolvedSchema(tableId);
+ if (optSchema.isPresent()) {
+ schema = optSchema.get();
+ schemaCache.put(tableId, schema);
+ } else {
+ throw new IllegalStateException(
+ "No schema available for table "
+ + tableId
+ + " in bucket assignment. "
+ + "Could not find schema from SchemaOperator coordinator.");
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException(
+ "Failed to retrieve schema for table " + tableId + " from SchemaOperator",
+ e);
+ }
+ }
+
+ // Create final reference for use in lambda
+ final Schema finalSchema = schema;
+
+ // Get or cache primary keys
+ List primaryKeys = finalSchema.primaryKeys();
+
+ if (primaryKeys.isEmpty()) {
+ throw new IllegalStateException(
+ "Cannot calculate bucket: table " + tableId + " has no primary keys");
+ }
+
+ // Get or create RowDataKeyGen for this table
+ RowDataKeyGen keyGen =
+ keyGenCache.computeIfAbsent(tableId, k -> RowDataUtils.createKeyGen(finalSchema));
+
+ // Get or create field getters for this table, lazily cached
+ List fieldGetters =
+ fieldGetterCache.computeIfAbsent(
+ tableId,
+ k -> RowDataUtils.createFieldGetters(finalSchema, ZoneId.systemDefault()));
+
+ // Convert DataChangeEvent to RowData for key extraction
+ RowData rowData = RowDataUtils.convertDataChangeEventToRowData(event, fieldGetters);
+
+ // Use RowDataKeyGen to extract record key and partition path
+ String recordKey = keyGen.getRecordKey(rowData);
+ String partition = keyGen.getPartitionPath(rowData);
+
+ // Calculate bucket using Hudi's logic (0 to numBuckets-1)
+ String tableIndexKeyFields = String.join(",", primaryKeys);
+ int bucketNumber = BucketIdentifier.getBucketId(recordKey, tableIndexKeyFields, numBuckets);
+
+ // partitionIndexFunc is designed for single table, events may come from different tables,
+ // prefix them with tableId e.g. tableId + "_" + partition
+ // Use partition function to map bucket to task index for balanced distribution
+ return partitionIndexFunc.apply(numBuckets, tableId + "_" + partition, bucketNumber);
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketWrapper.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketWrapper.java
new file mode 100644
index 00000000000..794f80a66b9
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketWrapper.java
@@ -0,0 +1,50 @@
+/*
+ * 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.hudi.sink.bucket;
+
+import org.apache.flink.cdc.common.event.Event;
+
+import java.io.Serializable;
+
+/**
+ * Wrapper class that implements Event and associates an event with a target bucket/task index. Used
+ * to enable bucket-based partitioning while allowing schema events to be broadcast.
+ *
+ * By implementing Event, this wrapper can be transparently passed through the operator chain
+ * while maintaining bidirectional communication for FlushSuccessEvent.
+ */
+public class BucketWrapper implements Event, Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private final int bucket;
+ private final Event event;
+
+ public BucketWrapper(int bucket, Event event) {
+ this.bucket = bucket;
+ this.event = event;
+ }
+
+ public int getBucket() {
+ return bucket;
+ }
+
+ public Event getEvent() {
+ return event;
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketWrapperFlushEvent.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketWrapperFlushEvent.java
new file mode 100644
index 00000000000..0aa53996bdb
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketWrapperFlushEvent.java
@@ -0,0 +1,84 @@
+/*
+ * 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.hudi.sink.bucket;
+
+import org.apache.flink.cdc.common.event.FlushEvent;
+import org.apache.flink.cdc.common.event.SchemaChangeEventType;
+import org.apache.flink.cdc.common.event.TableId;
+
+import java.util.List;
+import java.util.Objects;
+
+/** A wrapper class for {@link FlushEvent} to attach bucket id and task metadata. */
+public class BucketWrapperFlushEvent extends FlushEvent {
+
+ private final int bucket;
+ private final int bucketAssignTaskId;
+
+ public BucketWrapperFlushEvent(
+ int bucket,
+ int sourceSubTaskId,
+ int bucketAssignTaskId,
+ List tableIds,
+ SchemaChangeEventType schemaChangeEventType) {
+ super(sourceSubTaskId, tableIds, schemaChangeEventType);
+ this.bucket = bucket;
+ this.bucketAssignTaskId = bucketAssignTaskId;
+ }
+
+ public int getBucket() {
+ return bucket;
+ }
+
+ public int getBucketAssignTaskId() {
+ return bucketAssignTaskId;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ if (!super.equals(o)) {
+ return false;
+ }
+ BucketWrapperFlushEvent that = (BucketWrapperFlushEvent) o;
+ return bucket == that.bucket
+ && bucketAssignTaskId == that.bucketAssignTaskId
+ && getSourceSubTaskId() == that.getSourceSubTaskId();
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(super.hashCode(), bucket, bucketAssignTaskId);
+ }
+
+ @Override
+ public String toString() {
+ return "BucketWrapperFlushEvent{subTaskId="
+ + getSourceSubTaskId()
+ + ", bucketAssignTaskId="
+ + bucketAssignTaskId
+ + ", bucket="
+ + bucket
+ + '}';
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/FlushEventAlignmentOperator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/FlushEventAlignmentOperator.java
new file mode 100644
index 00000000000..615959916b8
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/FlushEventAlignmentOperator.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.connectors.hudi.sink.bucket;
+
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.common.event.FlushEvent;
+import org.apache.flink.cdc.runtime.operators.schema.regular.SchemaOperator;
+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.runtime.streamrecord.StreamRecord;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/** Align {@link FlushEvent}s broadcasted by {@link BucketAssignOperator}. */
+public class FlushEventAlignmentOperator extends AbstractStreamOperator
+ implements OneInputStreamOperator {
+
+ private transient int totalTasksNumber;
+
+ /**
+ * Key: subtask id of {@link SchemaOperator}, Value: subtask ids of {@link
+ * BucketAssignOperator}.
+ */
+ private transient Map> sourceTaskIdToAssignBucketSubTaskIds;
+
+ private transient int currentSubTaskId;
+
+ public FlushEventAlignmentOperator() {
+ // It's necessary to avoid unpredictable outcomes of Event shuffling.
+ this.chainingStrategy = ChainingStrategy.ALWAYS;
+ }
+
+ @Override
+ public void open() throws Exception {
+ super.open();
+ this.totalTasksNumber = getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks();
+ this.currentSubTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask();
+ sourceTaskIdToAssignBucketSubTaskIds = new HashMap<>();
+ }
+
+ @Override
+ public void processElement(StreamRecord streamRecord) {
+ BucketWrapper wrapper = streamRecord.getValue();
+ Event event = wrapper.getEvent();
+
+ if (event instanceof BucketWrapperFlushEvent) {
+ BucketWrapperFlushEvent bucketWrapperFlushEvent = (BucketWrapperFlushEvent) event;
+ int sourceSubTaskId = bucketWrapperFlushEvent.getSourceSubTaskId();
+ Set subTaskIds =
+ sourceTaskIdToAssignBucketSubTaskIds.getOrDefault(
+ sourceSubTaskId, new HashSet<>());
+ int subtaskId = bucketWrapperFlushEvent.getBucketAssignTaskId();
+ subTaskIds.add(subtaskId);
+ if (subTaskIds.size() == totalTasksNumber) {
+ LOG.info("{} send FlushEvent of {}", currentSubTaskId, sourceSubTaskId);
+ output.collect(
+ new StreamRecord<>(
+ new BucketWrapper(
+ wrapper.getBucket(),
+ new FlushEvent(
+ sourceSubTaskId,
+ bucketWrapperFlushEvent.getTableIds(),
+ bucketWrapperFlushEvent
+ .getSchemaChangeEventType()))));
+ sourceTaskIdToAssignBucketSubTaskIds.remove(sourceSubTaskId);
+ } else {
+ LOG.info(
+ "{} collect FlushEvent of {} with subtask {}",
+ currentSubTaskId,
+ sourceSubTaskId,
+ subtaskId);
+ sourceTaskIdToAssignBucketSubTaskIds.put(sourceSubTaskId, subTaskIds);
+ }
+ } else {
+ output.collect(streamRecord);
+ }
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/coordinator/MultiTableStreamWriteOperatorCoordinator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/coordinator/MultiTableStreamWriteOperatorCoordinator.java
new file mode 100644
index 00000000000..160ba07996f
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/coordinator/MultiTableStreamWriteOperatorCoordinator.java
@@ -0,0 +1,1005 @@
+/*
+ * 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.hudi.sink.coordinator;
+
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.connectors.hudi.sink.event.EnhancedWriteMetadataEvent;
+import org.apache.flink.cdc.connectors.hudi.sink.event.TableAwareCorrespondent.CreateTableRequest;
+import org.apache.flink.cdc.connectors.hudi.sink.event.TableAwareCorrespondent.MultiTableInstantTimeRequest;
+import org.apache.flink.cdc.connectors.hudi.sink.event.TableAwareCorrespondent.SchemaChangeRequest;
+import org.apache.flink.cdc.connectors.hudi.sink.event.TableAwareCorrespondent.SchemaChangeResponse;
+import org.apache.flink.cdc.connectors.hudi.sink.util.RowDataUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.CommitUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.SerializationUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.configuration.OptionsResolver;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
+import org.apache.hudi.sink.event.Correspondent;
+import org.apache.hudi.sink.event.WriteMetadataEvent;
+import org.apache.hudi.sink.utils.CoordinationResponseSerDe;
+import org.apache.hudi.sink.utils.EventBuffers;
+import org.apache.hudi.sink.utils.ExplicitClassloaderThreadFactory;
+import org.apache.hudi.sink.utils.NonThrownExecutor;
+import org.apache.hudi.util.AvroSchemaConverter;
+import org.apache.hudi.util.ClusteringUtil;
+import org.apache.hudi.util.CompactionUtil;
+import org.apache.hudi.util.FlinkWriteClients;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.configuration.FlinkOptions.COMPACTION_DELTA_COMMITS;
+
+/**
+ * A custom OperatorCoordinator that manages Hudi writes for multiple tables.
+ *
+ * This coordinator extends the default {@link StreamWriteOperatorCoordinator}. The parent class
+ * is designed for a single destination table, so its core logic (e.g., for commits and
+ * checkpointing) cannot be reused directly for a multi-table sink.
+ *
+ *
Therefore, this implementation overrides the essential lifecycle methods to manage a
+ * collection of per-table resources. It dynamically creates and manages a dedicated {@link
+ * HoodieFlinkWriteClient}, {@link EventBuffers}, and timeline for each table that appears in the
+ * upstream CDC data.
+ */
+public class MultiTableStreamWriteOperatorCoordinator extends StreamWriteOperatorCoordinator {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(MultiTableStreamWriteOperatorCoordinator.class);
+
+ /**
+ * Encapsulates all state and resources for a single table. This simplifies management by
+ * grouping related objects, making the coordinator logic cleaner and less prone to errors.
+ */
+ private static class TableContext implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ final transient HoodieFlinkWriteClient> writeClient;
+ final EventBuffers eventBuffers;
+ final TableState tableState;
+ final String tablePath;
+
+ TableContext(
+ HoodieFlinkWriteClient> writeClient,
+ EventBuffers eventBuffers,
+ TableState tableState,
+ String tablePath) {
+ this.writeClient = writeClient;
+ this.eventBuffers = eventBuffers;
+ this.tableState = tableState;
+ this.tablePath = tablePath;
+ }
+
+ void close() {
+ if (writeClient != null) {
+ try {
+ writeClient.close();
+ } catch (Exception e) {
+ LOG.error("Error closing write client for table path: {}", tablePath, e);
+ }
+ }
+ }
+ }
+
+ /** A container for table-specific configuration and state. */
+ private static class TableState implements Serializable {
+ private static final long serialVersionUID = 1L;
+ final String commitAction;
+ final boolean isOverwrite;
+ final WriteOperationType operationType;
+ final boolean scheduleCompaction;
+ final boolean scheduleClustering;
+ final boolean isDeltaTimeCompaction;
+
+ // Event-driven compaction tracking - tracks actual write activity
+ long commitsSinceLastCompaction = 0;
+ // For MOR tables, track log file growth
+ long totalLogBytesWritten = 0;
+
+ final int commitsThreshold;
+
+ TableState(Configuration conf) {
+ this.operationType =
+ WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION));
+ this.commitAction =
+ CommitUtils.getCommitActionType(
+ this.operationType,
+ HoodieTableType.valueOf(
+ conf.getString(FlinkOptions.TABLE_TYPE).toUpperCase()));
+ this.isOverwrite = WriteOperationType.isOverwrite(this.operationType);
+ this.scheduleCompaction = OptionsResolver.needsScheduleCompaction(conf);
+ this.scheduleClustering = OptionsResolver.needsScheduleClustering(conf);
+ this.isDeltaTimeCompaction = OptionsResolver.isDeltaTimeCompaction(conf);
+ this.commitsThreshold = conf.get(COMPACTION_DELTA_COMMITS);
+ }
+
+ /**
+ * Updates compaction metrics based on write statuses. Skips empty commits where no actual
+ * data was written.
+ *
+ * @param writeStatuses The write statuses from the latest commit
+ * @return true if this commit had actual writes, false if it was empty
+ */
+ boolean updateCompactionMetrics(List writeStatuses) {
+ if (writeStatuses == null || writeStatuses.isEmpty()) {
+ LOG.debug("No write statuses - skipping compaction metric update");
+ return false;
+ }
+
+ // Check if any actual writes occurred (skip empty commits)
+ long totalWrites =
+ writeStatuses.stream()
+ .map(WriteStatus::getStat)
+ .filter(stat -> stat != null)
+ .mapToLong(HoodieWriteStat::getNumWrites)
+ .sum();
+
+ if (totalWrites == 0) {
+ LOG.debug(
+ "Empty commit detected (numWrites=0) - skipping compaction metric update");
+ return false;
+ }
+
+ // Track log file bytes written (for MOR tables)
+ long bytesWritten =
+ writeStatuses.stream()
+ .map(WriteStatus::getStat)
+ .filter(stat -> stat != null)
+ .mapToLong(HoodieWriteStat::getTotalWriteBytes)
+ .sum();
+
+ commitsSinceLastCompaction++;
+ totalLogBytesWritten += bytesWritten;
+
+ LOG.debug(
+ "Updated compaction metrics: commits={}, bytes={}",
+ commitsSinceLastCompaction,
+ totalLogBytesWritten);
+ return true;
+ }
+
+ /** Resets compaction metrics after compaction is scheduled. */
+ void resetCompactionMetrics() {
+ commitsSinceLastCompaction = 0;
+ totalLogBytesWritten = 0;
+ }
+
+ /**
+ * Determines if compaction should be triggered based on write activity. Only triggers for
+ * MOR tables with actual data writes.
+ *
+ * @return true if compaction should be scheduled
+ */
+ boolean shouldTriggerCompaction() {
+ // Only trigger for MOR tables (DELTA_COMMIT means log files)
+ if (!commitAction.equals(HoodieTimeline.DELTA_COMMIT_ACTION)) {
+ return false;
+ }
+
+ return commitsSinceLastCompaction >= commitsThreshold;
+ }
+ }
+
+ /** The base Flink configuration. */
+ private final Configuration baseConfig;
+
+ /**
+ * A single, unified map holding the context for each managed table. The key is the {@link
+ * TableId}, providing a centralized place for all per-table resources.
+ */
+ private final Map tableContexts = new ConcurrentHashMap<>();
+
+ /** A reverse lookup map from table path to TableId for efficient event routing. */
+ private final Map pathToTableId = new ConcurrentHashMap<>();
+
+ /** Cache of schemas per table for config creation. */
+ private final Map tableSchemas = new ConcurrentHashMap<>();
+
+ /**
+ * Gateways for sending events to sub-tasks. This field is necessary because the parent's
+ * `gateways` array is private and not initialized if we don't call super.start().
+ */
+ private transient SubtaskGateway[] gateways;
+
+ /** A single-thread executor to handle instant time requests, mimicking the parent behavior. */
+ private transient NonThrownExecutor instantRequestExecutor;
+
+ public MultiTableStreamWriteOperatorCoordinator(Configuration conf, Context context) {
+ super(conf, context);
+ this.baseConfig = conf;
+ LOG.info(
+ "MultiTableStreamWriteOperatorCoordinator initialized for operator: {} with config: {}",
+ context.getOperatorId(),
+ baseConfig);
+ }
+
+ @Override
+ public void start() throws Exception {
+ // Hadoop's FileSystem API uses Java's ServiceLoader to find implementations for
+ // URI schemes (like 'file://'). The ServiceLoader relies on the thread's context
+ // classloader. The parent class sets this, but our overridden start() method must
+ // do so as well to ensure file system implementations can be found.
+ Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
+
+ // Initialize the executor service, which is a protected field in the parent class.
+ // This logic is borrowed from the parent's start() method as we cannot call super.start().
+ this.executor =
+ NonThrownExecutor.builder(LOG)
+ .threadFactory(
+ new ExplicitClassloaderThreadFactory(
+ "multi-table-coord-event-handler",
+ context.getUserCodeClassloader()))
+ .exceptionHook(
+ (errMsg, t) -> this.context.failJob(new HoodieException(errMsg, t)))
+ .waitForTasksFinish(true)
+ .build();
+
+ // Executor for handling instant requests.
+ this.instantRequestExecutor =
+ NonThrownExecutor.builder(LOG)
+ .threadFactory(
+ new ExplicitClassloaderThreadFactory(
+ "multi-table-instant-request",
+ context.getUserCodeClassloader()))
+ .exceptionHook(
+ (errMsg, t) -> this.context.failJob(new HoodieException(errMsg, t)))
+ .build();
+
+ // Initialize the gateways array to avoid NullPointerException when subtasks are ready.
+ this.gateways = new SubtaskGateway[context.currentParallelism()];
+
+ // Re-initialize transient fields after deserialization from a Flink checkpoint.
+ // When the coordinator is restored, the `tableContexts` map is deserialized, but all
+ // `writeClient` fields within it will be null because they are transient.
+ for (Map.Entry entry : tableContexts.entrySet()) {
+ TableId tableId = entry.getKey();
+ TableContext oldContext = entry.getValue();
+
+ try {
+ Configuration tableConfig = createTableSpecificConfig(tableId);
+ // Ensure the table's filesystem structure exists before creating a client.
+ StreamerUtil.initTableIfNotExists(tableConfig);
+ HoodieFlinkWriteClient> writeClient =
+ FlinkWriteClients.createWriteClient(tableConfig);
+
+ // Replace the old context (with a null client) with a new one containing the live
+ // client.
+ tableContexts.put(
+ tableId,
+ new TableContext(
+ writeClient,
+ oldContext.eventBuffers,
+ oldContext.tableState,
+ oldContext.tablePath));
+ LOG.info(
+ "Successfully re-initialized write client for recovered table: {}",
+ tableId);
+ } catch (Exception e) {
+ LOG.error(
+ "Failed to re-initialize write client for recovered table: {}", tableId, e);
+ context.failJob(e);
+ return; // Exit if initialization fails for any table
+ }
+ }
+ }
+
+ @Override
+ public CompletableFuture handleCoordinationRequest(
+ CoordinationRequest request) {
+ if (request instanceof MultiTableInstantTimeRequest) {
+ CompletableFuture future = new CompletableFuture<>();
+ instantRequestExecutor.execute(
+ () -> {
+ MultiTableInstantTimeRequest instantRequest =
+ (MultiTableInstantTimeRequest) request;
+ TableId tableId = instantRequest.getTableId();
+ long checkpointId = instantRequest.getCheckpointId();
+
+ TableContext tableContext = tableContexts.get(tableId);
+ if (tableContext == null) {
+ String errorMsg =
+ String.format(
+ "Received instant request for unknown table %s. The sink function should send a CreateTableEvent first.",
+ tableId);
+ LOG.error(errorMsg);
+ future.completeExceptionally(new IllegalStateException(errorMsg));
+ return;
+ }
+
+ Pair instantAndBuffer =
+ tableContext.eventBuffers.getInstantAndEventBuffer(checkpointId);
+ final String instantTime;
+
+ if (instantAndBuffer == null) {
+ // No instant yet for this checkpoint, create a new one.
+ instantTime = startInstantForTable(tableContext);
+ tableContext.eventBuffers.initNewEventBuffer(
+ checkpointId, instantTime, context.currentParallelism());
+ LOG.info(
+ "Created new instant [{}] for table [{}] at checkpoint [{}].",
+ instantTime,
+ tableId,
+ checkpointId);
+ } else {
+ // Instant already exists for this checkpoint, reuse it.
+ instantTime = instantAndBuffer.getLeft();
+ LOG.info(
+ "Reusing instant [{}] for table [{}] at checkpoint [{}].",
+ instantTime,
+ tableId,
+ checkpointId);
+ }
+ future.complete(
+ CoordinationResponseSerDe.wrap(
+ Correspondent.InstantTimeResponse.getInstance(
+ instantTime)));
+ },
+ "Handling instant time request for checkpoint %d",
+ ((MultiTableInstantTimeRequest) request).getCheckpointId());
+ return future;
+ } else if (request instanceof CreateTableRequest) {
+ CompletableFuture future = new CompletableFuture<>();
+ executor.execute(
+ () -> {
+ CreateTableRequest createTableRequest = (CreateTableRequest) request;
+ boolean isSuccess = handleCreateTableRequest(createTableRequest);
+ future.complete(
+ CoordinationResponseSerDe.wrap(
+ SchemaChangeResponse.of(
+ createTableRequest.getTableId(), isSuccess)));
+ },
+ "Handling create table request: ",
+ request);
+ return future;
+ } else if (request instanceof SchemaChangeRequest) {
+ CompletableFuture future = new CompletableFuture<>();
+ executor.execute(
+ () -> {
+ SchemaChangeRequest createTableRequest = (SchemaChangeRequest) request;
+ boolean isSuccess = handleSchemaChangeRequest(createTableRequest);
+ future.complete(
+ CoordinationResponseSerDe.wrap(
+ SchemaChangeResponse.of(
+ createTableRequest.getTableId(), isSuccess)));
+ },
+ "Handling create schema change request: ",
+ request);
+ return future;
+ } else {
+ LOG.warn("Received an unknown coordination request: {}", request.getClass().getName());
+ return super.handleCoordinationRequest(request);
+ }
+ }
+
+ private String startInstantForTable(TableContext tableContext) {
+ HoodieFlinkWriteClient> writeClient = tableContext.writeClient;
+ TableState tableState = tableContext.tableState;
+ HoodieTableMetaClient metaClient = writeClient.getHoodieTable().getMetaClient();
+
+ metaClient.reloadActiveTimeline();
+ final String newInstant = writeClient.startCommit(tableState.commitAction, metaClient);
+ metaClient
+ .getActiveTimeline()
+ .transitionRequestedToInflight(tableState.commitAction, newInstant);
+ return newInstant;
+ }
+
+ @Override
+ public void handleEventFromOperator(
+ int subtask, int attemptNumber, OperatorEvent operatorEvent) {
+ executor.execute(
+ () -> {
+ if (operatorEvent instanceof EnhancedWriteMetadataEvent) {
+ handleEnhancedWriteMetadataEvent(
+ (EnhancedWriteMetadataEvent) operatorEvent);
+ } else {
+ LOG.warn(
+ "Received an unhandled or non-enhanced OperatorEvent: {}",
+ operatorEvent);
+ }
+ },
+ "handling operator event %s",
+ operatorEvent);
+ }
+
+ private boolean handleCreateTableRequest(CreateTableRequest createTableRequest) {
+ TableId tableId = createTableRequest.getTableId();
+ // Store the schema for this table
+ tableSchemas.put(tableId, createTableRequest.getSchema());
+ LOG.info(
+ "Cached schema for table {}: {} columns",
+ tableId,
+ createTableRequest.getSchema().getColumnCount());
+
+ TableContext tableContext =
+ tableContexts.computeIfAbsent(
+ tableId,
+ tId -> {
+ LOG.info("New table detected: {}. Initializing Hudi resources.", tId);
+ try {
+ Configuration tableConfig = createTableSpecificConfig(tId);
+ String tablePath = tableConfig.getString(FlinkOptions.PATH);
+ pathToTableId.put(tablePath, tId);
+
+ // Create physical directory for Hudi table before initializing
+ createHudiTablePath(tableConfig);
+
+ StreamerUtil.initTableIfNotExists(tableConfig);
+ HoodieFlinkWriteClient> writeClient =
+ FlinkWriteClients.createWriteClient(tableConfig);
+ TableState tableState = new TableState(tableConfig);
+ EventBuffers eventBuffers = EventBuffers.getInstance(tableConfig);
+
+ LOG.info(
+ "Successfully initialized resources for table: {} at path: {}",
+ tId,
+ tablePath);
+ return new TableContext(
+ writeClient, eventBuffers, tableState, tablePath);
+ } catch (Exception e) {
+ LOG.error(
+ "Failed to initialize Hudi table resources for: {}",
+ tId,
+ e);
+ context.failJob(
+ new HoodieException(
+ "Failed to initialize Hudi writer for table " + tId,
+ e));
+ return null;
+ }
+ });
+ return tableContext != null;
+ }
+
+ /**
+ * Handles schema change events from the sink functions. Updates the cached schema and recreates
+ * the write client to ensure it uses the new schema.
+ *
+ * @param request The schema change request containing the table ID and new schema
+ */
+ private boolean handleSchemaChangeRequest(SchemaChangeRequest request) {
+ TableId tableId = request.getTableId();
+ Schema newSchema = request.getSchema();
+
+ LOG.info(
+ "Received schema change event for table {}: {} columns",
+ tableId,
+ newSchema.getColumnCount());
+
+ Schema oldSchema = tableSchemas.get(tableId);
+ if (Objects.equals(oldSchema, newSchema)) {
+ LOG.warn("Schema change already applied, tableId: {}, schema: {}.", tableId, newSchema);
+ return true;
+ }
+ // Update the cached schema
+ tableSchemas.put(tableId, newSchema);
+ LOG.info(
+ "Updated coordinator's schema cache for table: {}, new schema: {}",
+ tableId,
+ newSchema);
+
+ // Get the existing table context
+ TableContext oldContext = tableContexts.get(tableId);
+ if (oldContext == null) {
+ LOG.warn(
+ "Received schema change for unknown table: {}. Skipping write client update.",
+ tableId);
+ return true;
+ }
+
+ try {
+ // Close the old write client
+ if (oldContext.writeClient != null) {
+ oldContext.writeClient.close();
+ LOG.info("Closed old write client for table: {}", tableId);
+ }
+
+ // Create new configuration with updated schema
+ Configuration tableConfig = createTableSpecificConfig(tableId);
+
+ // Create new write client with updated schema
+ HoodieFlinkWriteClient> newWriteClient =
+ FlinkWriteClients.createWriteClient(tableConfig);
+ LOG.info("Created new write client with updated schema for table: {}", tableId);
+
+ // Update the table context with the new write client
+ // Keep the same eventBuffers, tableState, and tablePath
+ TableContext newContext =
+ new TableContext(
+ newWriteClient,
+ oldContext.eventBuffers,
+ oldContext.tableState,
+ oldContext.tablePath);
+ tableContexts.put(tableId, newContext);
+
+ LOG.info("Successfully updated write client for table {} after schema change", tableId);
+ return true;
+ } catch (Exception e) {
+ LOG.error("Failed to update write client for table {} after schema change", tableId, e);
+ context.failJob(
+ new HoodieException(
+ "Failed to update write client for table "
+ + tableId
+ + " after schema change",
+ e));
+ return false;
+ }
+ }
+
+ private void handleEnhancedWriteMetadataEvent(EnhancedWriteMetadataEvent enhancedEvent) {
+ String tablePath = enhancedEvent.getTablePath();
+ WriteMetadataEvent event = enhancedEvent.getOriginalEvent();
+ TableId tableId = pathToTableId.get(tablePath);
+
+ if (tableId == null) {
+ LOG.error("No tableId found for path: {}. Cannot process event.", tablePath);
+ context.failJob(new IllegalStateException("No tableId found for path: " + tablePath));
+ return;
+ }
+
+ TableContext tableContext = tableContexts.get(tableId);
+ if (tableContext == null) {
+ LOG.error("FATAL: Inconsistent state. No TableContext for table: {}.", tableId);
+ context.failJob(new IllegalStateException("No TableContext for table " + tableId));
+ return;
+ }
+
+ LOG.info(
+ "Fetching instant, but got null pair for : {}",
+ tableContext.eventBuffers.getInstantAndEventBuffer(event.getCheckpointId()));
+
+ // The instant should have been created by handleCoordinationRequest
+ if (tableContext.eventBuffers.getInstantAndEventBuffer(event.getCheckpointId()) == null) {
+ LOG.error(
+ "FATAL: Received WriteMetadataEvent for table {} at checkpoint {} before an instant was created. "
+ + "This should not happen. The sink function must request an instant before sending data.",
+ tableId,
+ event.getCheckpointId());
+ context.failJob(
+ new IllegalStateException(
+ "Received data for table "
+ + tableId
+ + " at checkpoint "
+ + event.getCheckpointId()
+ + " without a valid Hudi instant."));
+ return;
+ }
+
+ LOG.debug(
+ "Buffering event for table: {}, checkpoint: {}", tableId, event.getCheckpointId());
+ tableContext.eventBuffers.addEventToBuffer(event);
+ }
+
+ @Override
+ public void checkpointCoordinator(long checkpointId, CompletableFuture result) {
+ executor.execute(
+ () -> {
+ try {
+ Map>> allStates =
+ new HashMap<>();
+ tableContexts.forEach(
+ (tableId, tableContext) -> {
+ allStates.put(
+ tableId,
+ tableContext.eventBuffers.getAllCompletedEvents());
+ });
+
+ // Create a wrapper that includes both event buffers AND schemas
+ Map checkpointState = new HashMap<>();
+ checkpointState.put("eventBuffers", allStates);
+ checkpointState.put("schemas", new HashMap<>(tableSchemas));
+
+ byte[] serializedState = SerializationUtils.serialize(checkpointState);
+ result.complete(serializedState);
+ LOG.info(
+ "Successfully checkpointed coordinator state with {} schemas for checkpoint {}",
+ tableSchemas.size(),
+ checkpointId);
+ } catch (Throwable t) {
+ LOG.error(
+ "Failed to checkpoint coordinator state for checkpoint {}",
+ checkpointId,
+ t);
+ result.completeExceptionally(t);
+ }
+ },
+ "checkpointing coordinator state %d",
+ checkpointId);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void resetToCheckpoint(long checkpointId, byte[] checkpointData) {
+ if (checkpointData == null) {
+ LOG.info("No coordinator checkpoint data to restore for checkpoint {}.", checkpointId);
+ return;
+ }
+ try {
+ Map checkpointState = SerializationUtils.deserialize(checkpointData);
+ Map>> allStates =
+ (Map>>)
+ checkpointState.get("eventBuffers");
+ Map restoredSchemas =
+ (Map) checkpointState.get("schemas");
+
+ // Restore schemas
+ if (restoredSchemas != null && !restoredSchemas.isEmpty()) {
+ tableSchemas.clear();
+ tableSchemas.putAll(restoredSchemas);
+ LOG.info(
+ "Restored {} schemas from checkpoint: {}",
+ tableSchemas.size(),
+ tableSchemas.keySet());
+ }
+
+ allStates.forEach(
+ (tableId, completedEvents) -> {
+ // Lazily create table context if it doesn't exist.
+ // The actual write client is initialized in start().
+ tableContexts.computeIfAbsent(
+ tableId,
+ tId -> {
+ Configuration tableConfig = createTableSpecificConfig(tId);
+ String tablePath = tableConfig.getString(FlinkOptions.PATH);
+ pathToTableId.put(tablePath, tId);
+ TableState tableState = new TableState(tableConfig);
+ EventBuffers eventBuffers =
+ EventBuffers.getInstance(tableConfig);
+ return new TableContext(
+ null, eventBuffers, tableState, tablePath);
+ });
+ TableContext tableContext = tableContexts.get(tableId);
+ tableContext.eventBuffers.addEventsToBuffer(completedEvents);
+ });
+ LOG.info("Successfully restored coordinator state from checkpoint {}", checkpointId);
+ } catch (Throwable t) {
+ LOG.error("Failed to restore coordinator state from checkpoint {}", checkpointId, t);
+ context.failJob(new RuntimeException("Failed to restore coordinator state", t));
+ }
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) {
+ executor.execute(
+ () -> {
+ LOG.info(
+ "Checkpoint {} completed. Committing instants for all managed tables.",
+ checkpointId);
+ for (Map.Entry entry : tableContexts.entrySet()) {
+ TableId tableId = entry.getKey();
+ TableContext tableContext = entry.getValue();
+
+ tableContext
+ .eventBuffers
+ .getEventBufferStream()
+ .filter(e -> e.getKey() < checkpointId)
+ .forEach(
+ bufferEntry -> {
+ long ckpId = bufferEntry.getKey();
+ String instant = bufferEntry.getValue().getLeft();
+ WriteMetadataEvent[] events =
+ bufferEntry.getValue().getRight();
+ try {
+ commitInstantForTable(
+ tableId,
+ tableContext,
+ ckpId,
+ instant,
+ events);
+ } catch (Exception e) {
+ LOG.error(
+ "Exception while committing instant {} for table {}",
+ instant,
+ tableId,
+ e);
+ MultiTableStreamWriteOperatorCoordinator.this
+ .context.failJob(e);
+ }
+ });
+ }
+ },
+ "committing instants for checkpoint %d",
+ checkpointId);
+ }
+
+ private void commitInstantForTable(
+ TableId tableId,
+ TableContext tableContext,
+ long checkpointId,
+ String instant,
+ WriteMetadataEvent[] eventBuffer) {
+ final HoodieTimeline completedTimeline =
+ tableContext
+ .writeClient
+ .getHoodieTable()
+ .getMetaClient()
+ .getActiveTimeline()
+ .filterCompletedInstants();
+ if (completedTimeline.containsInstant(instant)) {
+ LOG.info(
+ "Instant {} already committed, table {}, checkpoint id: {}.",
+ instant,
+ tableId,
+ checkpointId);
+ return;
+ }
+ if (Arrays.stream(eventBuffer).allMatch(Objects::isNull)) {
+ LOG.info("No events for instant {}, table {}. Resetting buffer.", instant, tableId);
+ tableContext.eventBuffers.reset(checkpointId);
+ // Even with no events, we must clean up the inflight instant.
+ // A simple rollback handles this.
+ tableContext.writeClient.rollback(instant);
+ return;
+ }
+
+ List writeStatuses =
+ Arrays.stream(eventBuffer)
+ .filter(Objects::nonNull)
+ .map(WriteMetadataEvent::getWriteStatuses)
+ .flatMap(Collection::stream)
+ .collect(Collectors.toList());
+
+ if (writeStatuses.isEmpty() && !OptionsResolver.allowCommitOnEmptyBatch(baseConfig)) {
+ LOG.info(
+ "No data written for instant {}, table {}. Aborting commit and rolling back.",
+ instant,
+ tableId);
+ tableContext.eventBuffers.reset(checkpointId);
+ tableContext.writeClient.rollback(instant);
+ return;
+ }
+
+ doCommit(tableId, tableContext, checkpointId, instant, writeStatuses);
+ }
+
+ @SuppressWarnings("unchecked")
+ private void doCommit(
+ TableId tableId,
+ TableContext tableContext,
+ long checkpointId,
+ String instant,
+ List writeStatuses) {
+
+ TableState state = tableContext.tableState;
+ final Map> partitionToReplacedFileIds =
+ state.isOverwrite
+ ? tableContext.writeClient.getPartitionToReplacedFileIds(
+ state.operationType, writeStatuses)
+ : Collections.emptyMap();
+
+ HashMap checkpointCommitMetadata = new HashMap<>();
+ StreamerUtil.addFlinkCheckpointIdIntoMetaData(
+ baseConfig, checkpointCommitMetadata, checkpointId);
+
+ boolean success =
+ tableContext.writeClient.commit(
+ instant,
+ writeStatuses,
+ Option.of(checkpointCommitMetadata),
+ state.commitAction,
+ partitionToReplacedFileIds);
+
+ if (success) {
+ tableContext.eventBuffers.reset(checkpointId);
+ LOG.info("Successfully committed instant [{}] for table [{}]", instant, tableId);
+
+ // Update compaction metrics based on actual write activity
+ boolean hasWrites = tableContext.tableState.updateCompactionMetrics(writeStatuses);
+
+ // Event-driven table services scheduling - only if there were actual writes
+ if (hasWrites) {
+ scheduleTableServicesIfNeeded(tableId, tableContext);
+ } else {
+ LOG.debug(
+ "Skipping table services scheduling for table [{}] - empty commit",
+ tableId);
+ }
+ } else {
+ LOG.error("Failed to commit instant [{}] for table [{}]", instant, tableId);
+ MultiTableStreamWriteOperatorCoordinator.this.context.failJob(
+ new HoodieException(
+ String.format(
+ "Commit failed for instant %s, table %s", instant, tableId)));
+ }
+ }
+
+ /**
+ * Event-driven table services scheduling. Only schedules compaction/clustering when certain
+ * thresholds are met based on write metrics.
+ *
+ * @param tableId The table identifier
+ * @param tableContext The table's context containing write client and state
+ */
+ private void scheduleTableServicesIfNeeded(TableId tableId, TableContext tableContext) {
+ TableState state = tableContext.tableState;
+
+ // Event-driven compaction scheduling
+ if (state.scheduleCompaction && state.shouldTriggerCompaction()) {
+ try {
+ LOG.info(
+ "Triggering compaction for table [{}] - threshold met: commits={}/{}, bytes={} MB",
+ tableId,
+ state.commitsSinceLastCompaction,
+ state.commitsThreshold,
+ state.totalLogBytesWritten / (1024 * 1024));
+
+ CompactionUtil.scheduleCompaction(
+ tableContext.writeClient,
+ state.isDeltaTimeCompaction,
+ true); // committed = true since we just committed
+
+ // Reset metrics after scheduling
+ state.resetCompactionMetrics();
+
+ LOG.info("Successfully scheduled compaction for table [{}]", tableId);
+ } catch (Exception e) {
+ LOG.error("Failed to schedule compaction for table [{}]", tableId, e);
+ // Don't fail the job, just log the error
+ }
+ } else if (state.scheduleCompaction) {
+ LOG.debug(
+ "Compaction not triggered for table [{}] - commits={}/{}, bytes={} MB",
+ tableId,
+ state.commitsSinceLastCompaction,
+ state.commitsThreshold,
+ state.totalLogBytesWritten / (1024 * 1024));
+ }
+
+ // Clustering can remain on every commit or use similar metrics
+ if (state.scheduleClustering) {
+ try {
+ Configuration tableConfig = createTableSpecificConfig(tableId);
+ ClusteringUtil.scheduleClustering(
+ tableConfig,
+ tableContext.writeClient,
+ true); // committed = true since we just committed
+ LOG.info("Scheduled clustering for table [{}]", tableId);
+ } catch (Exception e) {
+ LOG.error("Failed to schedule clustering for table [{}]", tableId, e);
+ // Don't fail the job, just log the error
+ }
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ if (instantRequestExecutor != null) {
+ instantRequestExecutor.close();
+ }
+ tableContexts.values().forEach(TableContext::close);
+ tableContexts.clear();
+ pathToTableId.clear();
+ super.close();
+ LOG.info("MultiTableStreamWriteOperatorCoordinator closed.");
+ }
+
+ @Override
+ public void subtaskReady(int i, SubtaskGateway subtaskGateway) {
+ // Since the parent's `gateways` field is private, we must manage our own.
+ if (this.gateways == null) {
+ this.gateways = new SubtaskGateway[context.currentParallelism()];
+ }
+ this.gateways[i] = subtaskGateway;
+ }
+
+ // --- Helper Methods ---
+
+ /**
+ * Creates the physical directory for a Hudi table if it doesn't exist. This must be done on the
+ * coordinator side to avoid race conditions when multiple task managers try to create the same
+ * directory simultaneously.
+ *
+ * @param config The table-specific configuration containing the path
+ * @throws IOException if directory creation fails
+ */
+ private static void createHudiTablePath(Configuration config) throws IOException {
+ String tablePath = config.get(FlinkOptions.PATH);
+ Path path = Paths.get(tablePath);
+ if (!Files.exists(path)) {
+ Files.createDirectories(path);
+ LOG.info("Created physical directory for Hudi table at: {}", tablePath);
+ } else {
+ LOG.debug("Hudi table directory already exists at: {}", tablePath);
+ }
+ }
+
+ private Configuration createTableSpecificConfig(TableId tableId) {
+ Configuration tableConfig = new Configuration(baseConfig);
+ String rootPath = baseConfig.getString(FlinkOptions.PATH);
+ String tablePath =
+ String.format(
+ "%s/%s/%s", rootPath, tableId.getSchemaName(), tableId.getTableName());
+ tableConfig.setString(FlinkOptions.PATH, tablePath);
+ tableConfig.setString(FlinkOptions.TABLE_NAME, tableId.getTableName());
+ tableConfig.setString(FlinkOptions.DATABASE_NAME, tableId.getSchemaName());
+
+ // Set the table-specific schema from the cached schemas
+ Schema cdcSchema = tableSchemas.get(tableId);
+ if (cdcSchema != null) {
+ RowType rowType = RowDataUtils.toRowType(cdcSchema);
+ String tableAvroSchema = AvroSchemaConverter.convertToSchema(rowType).toString();
+ tableConfig.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, tableAvroSchema);
+ LOG.info(
+ "Set schema for table {} in coordinator config: {} business fields",
+ tableId,
+ rowType.getFieldCount());
+ } else {
+ LOG.warn(
+ "No schema found in cache for table {}. WriteClient may use incorrect schema!",
+ tableId);
+ }
+
+ return tableConfig;
+ }
+
+ /** Provider for {@link MultiTableStreamWriteOperatorCoordinator}. */
+ public static class Provider implements OperatorCoordinator.Provider {
+ private final OperatorID operatorId;
+ private final Configuration conf;
+
+ public Provider(OperatorID operatorId, Configuration conf) {
+ this.operatorId = operatorId;
+ this.conf = conf;
+ }
+
+ @Override
+ public OperatorID getOperatorId() {
+ return this.operatorId;
+ }
+
+ @Override
+ public OperatorCoordinator create(Context context) {
+ return new MultiTableStreamWriteOperatorCoordinator(this.conf, context);
+ }
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/EnhancedWriteMetadataEvent.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/EnhancedWriteMetadataEvent.java
new file mode 100644
index 00000000000..43ab83d1489
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/EnhancedWriteMetadataEvent.java
@@ -0,0 +1,83 @@
+/*
+ * 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.hudi.sink.event;
+
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+
+import org.apache.hudi.sink.event.WriteMetadataEvent;
+
+/**
+ * An {@link OperatorEvent} that enhances a standard Hudi {@link WriteMetadataEvent} with additional
+ * context required for multi-table sinking.
+ *
+ * The standard {@code WriteMetadataEvent} does not contain information about which destination
+ * table it belongs to. This event wraps the original event and adds the {@code tablePath}, allowing
+ * the {@code MultiTableStreamWriteOperatorCoordinator} to correctly route the write metadata to the
+ * timeline of the appropriate table.
+ */
+public class EnhancedWriteMetadataEvent implements OperatorEvent {
+
+ private static final long serialVersionUID = 1L;
+
+ /** The original event from the Hudi write function. */
+ private final WriteMetadataEvent originalEvent;
+
+ /** The filesystem path of the Hudi table this event belongs to. */
+ private final String tablePath;
+
+ /**
+ * Constructs a new EnhancedWriteMetadataEvent.
+ *
+ * @param originalEvent The original {@link WriteMetadataEvent} from the writer.
+ * @param tablePath The path of the Hudi table this metadata belongs to.
+ */
+ public EnhancedWriteMetadataEvent(WriteMetadataEvent originalEvent, String tablePath) {
+ this.originalEvent = originalEvent;
+ this.tablePath = tablePath;
+ }
+
+ /**
+ * Gets the original, un-enhanced event.
+ *
+ * @return The original {@link WriteMetadataEvent}.
+ */
+ public WriteMetadataEvent getOriginalEvent() {
+ return originalEvent;
+ }
+
+ /**
+ * Gets the path of the Hudi table.
+ *
+ * @return The table path string.
+ */
+ public String getTablePath() {
+ return tablePath;
+ }
+
+ @Override
+ public String toString() {
+ return "EnhancedWriteMetadataEvent{"
+ + "tablePath='"
+ + tablePath
+ + '\''
+ + ", instantTime='"
+ + originalEvent.getInstantTime()
+ + '\''
+ + '}';
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/HudiRecordEventSerializer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/HudiRecordEventSerializer.java
new file mode 100644
index 00000000000..1b0eeaaf604
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/HudiRecordEventSerializer.java
@@ -0,0 +1,177 @@
+/*
+ * 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.hudi.sink.event;
+
+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.SchemaChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.utils.SchemaUtils;
+import org.apache.flink.cdc.connectors.hudi.sink.util.RowDataUtils;
+
+import org.apache.hudi.client.model.HoodieFlinkInternalRow;
+import org.apache.hudi.sink.bulk.RowDataKeyGen;
+
+import java.time.ZoneId;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A {@link HudiRecordSerializer} for converting {@link Event} into {@link HoodieFlinkInternalRow}
+ * for Hudi writing.
+ *
+ *
This serializer maintains schema state per table and handles multi-table CDC events by:
+ *
+ *
+ * - Caching schemas from CreateTableEvent and SchemaChangeEvent
+ *
- Converting DataChangeEvent to HoodieFlinkInternalRow using cached schemas
+ *
- Using Hudi's RowDataKeyGen for record key and partition path extraction
+ *
- Supporting bucket-wrapped events from upstream operators
+ *
+ *
+ * Assumes that CreateTableEvent will always arrive before DataChangeEvent for each table,
+ * following the standard CDC pipeline startup sequence.
+ */
+public class HudiRecordEventSerializer implements HudiRecordSerializer {
+
+ /** Schema cache per table - populated from CreateTableEvent and SchemaChangeEvent. */
+ private final Map schemaMaps;
+
+ /** RowDataKeyGen cache per table for key and partition extraction. */
+ private final Map keyGenCache;
+
+ /** Zone ID for timestamp conversion. */
+ private final ZoneId zoneId;
+
+ public HudiRecordEventSerializer(ZoneId zoneId) {
+ this.schemaMaps = new HashMap<>();
+ this.keyGenCache = new HashMap<>();
+ this.zoneId = zoneId;
+ }
+
+ /**
+ * Serialize an Event into HoodieFlinkInternalRow.
+ *
+ * @param event The input event (can be BucketWrappedChangeEvent)
+ * @param fileId The file ID to assign to the record
+ * @param instantTime The instant time to assign to the record
+ * @return HoodieFlinkInternalRow or null for schema events
+ * @throws IllegalArgumentException if event type is unsupported
+ * @throws IllegalStateException if schema is not available for DataChangeEvent
+ */
+ @Override
+ public HoodieFlinkInternalRow serialize(Event event, String fileId, String instantTime) {
+ if (event instanceof CreateTableEvent) {
+ CreateTableEvent createTableEvent = (CreateTableEvent) event;
+ schemaMaps.put(createTableEvent.tableId(), createTableEvent.getSchema());
+ // Clear keyGenCache for this table since schema changed
+ keyGenCache.remove(createTableEvent.tableId());
+ // Schema events don't produce records
+ return null;
+
+ } else if (event instanceof SchemaChangeEvent) {
+ SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event;
+ Schema existingSchema = schemaMaps.get(schemaChangeEvent.tableId());
+ if (existingSchema != null
+ && !SchemaUtils.isSchemaChangeEventRedundant(
+ existingSchema, schemaChangeEvent)) {
+ Schema newSchema =
+ SchemaUtils.applySchemaChangeEvent(existingSchema, schemaChangeEvent);
+ schemaMaps.put(schemaChangeEvent.tableId(), newSchema);
+ // Clear keyGenCache for this table since schema changed
+ keyGenCache.remove(schemaChangeEvent.tableId());
+ }
+ // Schema events don't produce records
+ return null;
+
+ } else if (event instanceof DataChangeEvent) {
+ DataChangeEvent dataChangeEvent = (DataChangeEvent) event;
+ Schema schema = schemaMaps.get(dataChangeEvent.tableId());
+
+ if (schema == null) {
+ throw new IllegalStateException(
+ "No schema available for table "
+ + dataChangeEvent.tableId()
+ + ". CreateTableEvent should arrive before DataChangeEvent.");
+ }
+
+ // Get or create RowDataKeyGen for this table
+ RowDataKeyGen keyGen =
+ keyGenCache.computeIfAbsent(
+ dataChangeEvent.tableId(), tid -> RowDataUtils.createKeyGen(schema));
+
+ // Convert DataChangeEvent to HoodieFlinkInternalRow using RowDataKeyGen
+ return RowDataUtils.convertDataChangeEventToHoodieFlinkInternalRow(
+ dataChangeEvent, schema, zoneId, keyGen, fileId, instantTime);
+
+ } else {
+ throw new IllegalArgumentException(
+ "Unsupported event type for Hudi serialization: "
+ + event.getClass().getSimpleName());
+ }
+ }
+
+ /**
+ * Serialize an Event into HoodieFlinkInternalRow without fileId and instantTime. The fileId and
+ * instantTime will be set later by the caller.
+ *
+ * @param event The input event (can be BucketWrappedChangeEvent)
+ * @return HoodieFlinkInternalRow or null for schema events
+ * @throws IllegalArgumentException if event type is unsupported
+ * @throws IllegalStateException if schema is not available for DataChangeEvent
+ */
+ @Override
+ public HoodieFlinkInternalRow serialize(Event event) {
+ return serialize(event, "temp", "temp");
+ }
+
+ /**
+ * Get cached schema for a table.
+ *
+ * @param tableId The table identifier
+ * @return Schema or null if not cached
+ */
+ public Schema getSchema(TableId tableId) {
+ return schemaMaps.get(tableId);
+ }
+
+ /**
+ * Check if schema is cached for a table.
+ *
+ * @param tableId The table identifier
+ * @return true if schema is cached
+ */
+ public boolean hasSchema(TableId tableId) {
+ return schemaMaps.containsKey(tableId);
+ }
+
+ /**
+ * Set schema for a table. Used to initialize table-specific serializers with schema.
+ *
+ * @param tableId The table identifier
+ * @param schema The schema to set
+ */
+ public void setSchema(TableId tableId, Schema schema) {
+ schemaMaps.put(tableId, schema);
+ // Clear cached field getters and key gens for this table so they get recreated with the new
+ // schema
+ keyGenCache.remove(tableId);
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/HudiRecordSerializer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/HudiRecordSerializer.java
new file mode 100644
index 00000000000..90cc7f37d6b
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/HudiRecordSerializer.java
@@ -0,0 +1,48 @@
+/*
+ * 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.hudi.sink.event;
+
+import org.apache.hudi.client.model.HoodieFlinkInternalRow;
+
+/**
+ * A serializer interface for converting input records into {@link HoodieFlinkInternalRow} for Hudi
+ * writing.
+ *
+ * @param The input record type to be serialized
+ */
+public interface HudiRecordSerializer {
+
+ /**
+ * Serialize an input record into HoodieFlinkInternalRow.
+ *
+ * @param record The input record to serialize
+ * @param fileId The file ID to assign to the record
+ * @param instantTime The instant time to assign to the record
+ * @return HoodieFlinkInternalRow or null if the record doesn't produce a data record
+ */
+ HoodieFlinkInternalRow serialize(T record, String fileId, String instantTime);
+
+ /**
+ * Serialize an input record into HoodieFlinkInternalRow without fileId and instantTime. The
+ * fileId and instantTime will be set later by the caller.
+ *
+ * @param record The input record to serialize
+ * @return HoodieFlinkInternalRow or null if the record doesn't produce a data record
+ */
+ HoodieFlinkInternalRow serialize(T record);
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/SchemaChangeOperatorEvent.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/SchemaChangeOperatorEvent.java
new file mode 100644
index 00000000000..f7e2fa8a5af
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/SchemaChangeOperatorEvent.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.hudi.sink.event;
+
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+
+/**
+ * An operator event that encapsulates a schema change and the resulting new schema.
+ *
+ * This event is sent from the {@code MultiTableEventStreamWriteFunction} to the {@code
+ * MultiTableStreamWriteOperatorCoordinator} to signal that a table's schema has changed in the CDC
+ * stream. The coordinator uses this event to update its cached schema and recreate the write client
+ * to ensure subsequent operations use the correct schema.
+ */
+public class SchemaChangeOperatorEvent implements OperatorEvent {
+
+ private static final long serialVersionUID = 1L;
+
+ private final TableId tableId;
+ private final Schema newSchema;
+
+ /**
+ * Constructs a new SchemaChangeOperatorEvent.
+ *
+ * @param tableId The ID of the table whose schema changed
+ * @param newSchema The new schema after applying the schema change
+ */
+ public SchemaChangeOperatorEvent(TableId tableId, Schema newSchema) {
+ this.tableId = tableId;
+ this.newSchema = newSchema;
+ }
+
+ /**
+ * Gets the ID of the table whose schema changed.
+ *
+ * @return The table ID
+ */
+ public TableId getTableId() {
+ return tableId;
+ }
+
+ /**
+ * Gets the new schema after the change.
+ *
+ * @return The new schema
+ */
+ public Schema getNewSchema() {
+ return newSchema;
+ }
+
+ @Override
+ public String toString() {
+ return "SchemaChangeOperatorEvent{"
+ + "tableId="
+ + tableId
+ + ", newSchema columns="
+ + newSchema.getColumnCount()
+ + '}';
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/TableAwareCorrespondent.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/TableAwareCorrespondent.java
new file mode 100644
index 00000000000..1b572240986
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/TableAwareCorrespondent.java
@@ -0,0 +1,233 @@
+/*
+ * 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.hudi.sink.event;
+
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.connectors.hudi.sink.coordinator.MultiTableStreamWriteOperatorCoordinator;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.util.SerializedValue;
+
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sink.event.Correspondent;
+import org.apache.hudi.sink.utils.CoordinationResponseSerDe;
+
+/**
+ * A correspondent between a write task and the multi-table coordinator. This class is responsible
+ * for sending table-aware requests to the {@link MultiTableStreamWriteOperatorCoordinator}.
+ */
+public class TableAwareCorrespondent extends Correspondent {
+ private final OperatorID operatorID;
+ private final TaskOperatorEventGateway gateway;
+ private final TableId tableId;
+
+ private TableAwareCorrespondent(
+ OperatorID operatorID, TaskOperatorEventGateway gateway, TableId tableId) {
+ this.operatorID = operatorID;
+ this.gateway = gateway;
+ this.tableId = tableId;
+ }
+
+ /**
+ * Creates a coordinator correspondent.
+ *
+ * @param correspondent The original correspondent
+ * @param tableId The table ID
+ * @return an instance of {@code TableAwareCorrespondent}.
+ */
+ public static TableAwareCorrespondent getInstance(
+ Correspondent correspondent, TableId tableId) {
+ return new TableAwareCorrespondent(
+ correspondent.getOperatorID(), correspondent.getGateway(), tableId);
+ }
+
+ /**
+ * Sends a request to the coordinator to fetch the instant time for a specific table.
+ *
+ * @param checkpointId The current checkpoint ID.
+ * @return The instant time string allocated by the coordinator.
+ */
+ @Override
+ public String requestInstantTime(long checkpointId) {
+ try {
+ MultiTableInstantTimeRequest request =
+ new MultiTableInstantTimeRequest(checkpointId, tableId);
+ Correspondent.InstantTimeResponse response =
+ CoordinationResponseSerDe.unwrap(
+ this.gateway
+ .sendRequestToCoordinator(
+ this.operatorID, new SerializedValue<>(request))
+ .get());
+ return response.getInstant();
+ } catch (Exception e) {
+ throw new HoodieException(
+ "Error requesting the instant time from the coordinator for table " + tableId,
+ e);
+ }
+ }
+
+ /**
+ * A custom coordination request that includes the TableId to request an instant for a specific
+ * table.
+ */
+ public static class MultiTableInstantTimeRequest implements CoordinationRequest {
+ private static final long serialVersionUID = 1L;
+ private final long checkpointId;
+ private final TableId tableId;
+
+ public MultiTableInstantTimeRequest(long checkpointId, TableId tableId) {
+ this.checkpointId = checkpointId;
+ this.tableId = tableId;
+ }
+
+ public long getCheckpointId() {
+ return checkpointId;
+ }
+
+ public TableId getTableId() {
+ return tableId;
+ }
+ }
+
+ /**
+ * Send a request to coordinator to create a hudi table.
+ *
+ * @param createTableEvent The creating table event.
+ * @return Whether the table is created successfully.
+ */
+ public boolean requestCreatingTable(CreateTableEvent createTableEvent) {
+ try {
+ CreateTableRequest request = new CreateTableRequest(createTableEvent);
+ SchemaChangeResponse response =
+ CoordinationResponseSerDe.unwrap(
+ this.gateway
+ .sendRequestToCoordinator(
+ this.operatorID, new SerializedValue<>(request))
+ .get());
+ return response.isSuccess();
+ } catch (Exception e) {
+ throw new HoodieException(
+ "Error requesting the instant time from the coordinator for table " + tableId,
+ e);
+ }
+ }
+
+ /**
+ * Send a request to coordinator to apply the schema change.
+ *
+ * @param tableId the id of table
+ * @param newSchema the new table schema
+ * @return Whether the schema change is applied successfully.
+ */
+ public boolean requestSchemaChange(TableId tableId, Schema newSchema) {
+ try {
+ SchemaChangeRequest request = new SchemaChangeRequest(tableId, newSchema);
+ SchemaChangeResponse response =
+ CoordinationResponseSerDe.unwrap(
+ this.gateway
+ .sendRequestToCoordinator(
+ this.operatorID, new SerializedValue<>(request))
+ .get());
+ return response.isSuccess();
+ } catch (Exception e) {
+ throw new HoodieException(
+ "Error requesting the instant time from the coordinator for table " + tableId,
+ e);
+ }
+ }
+
+ /**
+ * A CoordinationRequest that encapsulates a {@link CreateTableEvent}.
+ *
+ *
This request is sent from the {@code MultiTableEventStreamWriteFunction} to the {@code
+ * MultiTableStreamWriteOperatorCoordinator} to signal that a new table has been discovered in
+ * the CDC stream. The coordinator uses this event to initialize all necessary resources for the
+ * new table, such as its dedicated write client and event buffers, before any data is written.
+ */
+ public static class CreateTableRequest extends SchemaChangeRequest {
+ private static final long serialVersionUID = 1L;
+
+ public CreateTableRequest(CreateTableEvent createTableEvent) {
+ super(createTableEvent.tableId(), createTableEvent.getSchema());
+ }
+ }
+
+ /**
+ * A CoordinationRequest that represents a request to change table schema.
+ *
+ *
This request is sent from the {@code MultiTableEventStreamWriteFunction} to the {@code
+ * MultiTableStreamWriteOperatorCoordinator} to signal that a schema change has been discovered
+ * in the CDC stream.
+ */
+ public static class SchemaChangeRequest implements CoordinationRequest {
+ private static final long serialVersionUID = 1L;
+
+ private final TableId tableId;
+ private final Schema schema;
+
+ public SchemaChangeRequest(TableId tableId, Schema schema) {
+ this.tableId = tableId;
+ this.schema = schema;
+ }
+
+ public TableId getTableId() {
+ return tableId;
+ }
+
+ public Schema getSchema() {
+ return schema;
+ }
+
+ @Override
+ public String toString() {
+ return "SchemaChangeRequest{" + "tableId=" + tableId + ", schema=" + schema + '}';
+ }
+ }
+
+ /**
+ * Response for a {@link CreateTableRequest} or {@link SchemaChangeRequest}. This response is
+ * sent from writer coordinator to indicate whether the schema change is applied successfully.
+ */
+ public static class SchemaChangeResponse implements CoordinationResponse {
+ private static final long serialVersionUID = 1L;
+ private final TableId tableId;
+ private final boolean success;
+
+ private SchemaChangeResponse(TableId tableId, boolean success) {
+ this.tableId = tableId;
+ this.success = success;
+ }
+
+ public boolean isSuccess() {
+ return success;
+ }
+
+ public TableId getTableId() {
+ return tableId;
+ }
+
+ public static SchemaChangeResponse of(TableId tableId, boolean success) {
+ return new SchemaChangeResponse(tableId, success);
+ }
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventProcessorFunction.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventProcessorFunction.java
new file mode 100644
index 00000000000..a07bf335212
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventProcessorFunction.java
@@ -0,0 +1,80 @@
+/*
+ * 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.hudi.sink.function;
+
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.common.event.FlushEvent;
+import org.apache.flink.cdc.common.event.SchemaChangeEvent;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Collector;
+
+/**
+ * Template interface for processing CDC events in a standardized way. Provides a consistent event
+ * handling pattern across different write function implementations.
+ *
+ *
All write functions should implement this interface to ensure uniform event processing with
+ * clear separation of concerns:
+ *
+ *
+ * - {@link #processDataChange(DataChangeEvent, ProcessFunction.Context, Collector)} - Handles
+ * DML operations (INSERT, UPDATE, DELETE)
+ *
- {@link #processSchemaChange(SchemaChangeEvent)} - Handles DDL operations (CREATE TABLE, ADD
+ * COLUMN, etc.)
+ *
- {@link #processFlush(FlushEvent)} - Handles coordinated flushing of buffered data
+ *
+ *
+ * Implementations of this interface are used in multi-table CDC pipelines to route and process
+ * events from different source tables to their corresponding Hudi tables.
+ *
+ * @see MultiTableEventStreamWriteFunction
+ */
+public interface EventProcessorFunction {
+
+ /**
+ * Process data change events (INSERT, UPDATE, DELETE operations).
+ *
+ *
This method handles DML operations from the CDC stream, converting them into Hudi records
+ * and collecting them for writing to the appropriate table.
+ *
+ * @param event The data change event containing the operation type and data
+ * @param ctx The process function context for accessing runtime information
+ * @param out The collector for emitting processed RowData records
+ */
+ void processDataChange(
+ DataChangeEvent event,
+ ProcessFunction.Context ctx,
+ Collector out);
+
+ /**
+ * Process schema change events (CREATE TABLE, ADD COLUMN, etc.).
+ *
+ * @param event The schema change event
+ * @throws Exception if processing fails
+ */
+ void processSchemaChange(SchemaChangeEvent event) throws Exception;
+
+ /**
+ * Process flush events for coordinated flushing.
+ *
+ * @param event The flush event
+ * @throws Exception if processing fails
+ */
+ void processFlush(FlushEvent event) throws Exception;
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/ExtendedBucketStreamWriteFunction.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/ExtendedBucketStreamWriteFunction.java
new file mode 100644
index 00000000000..32ec1ea52bc
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/ExtendedBucketStreamWriteFunction.java
@@ -0,0 +1,58 @@
+/*
+ * 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.hudi.sink.function;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.apache.hudi.sink.bucket.BucketStreamWriteFunction;
+
+/**
+ * Extended version of {@link BucketStreamWriteFunction} that exposes a public setter for the
+ * checkpoint ID.
+ *
+ * This class is necessary because the parent class's {@code checkpointId} field is protected and
+ * inaccessible from composition-based multi-table write functions. In a multi-table CDC sink, each
+ * table requires its own write function instance, and these instances must be updated with the
+ * current checkpoint ID for proper coordinator communication during checkpointing.
+ *
+ *
The public {@link #setCheckpointId(long)} method provides a clean API for parent write
+ * functions to update the checkpoint ID without resorting to reflection-based access.
+ *
+ * @see BucketStreamWriteFunction
+ * @see MultiTableEventStreamWriteFunction
+ */
+public class ExtendedBucketStreamWriteFunction extends BucketStreamWriteFunction {
+
+ public ExtendedBucketStreamWriteFunction(Configuration config, RowType rowType) {
+ super(config, rowType);
+ }
+
+ /**
+ * Sets the checkpoint ID for this write function.
+ *
+ *
This method provides public access to update the protected {@code checkpointId} field
+ * inherited from the parent class. The checkpoint ID is required for the write function to
+ * properly communicate with the coordinator during checkpoint operations.
+ *
+ * @param checkpointId the checkpoint ID to set
+ */
+ public void setCheckpointId(long checkpointId) {
+ this.checkpointId = checkpointId;
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/MultiTableEventStreamWriteFunction.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/MultiTableEventStreamWriteFunction.java
new file mode 100644
index 00000000000..cc92bcc5810
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/MultiTableEventStreamWriteFunction.java
@@ -0,0 +1,676 @@
+/*
+ * 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.hudi.sink.function;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+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.FlushEvent;
+import org.apache.flink.cdc.common.event.SchemaChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.utils.SchemaUtils;
+import org.apache.flink.cdc.connectors.hudi.sink.event.EnhancedWriteMetadataEvent;
+import org.apache.flink.cdc.connectors.hudi.sink.event.HudiRecordEventSerializer;
+import org.apache.flink.cdc.connectors.hudi.sink.event.TableAwareCorrespondent;
+import org.apache.flink.cdc.connectors.hudi.sink.util.RowDataUtils;
+import org.apache.flink.cdc.runtime.operators.sink.SchemaEvolutionClient;
+import org.apache.flink.cdc.runtime.serializer.TableIdSerializer;
+import org.apache.flink.cdc.runtime.serializer.schema.SchemaSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.OutputTag;
+
+import org.apache.hudi.client.model.HoodieFlinkInternalRow;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.sink.common.AbstractStreamWriteFunction;
+import org.apache.hudi.sink.event.WriteMetadataEvent;
+import org.apache.hudi.util.AvroSchemaConverter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.ZoneId;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Multi-table wrapper function that routes events to table-specific
+ * EventExtendedBucketStreamWriteFunction instances. This approach maintains table isolation by
+ * creating dedicated function instances per table while keeping the core write functions
+ * single-table focused.
+ */
+public class MultiTableEventStreamWriteFunction extends AbstractStreamWriteFunction
+ implements EventProcessorFunction {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(MultiTableEventStreamWriteFunction.class);
+
+ /** Table-specific write functions created dynamically when new tables are encountered. */
+ private transient Map tableFunctions;
+
+ /** Track tables that have been initialized to avoid duplicate initialization. */
+ private transient Map initializedTables;
+
+ /** Cache of schemas per table for RowType generation. */
+ private transient Map schemaMaps;
+
+ /** Persistent state for schemas to survive checkpoints/savepoints. */
+ private transient ListState> schemaState;
+
+ private transient Map tableConfigurations;
+
+ /** Schema evolution client to communicate with SchemaOperator. */
+ private transient SchemaEvolutionClient schemaEvolutionClient;
+
+ /** Serializer for converting Events to HoodieFlinkInternalRow. */
+ private transient HudiRecordEventSerializer recordSerializer;
+
+ /** Store the function initialization context for table functions. */
+ private transient FunctionInitializationContext functionInitializationContext;
+
+ public MultiTableEventStreamWriteFunction(Configuration config) {
+ super(config);
+ }
+
+ @Override
+ public void initializeState(FunctionInitializationContext context) throws Exception {
+ // NOTE: Do NOT call super.initializeState(context) here.
+ // The parent class (AbstractStreamWriteFunction) expects to manage a single Hudi table
+ // and tries to create a HoodieTableMetaClient during initialization.
+ // MultiTableEventStreamWriteFunction is a dispatcher that manages multiple tables
+ // dynamically, so it doesn't have a single table path. Each child function
+ // (ExtendedBucketStreamWriteFunction) handles its own state initialization.
+ this.functionInitializationContext = context;
+
+ // Initialize schema map before restoring state
+ if (this.schemaMaps == null) {
+ this.schemaMaps = new HashMap<>();
+ }
+
+ // Initialize schema state for persistence across checkpoints/savepoints
+ // Using operator state since this is not a keyed stream
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ TupleSerializer> tupleSerializer =
+ new TupleSerializer(
+ Tuple2.class,
+ new TypeSerializer[] {
+ TableIdSerializer.INSTANCE, SchemaSerializer.INSTANCE
+ });
+ ListStateDescriptor> schemaStateDescriptor =
+ new ListStateDescriptor<>("schemaState", tupleSerializer);
+ this.schemaState = context.getOperatorStateStore().getUnionListState(schemaStateDescriptor);
+
+ // Restore schemas from state if this is a restore operation
+ if (context.isRestored()) {
+ LOG.info("Restoring schemas from state");
+ for (Tuple2 entry : schemaState.get()) {
+ schemaMaps.put(entry.f0, entry.f1);
+ LOG.info("Restored schema for table: {}", entry.f0);
+ }
+ LOG.info("Restored {} schemas from state", schemaMaps.size());
+ }
+
+ LOG.info("MultiTableEventStreamWriteFunction state initialized");
+ }
+
+ /**
+ * Sets the SchemaEvolutionClient from the operator level since functions don't have direct
+ * access to TaskOperatorEventGateway.
+ */
+ public void setSchemaEvolutionClient(SchemaEvolutionClient schemaEvolutionClient) {
+ this.schemaEvolutionClient = schemaEvolutionClient;
+ LOG.info("SchemaEvolutionClient set for MultiTableEventStreamWriteFunction");
+ }
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ super.open(parameters);
+ this.tableFunctions = new HashMap<>();
+ this.initializedTables = new HashMap<>();
+ // Don't reinitialize schemaMaps if it already has restored schemas from state
+ if (this.schemaMaps == null) {
+ this.schemaMaps = new HashMap<>();
+ }
+ this.tableConfigurations = new HashMap<>();
+ // Initialize record serializer (must be done in open() since it's transient)
+ this.recordSerializer = new HudiRecordEventSerializer(ZoneId.systemDefault());
+
+ // Restore schemas to recordSerializer if they were restored from state
+ // recordSerializer is transient and does not persist across restarts
+ if (!schemaMaps.isEmpty()) {
+ LOG.info("Restoring {} schemas to recordSerializer", schemaMaps.size());
+ for (Map.Entry entry : schemaMaps.entrySet()) {
+ recordSerializer.setSchema(entry.getKey(), entry.getValue());
+ LOG.debug("Restored schema to recordSerializer for table: {}", entry.getKey());
+ }
+ }
+ }
+
+ @Override
+ public void processElement(Event event, Context ctx, Collector out) throws Exception {
+ LOG.debug("Processing event of type: {}", event.getClass().getSimpleName());
+
+ // Route event to appropriate handler based on type
+ if (event instanceof DataChangeEvent) {
+ processDataChange((DataChangeEvent) event, ctx, out);
+ } else if (event instanceof SchemaChangeEvent) {
+ processSchemaChange((SchemaChangeEvent) event);
+ } else if (event instanceof FlushEvent) {
+ processFlush((FlushEvent) event);
+ } else {
+ LOG.warn("Received unknown event type: {}", event.getClass().getName());
+ }
+ }
+
+ /**
+ * Processes schema events. For a {@link CreateTableEvent}, it ensures that the coordinator is
+ * notified and the physical Hudi table is created. For a {@link SchemaChangeEvent}, it updates
+ * the local schema cache.
+ *
+ * Implements {@link EventProcessorFunction#processSchemaChange(SchemaChangeEvent)}.
+ */
+ @Override
+ public void processSchemaChange(SchemaChangeEvent event) throws Exception {
+ TableId tableId = event.tableId();
+ try {
+ if (event instanceof CreateTableEvent) {
+ CreateTableEvent createTableEvent = (CreateTableEvent) event;
+ schemaMaps.put(tableId, createTableEvent.getSchema());
+ LOG.debug("Cached schema for new table: {}", tableId);
+
+ boolean createTableSuccess =
+ initializedTables.computeIfAbsent(
+ tableId,
+ tId -> {
+ try {
+ // Send an explicit event to the coordinator so it can
+ // prepare
+ // resources (including creating physical directory)
+ // *before* we
+ // attempt to write any data.
+ boolean success =
+ getTableAwareCorrespondent(tableId)
+ .requestCreatingTable(createTableEvent);
+ LOG.info(
+ "Sent CreateTableRequest to coordinator for new table: {}",
+ tId);
+ return success;
+ } catch (Exception e) {
+ // Re-throw to fail the Flink task if initialization fails.
+ throw new RuntimeException(
+ "Failed during first-time initialization for table: "
+ + tId,
+ e);
+ }
+ });
+
+ if (!createTableSuccess) {
+ throw new RuntimeException("Failed to create table: " + tableId);
+ }
+
+ // Ensure tableFunction is initialized
+ getOrCreateTableFunction(tableId);
+ return;
+ }
+
+ LOG.info("Schema change event received: {}", event);
+ Schema existingSchema = schemaMaps.get(tableId);
+ if (existingSchema == null
+ || SchemaUtils.isSchemaChangeEventRedundant(existingSchema, event)) {
+ return;
+ }
+
+ LOG.info("Schema change event received for table {}: {}", tableId, event);
+ LOG.info(
+ "Existing schema for table {} has {} columns: {}",
+ tableId,
+ existingSchema.getColumnCount(),
+ existingSchema.getColumnNames());
+
+ Schema newSchema = SchemaUtils.applySchemaChangeEvent(existingSchema, event);
+
+ LOG.info(
+ "New schema for table {} has {} columns: {}",
+ tableId,
+ newSchema.getColumnCount(),
+ newSchema.getColumnNames());
+
+ schemaMaps.put(tableId, newSchema);
+
+ // Update recordSerializer with the new schema immediately
+ // This ensures future DataChangeEvents are serialized with the new schema
+ recordSerializer.setSchema(tableId, newSchema);
+ LOG.info("Updated recordSerializer with new schema for table: {}", tableId);
+
+ // Invalidate cached table configuration so it gets recreated with NEW
+ // schema
+ // The tableConfigurations cache holds FlinkOptions.SOURCE_AVRO_SCHEMA which
+ // must be updated
+ tableConfigurations.remove(tableId);
+ LOG.info(
+ "Invalidated cached table configuration for {} to pick up new schema", tableId);
+
+ // If table function exists, close and remove it
+ // NOTE: Flushing should have been done earlier by a FlushEvent that was
+ // sent BEFORE this SchemaChangeEvent. We don't flush here because the
+ // table metadata may have already been updated to the new schema,
+ // which would cause a schema mismatch error.
+ // A new function with the updated schema will be created on the next
+ // DataChangeEvent
+ ExtendedBucketStreamWriteFunction tableFunction = tableFunctions.get(tableId);
+ if (tableFunction != null) {
+ LOG.info(
+ "Schema changed for table {}, closing and removing old table function",
+ tableId);
+
+ // Close the function to release resources (write client, etc.)
+ try {
+ tableFunction.close();
+ LOG.info("Closed old table function for table: {}", tableId);
+ } catch (Exception e) {
+ LOG.error("Failed to close table function for table: {}", tableId, e);
+ // Continue with removal even if close fails
+ }
+
+ // Remove the old function - a new one will be created with the new schema
+ tableFunctions.remove(tableId);
+ LOG.info(
+ "Removed old table function for table: {}. New function will be created with updated schema on next data event.",
+ tableId);
+ initializedTables.remove(tableId);
+ }
+
+ // Notify coordinator about schema change so it can update its write client
+ try {
+ getTableAwareCorrespondent(tableId).requestSchemaChange(tableId, newSchema);
+ LOG.info("Sent SchemaChangeOperatorEvent to coordinator for table: {}", tableId);
+ } catch (Exception e) {
+ LOG.error(
+ "Failed to send SchemaChangeOperatorEvent to coordinator for table: {}",
+ tableId,
+ e);
+ // Don't throw - schema change was applied locally, coordinator will
+ // update on next operation
+ }
+
+ LOG.debug("Updated schema for table: {}", tableId);
+ } catch (Exception e) {
+ LOG.error("Failed to process schema event for table: {}", tableId, e);
+ throw new RuntimeException("Failed to process schema event for table: " + tableId, e);
+ }
+ }
+
+ /**
+ * Processes change events with context and collector for writing. This triggers the actual Hudi
+ * write operations as side effects by delegating to table-specific functions.
+ */
+ @Override
+ public void processDataChange(
+ DataChangeEvent event,
+ ProcessFunction.Context ctx,
+ Collector out) {
+ TableId tableId = event.tableId();
+ try {
+ LOG.debug("Processing change event for table: {}", tableId);
+
+ // Check if schema is available before processing
+ if (!recordSerializer.hasSchema(event.tableId())) {
+ // Schema not available yet - CreateTableEvent hasn't arrived
+ throw new IllegalStateException(
+ "No schema available for table "
+ + event.tableId()
+ + ". CreateTableEvent should arrive before DataChangeEvent.");
+ }
+ HoodieFlinkInternalRow hoodieFlinkInternalRow = recordSerializer.serialize(event);
+
+ // Get or create table-specific function to handle this event
+ ExtendedBucketStreamWriteFunction tableFunction = getOrCreateTableFunction(tableId);
+
+ // Create context adapter to convert Event context to HoodieFlinkInternalRow context
+ ProcessFunction.Context adaptedContext =
+ new ContextAdapter(ctx);
+
+ tableFunction.processElement(hoodieFlinkInternalRow, adaptedContext, out);
+
+ LOG.debug("Successfully processed change event for table: {}", tableId);
+
+ } catch (Exception e) {
+ LOG.error("Failed to process change event for table: {}", tableId, e);
+ throw new RuntimeException("Failed to process change event for table: " + tableId, e);
+ }
+ }
+
+ /**
+ * Processes flush events for coordinated flushing across table functions. This handles both
+ * table-specific and global flush operations.
+ *
+ * Implements {@link EventProcessorFunction#processFlush(FlushEvent)}.
+ */
+ @Override
+ public void processFlush(FlushEvent event) throws Exception {
+ List tableIds = event.getTableIds();
+ try {
+ if (tableIds == null || tableIds.isEmpty()) {
+ LOG.info(
+ "Received global flush event, flushing all {} table functions",
+ tableFunctions.size());
+ for (Map.Entry entry :
+ tableFunctions.entrySet()) {
+ entry.getValue().flushRemaining(false);
+ LOG.debug("Flushed table function for: {}", entry.getKey());
+ }
+ } else {
+ LOG.info("Received flush event {} for {} specific tables", event, tableIds.size());
+ for (TableId tableId : tableIds) {
+ LOG.info(
+ "Flushing table {} with schema: {}",
+ tableId,
+ recordSerializer.getSchema(tableId));
+ ExtendedBucketStreamWriteFunction tableFunction = tableFunctions.get(tableId);
+ if (tableFunction != null) {
+ tableFunction.flushRemaining(false);
+ LOG.debug("Flushed table function for: {}", tableId);
+ }
+ }
+ }
+
+ if (schemaEvolutionClient == null) {
+ return;
+ }
+
+ int sinkSubtaskId = getRuntimeContext().getIndexOfThisSubtask();
+ int sourceSubtaskId = event.getSourceSubTaskId();
+
+ try {
+ schemaEvolutionClient.notifyFlushSuccess(sinkSubtaskId, sourceSubtaskId);
+ LOG.info(
+ "Sent FlushSuccessEvent to SchemaOperator from sink subtask {} for source subtask {}",
+ sinkSubtaskId,
+ sourceSubtaskId);
+ } catch (Exception e) {
+ LOG.error("Failed to send FlushSuccessEvent to SchemaOperator", e);
+ throw new RuntimeException("Failed to send FlushSuccessEvent to SchemaOperator", e);
+ }
+
+ } catch (Exception e) {
+ LOG.error("Failed to process flush event", e);
+ throw new RuntimeException("Failed to process flush event", e);
+ }
+ }
+
+ private ExtendedBucketStreamWriteFunction getOrCreateTableFunction(TableId tableId) {
+ ExtendedBucketStreamWriteFunction existingFunction = tableFunctions.get(tableId);
+ if (existingFunction != null) {
+ return existingFunction;
+ }
+
+ LOG.info("Creating new ExtendedBucketStreamWriteFunction for table: {}", tableId);
+ try {
+ ExtendedBucketStreamWriteFunction tableFunction = createTableFunction(tableId);
+ tableFunctions.put(tableId, tableFunction);
+ LOG.info("Successfully created and cached table function for: {}", tableId);
+ return tableFunction;
+ } catch (Exception e) {
+ LOG.error("Failed to create table function for table: {}", tableId, e);
+ throw new RuntimeException("Failed to create table function for table: " + tableId, e);
+ }
+ }
+
+ private ExtendedBucketStreamWriteFunction createTableFunction(TableId tableId)
+ throws Exception {
+ Schema schema = schemaMaps.get(tableId);
+ if (schema == null) {
+ throw new IllegalStateException(
+ "No schema found for table: "
+ + tableId
+ + ". CreateTableEvent must arrive before data events.");
+ }
+
+ if (functionInitializationContext == null) {
+ throw new IllegalStateException(
+ "FunctionInitializationContext not available for creating table function: "
+ + tableId);
+ }
+
+ Configuration tableConfig = createTableSpecificConfig(tableId);
+ RowType rowType = convertSchemaToFlinkRowType(schema);
+
+ // Log the schema being used for this new function
+ String avroSchemaInConfig = tableConfig.get(FlinkOptions.SOURCE_AVRO_SCHEMA);
+ LOG.info(
+ "Creating table function for {} with schema: {} columns, Avro schema in config: {}",
+ tableId,
+ schema.getColumnCount(),
+ avroSchemaInConfig);
+
+ ExtendedBucketStreamWriteFunction tableFunction =
+ new ExtendedBucketStreamWriteFunction(tableConfig, rowType);
+
+ tableFunction.setRuntimeContext(getRuntimeContext());
+
+ // Create a table-aware correspondent that can send MultiTableInstantTimeRequest
+ // Get the operator ID from the runtime context
+ TableAwareCorrespondent tableCorrespondent =
+ TableAwareCorrespondent.getInstance(correspondent, tableId);
+ tableFunction.setCorrespondent(tableCorrespondent);
+
+ // Instead of passing the raw gateway, we pass a proxy that intercepts and enhances events
+ // with the table path
+ String tablePath = tableConfig.getString(FlinkOptions.PATH);
+ tableFunction.setOperatorEventGateway(
+ new InterceptingGateway(this.getOperatorEventGateway(), tablePath));
+
+ try {
+ tableFunction.initializeState(functionInitializationContext);
+ if (this.checkpointId != -1) {
+ tableFunction.setCheckpointId(this.checkpointId);
+ }
+ LOG.info("Successfully initialized state for table function: {}", tableId);
+ } catch (Exception e) {
+ LOG.error("Failed to initialize state for table function: {}", tableId, e);
+ throw new RuntimeException(
+ "Failed to initialize state for table function: " + tableId, e);
+ }
+
+ tableFunction.open(tableConfig);
+
+ recordSerializer.setSchema(tableId, schema);
+ LOG.debug("Set schema for table function serializer: {}", tableId);
+
+ LOG.debug("Successfully created table function for: {}", tableId);
+ return tableFunction;
+ }
+
+ private RowType convertSchemaToFlinkRowType(Schema cdcSchema) {
+ return RowDataUtils.toRowType(cdcSchema);
+ }
+
+ private Configuration createTableSpecificConfig(TableId tableId) {
+ LOG.debug("Creating table specific config for table: {}", tableId);
+ return tableConfigurations.computeIfAbsent(
+ tableId,
+ k -> {
+ Configuration localTableConfig = new Configuration();
+ localTableConfig.addAll(this.config);
+
+ localTableConfig.set(FlinkOptions.DATABASE_NAME, tableId.getSchemaName());
+ localTableConfig.set(FlinkOptions.TABLE_NAME, tableId.getTableName());
+
+ RowType rowType = convertSchemaToFlinkRowType(schemaMaps.get(tableId));
+ String tableAvroSchema =
+ AvroSchemaConverter.convertToSchema(rowType).toString();
+ localTableConfig.set(FlinkOptions.SOURCE_AVRO_SCHEMA, tableAvroSchema);
+
+ String rootPath = this.config.get(FlinkOptions.PATH);
+ String tableBasePath =
+ String.format(
+ "%s/%s/%s",
+ rootPath, tableId.getSchemaName(), tableId.getTableName());
+ localTableConfig.set(FlinkOptions.PATH, tableBasePath);
+
+ return localTableConfig;
+ });
+ }
+
+ @Override
+ public void snapshotState() {
+ // This function acts as a dispatcher. It should not manage its own instant or buffer.
+ // Instead, it delegates the snapshot operation to each of its child, table-specific
+ // functions. Each child function will then handle its own buffer flushing and state
+ // snapshotting. The direct call to flushRemaining() is removed to prevent sending
+ // an invalid, generic instant request to the coordinator.
+ // flushRemaining(false);
+
+ // NOTE: This abstract method is intentionally empty for multi-table function.
+ // The actual delegation happens in snapshotState(FunctionSnapshotContext)
+ // to ensure child functions receive the correct checkpointId.
+ }
+
+ @Override
+ public void snapshotState(FunctionSnapshotContext context) throws Exception {
+ // Persist schemas to state for recovery
+ if (schemaState != null && schemaMaps != null) {
+ schemaState.clear();
+ for (Map.Entry entry : schemaMaps.entrySet()) {
+ schemaState.add(new Tuple2<>(entry.getKey(), entry.getValue()));
+ LOG.debug("Persisted schema for table: {}", entry.getKey());
+ }
+ LOG.info("Persisted {} schemas to state", schemaMaps.size());
+ }
+
+ for (Map.Entry entry :
+ tableFunctions.entrySet()) {
+ try {
+ ExtendedBucketStreamWriteFunction tableFunction = entry.getValue();
+ LOG.debug(
+ "Delegating snapshotState for table: {} with checkpointId: {}",
+ entry.getKey(),
+ context.getCheckpointId());
+ tableFunction.snapshotState(context);
+ LOG.debug("Successfully snapshotted state for table: {}", entry.getKey());
+ } catch (Exception e) {
+ LOG.error("Failed to snapshot state for table: {}", entry.getKey(), e);
+ throw new RuntimeException(
+ "Failed to snapshot state for table: " + entry.getKey(), e);
+ }
+ }
+ this.checkpointId = context.getCheckpointId();
+ }
+
+ @Override
+ public void close() throws Exception {
+ if (tableFunctions != null) {
+ for (ExtendedBucketStreamWriteFunction func : tableFunctions.values()) {
+ try {
+ func.close();
+ } catch (Exception e) {
+ LOG.error("Failed to close table function", e);
+ }
+ }
+ }
+ super.close();
+ }
+
+ public void endInput() {
+ super.endInput();
+ if (tableFunctions != null) {
+ for (ExtendedBucketStreamWriteFunction func : tableFunctions.values()) {
+ try {
+ func.endInput();
+ } catch (Exception e) {
+ LOG.error("Failed to complete endInput for table function", e);
+ }
+ }
+ }
+ }
+
+ private TableAwareCorrespondent getTableAwareCorrespondent(TableId tableId) {
+ return TableAwareCorrespondent.getInstance(correspondent, tableId);
+ }
+
+ /**
+ * Adapter to convert ProcessFunction Event RowData Context to ProcessFunction
+ * HoodieFlinkInternalRow RowData Context. This allows us to call
+ * ExtendedBucketStreamWriteFunction.processElement with the correct context type without
+ * managing its internal state.
+ */
+ private class ContextAdapter extends ProcessFunction.Context {
+ private final ProcessFunction.Context delegate;
+
+ ContextAdapter(ProcessFunction.Context delegate) {
+ this.delegate = delegate;
+ }
+
+ @Override
+ public Long timestamp() {
+ return delegate.timestamp();
+ }
+
+ @Override
+ public TimerService timerService() {
+ return delegate.timerService();
+ }
+
+ @Override
+ public void output(OutputTag outputTag, X value) {
+ delegate.output(outputTag, value);
+ }
+ }
+
+ /**
+ * A proxy {@link OperatorEventGateway} that intercepts {@link WriteMetadataEvent}s from child
+ * functions. It wraps them in an {@link EnhancedWriteMetadataEvent} to add the table path,
+ * which is essential for the multi-table coordinator to route the event correctly.
+ */
+ private static class InterceptingGateway implements OperatorEventGateway {
+ private final OperatorEventGateway delegate;
+ private final String tablePath;
+
+ InterceptingGateway(OperatorEventGateway delegate, String tablePath) {
+ this.delegate = delegate;
+ this.tablePath = tablePath;
+ }
+
+ @Override
+ public void sendEventToCoordinator(OperatorEvent event) {
+ if (event instanceof WriteMetadataEvent) {
+ // Wrap the original event with the table path so the coordinator knows
+ // which table this metadata belongs to.
+ EnhancedWriteMetadataEvent enhancedEvent =
+ new EnhancedWriteMetadataEvent((WriteMetadataEvent) event, tablePath);
+ delegate.sendEventToCoordinator(enhancedEvent);
+ } else {
+ delegate.sendEventToCoordinator(event);
+ }
+ }
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/operator/MultiTableWriteOperator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/operator/MultiTableWriteOperator.java
new file mode 100644
index 00000000000..dda33b99d7e
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/operator/MultiTableWriteOperator.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.hudi.sink.operator;
+
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.connectors.hudi.sink.function.MultiTableEventStreamWriteFunction;
+import org.apache.flink.cdc.connectors.hudi.sink.v2.OperatorIDGenerator;
+import org.apache.flink.cdc.runtime.operators.sink.SchemaEvolutionClient;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway;
+
+import org.apache.hudi.sink.common.AbstractWriteOperator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Multi-table write operator for Apache Hudi that handles CDC events from multiple tables. Extends
+ * AbstractWriteOperator with Event as the input type to support CDC multi-table scenarios.
+ *
+ * This operator:
+ *
+ *
+ * - Routes events to table-specific write functions
+ *
- Maintains proper coordinator setup for each table
+ *
- Passes events through to downstream operators
+ *
- Handles schema evolution across multiple tables
+ *
+ */
+public class MultiTableWriteOperator extends AbstractWriteOperator {
+
+ private static final long serialVersionUID = 1L;
+ private static final Logger LOG = LoggerFactory.getLogger(MultiTableWriteOperator.class);
+
+ private final String schemaOperatorUid;
+ private final MultiTableEventStreamWriteFunction multiTableWriteFunction;
+
+ /**
+ * Constructs a MultiTableWriteOperator.
+ *
+ * @param config Configuration for the operator
+ */
+ public MultiTableWriteOperator(Configuration config, String schemaOperatorUid) {
+ this(schemaOperatorUid, new MultiTableEventStreamWriteFunction(config));
+ }
+
+ private MultiTableWriteOperator(
+ String schemaOperatorUid, MultiTableEventStreamWriteFunction writeFunction) {
+ super(writeFunction);
+ this.schemaOperatorUid = schemaOperatorUid;
+ this.multiTableWriteFunction = writeFunction;
+ }
+
+ @Override
+ public void open() throws Exception {
+ super.open();
+
+ // Initialize SchemaEvolutionClient and set it on the MultiTableEventStreamWriteFunction
+ TaskOperatorEventGateway toCoordinator =
+ getContainingTask().getEnvironment().getOperatorCoordinatorEventGateway();
+ OperatorID schemaOperatorID = new OperatorIDGenerator(schemaOperatorUid).generate();
+ SchemaEvolutionClient schemaEvolutionClient =
+ new SchemaEvolutionClient(toCoordinator, schemaOperatorID);
+
+ // Set the SchemaEvolutionClient on the MultiTableEventStreamWriteFunction
+ multiTableWriteFunction.setSchemaEvolutionClient(schemaEvolutionClient);
+
+ // Register this sink subtask with the SchemaOperator
+ int subtaskIndex = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask();
+ try {
+ schemaEvolutionClient.registerSubtask(subtaskIndex);
+ LOG.info(
+ "Registered sink subtask {} with SchemaOperator {}",
+ subtaskIndex,
+ schemaOperatorUid);
+ } catch (Exception e) {
+ LOG.warn(
+ "Failed to register subtask with SchemaOperator, but continuing: {}",
+ e.getMessage());
+ }
+ }
+
+ /**
+ * Creates a MultiTableWriteOperatorFactory for multi-table Hudi write operations. This factory
+ * uses our extended StreamWriteOperatorCoordinator for multi-table support.
+ *
+ * @param conf Configuration for the operator
+ * @return MultiTableWriteOperatorFactory instance configured for multi-table support
+ */
+ public static MultiTableWriteOperatorFactory getFactory(
+ Configuration conf, String schemaOperatorUid) {
+ LOG.info("Creating multi-table write operator factory with extended coordinator support");
+ return MultiTableWriteOperatorFactory.instance(
+ conf, new MultiTableWriteOperator(conf, schemaOperatorUid));
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/operator/MultiTableWriteOperatorFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/operator/MultiTableWriteOperatorFactory.java
new file mode 100644
index 00000000000..93973bdf0c1
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/operator/MultiTableWriteOperatorFactory.java
@@ -0,0 +1,107 @@
+/*
+ * 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.hudi.sink.operator;
+
+import org.apache.flink.cdc.connectors.hudi.sink.coordinator.MultiTableStreamWriteOperatorCoordinator;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEventDispatcher;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.SimpleUdfStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.table.data.RowData;
+
+import org.apache.hudi.sink.common.AbstractWriteOperator;
+import org.apache.hudi.sink.event.Correspondent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Custom WriteOperatorFactory that creates our extended multi-table StreamWriteOperatorCoordinator
+ * instead of Hudi's original single-table coordinator.
+ *
+ * This factory ensures that multi-table CDC scenarios use the proper coordinator with:
+ *
+ *
+ * - Per-table client management
+ *
- Per-table instant tracking
+ *
- Dynamic table registration
+ *
- Table-aware event routing
+ *
+ *
+ * @param The input type for the write operator
+ */
+public class MultiTableWriteOperatorFactory extends SimpleUdfStreamOperatorFactory
+ implements CoordinatedOperatorFactory, OneInputStreamOperatorFactory {
+
+ private static final Logger LOG = LoggerFactory.getLogger(MultiTableWriteOperatorFactory.class);
+ private static final long serialVersionUID = 1L;
+
+ private final Configuration conf;
+ private final AbstractWriteOperator writeOperator;
+
+ public MultiTableWriteOperatorFactory(
+ Configuration conf, AbstractWriteOperator writeOperator) {
+ super(writeOperator);
+ this.conf = conf;
+ this.writeOperator = writeOperator;
+ }
+
+ public static MultiTableWriteOperatorFactory instance(
+ Configuration conf, AbstractWriteOperator writeOperator) {
+ return new MultiTableWriteOperatorFactory<>(conf, writeOperator);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public > T createStreamOperator(
+ StreamOperatorParameters parameters) {
+ LOG.info("Creating multi-table write operator with extended coordinator support");
+
+ // necessary setting for the operator.
+ super.createStreamOperator(parameters);
+
+ final OperatorID operatorID = parameters.getStreamConfig().getOperatorID();
+ final OperatorEventDispatcher eventDispatcher = parameters.getOperatorEventDispatcher();
+
+ this.writeOperator.setCorrespondent(
+ Correspondent.getInstance(
+ operatorID,
+ parameters
+ .getContainingTask()
+ .getEnvironment()
+ .getOperatorCoordinatorEventGateway()));
+ this.writeOperator.setOperatorEventGateway(
+ eventDispatcher.getOperatorEventGateway(operatorID));
+ eventDispatcher.registerEventHandler(operatorID, writeOperator);
+ return (T) writeOperator;
+ }
+
+ @Override
+ public OperatorCoordinator.Provider getCoordinatorProvider(
+ String operatorName, OperatorID operatorID) {
+ LOG.info(
+ "Creating multi-table StreamWriteOperatorCoordinator provider for operator: {}",
+ operatorName);
+ return new MultiTableStreamWriteOperatorCoordinator.Provider(operatorID, conf);
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/util/RowDataUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/util/RowDataUtils.java
new file mode 100644
index 00000000000..336d584249a
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/util/RowDataUtils.java
@@ -0,0 +1,604 @@
+/*
+ * 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.hudi.sink.util;
+
+import org.apache.flink.cdc.common.data.DecimalData;
+import org.apache.flink.cdc.common.data.RecordData;
+import org.apache.flink.cdc.common.data.RecordData.FieldGetter;
+import org.apache.flink.cdc.common.event.ChangeEvent;
+import org.apache.flink.cdc.common.event.DataChangeEvent;
+import org.apache.flink.cdc.common.event.OperationType;
+import org.apache.flink.cdc.common.schema.Column;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.types.DataType;
+import org.apache.flink.cdc.common.types.DataTypeChecks;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.types.RowKind;
+
+import org.apache.hudi.client.model.HoodieFlinkInternalRow;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.sink.bulk.RowDataKeyGen;
+
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.cdc.common.types.DataTypeChecks.getFieldCount;
+import static org.apache.flink.cdc.common.types.DataTypeChecks.getPrecision;
+
+/** Utils for converting {@link RowData} and {@link DataChangeEvent}. */
+public class RowDataUtils {
+
+ /** Convert {@link DataChangeEvent} to {@link RowData}. */
+ public static RowData convertDataChangeEventToRowData(
+ ChangeEvent changeEvent, List fieldGetters) {
+
+ if (!(changeEvent instanceof DataChangeEvent)) {
+ throw new IllegalArgumentException("ChangeEvent must be of type DataChangeEvent");
+ }
+
+ DataChangeEvent dataChangeEvent = (DataChangeEvent) changeEvent;
+
+ RecordData recordData;
+ RowKind kind;
+ switch (dataChangeEvent.op()) {
+ case INSERT:
+ case UPDATE:
+ case REPLACE:
+ {
+ recordData = dataChangeEvent.after();
+ kind = RowKind.INSERT;
+ break;
+ }
+ case DELETE:
+ {
+ recordData = dataChangeEvent.before();
+ kind = RowKind.DELETE;
+ break;
+ }
+ default:
+ throw new IllegalArgumentException("don't support type of " + dataChangeEvent.op());
+ }
+ GenericRowData genericRowData = new GenericRowData(recordData.getArity());
+ genericRowData.setRowKind(kind);
+ for (int i = 0; i < recordData.getArity(); i++) {
+ genericRowData.setField(i, fieldGetters.get(i).getFieldOrNull(recordData));
+ }
+ return genericRowData;
+ }
+
+ public static List createFieldGetters(Schema schema, ZoneId zoneId) {
+ List columns = schema.getColumns();
+ List fieldGetters = new ArrayList<>(columns.size());
+ for (int i = 0; i < columns.size(); i++) {
+ fieldGetters.add(createFieldGetter(columns.get(i).getType(), i, zoneId));
+ }
+ return fieldGetters;
+ }
+
+ /** Create a {@link FieldGetter} for the given {@link DataType}. */
+ public static FieldGetter createFieldGetter(DataType fieldType, int fieldPos, ZoneId zoneId) {
+ final FieldGetter fieldGetter;
+ // ordered by type root definition
+ switch (fieldType.getTypeRoot()) {
+ case CHAR:
+ case VARCHAR:
+ fieldGetter =
+ row ->
+ org.apache.flink.table.data.StringData.fromString(
+ row.getString(fieldPos).toString());
+ break;
+ case BOOLEAN:
+ fieldGetter = row -> row.getBoolean(fieldPos);
+ break;
+ case BINARY:
+ case VARBINARY:
+ fieldGetter = row -> row.getBinary(fieldPos);
+ break;
+ case DECIMAL:
+ final int decimalScale = DataTypeChecks.getScale(fieldType);
+ int precision = getPrecision(fieldType);
+ fieldGetter =
+ row -> {
+ DecimalData decimalData =
+ row.getDecimal(fieldPos, precision, decimalScale);
+ return org.apache.flink.table.data.DecimalData.fromBigDecimal(
+ decimalData.toBigDecimal(), precision, decimalScale);
+ };
+ break;
+ case TINYINT:
+ fieldGetter = row -> row.getBoolean(fieldPos);
+ break;
+ case SMALLINT:
+ fieldGetter = row -> row.getInt(fieldPos);
+ break;
+ case BIGINT:
+ fieldGetter = row -> row.getLong(fieldPos);
+ break;
+ case FLOAT:
+ fieldGetter = row -> row.getFloat(fieldPos);
+ break;
+ case DOUBLE:
+ fieldGetter = row -> row.getDouble(fieldPos);
+ break;
+ case INTEGER:
+ case DATE:
+ case TIME_WITHOUT_TIME_ZONE:
+ fieldGetter = (row) -> row.getInt(fieldPos);
+ break;
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ fieldGetter =
+ (row) ->
+ TimestampData.fromTimestamp(
+ row.getTimestamp(fieldPos, getPrecision(fieldType))
+ .toTimestamp());
+ break;
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ fieldGetter =
+ row ->
+ TimestampData.fromInstant(
+ row.getLocalZonedTimestampData(
+ fieldPos,
+ DataTypeChecks.getPrecision(fieldType))
+ .toInstant());
+ break;
+ case TIMESTAMP_WITH_TIME_ZONE:
+ fieldGetter =
+ (row) ->
+ TimestampData.fromTimestamp(
+ row.getZonedTimestamp(fieldPos, getPrecision(fieldType))
+ .toTimestamp());
+ break;
+ case ROW:
+ final int rowFieldCount = getFieldCount(fieldType);
+ fieldGetter = row -> row.getRow(fieldPos, rowFieldCount);
+ break;
+ default:
+ throw new IllegalArgumentException(
+ "don't support type of " + fieldType.getTypeRoot());
+ }
+ if (!fieldType.isNullable()) {
+ return fieldGetter;
+ }
+ return row -> {
+ if (row.isNullAt(fieldPos)) {
+ return null;
+ }
+ return fieldGetter.getFieldOrNull(row);
+ };
+ }
+
+ /**
+ * Convert a DataChangeEvent to a HoodieFlinkInternalRow with automatic record key and partition
+ * path extraction using Hudi's RowDataKeyGen. This is the preferred method as it uses Hudi's
+ * built-in key generation logic.
+ *
+ * @param dataChangeEvent The DataChangeEvent to convert
+ * @param schema Schema for the table
+ * @param zoneId Time zone for timestamp conversion
+ * @param keyGen Hudi's RowDataKeyGen for extracting record keys and partition paths
+ * @param fileId The file ID for the record
+ * @param instantTime The instant time for the record
+ * @return HoodieFlinkInternalRow containing the converted data
+ */
+ public static HoodieFlinkInternalRow convertDataChangeEventToHoodieFlinkInternalRow(
+ DataChangeEvent dataChangeEvent,
+ Schema schema,
+ ZoneId zoneId,
+ org.apache.hudi.sink.bulk.RowDataKeyGen keyGen,
+ String fileId,
+ String instantTime) {
+
+ // Convert DataChangeEvent to RowData using existing utility
+ List fieldGetters = createFieldGetters(schema, zoneId);
+ RowData rowData = convertDataChangeEventToRowData(dataChangeEvent, fieldGetters);
+
+ // Use Hudi's RowDataKeyGen to extract record key and partition path
+ String recordKey = keyGen.getRecordKey(rowData);
+ String partitionPath = keyGen.getPartitionPath(rowData);
+
+ // Map CDC operation to Hudi operation type
+ String operationType = mapCdcOperationToHudiOperation(dataChangeEvent.op());
+
+ // Create and return HoodieFlinkInternalRow
+ return new HoodieFlinkInternalRow(
+ recordKey, // Record key
+ partitionPath, // Partition path
+ fileId, // File ID
+ instantTime, // Instant time
+ operationType, // Operation type
+ false, // isIndexRecord
+ rowData // Row data
+ );
+ }
+
+ /** Map CDC operation type to Hudi operation type string. */
+ private static String mapCdcOperationToHudiOperation(OperationType cdcOp) {
+ switch (cdcOp) {
+ case INSERT:
+ return "I";
+ case UPDATE:
+ case REPLACE:
+ return "U";
+ case DELETE:
+ return "D";
+ default:
+ throw new IllegalArgumentException("Unsupported CDC operation: " + cdcOp);
+ }
+ }
+
+ /**
+ * Extract record key from DataChangeEvent based on primary key fields in schema. Public utility
+ * method for use by operators that need to calculate record keys.
+ *
+ * @param dataChangeEvent The DataChangeEvent to extract record key from
+ * @param schema The table schema containing primary key definitions
+ * @return The record key string in format "field1:value1,field2:value2"
+ */
+ public static String extractRecordKeyFromDataChangeEvent(
+ DataChangeEvent dataChangeEvent, Schema schema) {
+ List primaryKeyFields = schema.primaryKeys();
+ if (primaryKeyFields.isEmpty()) {
+ throw new IllegalStateException(
+ "Table " + dataChangeEvent.tableId() + " has no primary keys");
+ }
+
+ // Get the record data to extract from (after for INSERT/UPDATE/REPLACE, before for DELETE)
+ RecordData recordData;
+ switch (dataChangeEvent.op()) {
+ case INSERT:
+ case UPDATE:
+ case REPLACE:
+ recordData = dataChangeEvent.after();
+ break;
+ case DELETE:
+ recordData = dataChangeEvent.before();
+ break;
+ default:
+ throw new IllegalArgumentException(
+ "Unsupported operation: " + dataChangeEvent.op());
+ }
+
+ if (recordData == null) {
+ throw new IllegalStateException(
+ "Record data is null for operation: " + dataChangeEvent.op());
+ }
+
+ List recordKeyPairs = new ArrayList<>(primaryKeyFields.size());
+ for (String primaryKeyField : primaryKeyFields) {
+ int fieldIndex = schema.getColumnNames().indexOf(primaryKeyField);
+ if (fieldIndex == -1) {
+ throw new IllegalStateException(
+ "Primary key field '"
+ + primaryKeyField
+ + "' not found in schema for table "
+ + dataChangeEvent.tableId());
+ }
+
+ Object fieldValue =
+ recordData.isNullAt(fieldIndex)
+ ? null
+ : getFieldValue(
+ recordData,
+ fieldIndex,
+ schema.getColumns().get(fieldIndex).getType());
+
+ if (fieldValue == null) {
+ throw new IllegalStateException(
+ "Primary key field '" + primaryKeyField + "' is null in record");
+ }
+
+ // Format as "fieldName:value" to match BucketAssignOperator format
+ recordKeyPairs.add(primaryKeyField + ":" + fieldValue);
+ }
+
+ return String.join(",", recordKeyPairs);
+ }
+
+ /**
+ * Extract partition path from DataChangeEvent based on partition key fields in schema. Public
+ * utility method for use by operators that need to calculate partition paths.
+ *
+ * If the schema has partition keys defined:
+ *
+ *
+ * - Extracts partition field values from the record data
+ *
- Formats them as "field1=value1/field2=value2" (Hive-style partitioning)
+ *
+ *
+ * If no partition keys are defined, returns empty string (for unpartitioned tables).
+ *
+ * @param dataChangeEvent The DataChangeEvent to extract partition from
+ * @param schema The table schema containing partition key definitions
+ * @return The partition path string (empty string for unpartitioned tables)
+ */
+ public static String extractPartitionPathFromDataChangeEvent(
+ DataChangeEvent dataChangeEvent, Schema schema) {
+ List partitionKeys = schema.partitionKeys();
+ if (partitionKeys == null || partitionKeys.isEmpty()) {
+ // Hudi convention: unpartitioned tables use empty string, not "default"
+ return "";
+ }
+
+ // Get the record data to extract from (after for INSERT/UPDATE/REPLACE, before for DELETE)
+ RecordData recordData;
+ switch (dataChangeEvent.op()) {
+ case INSERT:
+ case UPDATE:
+ case REPLACE:
+ recordData = dataChangeEvent.after();
+ break;
+ case DELETE:
+ recordData = dataChangeEvent.before();
+ break;
+ default:
+ throw new IllegalArgumentException(
+ "Unsupported operation: " + dataChangeEvent.op());
+ }
+
+ if (recordData == null) {
+ throw new IllegalStateException(
+ "Cannot extract partition path: "
+ + dataChangeEvent.op()
+ + " event has null data");
+ }
+
+ // Extract partition values and build partition path
+ List partitionParts = new ArrayList<>(partitionKeys.size());
+ for (String partitionKey : partitionKeys) {
+ int fieldIndex = schema.getColumnNames().indexOf(partitionKey);
+ if (fieldIndex == -1) {
+ throw new IllegalStateException(
+ "Partition key field '"
+ + partitionKey
+ + "' not found in schema for table "
+ + dataChangeEvent.tableId());
+ }
+
+ // Get field value
+ Object fieldValue;
+ if (recordData.isNullAt(fieldIndex)) {
+ // Handle null partition values - use "__HIVE_DEFAULT_PARTITION__" as per Hive
+ // convention
+ fieldValue = "__HIVE_DEFAULT_PARTITION__";
+ } else {
+ // Get the field value based on the field type
+ DataType fieldType = schema.getColumns().get(fieldIndex).getType();
+ fieldValue = getFieldValue(recordData, fieldIndex, fieldType);
+ }
+
+ // Format as "key=value" (Hive-style partitioning)
+ partitionParts.add(partitionKey + "=" + fieldValue);
+ }
+
+ // Join partition parts with "/"
+ return String.join("/", partitionParts);
+ }
+
+ /** Get field value from RecordData based on field type. */
+ private static Object getFieldValue(RecordData recordData, int fieldIndex, DataType fieldType) {
+ switch (fieldType.getTypeRoot()) {
+ case CHAR:
+ case VARCHAR:
+ return recordData.getString(fieldIndex);
+ case BOOLEAN:
+ return recordData.getBoolean(fieldIndex);
+ case BINARY:
+ case VARBINARY:
+ return recordData.getBinary(fieldIndex);
+ case DECIMAL:
+ return recordData.getDecimal(
+ fieldIndex,
+ DataTypeChecks.getPrecision(fieldType),
+ DataTypeChecks.getScale(fieldType));
+ case TINYINT:
+ return recordData.getByte(fieldIndex);
+ case SMALLINT:
+ return recordData.getShort(fieldIndex);
+ case INTEGER:
+ case DATE:
+ case TIME_WITHOUT_TIME_ZONE:
+ return recordData.getInt(fieldIndex);
+ case BIGINT:
+ return recordData.getLong(fieldIndex);
+ case FLOAT:
+ return recordData.getFloat(fieldIndex);
+ case DOUBLE:
+ return recordData.getDouble(fieldIndex);
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ return recordData.getTimestamp(fieldIndex, DataTypeChecks.getPrecision(fieldType));
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ return recordData.getLocalZonedTimestampData(
+ fieldIndex, DataTypeChecks.getPrecision(fieldType));
+ case TIMESTAMP_WITH_TIME_ZONE:
+ return recordData.getZonedTimestamp(
+ fieldIndex, DataTypeChecks.getPrecision(fieldType));
+ case ROW:
+ return recordData.getRow(fieldIndex, DataTypeChecks.getFieldCount(fieldType));
+ default:
+ throw new IllegalArgumentException(
+ "Unsupported field type: " + fieldType.getTypeRoot());
+ }
+ }
+
+ /**
+ * Converts a Flink CDC Schema to a Flink Table RowType.
+ *
+ * @param schema The input org.apache.flink.cdc.common.schema.Schema
+ * @return The corresponding org.apache.flink.table.types.logical.RowType
+ */
+ public static RowType toRowType(Schema schema) {
+ List fields =
+ schema.getColumns().stream()
+ .map(
+ column ->
+ new RowType.RowField(
+ column.getName(), toLogicalType(column.getType())))
+ .collect(Collectors.toList());
+
+ return new RowType(false, fields);
+ }
+
+ /**
+ * Maps a Flink CDC DataType to a Flink Table LogicalType. This method covers a wide range of
+ * common types.
+ *
+ * @param cdcType The CDC data type
+ * @return The corresponding LogicalType
+ */
+ public static LogicalType toLogicalType(DataType cdcType) {
+ // The isNullable property is carried over.
+ boolean isNullable = cdcType.isNullable();
+
+ switch (cdcType.getTypeRoot()) {
+ case CHAR:
+ return new CharType(
+ isNullable,
+ ((org.apache.flink.cdc.common.types.CharType) cdcType).getLength());
+ case VARCHAR:
+ // STRING() in CDC is a VARCHAR with max length.
+ return new VarCharType(
+ isNullable,
+ ((org.apache.flink.cdc.common.types.VarCharType) cdcType).getLength());
+ case BOOLEAN:
+ return new BooleanType(isNullable);
+ case BINARY:
+ return new BinaryType(
+ isNullable,
+ ((org.apache.flink.cdc.common.types.BinaryType) cdcType).getLength());
+ case VARBINARY:
+ // BYTES() in CDC is a VARBINARY with max length.
+ return new VarBinaryType(
+ isNullable,
+ ((org.apache.flink.cdc.common.types.VarBinaryType) cdcType).getLength());
+ case DECIMAL:
+ org.apache.flink.cdc.common.types.DecimalType decimalType =
+ (org.apache.flink.cdc.common.types.DecimalType) cdcType;
+ return new org.apache.flink.table.types.logical.DecimalType(
+ isNullable, decimalType.getPrecision(), decimalType.getScale());
+ case TINYINT:
+ return new TinyIntType(isNullable);
+ case SMALLINT:
+ return new SmallIntType(isNullable);
+ case INTEGER:
+ return new IntType(isNullable);
+ case BIGINT:
+ return new BigIntType(isNullable);
+ case FLOAT:
+ return new FloatType(isNullable);
+ case DOUBLE:
+ return new DoubleType(isNullable);
+ case DATE:
+ return new DateType(isNullable);
+ case TIME_WITHOUT_TIME_ZONE:
+ org.apache.flink.cdc.common.types.TimeType timeType =
+ (org.apache.flink.cdc.common.types.TimeType) cdcType;
+ return new TimeType(isNullable, timeType.getPrecision());
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ org.apache.flink.cdc.common.types.TimestampType timestampType =
+ (org.apache.flink.cdc.common.types.TimestampType) cdcType;
+ return new TimestampType(isNullable, timestampType.getPrecision());
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ org.apache.flink.cdc.common.types.LocalZonedTimestampType ltzTimestampType =
+ (org.apache.flink.cdc.common.types.LocalZonedTimestampType) cdcType;
+ return new LocalZonedTimestampType(isNullable, ltzTimestampType.getPrecision());
+ case ARRAY:
+ org.apache.flink.cdc.common.types.ArrayType arrayType =
+ (org.apache.flink.cdc.common.types.ArrayType) cdcType;
+ return new org.apache.flink.table.types.logical.ArrayType(
+ isNullable, toLogicalType(arrayType.getElementType()));
+ case MAP:
+ org.apache.flink.cdc.common.types.MapType mapType =
+ (org.apache.flink.cdc.common.types.MapType) cdcType;
+ return new org.apache.flink.table.types.logical.MapType(
+ isNullable,
+ toLogicalType(mapType.getKeyType()),
+ toLogicalType(mapType.getValueType()));
+ case ROW:
+ org.apache.flink.cdc.common.types.RowType cdcRowType =
+ (org.apache.flink.cdc.common.types.RowType) cdcType;
+ List fields =
+ cdcRowType.getFields().stream()
+ .map(
+ field ->
+ new RowType.RowField(
+ field.getName(),
+ toLogicalType(field.getType()),
+ field.getDescription()))
+ .collect(Collectors.toList());
+ return new org.apache.flink.table.types.logical.RowType(isNullable, fields);
+ default:
+ throw new UnsupportedOperationException(
+ "Unsupported CDC type: " + cdcType.getTypeRoot());
+ }
+ }
+
+ /**
+ * Create a RowDataKeyGen for a table based on its schema.
+ *
+ * @param schema The table schema
+ * @return RowDataKeyGen configured with record key and partition fields from schema
+ */
+ public static RowDataKeyGen createKeyGen(Schema schema) {
+ Configuration config = new Configuration();
+
+ // Set record key fields from primary keys
+ List primaryKeys = schema.primaryKeys();
+ if (primaryKeys == null || primaryKeys.isEmpty()) {
+ throw new IllegalStateException(
+ "Table schema has no primary keys - cannot create RowDataKeyGen");
+ }
+ config.setString(FlinkOptions.RECORD_KEY_FIELD, String.join(",", primaryKeys));
+
+ // Set partition path fields from partition keys
+ List partitionKeys = schema.partitionKeys();
+ if (partitionKeys != null && !partitionKeys.isEmpty()) {
+ config.setString(FlinkOptions.PARTITION_PATH_FIELD, String.join(",", partitionKeys));
+ } else {
+ // For unpartitioned tables, use empty string
+ config.setString(FlinkOptions.PARTITION_PATH_FIELD, "");
+ }
+
+ // Convert schema to RowType
+ RowType rowType = toRowType(schema);
+
+ // Create and return RowDataKeyGen using static factory method
+ return RowDataKeyGen.instance(config, rowType);
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/v2/HudiSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/v2/HudiSink.java
new file mode 100644
index 00000000000..753e3af9592
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/v2/HudiSink.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.hudi.sink.v2;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.api.connector.sink2.WriterInitContext;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.connectors.hudi.sink.bucket.BucketAssignOperator;
+import org.apache.flink.cdc.connectors.hudi.sink.bucket.BucketWrapper;
+import org.apache.flink.cdc.connectors.hudi.sink.bucket.FlushEventAlignmentOperator;
+import org.apache.flink.cdc.connectors.hudi.sink.operator.MultiTableWriteOperator;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Collector;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.time.ZoneId;
+
+/** A {@link Sink} implementation for Apache Hudi. */
+public class HudiSink implements Sink, WithPreWriteTopology {
+
+ private static final Logger LOG = LoggerFactory.getLogger(HudiSink.class);
+
+ private final Configuration conf;
+
+ private final String schemaOperatorUid;
+
+ public HudiSink(Configuration conf, String schemaOperatorUid, ZoneId zoneId) {
+ LOG.info("Creating Hoodie sink with conf: {}", conf);
+ this.conf = conf;
+ this.schemaOperatorUid = schemaOperatorUid;
+ }
+
+ @Override
+ public SinkWriter createWriter(InitContext context) throws IOException {
+ return DummySinkWriter.INSTANCE;
+ }
+
+ @Override
+ public SinkWriter createWriter(WriterInitContext context) throws IOException {
+ return DummySinkWriter.INSTANCE;
+ }
+
+ @Override
+ public DataStream addPreWriteTopology(DataStream dataStream) {
+ LOG.info("Building Hudi pre-write topology with bucket assignment and partitioning");
+
+ // Step 1: Bucket assignment operator
+ // - Calculates bucket for DataChangeEvents
+ // - Broadcasts schema events to all tasks
+ // - Wraps events in BucketWrapper for downstream partitioning
+ DataStream bucketAssignedStream =
+ dataStream
+ .transform(
+ "bucket_assign",
+ TypeInformation.of(BucketWrapper.class),
+ new BucketAssignOperator(conf, schemaOperatorUid))
+ .uid("bucket_assign");
+
+ // Step 2: Partition by bucket index
+ // - Routes events to tasks based on bucket index
+ // - Schema events are broadcast (sent to all bucket indices)
+ // - Data events go to their specific bucket's task
+ DataStream partitionedStream =
+ bucketAssignedStream.partitionCustom(
+ (key, numPartitions) -> key % numPartitions,
+ (KeySelector) BucketWrapper::getBucket);
+
+ // Step 3: Flush event alignment
+ // - Aligns FlushEvents from multiple BucketAssignOperator instances
+ // - Ensures each writer receives only one FlushEvent per source
+ DataStream alignedStream =
+ partitionedStream
+ .transform(
+ "flush_event_alignment",
+ TypeInformation.of(BucketWrapper.class),
+ new FlushEventAlignmentOperator())
+ .uid("flush_event_alignment");
+
+ // Step 4: Unwrap and write to Hudi
+ // Use map to unwrap BucketWrapper before passing to MultiTableWriteOperator
+ DataStream unwrappedStream =
+ alignedStream.map(wrapper -> wrapper.getEvent(), TypeInformation.of(Event.class));
+
+ return unwrappedStream
+ .transform(
+ "multi_table_write",
+ TypeInformation.of(RowData.class),
+ MultiTableWriteOperator.getFactory(conf, schemaOperatorUid))
+ .uid("multi_table_write")
+ .flatMap(
+ (RowData rowData, Collector out) -> {
+ // Write side effects are handled by the operator, no events emitted
+ // downstream
+ })
+ .returns(TypeInformation.of(Event.class));
+ }
+
+ /** Dummy sink writer that does nothing. */
+ private static class DummySinkWriter implements SinkWriter {
+ private static final SinkWriter INSTANCE = new DummySinkWriter();
+
+ @Override
+ public void write(Event element, Context context) {
+ // do nothing
+ }
+
+ @Override
+ public void flush(boolean endOfInput) {
+ // do nothing
+ }
+
+ @Override
+ public void close() {
+ // do nothing
+ }
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/v2/OperatorIDGenerator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/v2/OperatorIDGenerator.java
new file mode 100644
index 00000000000..c00aa751608
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/v2/OperatorIDGenerator.java
@@ -0,0 +1,44 @@
+/*
+ * 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.hudi.sink.v2;
+
+import org.apache.flink.runtime.jobgraph.OperatorID;
+
+import org.apache.flink.shaded.guava31.com.google.common.hash.Hashing;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/** Generator for creating deterministic OperatorIDs from UIDs. */
+public class OperatorIDGenerator {
+
+ private final String transformationUid;
+
+ public OperatorIDGenerator(String transformationUid) {
+ this.transformationUid = transformationUid;
+ }
+
+ public OperatorID generate() {
+ byte[] hash =
+ Hashing.murmur3_128(0)
+ .newHasher()
+ .putString(transformationUid, UTF_8)
+ .hash()
+ .asBytes();
+ return new OperatorID(hash);
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory
new file mode 100644
index 00000000000..c0b308d0b68
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.flink.cdc.connectors.hudi.sink.HudiDataSinkFactory
diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml
index 73869eec91a..725db6dbe2d 100644
--- a/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml
+++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml
@@ -40,6 +40,7 @@ limitations under the License.
flink-cdc-pipeline-connector-maxcompute
flink-cdc-pipeline-connector-iceberg
flink-cdc-pipeline-connector-fluss
+ flink-cdc-pipeline-connector-hudi
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 0e905e5af78..ff3478cb471 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
@@ -43,6 +43,7 @@ limitations under the License.
1.6.1
2.3.9
0.7.0
+ 1.1.0
@@ -138,6 +139,13 @@ limitations under the License.
test-jar
test
+
+ org.apache.flink
+ flink-cdc-pipeline-connector-hudi
+ ${project.version}
+ test-jar
+ test
+
org.apache.flink
flink-cdc-pipeline-connector-kafka
@@ -621,6 +629,42 @@ limitations under the License.
${project.build.directory}/dependencies
+
+ org.apache.flink
+ flink-cdc-pipeline-connector-hudi
+ ${project.version}
+ hudi-cdc-pipeline-connector.jar
+ jar
+ ${project.build.directory}/dependencies
+
+
+
+ org.apache.hudi
+ hudi-flink1.20-bundle
+ ${hudi.version}
+ hudi-sql-connector.jar
+ jar
+ ${project.build.directory}/dependencies
+
+
+
+ org.apache.hudi
+ hudi-hadoop-common
+ ${hudi.version}
+ hudi-hadoop-common.jar
+ jar
+ ${project.build.directory}/dependencies
+
+
+
+ org.apache.flink
+ flink-parquet
+ ${flink.version}
+ flink-parquet.jar
+ jar
+ ${project.build.directory}/dependencies
+
+
org.apache.flink
flink-shaded-hadoop-2-uber
@@ -630,6 +674,24 @@ limitations under the License.
${project.build.directory}/dependencies
+
+ org.apache.flink
+ flink-hadoop-compatibility_2.12
+ ${flink.version}
+ flink-hadoop-compatibility.jar
+ jar
+ ${project.build.directory}/dependencies
+
+
+
+ org.apache.flink
+ flink-metrics-dropwizard
+ ${flink.version}
+ flink-metrics-dropwizard.jar
+ jar
+ ${project.build.directory}/dependencies
+
+
org.apache.paimon
paimon-flink-${flink-major-1.19}
diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToHudiE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToHudiE2eITCase.java
new file mode 100644
index 00000000000..d24eff0a639
--- /dev/null
+++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToHudiE2eITCase.java
@@ -0,0 +1,894 @@
+/*
+ * 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.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.cdc.common.test.utils.TestUtils;
+import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase;
+import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment;
+import org.apache.flink.cdc.pipeline.tests.utils.TarballFetcher;
+import org.apache.flink.client.program.rest.RestClusterClient;
+import org.apache.flink.core.execution.SavepointFormatType;
+import org.apache.flink.runtime.client.JobStatusMessage;
+import org.apache.flink.table.api.ValidationException;
+
+import org.apache.hudi.common.model.HoodieTableType;
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.BindMode;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.FixedHostPortGenericContainer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.output.ToStringConsumer;
+import org.testcontainers.images.builder.Transferable;
+import org.testcontainers.lifecycle.Startables;
+import org.testcontainers.utility.MountableFile;
+
+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.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/** End-to-end tests for mysql cdc to Iceberg pipeline job. */
+public class MySqlToHudiE2eITCase extends PipelineTestEnvironment {
+
+ private static final Logger LOG = LoggerFactory.getLogger(MySqlToHudiE2eITCase.class);
+
+ private static final Duration HUDI_TESTCASE_TIMEOUT = Duration.ofMinutes(20);
+
+ private static final String FLINK_LIB_DIR = "/opt/flink/lib";
+
+ private static final String PEEK_SQL_FILE = "peek-hudi.sql";
+
+ private static final String TABLE_TYPE = HoodieTableType.MERGE_ON_READ.name();
+
+ // Custom Flink properties for Hudi tests with increased metaspace and heap for heavy
+ // dependencies
+ private static final String HUDI_FLINK_PROPERTIES =
+ FLINK_PROPERTIES
+ + "\n"
+ + "taskmanager.memory.jvm-metaspace.size: 512M"
+ + "\n"
+ + "taskmanager.memory.task.heap.size: 1024M"
+ + "\n"
+ + "taskmanager.memory.process.size: 4GB";
+
+ protected final UniqueDatabase inventoryDatabase =
+ new UniqueDatabase(MYSQL, "hudi_inventory", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD);
+
+ private String warehouse;
+
+ private final boolean debug = false;
+
+ @BeforeAll
+ public static void initializeContainers() {
+ LOG.info("Starting containers...");
+ Startables.deepStart(Stream.of(MYSQL)).join();
+ LOG.info("Containers are started.");
+ }
+
+ /*
+ * The Flink SQL Client requires certain core dependencies, like Hadoop's FileSystem,
+ * on its main classpath (`/lib`) to be discovered correctly by the ServiceLoader.
+ * Adding them as temporary session JARs via the `--jar` flag is unreliable for these
+ * low-level services.
+ *
+ * By copying these dependencies directly into the container's `/opt/flink/lib`
+ * directory, we ensure they are loaded by Flink's main classloader, which
+ * permanently resolves the `No FileSystem for scheme: file` error during validation.
+ */
+ @BeforeEach
+ @Override
+ public void before() throws Exception {
+ LOG.info("Starting containers...");
+
+ // 2. Instantiate the correct class and apply class-specific methods
+ if (debug) {
+ // Use FixedHost instead of GenericContainer to ensure that ports are fixed for easier
+ // debugging during dev
+ jobManager =
+ new FixedHostPortGenericContainer<>(getFlinkDockerImageTag())
+ .withFixedExposedPort(8081, JOB_MANAGER_REST_PORT)
+ .withFixedExposedPort(9005, 9005)
+ .withEnv(
+ "FLINK_ENV_JAVA_OPTS",
+ "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:9005");
+ taskManager =
+ new FixedHostPortGenericContainer<>(getFlinkDockerImageTag())
+ .withFixedExposedPort(9006, 9006)
+ .withEnv(
+ "FLINK_ENV_JAVA_OPTS",
+ "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:9006");
+ } else {
+ jobManager =
+ new GenericContainer<>(getFlinkDockerImageTag())
+ // Expose ports for random mapping by Docker
+ .withExposedPorts(JOB_MANAGER_REST_PORT);
+ taskManager = new FixedHostPortGenericContainer<>(getFlinkDockerImageTag());
+ }
+
+ jobManagerConsumer = new ToStringConsumer();
+ jobManager
+ .withCommand("jobmanager")
+ .withNetwork(NETWORK)
+ .withNetworkAliases(INTER_CONTAINER_JM_ALIAS)
+ .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES)
+ .withCreateContainerCmdModifier(cmd -> cmd.withVolumes(sharedVolume))
+ .withLogConsumer(jobManagerConsumer);
+ Startables.deepStart(Stream.of(jobManager)).join();
+ runInContainerAsRoot(jobManager, "chmod", "0777", "-R", sharedVolume.toString());
+
+ taskManagerConsumer = new ToStringConsumer();
+ taskManager
+ .withCommand("taskmanager")
+ .withNetwork(NETWORK)
+ .withNetworkAliases(INTER_CONTAINER_TM_ALIAS)
+ .withEnv("FLINK_PROPERTIES", HUDI_FLINK_PROPERTIES)
+ .dependsOn(jobManager)
+ .withVolumesFrom(jobManager, BindMode.READ_WRITE)
+ .withLogConsumer(taskManagerConsumer);
+ Startables.deepStart(Stream.of(taskManager)).join();
+ runInContainerAsRoot(taskManager, "chmod", "0777", "-R", sharedVolume.toString());
+
+ TarballFetcher.fetchLatest(jobManager);
+ LOG.info("CDC executables deployed.");
+
+ inventoryDatabase.createAndInitialize();
+
+ jobManager.copyFileToContainer(
+ MountableFile.forHostPath(TestUtils.getResource(getHudiSQLConnectorResourceName())),
+ FLINK_LIB_DIR + "/" + getHudiSQLConnectorResourceName());
+ jobManager.copyFileToContainer(
+ MountableFile.forHostPath(TestUtils.getResource("flink-shade-hadoop.jar")),
+ FLINK_LIB_DIR + "/flink-shade-hadoop.jar");
+ jobManager.copyFileToContainer(
+ MountableFile.forHostPath(TestUtils.getResource("hudi-hadoop-common.jar")),
+ FLINK_LIB_DIR + "/hudi-hadoop-common.jar");
+ jobManager.copyFileToContainer(
+ MountableFile.forHostPath(TestUtils.getResource("flink-hadoop-compatibility.jar")),
+ FLINK_LIB_DIR + "/flink-hadoop-compatibility.jar");
+ jobManager.copyFileToContainer(
+ MountableFile.forHostPath(TestUtils.getResource("flink-parquet.jar")),
+ FLINK_LIB_DIR + "/flink-parquet.jar");
+ }
+
+ @AfterEach
+ public void after() {
+ try {
+ super.after();
+ inventoryDatabase.dropDatabase();
+ } catch (Exception e) {
+ LOG.error("Failed to clean up resources", e);
+ }
+ }
+
+ @Test
+ public void testSyncWholeDatabase() throws Exception {
+ warehouse = sharedVolume.toString() + "/hudi_warehouse_" + UUID.randomUUID();
+ String database = inventoryDatabase.getDatabaseName();
+
+ LOG.info("Preparing Hudi warehouse directory: {}", warehouse);
+ runInContainerAsRoot(jobManager, "mkdir", "-p", warehouse);
+ runInContainerAsRoot(jobManager, "chmod", "-R", "0777", warehouse);
+
+ String pipelineJob =
+ String.format(
+ "source:\n"
+ + " type: mysql\n"
+ + " hostname: mysql\n"
+ + " port: 3306\n"
+ + " username: %s\n"
+ + " password: %s\n"
+ + " tables: %s.\\.*\n"
+ + " server-id: 5400-5404\n"
+ + " server-time-zone: UTC\n"
+ + "\n"
+ + "sink:\n"
+ + " type: hudi\n"
+ + " path: %s\n"
+ + " hoodie.datasource.write.recordkey.field: id\n"
+ + " hoodie.table.type: "
+ + TABLE_TYPE
+ + " \n"
+ + " hoodie.schema.on.read.enable: true\n"
+ + " write.bucket_assign.tasks: 2\n"
+ + " write.tasks: 2\n"
+ + " compaction.delta_commits: 2\n"
+ + "\n"
+ + "pipeline:\n"
+ + " schema.change.behavior: evolve\n"
+ + " parallelism: %s",
+ MYSQL_TEST_USER,
+ MYSQL_TEST_PASSWORD,
+ database,
+ warehouse,
+ parallelism);
+ Path hudiCdcConnector = TestUtils.getResource("hudi-cdc-pipeline-connector.jar");
+ // Path hudiHadoopCommonJar = TestUtils.getResource("hudi-hadoop-common.jar");
+ Path hadoopJar = TestUtils.getResource("flink-shade-hadoop.jar");
+ Path hadoopCompatibilityJar = TestUtils.getResource("flink-hadoop-compatibility.jar");
+ Path dropMetricsJar = TestUtils.getResource("flink-metrics-dropwizard.jar");
+ Path flinkParquet = TestUtils.getResource("flink-parquet.jar");
+ JobID pipelineJobID =
+ submitPipelineJob(
+ pipelineJob,
+ hudiCdcConnector,
+ hadoopJar,
+ hadoopCompatibilityJar,
+ dropMetricsJar,
+ flinkParquet);
+ waitUntilJobRunning(pipelineJobID, Duration.ofSeconds(60));
+ LOG.info("Pipeline job is running");
+
+ // Validate that source records from RDB have been initialized properly and landed in sink
+ validateSinkResult(warehouse, database, "products", getProductsExpectedSinkResults());
+ validateSinkResult(warehouse, database, "customers", getCustomersExpectedSinkResults());
+
+ // Generate binlogs
+ LOG.info("Begin incremental reading stage.");
+ String mysqlJdbcUrl =
+ String.format(
+ "jdbc:mysql://%s:%s/%s",
+ MYSQL.getHost(), MYSQL.getDatabasePort(), database);
+ List recordsInIncrementalPhase;
+ try (Connection conn =
+ DriverManager.getConnection(
+ mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD);
+ Statement stat = conn.createStatement()) {
+
+ stat.execute(
+ "INSERT INTO products VALUES (default,'Ten','Jukebox',0.2, null, null, null);"); // 110
+ stat.execute("UPDATE products SET description='Fay' WHERE id=106;");
+ stat.execute("UPDATE products SET weight='5.125' WHERE id=107;");
+
+ // modify table schema
+ stat.execute("ALTER TABLE products DROP COLUMN point_c;");
+ stat.execute("DELETE FROM products WHERE id=101;");
+
+ stat.execute(
+ "INSERT INTO products VALUES (default,'Eleven','Kryo',5.18, null, null);"); // 111
+ stat.execute(
+ "INSERT INTO products VALUES (default,'Twelve', 'Lily', 2.14, null, null);"); // 112
+
+ validateSinkResult(
+ warehouse, database, "products", getProductsExpectedAfterDropSinkResults());
+
+ recordsInIncrementalPhase = createChangesAndValidate(stat);
+ } catch (SQLException e) {
+ LOG.error("Update table for CDC failed.", e);
+ throw e;
+ }
+
+ // Build expected results
+ List recordsInSnapshotPhase = getProductsExpectedAfterAddModSinkResults();
+ recordsInSnapshotPhase.addAll(recordsInIncrementalPhase);
+
+ validateSinkResult(warehouse, database, "products", recordsInSnapshotPhase);
+
+ // Verify that compaction was scheduled for at least one table (only for MOR tables)
+ LOG.info("Verifying compaction scheduling for MOR tables...");
+ if (TABLE_TYPE.equals(HoodieTableType.MERGE_ON_READ.name())) {
+ assertCompactionScheduled(warehouse, database, Arrays.asList("products", "customers"));
+ }
+ }
+
+ /**
+ * Executes a series of DDL (Data Definition Language) and DML (Data Manipulation Language)
+ * operations on the {@code products} table to simulate schema evolution and data loading.
+ *
+ * The method performs two primary phases:
+ *
+ *
+ * - Column Addition: It sequentially adds 10 new columns, named {@code point_c_0}
+ * through {@code point_c_9}, each with a {@code VARCHAR(10)} type. After each column is
+ * added, it executes a batch of 1000 {@code INSERT} statements, populating the columns
+ * that exist at that point.
+ *
- Column Modification: After all columns are added, it enters a second phase. In
+ * each of the 10 iterations, it first inserts another 1000 rows and then modifies the
+ * data type of the first new column ({@code point_c_0}), progressively increasing its
+ * size from {@code VARCHAR(10)} to {@code VARCHAR(19)}.
+ *
+ *
+ * Throughout this process, the method constructs and returns a list of strings. Each string
+ * represents the expected data for each inserted row in a comma-separated format, which can be
+ * used for validation.
+ *
+ * @param stat The JDBC {@link Statement} object used to execute the SQL commands.
+ * @return A {@link List} of strings, where each string is a CSV representation of an inserted
+ * row, reflecting the expected state in the database.
+ * @throws SQLException if a database access error occurs or the executed SQL is invalid.
+ */
+ private List createChangesAndValidate(Statement stat) throws SQLException {
+ List result = new ArrayList<>();
+ StringBuilder sqlFields = new StringBuilder();
+
+ // Auto-increment id will start from this
+ int currentId = 113;
+ final int statementBatchCount = 1000;
+
+ // Step 1 - Add Column: Add 10 columns with VARCHAR(10) sequentially
+ for (int addColumnRepeat = 0; addColumnRepeat < 10; addColumnRepeat++) {
+ String addColAlterTableCmd =
+ String.format(
+ "ALTER TABLE products ADD COLUMN point_c_%s VARCHAR(10);",
+ addColumnRepeat);
+ stat.execute(addColAlterTableCmd);
+ LOG.info("Executed: {}", addColAlterTableCmd);
+ sqlFields.append(", '1'");
+ StringBuilder resultFields = new StringBuilder();
+ for (int addedFieldCount = 0; addedFieldCount < 10; addedFieldCount++) {
+ if (addedFieldCount <= addColumnRepeat) {
+ resultFields.append(", 1");
+ } else {
+ resultFields.append(", null");
+ }
+ }
+
+ for (int statementCount = 0; statementCount < statementBatchCount; statementCount++) {
+ stat.addBatch(
+ String.format(
+ "INSERT INTO products VALUES (default,'finally', null, 2.14, null, null %s);",
+ sqlFields));
+ result.add(
+ String.format(
+ "%s, finally, null, 2.14, null, null%s", currentId, resultFields));
+ currentId++;
+ }
+ stat.executeBatch();
+ }
+
+ // Step 2 - Modify type for the columns added in Step 1, increasing the VARCHAR length
+ for (int modifyColumnRepeat = 0; modifyColumnRepeat < 10; modifyColumnRepeat++) {
+ // Perform 1000 inserts as a batch, continuing the ID sequence from Step 1
+ for (int statementCount = 0; statementCount < statementBatchCount; statementCount++) {
+ stat.addBatch(
+ String.format(
+ "INSERT INTO products VALUES (default,'finally', null, 2.14, null, null %s);",
+ sqlFields));
+
+ result.add(
+ String.format(
+ "%s, finally, null, 2.14, null, null%s",
+ currentId, ", 1, 1, 1, 1, 1, 1, 1, 1, 1, 1"));
+ // Continue incrementing the counter for each insert
+ currentId++;
+ }
+ stat.executeBatch();
+
+ String modifyColTypeAlterCmd =
+ String.format(
+ "ALTER TABLE products MODIFY point_c_0 VARCHAR(%s);",
+ 10 + modifyColumnRepeat);
+ stat.execute(modifyColTypeAlterCmd);
+ LOG.info("Executed: {}", modifyColTypeAlterCmd);
+ }
+
+ return result;
+ }
+
+ private List fetchHudiTableRows(String warehouse, String databaseName, String tableName)
+ throws Exception {
+ String template =
+ readLines("docker/" + PEEK_SQL_FILE).stream()
+ .filter(line -> !line.startsWith("--"))
+ .collect(Collectors.joining("\n"));
+ String sql = String.format(template, warehouse, databaseName, tableName);
+ String containerSqlPath = sharedVolume.toString() + "/" + PEEK_SQL_FILE;
+ jobManager.copyFileToContainer(Transferable.of(sql), containerSqlPath);
+ LOG.info("Executing SQL client in container with Hudi connector and Hadoop JARs");
+
+ // Pass in empty FLINK_ENV_JAVA_OPTS so that we do not launch a new JVM (for SQL
+ // submission/parsing) inheriting environment variables which will cause it to bind to the
+ // same debug port, causing the port already in use error
+ String[] commandToExecute = {
+ "bash",
+ "-c",
+ "FLINK_ENV_JAVA_OPTS='' /opt/flink/bin/sql-client.sh"
+ + " --jar "
+ + FLINK_LIB_DIR
+ + "/"
+ + getHudiSQLConnectorResourceName()
+ + " --jar "
+ + FLINK_LIB_DIR
+ + "/flink-shade-hadoop.jar"
+ + " -f "
+ + containerSqlPath
+ };
+ LOG.debug("Executing command: {}", String.join(" ", commandToExecute));
+ Container.ExecResult result = jobManager.execInContainer(commandToExecute);
+
+ LOG.debug("SQL client execution completed with exit code: {}", result.getExitCode());
+ LOG.debug("SQL client stdout: {}", result.getStdout());
+ LOG.debug("SQL client stderr: {}", result.getStderr());
+
+ if (result.getExitCode() != 0) {
+ LOG.error("SQL client execution failed!");
+ LOG.error("Exit code: {}", result.getExitCode());
+ LOG.error("Stdout: {}", result.getStdout());
+ LOG.error("Stderr: {}", result.getStderr());
+ throw new RuntimeException(
+ "Failed to execute Hudi peek script. Exit code: "
+ + result.getExitCode()
+ + ". Stdout: "
+ + result.getStdout()
+ + "; Stderr: "
+ + result.getStderr());
+ }
+
+ return Arrays.stream(result.getStdout().split("\n"))
+ .filter(line -> line.startsWith("|"))
+ .skip(1)
+ .map(MySqlToHudiE2eITCase::extractRow)
+ .map(row -> String.format("%s", String.join(", ", row)))
+ .collect(Collectors.toList());
+ }
+
+ private static String[] extractRow(String row) {
+ return Arrays.stream(row.split("\\|"))
+ .map(String::trim)
+ .filter(col -> !col.isEmpty())
+ .map(col -> col.equals("") ? "null" : col)
+ .toArray(String[]::new);
+ }
+
+ protected String getHudiSQLConnectorResourceName() {
+ return "hudi-sql-connector.jar";
+ }
+
+ @Override
+ public String stopJobWithSavepoint(JobID jobID) {
+ String savepointPath = "/opt/flink/";
+ try {
+ // Use REST API to stop with savepoint to avoid CLI classpath conflicts
+ // (Hadoop/Hudi JARs in FLINK_LIB_DIR conflict with Flink's commons-cli)
+ LOG.info("Stopping job {} with savepoint to {}", jobID, savepointPath);
+
+ String savepointLocation =
+ getRestClusterClient()
+ .stopWithSavepoint(
+ jobID, false, savepointPath, SavepointFormatType.CANONICAL)
+ .get(60, java.util.concurrent.TimeUnit.SECONDS);
+
+ LOG.info("Savepoint completed at: {}", savepointLocation);
+ return savepointLocation;
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to stop job with savepoint", e);
+ }
+ }
+
+ private void validateSinkResult(
+ String warehouse, String database, String table, List expected)
+ throws InterruptedException {
+ LOG.info("Verifying Hudi {}::{}::{} results...", warehouse, database, table);
+ long deadline = System.currentTimeMillis() + HUDI_TESTCASE_TIMEOUT.toMillis();
+ List results = Collections.emptyList();
+ while (System.currentTimeMillis() < deadline) {
+ try {
+ results = fetchHudiTableRows(warehouse, database, table);
+ Assertions.assertThat(results).containsExactlyInAnyOrderElementsOf(expected);
+ LOG.info(
+ "Successfully verified {} records in {} seconds for {}::{}.",
+ expected.size(),
+ (System.currentTimeMillis() - deadline + HUDI_TESTCASE_TIMEOUT.toMillis())
+ / 1000,
+ database,
+ table);
+ return;
+ } catch (Exception e) {
+ LOG.warn("Validate failed, waiting for the next loop...", e);
+ } catch (AssertionError ignored) {
+ // AssertionError contains way too much records and might flood the log output.
+ if (expected.size() == results.size()) {
+ // Size of rows match up, print the contents
+ final int rowsToPrint = 100;
+ LOG.warn(
+ "Result expected: {}, but got {}",
+ expected.stream()
+ .sorted()
+ .limit(rowsToPrint)
+ .collect(Collectors.toList()),
+ results.stream()
+ .sorted()
+ .limit(rowsToPrint)
+ .collect(Collectors.toList()));
+ } else {
+ LOG.warn(
+ "Results mismatch, expected {} records, but got {} actually. Waiting for the next loop...",
+ expected.size(),
+ results.size());
+ }
+ }
+
+ Thread.sleep(10000L);
+ }
+ Assertions.assertThat(results).containsExactlyInAnyOrderElementsOf(expected);
+ }
+
+ @Test
+ public void testStopAndRestartFromSavepoint() throws Exception {
+ warehouse = sharedVolume.toString() + "/hudi_warehouse_savepoint_" + UUID.randomUUID();
+ String database = inventoryDatabase.getDatabaseName();
+
+ LOG.info("Preparing Hudi warehouse directory: {}", warehouse);
+ runInContainerAsRoot(jobManager, "mkdir", "-p", warehouse);
+ runInContainerAsRoot(jobManager, "chmod", "-R", "0777", warehouse);
+
+ // Configure pipeline with checkpointing
+ String pipelineJob =
+ String.format(
+ "source:\n"
+ + " type: mysql\n"
+ + " hostname: mysql\n"
+ + " port: 3306\n"
+ + " username: %s\n"
+ + " password: %s\n"
+ + " tables: %s.\\.*\n"
+ + " server-id: 5600-5604\n"
+ + " server-time-zone: UTC\n"
+ + "\n"
+ + "sink:\n"
+ + " type: hudi\n"
+ + " path: %s\n"
+ + " hoodie.datasource.write.recordkey.field: id\n"
+ + " hoodie.table.type: MERGE_ON_READ\n"
+ + " hoodie.schema.on.read.enable: true\n"
+ + " write.bucket_assign.tasks: 2\n"
+ + " write.tasks: 2\n"
+ + "\n"
+ + "pipeline:\n"
+ + " schema.change.behavior: evolve\n"
+ + " parallelism: %s\n"
+ + "\n",
+ MYSQL_TEST_USER, MYSQL_TEST_PASSWORD, database, warehouse, parallelism);
+
+ Path hudiCdcConnector = TestUtils.getResource("hudi-cdc-pipeline-connector.jar");
+ Path hudiHadoopCommonJar = TestUtils.getResource("hudi-hadoop-common.jar");
+ Path hadoopJar = TestUtils.getResource("flink-shade-hadoop.jar");
+ Path hadoopCompatibilityJar = TestUtils.getResource("flink-hadoop-compatibility.jar");
+ Path dropMetricsJar = TestUtils.getResource("flink-metrics-dropwizard.jar");
+ Path flinkParquet = TestUtils.getResource("flink-parquet.jar");
+
+ // Start the pipeline job
+ LOG.info("Phase 1: Starting initial pipeline job");
+ JobID pipelineJobID1 =
+ submitPipelineJob(
+ pipelineJob,
+ hudiCdcConnector,
+ hudiHadoopCommonJar,
+ hadoopJar,
+ hadoopCompatibilityJar,
+ dropMetricsJar,
+ flinkParquet);
+ waitUntilJobRunning(pipelineJobID1, Duration.ofSeconds(60));
+
+ // Store the jobID of the submitted job, we will need it for stopping the job later
+ Collection jobs =
+ getRestClusterClient().listJobs().get(10, TimeUnit.SECONDS);
+ Assertions.assertThat(jobs).hasSize(1);
+ JobStatusMessage pipelineJobMessage = jobs.iterator().next();
+ LOG.info(
+ "Pipeline job: ID={}, Name={}, Status={}",
+ pipelineJobMessage.getJobId(),
+ pipelineJobMessage.getJobName(),
+ pipelineJobMessage.getJobState());
+
+ // Validate initial snapshot data for both tables
+ validateSinkResult(warehouse, database, "products", getProductsExpectedSinkResults());
+ validateSinkResult(warehouse, database, "customers", getCustomersExpectedSinkResults());
+ LOG.info("Phase 1: Initial snapshot validated successfully");
+
+ // Phase 2: Insert incremental data before stopping
+ String mysqlJdbcUrl =
+ String.format(
+ "jdbc:mysql://%s:%s/%s",
+ MYSQL.getHost(), MYSQL.getDatabasePort(), database);
+ try (Connection conn =
+ DriverManager.getConnection(
+ mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD);
+ Statement stat = conn.createStatement()) {
+
+ stat.execute(
+ "INSERT INTO products VALUES (default,'Pre-Stop Product','Description',1.23, null, null, null);");
+ stat.execute(
+ "INSERT INTO customers VALUES (105, 'user_pre_stop', 'Beijing', '987654321');");
+ LOG.info("Phase 2: Incremental data inserted before stop");
+
+ // Wait for data to be checkpointed
+ Thread.sleep(5000);
+ }
+
+ // Validate data before stopping
+ List expectedProductsBeforeStop = new ArrayList<>(getProductsExpectedSinkResults());
+ expectedProductsBeforeStop.add(
+ "110, Pre-Stop Product, Description, 1.23, null, null, null");
+
+ List expectedCustomersBeforeStop =
+ new ArrayList<>(getCustomersExpectedSinkResults());
+ expectedCustomersBeforeStop.add("105, user_pre_stop, Beijing, 987654321");
+
+ validateSinkResult(warehouse, database, "products", expectedProductsBeforeStop);
+ validateSinkResult(warehouse, database, "customers", expectedCustomersBeforeStop);
+ LOG.info("Phase 2: Data validated before stop");
+
+ // Phase 3: Stop job with savepoint
+ LOG.info("Phase 3: Stopping job with savepoint");
+ Collection runningJobs =
+ getRestClusterClient().listJobs().get(10, TimeUnit.SECONDS).stream()
+ .filter(j -> j.getJobState().equals(JobStatus.RUNNING))
+ .collect(Collectors.toList());
+
+ if (runningJobs.isEmpty()) {
+ throw new RuntimeException("No running jobs found!");
+ }
+
+ String savepointPath = stopJobWithSavepoint(pipelineJobMessage.getJobId());
+ LOG.info("Job stopped with savepoint at: {}", savepointPath);
+
+ // Phase 4: Restart from savepoint
+ LOG.info("Phase 4: Restarting job from savepoint");
+ JobID pipelineJobID2 =
+ submitPipelineJob(
+ pipelineJob,
+ savepointPath,
+ false,
+ hudiCdcConnector,
+ hudiHadoopCommonJar,
+ hadoopJar,
+ hadoopCompatibilityJar,
+ dropMetricsJar,
+ flinkParquet);
+ waitUntilJobRunning(pipelineJobID2, Duration.ofSeconds(60));
+ LOG.info("Job restarted from savepoint");
+
+ // Wait for Hudi to stabilize after restart
+ Thread.sleep(5000);
+
+ // Validate data after restart - should be the same as before stop
+ validateSinkResult(warehouse, database, "products", expectedProductsBeforeStop);
+ validateSinkResult(warehouse, database, "customers", expectedCustomersBeforeStop);
+ LOG.info("Phase 4: Data consistency validated after restart from savepoint");
+
+ // Phase 5: Continue with post-restart data to ensure pipeline still works
+ try (Connection conn =
+ DriverManager.getConnection(
+ mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD);
+ Statement stat = conn.createStatement()) {
+
+ stat.execute(
+ "INSERT INTO products VALUES (default,'Post-Restart Product','New Description',4.56, null, null, null);");
+ stat.execute("UPDATE products SET description='Updated Description' WHERE id=110;");
+ stat.execute(
+ "INSERT INTO customers VALUES (106, 'user_post_restart', 'Guangzhou', '111222333');");
+ stat.execute("DELETE FROM customers WHERE id=101;");
+ LOG.info("Phase 5: Post-restart data changes applied");
+ }
+
+ // Phase 6: Final validation
+ List expectedProductsFinal = new ArrayList<>(expectedProductsBeforeStop);
+ // Update the pre-stop product description
+ expectedProductsFinal.removeIf(row -> row.startsWith("110,"));
+ expectedProductsFinal.add(
+ "110, Pre-Stop Product, Updated Description, 1.23, null, null, null");
+ expectedProductsFinal.add(
+ "111, Post-Restart Product, New Description, 4.56, null, null, null");
+
+ List expectedCustomersFinal = new ArrayList<>(expectedCustomersBeforeStop);
+ // Remove deleted customer
+ expectedCustomersFinal.removeIf(row -> row.startsWith("101,"));
+ expectedCustomersFinal.add("106, user_post_restart, Guangzhou, 111222333");
+
+ validateSinkResult(warehouse, database, "products", expectedProductsFinal);
+ validateSinkResult(warehouse, database, "customers", expectedCustomersFinal);
+ LOG.info(
+ "Phase 6: Final validation successful - stop/restart with savepoint working correctly for multiple tables");
+ }
+
+ private static List getProductsExpectedSinkResults() {
+ return Arrays.asList(
+ "101, One, Alice, 3.202, red, {\"key1\": \"value1\"}, null",
+ "102, Two, Bob, 1.703, white, {\"key2\": \"value2\"}, null",
+ "103, Three, Cecily, 4.105, red, {\"key3\": \"value3\"}, null",
+ "104, Four, Derrida, 1.857, white, {\"key4\": \"value4\"}, null",
+ "105, Five, Evelyn, 5.211, red, {\"K\": \"V\", \"k\": \"v\"}, null",
+ "106, Six, Ferris, 9.813, null, null, null",
+ "107, Seven, Grace, 2.117, null, null, null",
+ "108, Eight, Hesse, 6.819, null, null, null",
+ "109, Nine, IINA, 5.223, null, null, null");
+ }
+
+ private static List getProductsExpectedAfterDropSinkResults() {
+ return Arrays.asList(
+ "102, Two, Bob, 1.703, white, {\"key2\": \"value2\"}",
+ "103, Three, Cecily, 4.105, red, {\"key3\": \"value3\"}",
+ "104, Four, Derrida, 1.857, white, {\"key4\": \"value4\"}",
+ "105, Five, Evelyn, 5.211, red, {\"K\": \"V\", \"k\": \"v\"}",
+ "106, Six, Fay, 9.813, null, null",
+ "107, Seven, Grace, 5.125, null, null",
+ "108, Eight, Hesse, 6.819, null, null",
+ "109, Nine, IINA, 5.223, null, null",
+ "110, Ten, Jukebox, 0.2, null, null",
+ "111, Eleven, Kryo, 5.18, null, null",
+ "112, Twelve, Lily, 2.14, null, null");
+ }
+
+ private static List getProductsExpectedAfterAddModSinkResults() {
+ // We need this list to be mutable, i.e. not fixed sized
+ // Arrays.asList returns a fixed size list which is not mutable
+ return new ArrayList<>(
+ Arrays.asList(
+ "102, Two, Bob, 1.703, white, {\"key2\": \"value2\"}, null, null, null, null, null, null, null, null, null, null",
+ "103, Three, Cecily, 4.105, red, {\"key3\": \"value3\"}, null, null, null, null, null, null, null, null, null, null",
+ "104, Four, Derrida, 1.857, white, {\"key4\": \"value4\"}, null, null, null, null, null, null, null, null, null, null",
+ "105, Five, Evelyn, 5.211, red, {\"K\": \"V\", \"k\": \"v\"}, null, null, null, null, null, null, null, null, null, null",
+ "106, Six, Fay, 9.813, null, null, null, null, null, null, null, null, null, null, null, null",
+ "107, Seven, Grace, 5.125, null, null, null, null, null, null, null, null, null, null, null, null",
+ "108, Eight, Hesse, 6.819, null, null, null, null, null, null, null, null, null, null, null, null",
+ "109, Nine, IINA, 5.223, null, null, null, null, null, null, null, null, null, null, null, null",
+ "110, Ten, Jukebox, 0.2, null, null, null, null, null, null, null, null, null, null, null, null",
+ "111, Eleven, Kryo, 5.18, null, null, null, null, null, null, null, null, null, null, null, null",
+ "112, Twelve, Lily, 2.14, null, null, null, null, null, null, null, null, null, null, null, null"));
+ }
+
+ private static List getCustomersExpectedSinkResults() {
+ return Arrays.asList(
+ "101, user_1, Shanghai, 123567891234",
+ "102, user_2, Shanghai, 123567891234",
+ "103, user_3, Shanghai, 123567891234",
+ "104, user_4, Shanghai, 123567891234");
+ }
+
+ public void waitUntilJobRunning(JobID jobId, Duration timeout) {
+ waitUntilJobState(jobId, timeout, JobStatus.RUNNING);
+ }
+
+ public void waitUntilJobFinished(JobID jobId, Duration timeout) {
+ waitUntilJobState(jobId, timeout, JobStatus.FINISHED);
+ }
+
+ public void waitUntilJobState(JobID jobId, Duration timeout, JobStatus expectedStatus) {
+ RestClusterClient> clusterClient = getRestClusterClient();
+ Deadline deadline = Deadline.fromNow(timeout);
+ while (deadline.hasTimeLeft()) {
+ Collection jobStatusMessages;
+ try {
+ jobStatusMessages = clusterClient.listJobs().get(10, TimeUnit.SECONDS);
+ } catch (Exception e) {
+ LOG.warn("Error when fetching job status.", e);
+ continue;
+ }
+
+ if (jobStatusMessages == null || jobStatusMessages.isEmpty()) {
+ continue;
+ }
+
+ Optional optMessage =
+ jobStatusMessages.stream().filter(j -> j.getJobId().equals(jobId)).findFirst();
+
+ if (!optMessage.isPresent()) {
+ LOG.warn("Job: {} not found, waiting for the next loop...", jobId);
+ continue;
+ }
+
+ JobStatusMessage message = optMessage.get();
+ JobStatus jobStatus = message.getJobState();
+ if (!expectedStatus.isTerminalState() && jobStatus.isTerminalState()) {
+ throw new ValidationException(
+ String.format(
+ "Job has been terminated! JobName: %s, JobID: %s, Status: %s",
+ message.getJobName(), message.getJobId(), message.getJobState()));
+ } else if (jobStatus == expectedStatus) {
+ return;
+ }
+ }
+ }
+
+ /**
+ * Asserts that compaction was scheduled for the given tables by checking for
+ * .compaction.requested files in the Hudi timeline directory inside the container.
+ *
+ * Should only be invoked for MERGE_ON_READ tables.
+ *
+ * @param warehouse The warehouse directory path
+ * @param database The database name
+ * @param tables List of table names to check
+ */
+ private void assertCompactionScheduled(String warehouse, String database, List tables)
+ throws Exception {
+ boolean compactionFound = false;
+ StringBuilder debugInfo = new StringBuilder();
+
+ for (String table : tables) {
+ // This will exclude metadata table timeline results
+ String timelinePath =
+ String.format("%s/%s/%s/.hoodie/timeline", warehouse, database, table);
+ debugInfo.append(
+ String.format(
+ "\nChecking timeline for %s.%s at: %s", database, table, timelinePath));
+
+ // Check if timeline directory exists in container
+ Container.ExecResult lsResult = jobManager.execInContainer("ls", "-la", timelinePath);
+ if (lsResult.getExitCode() != 0) {
+ debugInfo.append(
+ String.format(
+ " - Timeline directory does not exist or cannot be accessed: %s",
+ lsResult.getStderr()));
+ continue;
+ }
+
+ // Find .compaction.requested files
+ Container.ExecResult findResult =
+ jobManager.execInContainer(
+ "find", timelinePath, "-name", "*.compaction.requested");
+
+ if (findResult.getExitCode() == 0 && !findResult.getStdout().trim().isEmpty()) {
+ compactionFound = true;
+ String[] compactionFiles = findResult.getStdout().trim().split("\n");
+ debugInfo.append(
+ String.format(
+ " - Found %d compaction file(s): %s",
+ compactionFiles.length, Arrays.toString(compactionFiles)));
+ LOG.info(
+ "Compaction scheduled for table {}.{}: {}",
+ database,
+ table,
+ Arrays.toString(compactionFiles));
+ } else {
+ debugInfo.append(" - No compaction.requested files found");
+
+ // List all timeline files for debugging
+ Container.ExecResult allFilesResult =
+ jobManager.execInContainer("ls", "-1", timelinePath);
+ if (allFilesResult.getExitCode() == 0) {
+ debugInfo.append(
+ String.format(
+ "\n All timeline files: %s",
+ allFilesResult.getStdout().replace("\n", ", ")));
+ }
+ }
+ }
+
+ if (!compactionFound) {
+ LOG.error("Compaction verification failed. Debug info:{}", debugInfo);
+ Assertions.fail(
+ "No compaction.requested files found in any table timeline. "
+ + "Expected at least one compaction to be scheduled."
+ + debugInfo);
+ } else {
+ LOG.info("Compaction verification successful!");
+ }
+ }
+}
diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/hudi_inventory.sql b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/hudi_inventory.sql
new file mode 100644
index 00000000000..82dc6fd097e
--- /dev/null
+++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/hudi_inventory.sql
@@ -0,0 +1,55 @@
+-- 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: mysql_inventory
+-- ----------------------------------------------------------------------------------------------------------------
+
+-- Create and populate our products using a single insert with many rows
+CREATE TABLE products (
+ id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
+ name VARCHAR(255) NOT NULL DEFAULT 'flink',
+ description VARCHAR(512),
+ weight FLOAT,
+ enum_c enum('red', 'white') default 'red', -- test some complex types as well,
+ json_c JSON, -- because we use additional dependencies to deserialize complex types.
+ point_c POINT
+);
+ALTER TABLE products AUTO_INCREMENT = 101;
+
+INSERT INTO products
+VALUES (default,"One", "Alice", 3.202, 'red', '{"key1": "value1"}', null),
+ (default,"Two", "Bob", 1.703, 'white', '{"key2": "value2"}', null),
+ (default,"Three", "Cecily", 4.105, 'red', '{"key3": "value3"}', null),
+ (default,"Four", "Derrida", 1.857, 'white', '{"key4": "value4"}', null),
+ (default,"Five", "Evelyn", 5.211, 'red', '{"K": "V", "k": "v"}', null),
+ (default,"Six", "Ferris", 9.813, null, null, null),
+ (default,"Seven", "Grace", 2.117, null, null, null),
+ (default,"Eight", "Hesse", 6.819, null, null, null),
+ (default,"Nine", "IINA", 5.223, null, null, null);
+
+-- Create and populate our customers 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"),
+ (104,"user_4","Shanghai","123567891234");
diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/docker/peek-hudi.sql b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/docker/peek-hudi.sql
new file mode 100644
index 00000000000..092ac66e4aa
--- /dev/null
+++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/docker/peek-hudi.sql
@@ -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.
+
+-- Format this file with the following arguments:
+-- Warehouse Path, Database Name, and Table Name.
+
+SET 'sql-client.execution.result-mode' = 'tableau';
+SET 'table.display.max-column-width' = '100000';
+SET 'execution.runtime-mode' = 'batch';
+
+CREATE CATALOG hoodie_catalog WITH (
+ 'type' = 'hudi',
+ 'catalog.path' = '%s',
+ 'mode' = 'dfs'
+);
+
+USE CATALOG hoodie_catalog;
+
+SELECT * FROM %s.%s;
\ No newline at end of file