From aaa4c6074dee32172eb50bb85d591a786182e00a Mon Sep 17 00:00:00 2001 From: voon Date: Fri, 8 Aug 2025 15:51:12 +0800 Subject: [PATCH 01/33] Flink-CDC checkin 2 --- .idea/vcs.xml | 4 +- .../flink-cdc-pipeline-connector-hudi/pom.xml | 385 +++++++++ .../cdc/connectors/hudi/sink/HudiConfig.java | 140 ++++ .../connectors/hudi/sink/HudiDataSink.java | 90 +++ .../hudi/sink/HudiDataSinkFactory.java | 76 ++ .../hudi/sink/HudiDataSinkOptions.java | 26 + .../hudi/sink/HudiMetadataApplier.java | 707 +++++++++++++++++ .../sink/bucket/BucketAssignOperator.java | 256 ++++++ .../hudi/sink/bucket/BucketWrapper.java | 50 ++ .../sink/bucket/BucketWrapperFlushEvent.java | 84 ++ .../bucket/FlushEventAlignmentOperator.java | 96 +++ ...tiTableStreamWriteOperatorCoordinator.java | 734 ++++++++++++++++++ .../sink/event/CreateTableOperatorEvent.java | 59 ++ .../event/EnhancedWriteMetadataEvent.java | 83 ++ .../sink/event/HudiRecordEventSerializer.java | 228 ++++++ .../hudi/sink/event/HudiRecordSerializer.java | 48 ++ .../sink/event/TableAwareCorrespondent.java | 85 ++ .../EventBucketStreamWriteFunction.java | 337 ++++++++ .../sink/function/EventProcessorFunction.java | 64 ++ .../function/EventStreamWriteFunction.java | 702 +++++++++++++++++ .../MultiTableEventStreamWriteFunction.java | 590 ++++++++++++++ .../sink/model/BucketAssignmentIndex.java | 82 ++ .../operator/MultiTableWriteOperator.java | 161 ++++ .../MultiTableWriteOperatorFactory.java | 103 +++ .../hudi/sink/util/RowDataUtils.java | 472 +++++++++++ .../cdc/connectors/hudi/sink/v2/HudiSink.java | 136 ++++ .../hudi/sink/v2/OperatorIDGenerator.java | 43 + ....apache.flink.cdc.common.factories.Factory | 16 + .../flink-cdc-pipeline-connectors/pom.xml | 1 + .../flink-cdc-pipeline-e2e-tests/pom.xml | 62 ++ .../pipeline/tests/MySqlToHudiE2eITCase.java | 517 ++++++++++++ .../tests/MySqlToIcebergE2eITCase.java | 3 +- .../tests/utils/PipelineTestEnvironment.java | 21 +- .../src/test/resources/ddl/hudi_inventory.sql | 55 ++ .../src/test/resources/docker/peek-hudi.sql | 31 + pom.xml | 8 + 36 files changed, 6545 insertions(+), 10 deletions(-) create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiConfig.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSink.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactory.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkOptions.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiMetadataApplier.java create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/CreateTableOperatorEvent.java create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventBucketStreamWriteFunction.java create mode 100644 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 create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventStreamWriteFunction.java create mode 100644 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 create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/model/BucketAssignmentIndex.java create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToHudiE2eITCase.java create mode 100644 flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/hudi_inventory.sql create mode 100644 flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/docker/peek-hudi.sql diff --git a/.idea/vcs.xml b/.idea/vcs.xml index 818d136a83f..08d52800435 100644 --- a/.idea/vcs.xml +++ b/.idea/vcs.xml @@ -20,6 +20,8 @@ + + - + \ No newline at end of file diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml new file mode 100644 index 00000000000..a8926185213 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml @@ -0,0 +1,385 @@ + + + + 4.0.0 + + flink-cdc-pipeline-connectors + org.apache.flink + ${revision} + + + flink-cdc-pipeline-connector-hudi + jar + + flink-cdc-pipeline-connector-hudi + + + 1.1.0-SNAPSHOT + 2.10.2 + 3.4.6 + 4.1.1 + 2.9.1 + 9.4.57.v20241219 + 4.6.7 + 1.11.4 + 4.4.16 + 4.5.14 + + + + + org.apache.hudi + hudi-common + ${hudi.version} + + + + org.apache.hudi + hudi-flink + ${hudi.version} + + + + org.apache.hudi + hudi-io + ${hudi.version} + + + + org.apache.hudi + hudi-client-common + ${hudi.version} + + + + org.apache.hudi + hudi-flink-client + ${hudi.version} + + + + org.apache.hudi + hudi-hive-sync + ${hudi.version} + + + + org.apache.hudi + hudi-sync-common + ${hudi.version} + + + + org.apache.hudi + hudi-timeline-service + ${hudi.version} + + + + org.apache.hudi + hudi-flink1.20.x + ${hudi.version} + + + + org.apache.hudi + hudi-hadoop-common + ${hudi.version} + provided + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + provided + + + + org.apache.flink + flink-hadoop-compatibility_2.12 + ${flink.version} + provided + + + + org.apache.flink + flink-table-runtime + ${flink.version} + provided + + + + org.apache.flink + flink-parquet + ${flink.version} + provided + + + + + org.apache.parquet + parquet-avro + 1.13.1 + + + + + io.dropwizard.metrics + metrics-core + ${metrics.version} + + + + + com.github.ben-manes.caffeine + caffeine + ${caffeine.version} + + + + + org.eclipse.jetty + jetty-util + ${jetty.version} + + + + + io.javalin + javalin + ${javalin.version} + + + + org.eclipse.jetty + jetty-server + ${jetty.version} + + + + + org.apache.avro + avro + ${avro.version} + + + + + com.fasterxml.jackson.module + jackson-module-afterburner + + + + + org.mockito + mockito-inline + ${mockito.version} + test + + + org.mockito + mockito-core + ${mockito.version} + test + + + org.mockito + mockito-junit-jupiter + ${mockito.version} + test + + + + + com.esotericsoftware + kryo-shaded + 4.0.2 + compile + + + + + org.apache.httpcomponents + fluent-hc + ${httpclient.version} + + + org.apache.httpcomponents + httpcore + ${httpcore.version} + + + org.apache.httpcomponents + httpclient + ${httpclient.version} + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + test-jar + + test-jar + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.2.4 + + + package + + shade + + + false + + + org.apache.hudi:* + + + com.esotericsoftware:kryo-shaded + com.esotericsoftware:minlog + org.objenesis:objenesis + + com.yammer.metrics:metrics-core + com.beust:jcommander + io.javalin:javalin + org.jetbrains.kotlin:* + org.rocksdb:rocksdbjni + org.antlr:stringtemplate + + org.apache.parquet:parquet-avro + org.apache.parquet:parquet-hadoop + org.apache.parquet:parquet-column + org.apache.parquet:parquet-common + org.apache.parquet:parquet-format-structures + org.apache.parquet:parquet-encoding + org.apache.parquet:parquet-jackson + + org.apache.avro:avro + joda-time:joda-time + com.fasterxml.jackson.core:jackson-annotations + com.fasterxml.jackson.core:jackson-core + com.fasterxml.jackson.core:jackson-databind + com.fasterxml.jackson.datatype:jackson-datatype-jsr310 + + com.lmax:disruptor + com.github.davidmoten:guava-mini + com.github.davidmoten:hilbert-curve + com.github.ben-manes.caffeine:caffeine + com.twitter:bijection-avro_${scala.binary.version} + com.twitter:bijection-core_${scala.binary.version} + io.dropwizard.metrics:metrics-core + io.dropwizard.metrics:metrics-graphite + io.dropwizard.metrics:metrics-jmx + io.prometheus:simpleclient + io.prometheus:simpleclient_httpserver + io.prometheus:simpleclient_dropwizard + io.prometheus:simpleclient_pushgateway + io.prometheus:simpleclient_common + com.uber.m3:tally-m3 + com.uber.m3:tally-core + + + org.eclipse.jetty:* + org.eclipse.jetty.websocket:* + javax.servlet:javax.servlet-api + org.apache.httpcomponents:* + + + org.apache.flink:flink-json + + org.apache.flink:flink-metrics-dropwizard + + org.apache.hive:hive-common + org.apache.hive:hive-service + org.apache.hive:hive-service-rpc + org.apache.hive:hive-exec + org.apache.hive:hive-standalone-metastore + org.apache.hive:hive-metastore + org.apache.hive:hive-jdbc + org.datanucleus:datanucleus-core + org.datanucleus:datanucleus-api-jdo + org.apache.thrift:libfb303 + org.apache.orc:orc-core + + commons-codec:commons-codec + commons-io:commons-io + org.openjdk.jol:jol-core + + + com.fasterxml.jackson.module:jackson-module-afterburner + io.airlift:aircompressor + + + + + + + org.apache.avro + org.apache.flink.cdc.connectors.hudi.shaded.avro + + + + com.esotericsoftware.kryo. + org.apache.flink.cdc.connectors.hudi.shaded.com.esotericsoftware.kryo. + + + com.esotericsoftware.reflectasm. + org.apache.flink.cdc.connectors.hudi.shaded.com.esotericsoftware.reflectasm. + + + com.esotericsoftware.minlog. + org.apache.flink.cdc.connectors.hudi.shaded.com.esotericsoftware.minlog. + + + org.objenesis. + org.apache.flink.cdc.connectors.hudi.shaded.org.objenesis. + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + \ No newline at end of file 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/HudiConfig.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiConfig.java new file mode 100644 index 00000000000..1dd2d31649b --- /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/HudiConfig.java @@ -0,0 +1,140 @@ +package org.apache.flink.cdc.connectors.hudi.sink; + +import org.apache.flink.cdc.common.configuration.ConfigOption; +import org.apache.flink.cdc.common.configuration.ConfigOptions; +import org.apache.flink.configuration.description.Description; +import org.apache.hudi.common.config.HoodieCommonConfig; +import org.apache.hudi.configuration.FlinkOptions; + +/** + * A utility class that holds all the configuration options for the Hudi sink. It wraps Hudi's + * {@link FlinkOptions} to provide a consistent interface within the CDC framework, using helper + * methods to reduce boilerplate. + */ +public class HudiConfig { + + // ----- Helper Methods for Option Creation ----- + + private static ConfigOption stringOption(String key, Description description) { + return ConfigOptions.key(key) + .stringType() + .noDefaultValue() + .withDescription(description.toString()); + } + + private static ConfigOption stringOption( + String key, String defaultValue, Description description) { + return ConfigOptions.key(key) + .stringType() + .defaultValue(defaultValue) + .withDescription(description.toString()); + } + + private static ConfigOption intOption(String key, Description description) { + return ConfigOptions.key(key) + .intType() + .noDefaultValue() + .withDescription(description.toString()); + } + + private static ConfigOption booleanOption( + String key, boolean defaultValue, Description description) { + return ConfigOptions.key(key) + .booleanType() + .defaultValue(defaultValue) + .withDescription(description.toString()); + } + + // ----- Public Configuration Options ----- + + // Core Hudi Options + public static final ConfigOption PATH = + stringOption(FlinkOptions.PATH.key(), FlinkOptions.PATH.description()); + + // public static final ConfigOption TABLE_TYPE = + // stringOption( + // FlinkOptions.TABLE_TYPE.key(), + // FlinkOptions.TABLE_TYPE.defaultValue(), + // FlinkOptions.TABLE_TYPE.description()); + public static final ConfigOption TABLE_TYPE = + stringOption( + "hoodie.table.type", + FlinkOptions.TABLE_TYPE.defaultValue(), + FlinkOptions.TABLE_TYPE.description()); + + // Required Fields for CDC + public static final ConfigOption RECORD_KEY_FIELD = + stringOption( + FlinkOptions.RECORD_KEY_FIELD.key(), + FlinkOptions.RECORD_KEY_FIELD.description()); + + public static final ConfigOption ORDERING_FIELDS = + stringOption( + FlinkOptions.ORDERING_FIELDS.key(), FlinkOptions.ORDERING_FIELDS.description()); + + public static final ConfigOption PARTITION_PATH_FIELD = + stringOption( + FlinkOptions.PARTITION_PATH_FIELD.key(), + "", + FlinkOptions.PARTITION_PATH_FIELD.description()); + + // Bucket Index Options + public static final ConfigOption INDEX_TYPE = + stringOption( + FlinkOptions.INDEX_TYPE.key(), "BUCKET", FlinkOptions.INDEX_TYPE.description()); + + public static final ConfigOption INDEX_BUCKET_TARGET = + stringOption( + FlinkOptions.INDEX_KEY_FIELD.key(), FlinkOptions.INDEX_KEY_FIELD.description()); + + public static final ConfigOption BUCKET_INDEX_NUM_BUCKETS = + intOption( + FlinkOptions.BUCKET_INDEX_NUM_BUCKETS.key(), + FlinkOptions.BUCKET_INDEX_NUM_BUCKETS.description()); + + // Hive Sync Options + public static final ConfigOption HIVE_SYNC_ENABLED = + booleanOption( + FlinkOptions.HIVE_SYNC_ENABLED.key(), + false, + FlinkOptions.HIVE_SYNC_ENABLED.description()); + + public static final ConfigOption HIVE_SYNC_METASTORE_URIS = + stringOption( + FlinkOptions.HIVE_SYNC_METASTORE_URIS.key(), + FlinkOptions.HIVE_SYNC_METASTORE_URIS.description()); + + public static final ConfigOption HIVE_SYNC_DB = + stringOption(FlinkOptions.HIVE_SYNC_DB.key(), FlinkOptions.HIVE_SYNC_DB.description()); + + public static final ConfigOption HIVE_SYNC_TABLE = + stringOption( + FlinkOptions.HIVE_SYNC_TABLE.key(), FlinkOptions.HIVE_SYNC_TABLE.description()); + + public static final ConfigOption SCHEMA_OPERATOR_UID = + ConfigOptions.key("schema.operator.uid") + .stringType() + .defaultValue("schema-operator-uid") + .withDescription( + "A unique ID for the schema operator, used by the BucketAssignerOperator to create a SchemaEvolutionClient."); + + public static final ConfigOption TABLE_SCHEMA = + ConfigOptions.key("table.schema") + .stringType() + .noDefaultValue() + .withDescription("The table schema in JSON format for the Hudi table."); + + public static final ConfigOption BUCKET_ASSIGN_TASKS = + intOption( + FlinkOptions.BUCKET_ASSIGN_TASKS.key(), + FlinkOptions.BUCKET_ASSIGN_TASKS.description()); + + public static final ConfigOption WRITE_TASKS = + intOption(FlinkOptions.WRITE_TASKS.key(), FlinkOptions.WRITE_TASKS.description()); + + public static final ConfigOption SCHEMA_ON_READ_ENABLE = + booleanOption( + HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.key(), + false, + Description.builder().build()); +} 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/HudiDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSink.java new file mode 100644 index 00000000000..e58d7115381 --- /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/HudiDataSink.java @@ -0,0 +1,90 @@ +package org.apache.flink.cdc.connectors.hudi.sink; + +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.sink.DataSink; +import org.apache.flink.cdc.common.sink.EventSinkProvider; +import org.apache.flink.cdc.common.sink.FlinkSinkProvider; +import org.apache.flink.cdc.common.sink.MetadataApplier; +import org.apache.flink.cdc.connectors.hudi.sink.v2.HudiSink; +import org.apache.flink.table.types.logical.RowType; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.time.ZoneId; + +/** + * A {@link DataSink} for Apache Hudi that provides the main entry point for the Flink CDC + * framework. + */ +public class HudiDataSink implements DataSink, Serializable { + + private static final Logger LOG = LoggerFactory.getLogger(HudiDataSink.class); + + private final Configuration config; + + private final String schemaOperatorUid; + + public HudiDataSink(Configuration config, String schemaOperatorUid) { + LOG.info("Creating HudiDataSink with universal configuration {}", config); + this.config = config; + this.schemaOperatorUid = schemaOperatorUid; + } + + /** Provides the core sink implementation that handles the data flow of events. */ + @Override + public EventSinkProvider getEventSinkProvider() { + LOG.info("Creating HudiDataSinkProvider with universal configuration {}", config); + // For CDC pipelines, we don't have a pre-configured schema since tables are created + // dynamically + // Instead, we use a multi-table sink that handles schema discovery and table creation + + // Convert CDC configuration to Flink configuration for HoodieSink + org.apache.flink.configuration.Configuration flinkConfig = toFlinkConfig(config); + + // Extract configuration options + java.util.Map configMap = config.toMap(); + boolean overwrite = "insert_overwrite".equals(configMap.get("write.operation")); + boolean isBounded = "BATCH".equals(configMap.get("execution.checkpointing.mode")); + + // Create the HudiSink with multi-table support via wrapper pattern + // Use empty RowType since tables are created dynamically + HudiSink hudiSink = + new HudiSink( + flinkConfig, + RowType.of(), // Empty row type for dynamic multi-table support + overwrite, + isBounded, + schemaOperatorUid, + ZoneId.systemDefault()); + + return FlinkSinkProvider.of(hudiSink); + } + + /** + * Provides the metadata applier. In our design, this has a passive role (e.g., logging), as + * transactional metadata operations are handled by the HudiCommitter. + */ + @Override + public MetadataApplier getMetadataApplier() { + return new HudiMetadataApplier(config); + } + + /** + * Converts a {@link org.apache.flink.cdc.common.configuration.Configuration} to a {@link + * org.apache.flink.configuration.Configuration}. + * + * @param cdcConfig The input CDC configuration. + * @return A new Flink configuration containing the same key-value pairs. + */ + private static org.apache.flink.configuration.Configuration toFlinkConfig( + Configuration cdcConfig) { + final org.apache.flink.configuration.Configuration flinkConfig = + new org.apache.flink.configuration.Configuration(); + if (cdcConfig != null) { + cdcConfig.toMap().forEach(flinkConfig::setString); + } + return flinkConfig; + } +} 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/HudiDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactory.java new file mode 100644 index 00000000000..c9461fb6a1d --- /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/HudiDataSinkFactory.java @@ -0,0 +1,76 @@ +package org.apache.flink.cdc.connectors.hudi.sink; + +import static org.apache.flink.cdc.connectors.hudi.sink.HudiDataSinkOptions.PREFIX_CATALOG_PROPERTIES; +import static org.apache.flink.cdc.connectors.hudi.sink.HudiDataSinkOptions.PREFIX_TABLE_PROPERTIES; + +import org.apache.flink.cdc.common.configuration.ConfigOption; +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.factories.DataSinkFactory; +import org.apache.flink.cdc.common.factories.FactoryHelper; +import org.apache.flink.cdc.common.pipeline.PipelineOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.Set; + +/** + * Factory for creating {@link HudiDataSink}. This class defines the configuration options and + * instantiates the sink by delegating option definitions to {@link HudiConfig}. + */ +public class HudiDataSinkFactory implements DataSinkFactory { + + private static final Logger LOG = LoggerFactory.getLogger(HudiDataSinkFactory.class); + + public static final String IDENTIFIER = "hudi"; + + @Override + public String identifier() { + return IDENTIFIER; + } + + @Override + public HudiDataSink createDataSink(Context context) { + LOG.info("Creating HudiDataSink for {}", context); + + FactoryHelper.createFactoryHelper(this, context) + .validateExcept(PREFIX_TABLE_PROPERTIES, PREFIX_CATALOG_PROPERTIES); + + FactoryHelper.DefaultContext factoryContext = (FactoryHelper.DefaultContext) context; + Configuration config = factoryContext.getFactoryConfiguration(); + + String schemaOperatorUid = + context.getPipelineConfiguration() + .get(PipelineOptions.PIPELINE_SCHEMA_OPERATOR_UID); + + return new HudiDataSink(config, schemaOperatorUid); + } + + @Override + public Set> requiredOptions() { + Set> options = new HashSet<>(); + options.add(HudiConfig.PATH); + options.add(HudiConfig.RECORD_KEY_FIELD); + // options.add(HudiConfig.PRECOMBINE_FIELD); + // options.add(HudiConfig.BUCKET_INDEX_NUM_BUCKETS); + return options; + } + + @Override + public Set> optionalOptions() { + Set> 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); + 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..d42650ba958 --- /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,26 @@ +/* + * 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; + +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..b7ebd4166e5 --- /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,707 @@ +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.*; +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.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StorageConfiguration; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; +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.*; +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); + } + + /** + * Updates the actual Hudi table schema stored in hoodie.properties. This is necessary because + * catalog.alterTable() only updates table_option.properties, not the actual Hudi table schema. + */ + private void updateHudiTableSchema(ObjectPath objectPath, ResolvedSchema newSchema) + throws Exception { + // Get table path + String basePath = catalogConfig.getString(CatalogOptions.CATALOG_PATH.key(), null); + String tablePath = + String.format( + "%s/%s/%s", + basePath, objectPath.getDatabaseName(), objectPath.getObjectName()); + + LOG.info("Updating Hudi table schema at path: {}", tablePath); + + // Convert Flink schema to Avro schema + org.apache.avro.Schema avroSchema = convertFlinkSchemaToAvro(newSchema); + + // Update the table schema using HoodieTableMetaClient + org.apache.hadoop.conf.Configuration hadoopConf = + new org.apache.hadoop.conf.Configuration(); + StorageConfiguration storageConf = new HadoopStorageConfiguration(hadoopConf); + + org.apache.hudi.common.table.HoodieTableMetaClient metaClient = + org.apache.hudi.common.table.HoodieTableMetaClient.builder() + .setConf(storageConf) + .setBasePath(tablePath) + .build(); + + // Commit the schema change by updating table config + HoodieTableConfig tableConfig = metaClient.getTableConfig(); + HoodieStorage storage = metaClient.getStorage(); + + // Write the updated schema to hoodie.properties + // Hudi stores the schema under "hoodie.table.schema" which is what HoodieCatalog reads + Properties props = new Properties(); + props.putAll(tableConfig.getProps()); + + String schemaStr = avroSchema.toString(); + // Update the create schema (historical record) + props.setProperty(HoodieTableConfig.CREATE_SCHEMA.key(), schemaStr); + // Also update the current schema property that the catalog reads + props.setProperty("hoodie.table.schema", schemaStr); + + StoragePath propsPath = + new StoragePath(metaClient.getMetaPath(), HoodieTableConfig.HOODIE_PROPERTIES_FILE); + + try (java.io.OutputStream outputStream = storage.create(propsPath, true)) { + props.store(outputStream, "Updated by Flink CDC HudiMetadataApplier"); + } + + LOG.info("Successfully updated Hudi table schema"); + } + + /** Converts Flink ResolvedSchema to Avro Schema for Hudi. */ + private org.apache.avro.Schema convertFlinkSchemaToAvro(ResolvedSchema flinkSchema) { + List avroFields = new ArrayList<>(); + + for (org.apache.flink.table.catalog.Column column : flinkSchema.getColumns()) { + org.apache.avro.Schema fieldSchema = convertFlinkTypeToAvro(column.getDataType()); + org.apache.avro.Schema.Field field = + new org.apache.avro.Schema.Field( + column.getName(), fieldSchema, null, (Object) null); + avroFields.add(field); + } + + org.apache.avro.Schema avroSchema = + org.apache.avro.Schema.createRecord( + "HudiSchema", null, "org.apache.hudi", false, avroFields); + + return avroSchema; + } + + /** 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 { + // TODO: Check if this is necessary + // updateHudiTableSchema(objectPath, newSchema); + LOG.info("Successfully modified schema for table: {}", tableId); + + 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); + + // TODO: enable this in the future + // Disable metadata table usage in catalog commit + flinkConfig.setString(HoodieMetadataConfig.ENABLE.key(), "false"); + + // 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..776df24b3be --- /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,256 @@ +/* + * 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.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.OperationType; +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.types.DataType; +import org.apache.flink.cdc.common.utils.SchemaUtils; +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.hudi.configuration.FlinkOptions; +import org.apache.hudi.index.bucket.BucketIdentifier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +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; + + /** 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<>(); + + /** Cache of primary key fields per table */ + private final Map> primaryKeyCache = new HashMap<>(); + + /** Cache of field getters per table */ + private final Map> fieldGetterCache = new HashMap<>(); + + public BucketAssignOperator(Configuration conf, String schemaOperatorUid) { + this.numBuckets = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS); + this.schemaOperatorUid = schemaOperatorUid; + // Use ALWAYS like Paimon does - allows chaining with both upstream and downstream + 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(); + 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 + fieldGetterCache.remove(schemaEvent.tableId()); + primaryKeyCache.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 bucket = calculateBucket(dataEvent); + output.collect(new StreamRecord<>(new BucketWrapper(bucket, 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))); + } + } + + private int calculateBucket(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 = primaryKeyCache.computeIfAbsent(tableId, k -> finalSchema.primaryKeys()); + + if (primaryKeys.isEmpty()) { + throw new IllegalStateException( + "Cannot calculate bucket: table " + tableId + " has no primary keys"); + } + + // Create final references for use in lambda + final List finalPrimaryKeys = primaryKeys; + + // Get or cache field getters + List fieldGetters = fieldGetterCache.computeIfAbsent(tableId, k -> { + List getters = new ArrayList<>(finalPrimaryKeys.size()); + for (String primaryKeyField : finalPrimaryKeys) { + int fieldIndex = finalSchema.getColumnNames().indexOf(primaryKeyField); + if (fieldIndex == -1) { + throw new IllegalStateException( + "Primary key field '" + primaryKeyField + + "' not found in schema for table " + tableId); + } + DataType fieldType = finalSchema.getColumns().get(fieldIndex).getType(); + getters.add(RecordData.createFieldGetter(fieldType, fieldIndex)); + } + return getters; + }); + + // Extract record key + String recordKey = extractRecordKey(event, primaryKeys, fieldGetters); + + // Calculate bucket using Hudi's logic + String tableIndexKeyFields = String.join(",", primaryKeys); + return BucketIdentifier.getBucketId(recordKey, tableIndexKeyFields, numBuckets); + } + + private String extractRecordKey(DataChangeEvent event, List primaryKeys, + List fieldGetters) { + // For DELETE, use 'before' data; for INSERT/UPDATE, use 'after' data + RecordData recordData = event.op() == OperationType.DELETE + ? event.before() + : event.after(); + + if (recordData == null) { + throw new IllegalStateException( + "Cannot extract record key: " + event.op() + " event has null data"); + } + + List recordKeyPairs = new ArrayList<>(primaryKeys.size()); + for (int i = 0; i < primaryKeys.size(); i++) { + RecordData.FieldGetter fieldGetter = fieldGetters.get(i); + Object fieldValue = fieldGetter.getFieldOrNull(recordData); + + if (fieldValue == null) { + throw new IllegalStateException( + "Primary key field '" + primaryKeys.get(i) + "' is null in record"); + } + + // Format as "fieldName:value" + recordKeyPairs.add(primaryKeys.get(i) + ":" + fieldValue); + } + + return String.join(",", recordKeyPairs); + } +} \ No newline at end of file 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..75f12e9693c --- /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; + } +} \ No newline at end of file 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..391005b57f3 --- /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 + + '}'; + } +} \ No newline at end of file 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..67575358c96 --- /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,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.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); + } + } +} \ No newline at end of file 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..d9f2ba3b27c --- /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,734 @@ +/* + * 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.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.event.CreateTableOperatorEvent; +import org.apache.flink.cdc.connectors.hudi.sink.event.EnhancedWriteMetadataEvent; +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.config.HoodieMetadataConfig; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +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.config.HoodieWriteConfig; +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.FlinkWriteClients; +import org.apache.hudi.util.StreamerUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +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; + +/** + * 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); + + /** + * A custom coordination request that includes the TableId to request an instant for a specific + * table. + */ + public static class MultiTableInstantTimeRequest implements CoordinationRequest, Serializable { + 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; + } + } + + /** + * 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; + + 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); + } + } + + /** 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 dedicated write client whose only job is to run the embedded timeline server. This ensures + * there is only one timeline server for the entire job. + */ + private transient HoodieFlinkWriteClient timelineServerClient; + + /** 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); + conf.setString("fs.file.impl", "org.apache.hadoop.fs.LocalFileSystem"); + 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()]; + + // Initialize a single write client for the coordinator path. + // Its primary role is to start and manage the embedded timeline server. + try { + // The baseConfig points to the dummy coordinator path. + // A .hoodie directory is required for the timeline server to start. + StreamerUtil.initTableIfNotExists(this.baseConfig); + this.timelineServerClient = FlinkWriteClients.createWriteClient(this.baseConfig); + LOG.info("Successfully started timeline server on coordinator."); + } catch (Exception e) { + LOG.error("Failed to start timeline server on coordinator.", e); + context.failJob(e); + return; + } + + // 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 { + 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 CreateTableOperatorEvent) { + handleCreateTableEvent((CreateTableOperatorEvent) operatorEvent); + } else if (operatorEvent instanceof EnhancedWriteMetadataEvent) { + handleEnhancedWriteMetadataEvent( + (EnhancedWriteMetadataEvent) operatorEvent); + } else { + LOG.warn( + "Received an unhandled or non-enhanced OperatorEvent: {}", + operatorEvent); + } + }, + "handling operator event %s", + operatorEvent); + } + + private void handleCreateTableEvent(CreateTableOperatorEvent createTableOperatorEvent) { + CreateTableEvent event = createTableOperatorEvent.getCreateTableEvent(); + TableId tableId = event.tableId(); + + // Store the schema for this table + tableSchemas.put(tableId, event.getSchema()); + LOG.info( + "Cached schema for table {}: {} columns", + tableId, + event.getSchema().getColumnCount()); + + 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); + + 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; + } + }); + } + + private void handleEnhancedWriteMetadataEvent(EnhancedWriteMetadataEvent enhancedEvent) { + String tablePath = enhancedEvent.getTablePath(); + WriteMetadataEvent event = enhancedEvent.getOriginalEvent(); + TableId tableId = pathToTableId.get(tablePath); + + if (tableId == null) { + LOG.warn("No tableId found for path: {}. Cannot process event.", 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; + } + // 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()); + }); + + byte[] serializedState = + SerializationUtils.serialize((Serializable) allStates); + result.complete(serializedState); + LOG.info( + "Successfully checkpointed coordinator state for checkpoint {}", + 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>> allStates = + SerializationUtils.deserialize(checkpointData); + 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) { + + 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); + } 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))); + } + } + + @Override + public void close() throws Exception { + if (timelineServerClient != null) { + timelineServerClient.close(); + } + 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 --- + private Configuration createTableSpecificConfig(TableId tableId) { + Configuration tableConfig = new Configuration(baseConfig); + String coordinatorPath = baseConfig.getString(FlinkOptions.PATH); + // Use the same logic as MultiTableEventStreamWriteFunction to strip "/coordinator" + String rootPath = coordinatorPath.split("/coordinator")[0]; + 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); + } + + // Disable both embedded timeline server and metadata table for per-table clients. + // The central coordinator manages the only timeline server. + tableConfig.setBoolean(HoodieWriteConfig.EMBEDDED_TIMELINE_SERVER_ENABLE.key(), false); + // For simplicity, let's keep it consistent. + tableConfig.setBoolean(HoodieMetadataConfig.ENABLE.key(), false); + + // Use memory-based file system view since each client is lightweight. + tableConfig.setString(FileSystemViewStorageConfig.VIEW_TYPE.key(), "MEMORY"); + + 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/CreateTableOperatorEvent.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/CreateTableOperatorEvent.java new file mode 100644 index 00000000000..45043eb06dc --- /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/CreateTableOperatorEvent.java @@ -0,0 +1,59 @@ +/* + * 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.runtime.operators.coordination.OperatorEvent; + +/** + * An operator event that encapsulates a {@link CreateTableEvent}. + * + *

This event 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 class CreateTableOperatorEvent implements OperatorEvent { + + private static final long serialVersionUID = 1L; + + private final CreateTableEvent createTableEvent; + + /** + * Constructs a new CreateTableOperatorEvent. + * + * @param createTableEvent The original CDC event that triggered this operator event. + */ + public CreateTableOperatorEvent(CreateTableEvent createTableEvent) { + this.createTableEvent = createTableEvent; + } + + /** + * Gets the encapsulated {@link CreateTableEvent}. + * + * @return The original create table event. + */ + public CreateTableEvent getCreateTableEvent() { + return createTableEvent; + } + + @Override + public String toString() { + return "CreateTableOperatorEvent{" + "tableId=" + createTableEvent.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/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..b8118ab79ce --- /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,228 @@ +/* + * 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.data.RecordData; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.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 java.time.ZoneId; +import java.util.HashMap; +import java.util.List; +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 + *
  • 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; + + /** Field getter cache per table for efficient conversion */ + private final Map> fieldGetterCache; + + /** Zone ID for timestamp conversion */ + private final ZoneId zoneId; + + public HudiRecordEventSerializer(ZoneId zoneId) { + this.schemaMaps = new HashMap<>(); + this.fieldGetterCache = 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 field getter cache for this table since schema changed + fieldGetterCache.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 field getter cache for this table since schema changed + fieldGetterCache.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."); + } + + // Convert DataChangeEvent to HoodieFlinkInternalRow using utility function + return RowDataUtils.convertDataChangeEventToHoodieFlinkInternalRow( + dataChangeEvent, schema, zoneId, 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) { + if (event instanceof CreateTableEvent) { + CreateTableEvent createTableEvent = (CreateTableEvent) event; + schemaMaps.put(createTableEvent.tableId(), createTableEvent.getSchema()); + // Clear field getter cache for this table since schema changed + fieldGetterCache.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 field getter cache for this table since schema changed + fieldGetterCache.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."); + } + + // Convert DataChangeEvent to HoodieFlinkInternalRow using utility function + // Use temporary values that will be overridden later + return RowDataUtils.convertDataChangeEventToHoodieFlinkInternalRow( + dataChangeEvent, schema, zoneId, "temp", "temp"); + } else { + throw new IllegalArgumentException( + "Unsupported event type for Hudi serialization: " + + event.getClass().getSimpleName()); + } + } + + /** + * 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); + } + + /** + * Get cached field getters for a table, creating them if needed. + * + * @param tableId The table identifier + * @return List of field getters or null if schema not available + */ + public List getFieldGetters(TableId tableId) { + Schema schema = schemaMaps.get(tableId); + if (schema == null) { + return null; + } + + return fieldGetterCache.computeIfAbsent( + tableId, k -> RowDataUtils.createFieldGetters(schema, zoneId)); + } + + /** + * 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 for this table so they get recreated with the new schema + fieldGetterCache.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/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..c1f95e99b47 --- /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,85 @@ +/* + * 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.connectors.hudi.sink.coordinator.MultiTableStreamWriteOperatorCoordinator; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; +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 { + MultiTableStreamWriteOperatorCoordinator.MultiTableInstantTimeRequest request = + new MultiTableStreamWriteOperatorCoordinator.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); + } + } +} 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/EventBucketStreamWriteFunction.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/EventBucketStreamWriteFunction.java new file mode 100644 index 00000000000..a9266db3264 --- /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/EventBucketStreamWriteFunction.java @@ -0,0 +1,337 @@ +package org.apache.flink.cdc.connectors.hudi.sink.function; + +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.OperationType; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.connectors.hudi.sink.event.HudiRecordEventSerializer; +import org.apache.flink.cdc.connectors.hudi.sink.event.HudiRecordSerializer; +import org.apache.flink.cdc.connectors.hudi.sink.model.BucketAssignmentIndex; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hudi.client.model.HoodieFlinkInternalRow; +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.configuration.OptionsResolver; +import org.apache.hudi.index.bucket.BucketIdentifier; +import org.apache.hudi.index.bucket.partition.NumBucketsFunction; +import org.apache.hudi.utils.RuntimeContextUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class EventBucketStreamWriteFunction extends EventStreamWriteFunction { + + private static final Logger LOG = LoggerFactory.getLogger(EventBucketStreamWriteFunction.class); + + private int parallelism; + + private boolean isNonBlockingConcurrencyControl; + + /** BucketID to file group mapping in each partition of a tableId. */ + private BucketAssignmentIndex bucketAssignmentIndex; + + /** + * Incremental bucket index of the current checkpoint interval, it is needed because the bucket + * type('I' or 'U') should be decided based on the committed files view, all the records in one + * bucket should have the same bucket type. + */ + private Set incBucketIndexes; + + /** Serializer for converting Events to HoodieFlinkInternalRow for single table */ + private HudiRecordEventSerializer recordSerializer; + + /** Function for calculating the task partition to dispatch. */ + private Functions.Function3 partitionIndexFunc; + + /** Function to calculate num buckets per partition. */ + private NumBucketsFunction numBucketsFunction; + + /** Cached primary key fields for this table */ + private transient List primaryKeyFields; + + /** Cached field getters for primary key fields */ + private transient List primaryKeyFieldGetters; + + /** Cached schema for this table */ + private transient Schema cachedSchema; + + /** Number of buckets for this function */ + private int numBuckets; + + /** + * Constructs a BucketStreamWriteFunction. + * + * @param config The config options + */ + public EventBucketStreamWriteFunction(Configuration config, RowType rowType) { + super(config, rowType); + } + + @Override + public void open(Configuration parameters) throws IOException { + super.open(parameters); + this.isNonBlockingConcurrencyControl = + OptionsResolver.isNonBlockingConcurrencyControl(config); + this.taskID = RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); + this.parallelism = RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()); + this.bucketAssignmentIndex = new BucketAssignmentIndex(); + this.incBucketIndexes = new HashSet<>(); + this.partitionIndexFunc = BucketIndexUtil.getPartitionIndexFunc(parallelism); + this.numBucketsFunction = + new NumBucketsFunction( + config.get(FlinkOptions.BUCKET_INDEX_PARTITION_EXPRESSIONS), + config.get(FlinkOptions.BUCKET_INDEX_PARTITION_RULE), + config.get(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS)); + + this.numBuckets = config.get(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS); + + // Initialize record serializer with system default zone ID + this.recordSerializer = new HudiRecordEventSerializer(ZoneId.systemDefault()); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + super.initializeState(context); + // Bootstrap will reload file groups from Hudi on startup + } + + @Override + public void snapshotState() { + LOG.info("Triggering snapshotState"); + super.snapshotState(); + this.incBucketIndexes.clear(); + } + + @Override + public void processDataChange(DataChangeEvent event) throws Exception { + // 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); + // Calculate bucket from event data for bucket assignment + int bucket = calculateBucketFromEvent(event); + + // Define record location (file ID, instant time) based on bucket assignment + defineRecordLocation(bucket, hoodieFlinkInternalRow); + + // Buffer the record for writing + bufferRecord(hoodieFlinkInternalRow); + + LOG.debug( + "Processed DataChangeEvent for table {}: partition={}, fileId={}, instantTime={}", + event.tableId(), + hoodieFlinkInternalRow.getPartitionPath(), + hoodieFlinkInternalRow.getFileId(), + hoodieFlinkInternalRow.getInstantTime()); + } + + @Override + public void processSchemaChange(SchemaChangeEvent event) throws Exception { + // Single-table functions typically receive schema via serializer setup + // This is called when CreateTableEvent arrives + LOG.info("Schema change event received: {}", event); + + // Handle schema events (CreateTableEvent, SchemaChangeEvent) - they don't produce records + // null will be returned from serialize + recordSerializer.serialize(event); + } + + private void defineRecordLocation(int bucketNum, HoodieFlinkInternalRow record) { + final String partition = record.getPartitionPath(); + + // Check if this task should handle this bucket + if (!isBucketToLoad(bucketNum, partition)) { + throw new IllegalStateException( + String.format( + "Task %d received record for bucket %d which should not be handled by this task. " + + "This indicates a partitioning problem - records must be routed to the correct task.", + taskID, bucketNum)); + } + + bootstrapIndexIfNeed(partition); + Map bucketToFileId = bucketAssignmentIndex.getBucketToFileIdMap(partition); + final String bucketId = partition + "/" + bucketNum; + + if (incBucketIndexes.contains(bucketId)) { + record.setInstantTime("I"); + record.setFileId(bucketToFileId.get(bucketNum)); + } else if (bucketToFileId.containsKey(bucketNum)) { + record.setInstantTime("U"); + record.setFileId(bucketToFileId.get(bucketNum)); + } else { + String newFileId = + isNonBlockingConcurrencyControl + ? BucketIdentifier.newBucketFileIdForNBCC(bucketNum) + : BucketIdentifier.newBucketFileIdPrefix(bucketNum); + record.setInstantTime("I"); + record.setFileId(newFileId); + bucketToFileId.put(bucketNum, newFileId); + incBucketIndexes.add(bucketId); + } + } + + /** + * Determine whether the current fileID belongs to the current task. partitionIndex == this + * taskID belongs to this task. + */ + public boolean isBucketToLoad(int bucketNumber, String partition) { + int numBuckets = numBucketsFunction.getNumBuckets(partition); + return partitionIndexFunc.apply(numBuckets, partition, bucketNumber) == taskID; + } + + /** + * Get partition_bucket -> fileID mapping from the existing hudi table. This is a required + * operation for each restart to avoid having duplicate file ids for one bucket. + */ + private void bootstrapIndexIfNeed(String partition) { + if (bucketAssignmentIndex.containsPartition(partition)) { + return; + } + LOG.info( + "Loading Hoodie Table {}, with path {}/{}", + this.metaClient.getTableConfig().getTableName(), + this.metaClient.getBasePath(), + partition); + + // Load existing fileID belongs to this task + Map bucketToFileIDMap = new HashMap<>(); + this.writeClient + .getHoodieTable() + .getHoodieView() + .getLatestFileSlices(partition) + .forEach( + fileSlice -> { + String fileId = fileSlice.getFileId(); + int bucketNumber = BucketIdentifier.bucketIdFromFileId(fileId); + if (isBucketToLoad(bucketNumber, partition)) { + LOG.info( + String.format( + "Should load this partition bucket %s with fileId %s", + bucketNumber, fileId)); + // Validate that one bucketId has only ONE fileId + if (bucketToFileIDMap.containsKey(bucketNumber)) { + throw new RuntimeException( + String.format( + "Duplicate fileId %s from bucket %s of partition %s found " + + "during the BucketStreamWriteFunction index bootstrap.", + fileId, bucketNumber, partition)); + } else { + LOG.info( + String.format( + "Adding fileId %s to the bucket %s of partition %s.", + fileId, bucketNumber, partition)); + bucketToFileIDMap.put(bucketNumber, fileId); + } + } + }); + bucketAssignmentIndex.bootstrapPartition(partition, bucketToFileIDMap); + } + + /** Calculate bucket from DataChangeEvent using primary key fields. */ + private int calculateBucketFromEvent(DataChangeEvent dataChangeEvent) { + // Initialize cache on first call + if (cachedSchema == null) { + cachedSchema = recordSerializer.getSchema(dataChangeEvent.tableId()); + if (cachedSchema == null) { + throw new IllegalStateException( + "No schema available for table " + dataChangeEvent.tableId()); + } + + // Cache primary key fields + primaryKeyFields = cachedSchema.primaryKeys(); + if (primaryKeyFields.isEmpty()) { + throw new IllegalStateException( + "Cannot calculate bucket: table " + + dataChangeEvent.tableId() + + " has no primary keys"); + } + + // Cache field getters for primary key fields + primaryKeyFieldGetters = new ArrayList<>(primaryKeyFields.size()); + for (String primaryKeyField : primaryKeyFields) { + int fieldIndex = cachedSchema.getColumnNames().indexOf(primaryKeyField); + if (fieldIndex == -1) { + throw new IllegalStateException( + "Primary key field '" + + primaryKeyField + + "' not found in schema for table " + + dataChangeEvent.tableId()); + } + DataType fieldType = cachedSchema.getColumns().get(fieldIndex).getType(); + primaryKeyFieldGetters.add(RecordData.createFieldGetter(fieldType, fieldIndex)); + } + } + + // Extract record key from event data using cached field getters + String recordKey = extractRecordKeyFromEvent(dataChangeEvent); + + // Calculate bucket using Hudi's bucket logic + return calculateBucketFromRecordKey(recordKey, primaryKeyFields); + } + + /** + * Extract record key from CDC event data using cached field getters for optimal performance. + */ + private String extractRecordKeyFromEvent(DataChangeEvent dataChangeEvent) { + // For DELETE operations, use 'before' data; for INSERT/UPDATE, use 'after' data + RecordData recordData = + dataChangeEvent.op() == OperationType.DELETE + ? dataChangeEvent.before() + : dataChangeEvent.after(); + + if (recordData == null) { + throw new IllegalStateException( + "Cannot extract record key: " + dataChangeEvent.op() + " event has null data"); + } + + // Use cached field getters for optimal performance + List recordKeyPairs = new ArrayList<>(primaryKeyFields.size()); + for (int i = 0; i < primaryKeyFields.size(); i++) { + RecordData.FieldGetter fieldGetter = primaryKeyFieldGetters.get(i); + Object fieldValue = fieldGetter.getFieldOrNull(recordData); + + if (fieldValue == null) { + throw new IllegalStateException( + "Primary key field '" + primaryKeyFields.get(i) + "' is null in record"); + } + + // Format as "fieldName:value" + recordKeyPairs.add(primaryKeyFields.get(i) + ":" + fieldValue); + } + + // Join primary key pairs with comma (recordKey1:val1,recordKey2:val2) + return String.join(",", recordKeyPairs); + } + + /** Calculate bucket ID from record key using Hudi's bucket logic. */ + private int calculateBucketFromRecordKey(String recordKey, List primaryKeyFields) { + // Convert primary key field list to comma-separated string for Hudi bucket calculation + String tableIndexKeyFields = String.join(",", primaryKeyFields); + return BucketIdentifier.getBucketId(recordKey, tableIndexKeyFields, numBuckets); + } + + /** Get the record serializer for schema setup. */ + public HudiRecordSerializer getRecordSerializer() { + return recordSerializer; + } +} 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..7da23baba4f --- /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,64 @@ +/* + * 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.FlushEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; + +/** + * 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)} - Handles INSERT/UPDATE/DELETE operations + *
  • {@link #processSchemaChange(SchemaChangeEvent)} - Handles DDL operations (CREATE TABLE, + * ADD COLUMN, etc.) + *
  • {@link #processFlush(FlushEvent)} - Handles coordinated flushing of buffered data + *
+ */ +public interface EventProcessorFunction { + + /** + * Process data change events (INSERT/UPDATE/DELETE). This is where actual data is buffered and + * written. + * + * @param event The data change event + * @throws Exception if processing fails + */ + void processDataChange(DataChangeEvent event) throws Exception; + + /** + * 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/EventStreamWriteFunction.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/EventStreamWriteFunction.java new file mode 100644 index 00000000000..668a4ace2cf --- /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/EventStreamWriteFunction.java @@ -0,0 +1,702 @@ +/* + * 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.avro.Schema; +import org.apache.flink.cdc.common.event.*; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.runtime.util.MemorySegmentPool; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.model.HoodieFlinkInternalRow; +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.model.HoodieOperation; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.read.BufferedRecordMerger; +import org.apache.hudi.common.table.read.BufferedRecordMergerFactory; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.VisibleForTesting; +import org.apache.hudi.common.util.collection.MappingIterator; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.metrics.FlinkStreamWriteMetrics; +import org.apache.hudi.sink.buffer.MemorySegmentPoolFactory; +import org.apache.hudi.sink.buffer.RowDataBucket; +import org.apache.hudi.sink.buffer.TotalSizeTracer; +import org.apache.hudi.sink.bulk.RowDataKeyGen; +import org.apache.hudi.sink.common.AbstractStreamWriteFunction; +import org.apache.hudi.sink.event.WriteMetadataEvent; +import org.apache.hudi.sink.exception.MemoryPagesExhaustedException; +import org.apache.hudi.sink.transform.RecordConverter; +import org.apache.hudi.sink.utils.BufferUtils; +import org.apache.hudi.table.action.commit.BucketInfo; +import org.apache.hudi.table.action.commit.BucketType; +import org.apache.hudi.table.action.commit.FlinkWriteHelper; +import org.apache.hudi.util.FlinkWriteClients; +import org.apache.hudi.util.MutableIteratorWrapperIterator; +import org.apache.hudi.util.StreamerUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; + +public abstract class EventStreamWriteFunction extends AbstractStreamWriteFunction + implements EventProcessorFunction { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(EventStreamWriteFunction.class); + + /** Write buffer as buckets for a checkpoint. The key is bucket ID (partition path + fileID). */ + protected transient Map buckets; + + /** Write function to trigger the actual write action. */ + protected transient WriteFunction writeFunction; + + private transient BufferedRecordMerger recordMerger; + private transient HoodieReaderContext readerContext; + private transient List orderingFieldNames; + + protected RowType rowType; + + protected final RowDataKeyGen keyGen; + + /** Total size tracer. */ + private transient TotalSizeTracer tracer; + + /** Metrics for flink stream write. */ + protected transient FlinkStreamWriteMetrics writeMetrics; + + /** Table ID for table-specific coordination requests. */ + protected TableId tableId; + + protected transient MemorySegmentPool memorySegmentPool; + + protected transient RecordConverter recordConverter; + + /** + * Constructs an EventStreamWriteFunction. + * + * @param config The config options + */ + public EventStreamWriteFunction(Configuration config, RowType rowType) { + super(config); + this.rowType = rowType; + this.keyGen = RowDataKeyGen.instance(config, rowType); + } + + /** + * Sets the table ID for this function. This is used for table-specific coordination requests. + * + * @param tableId The table ID + */ + public void setTableId(TableId tableId) { + this.tableId = tableId; + } + + @Override + public void open(Configuration parameters) throws IOException { + this.tracer = new TotalSizeTracer(this.config); + initBuffer(); + initWriteFunction(); + initMergeClass(); + initRecordConverter(); + initWriteMetrics(); + } + + @Override + public void snapshotState() { + // Based on the fact that the coordinator starts the checkpoint first, + // it would check the validity. + // wait for the buffer data flush out and request a new instant + LOG.info("Triggered snapshotState"); + flushRemaining(false); + } + + @Override + public final void processElement( + Event event, ProcessFunction.Context ctx, Collector out) + throws Exception { + // Route event to appropriate handler based on type + if (event instanceof DataChangeEvent) { + processDataChange((DataChangeEvent) event); + } 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()); + } + } + + /** + * Process data change events (INSERT/UPDATE/DELETE). This is where actual data is buffered and + * written. + * + *

Implements {@link EventProcessorFunction#processDataChange(DataChangeEvent)}. + * + * @param event The data change event + */ + @Override + public abstract void processDataChange(DataChangeEvent event) throws Exception; + + /** + * Process schema change events (CREATE TABLE, ADD COLUMN, etc.). Default: No-op. Override if + * schema evolution is needed. + * + *

Implements {@link EventProcessorFunction#processSchemaChange(SchemaChangeEvent)}. + * + * @param event The schema change event + */ + @Override + public void processSchemaChange(SchemaChangeEvent event) throws Exception { + LOG.debug("Schema change event not handled by {}: {}", getClass().getSimpleName(), event); + } + + /** + * Process flush events for coordinated flushing. Default: Flush all buffered data. + * + *

Implements {@link EventProcessorFunction#processFlush(FlushEvent)}. + * + * @param event The flush event + */ + @Override + public void processFlush(FlushEvent event) throws Exception { + LOG.info("Received a flush event, flushing all remaining data."); + flushRemaining(false); + } + + @Override + public void close() { + if (this.writeClient != null) { + this.writeClient.close(); + } + } + + /** End input action for batch source. */ + public void endInput() { + super.endInput(); + flushRemaining(true); + this.writeClient.cleanHandles(); + this.writeStatuses.clear(); + } + + // ------------------------------------------------------------------------- + // Utilities + // ------------------------------------------------------------------------- + + private void initBuffer() { + this.buckets = new LinkedHashMap<>(); + this.memorySegmentPool = MemorySegmentPoolFactory.createMemorySegmentPool(config); + } + + private void initWriteFunction() { + final String writeOperation = this.config.get(FlinkOptions.OPERATION); + switch (WriteOperationType.fromValue(writeOperation)) { + case INSERT: + this.writeFunction = + (records, bucketInfo, instantTime) -> + this.writeClient.insert(records, bucketInfo, instantTime); + break; + case UPSERT: + case DELETE: // shares the code path with UPSERT + case DELETE_PREPPED: + this.writeFunction = + (records, bucketInfo, instantTime) -> + this.writeClient.upsert(records, bucketInfo, instantTime); + break; + case INSERT_OVERWRITE: + this.writeFunction = + (records, bucketInfo, instantTime) -> + this.writeClient.insertOverwrite(records, bucketInfo, instantTime); + break; + case INSERT_OVERWRITE_TABLE: + this.writeFunction = + (records, bucketInfo, instantTime) -> + this.writeClient.insertOverwriteTable( + records, bucketInfo, instantTime); + break; + default: + throw new RuntimeException("Unsupported write operation : " + writeOperation); + } + } + + private void initWriteMetrics() { + MetricGroup metrics = getRuntimeContext().getMetricGroup(); + this.writeMetrics = new FlinkStreamWriteMetrics(metrics); + this.writeMetrics.registerMetrics(); + } + + private void initRecordConverter() { + this.recordConverter = RecordConverter.getInstance(keyGen); + } + + private void initMergeClass() { + readerContext = + writeClient + .getEngineContext() + .getReaderContextFactory(metaClient) + .getContext(); + readerContext.initRecordMergerForIngestion(writeClient.getConfig().getProps()); + + recordMerger = + BufferedRecordMergerFactory.create( + readerContext, + readerContext.getMergeMode(), + false, + readerContext.getRecordMerger(), + new Schema.Parser().parse(writeClient.getConfig().getSchema()), + readerContext.getPayloadClasses(writeClient.getConfig().getProps()), + writeClient.getConfig().getProps(), + metaClient.getTableConfig().getPartialUpdateMode()); + LOG.info("init hoodie merge with class [{}]", recordMerger.getClass().getName()); + } + + private boolean doBufferRecord(String bucketID, HoodieFlinkInternalRow record) + throws IOException { + try { + RowDataBucket bucket = + this.buckets.computeIfAbsent( + bucketID, + k -> + new RowDataBucket( + bucketID, + BufferUtils.createBuffer(rowType, memorySegmentPool), + getBucketInfo(record), + this.config.get(FlinkOptions.WRITE_BATCH_SIZE))); + + return bucket.writeRow(record.getRowData()); + } catch (MemoryPagesExhaustedException e) { + LOG.info( + "There is no enough free pages in memory pool to create buffer, need flushing first.", + e); + return false; + } + } + + /** + * Buffers the given record. + * + *

Flush the data bucket first if the bucket records size is greater than the configured + * value {@link FlinkOptions#WRITE_BATCH_SIZE}. + * + *

Flush the max size data bucket if the total buffer size exceeds the configured threshold + * {@link FlinkOptions#WRITE_TASK_MAX_SIZE}. + * + * @param record HoodieFlinkInternalRow + */ + protected void bufferRecord(HoodieFlinkInternalRow record) throws IOException { + writeMetrics.markRecordIn(); + // set operation type into rowkind of row. + record.getRowData() + .setRowKind( + RowKind.fromByteValue( + HoodieOperation.fromName(record.getOperationType()).getValue())); + final String bucketID = getBucketID(record.getPartitionPath(), record.getFileId()); + + // 1. try buffer the record into the memory pool + boolean success = doBufferRecord(bucketID, record); + if (!success) { + // 2. flushes the bucket if the memory pool is full + RowDataBucket bucketToFlush = + this.buckets.values().stream() + .max(Comparator.comparingLong(RowDataBucket::getBufferSize)) + .orElseThrow(NoSuchElementException::new); + if (flushBucket(bucketToFlush)) { + // 2.1 flushes the data bucket with maximum size + this.tracer.countDown(bucketToFlush.getBufferSize()); + disposeBucket(bucketToFlush.getBucketId()); + } else { + LOG.warn( + "The buffer size hits the threshold {}, but still flush the max size data bucket failed!", + this.tracer.maxBufferSize); + } + // 2.2 try to write row again + success = doBufferRecord(bucketID, record); + if (!success) { + throw new RuntimeException("Buffer is too small to hold a single record."); + } + } + RowDataBucket bucket = this.buckets.get(bucketID); + this.tracer.trace(bucket.getLastRecordSize()); + // 3. flushes the bucket if it is full + if (bucket.isFull()) { + if (flushBucket(bucket)) { + this.tracer.countDown(bucket.getBufferSize()); + disposeBucket(bucket.getBucketId()); + } + } + // update buffer metrics after tracing buffer size + writeMetrics.setWriteBufferedSize(this.tracer.bufferSize); + } + + private void disposeBucket(String bucketID) { + RowDataBucket bucket = this.buckets.remove(bucketID); + if (bucket != null) { + bucket.dispose(); + } + } + + private String getBucketID(String partitionPath, String fileId) { + return StreamerUtil.generateBucketKey(partitionPath, fileId); + } + + private static BucketInfo getBucketInfo(HoodieFlinkInternalRow internalRow) { + BucketType bucketType; + switch (internalRow.getInstantTime()) { + case "I": + bucketType = BucketType.INSERT; + break; + case "U": + bucketType = BucketType.UPDATE; + break; + default: + throw new HoodieException( + "Unexpected bucket type: " + internalRow.getInstantTime()); + } + return new BucketInfo(bucketType, internalRow.getFileId(), internalRow.getPartitionPath()); + } + + private boolean hasData() { + return !this.buckets.isEmpty() + && this.buckets.values().stream().anyMatch(bucket -> !bucket.isEmpty()); + } + + private boolean flushBucket(RowDataBucket bucket) { + return flushBucket(bucket, this.rowType); + } + + private boolean flushBucket(RowDataBucket bucket, RowType schemaToUse) { + String instant = instantToWriteForTable(true); + + if (instant == null) { + LOG.info("No inflight instant when flushing data, skip."); + return false; + } + + ValidationUtils.checkState( + !bucket.isEmpty(), "Data bucket to flush has no buffering records"); + final List writeStatus = writeRecords(instant, bucket, schemaToUse); + final WriteMetadataEvent event = + WriteMetadataEvent.builder() + .taskID(taskID) + .checkpointId(this.checkpointId) + .instantTime(instant) + .writeStatus(writeStatus) + .lastBatch(false) + .endInput(false) + .build(); + + this.eventGateway.sendEventToCoordinator(event); + writeStatuses.addAll(writeStatus); + return true; + } + + protected void flushRemaining(boolean endInput) { + writeMetrics.startDataFlush(); + this.currentInstant = instantToWriteForTable(hasData()); + if (this.currentInstant == null) { + if (hasData()) { + throw new HoodieException("No inflight instant when flushing data!"); + } else { + LOG.info("No data to flush and no inflight instant, sending empty commit metadata"); + final WriteMetadataEvent event = + WriteMetadataEvent.builder() + .taskID(taskID) + .checkpointId(checkpointId) + .instantTime(instantToWrite(false)) + .writeStatus(Collections.emptyList()) + .lastBatch(true) + .endInput(endInput) + .build(); + this.eventGateway.sendEventToCoordinator(event); + return; + } + } + final List writeStatus; + if (!buckets.isEmpty()) { + writeStatus = new ArrayList<>(); + // Create a snapshot of bucket IDs to avoid issues with disposed buckets + List bucketIds = new ArrayList<>(buckets.keySet()); + for (String bucketId : bucketIds) { + RowDataBucket bucket = buckets.get(bucketId); + if (bucket != null && !bucket.isEmpty()) { + writeStatus.addAll(writeRecords(currentInstant, bucket)); + } + // Remove and dispose bucket immediately after writing + disposeBucket(bucketId); + } + } else { + LOG.info("No data to write in subtask [{}] for instant [{}]", taskID, currentInstant); + writeStatus = Collections.emptyList(); + } + final WriteMetadataEvent event = + WriteMetadataEvent.builder() + .taskID(taskID) + .checkpointId(checkpointId) + .instantTime(currentInstant) + .writeStatus(writeStatus) + .lastBatch(true) + .endInput(endInput) + .build(); + + this.eventGateway.sendEventToCoordinator(event); + this.buckets.clear(); + this.tracer.reset(); + this.writeClient.cleanHandles(); + this.writeStatuses.addAll(writeStatus); + + writeMetrics.endDataFlush(); + writeMetrics.resetAfterCommit(); + } + + protected List writeRecords(String instant, RowDataBucket rowDataBucket) { + return writeRecords(instant, rowDataBucket, this.rowType); + } + + protected List writeRecords( + String instant, RowDataBucket rowDataBucket, RowType schemaToUse) { + writeMetrics.startFileFlush(); + + Iterator rowItr = + new MutableIteratorWrapperIterator<>( + rowDataBucket.getDataIterator(), + () -> new BinaryRowData(schemaToUse.getFieldCount())); + Iterator recordItr = + new MappingIterator<>( + rowItr, + rowData -> recordConverter.convert(rowData, rowDataBucket.getBucketInfo())); + + List statuses = + writeFunction.write( + deduplicateRecordsIfNeeded(recordItr), + rowDataBucket.getBucketInfo(), + instant); + writeMetrics.endFileFlush(); + writeMetrics.increaseNumOfFilesWritten(); + return statuses; + } + + protected Iterator deduplicateRecordsIfNeeded(Iterator records) { + if (config.get(FlinkOptions.PRE_COMBINE)) { + return FlinkWriteHelper.newInstance() + .deduplicateRecords( + records, + null, + -1, + this.writeClient.getConfig().getSchema(), + this.writeClient.getConfig().getProps(), + recordMerger, + readerContext, + orderingFieldNames.toArray(new String[0])); + } else { + return records; + } + } + + /** + * Table-specific version of instantToWrite that delegates to the parent's instantToWrite + * method. The table information is passed through the TableAwareCorrespondent that was set by + * MultiTableEventStreamWriteFunction. + */ + protected String instantToWriteForTable(boolean hasData) { + if (!hasData) { + return null; + } + + if (tableId == null) { + throw new IllegalStateException( + "TableId must be set before requesting instant from coordinator"); + } + + // Use the parent's instant request mechanism + // The TableAwareCorrespondent handles sending the table-specific requests + return instantToWrite(hasData); + } + + /** + * Flush all buckets immediately. Called when schema changes to ensure no data with old schema + * remains in buffers. + * + * @param schemaToUse The RowType schema to use for flushing (should be the OLD schema before + * the change) + */ + public void flushAllBuckets(RowType schemaToUse) { + LOG.info( + "Flushing all {} buckets with schema containing {} fields due to schema change", + buckets.size(), + schemaToUse.getFieldCount()); + if (buckets.isEmpty()) { + LOG.debug("No buckets to flush"); + return; + } + + // Create a snapshot of bucket IDs to avoid concurrent modification + List bucketIds = new ArrayList<>(buckets.keySet()); + + // Flush and dispose all buckets using the provided schema + for (String bucketId : bucketIds) { + RowDataBucket bucket = buckets.get(bucketId); + if (bucket != null && !bucket.isEmpty()) { + try { + flushBucket(bucket, schemaToUse); + } catch (Exception e) { + LOG.error("Failed to flush bucket {} during schema change", bucketId, e); + // Continue flushing other buckets even if one fails + } + } + // Dispose and remove bucket immediately to prevent access to disposed buckets + disposeBucket(bucketId); + } + + tracer.reset(); + LOG.info("All buckets flushed and cleared"); + } + + /** + * Update the rowType when schema evolves. This ensures new buffers are created with the correct + * schema. Note: keyGen is not updated since primary keys cannot change during schema evolution. + * + * @param newRowType The new RowType after schema evolution + */ + public void updateRowType(RowType newRowType) { + LOG.info( + "Updating RowType from {} fields to {} fields", + rowType.getFieldCount(), + newRowType.getFieldCount()); + this.rowType = newRowType; + + // Note: We do NOT call initMergeClass() here because: + // 1. We just flushed buffered data with OLD schema to parquet files + // 2. If we reinit merge components now, Hudi will expect NEW schema + // 3. During the next checkpoint, Hudi may need to read those files for merging + // 4. Reading old files with new converters may cause IndexOutOfBoundsException + // + // The merge components will use the old Avro schema until the next open() or + // until we explicitly update Hudi's table schema metadata via HudiMetadataApplier + + // Log active timeline state for debugging + logActiveTimeline(); + + LOG.info("RowType updated successfully"); + } + + public void updateWriteClientWithNewSchema(String newAvroSchema) { + this.config.setString(FlinkOptions.SOURCE_AVRO_SCHEMA.key(), newAvroSchema); + this.writeClient = FlinkWriteClients.createWriteClient(this.config, getRuntimeContext()); + } + + /** Logs the current state of the active timeline for debugging purposes. */ + private void logActiveTimeline() { + try { + if (metaClient != null) { + metaClient.reloadActiveTimeline(); + var activeTimeline = metaClient.getActiveTimeline(); + + LOG.info("Active timeline state for table {}:", tableId); + LOG.info( + " - Completed commits: {}", + activeTimeline + .getCommitsTimeline() + .filterCompletedInstants() + .countInstants()); + LOG.info( + " - Pending commits: {}", + activeTimeline + .getCommitsTimeline() + .filterPendingExcludingCompaction() + .countInstants()); + + List instantsInfo = new ArrayList<>(); + activeTimeline + .getInstants() + .forEach( + instant -> + instantsInfo.add( + instant.requestedTime() + + "(" + + instant.getState() + + ")")); + LOG.info(" - All instants: {}", instantsInfo); + + LOG.info( + " - Latest completed commit: {}", + activeTimeline + .getCommitsTimeline() + .filterCompletedInstants() + .lastInstant() + .map(instant -> instant.requestedTime()) + .orElse("None")); + } + } catch (Exception e) { + LOG.warn("Failed to log active timeline state", e); + } + } + + // metrics are now created per table via getOrCreateWriteMetrics(TableId) when needed + + // ------------------------------------------------------------------------- + // Getter/Setter + // ------------------------------------------------------------------------- + + @VisibleForTesting + @SuppressWarnings("rawtypes") + public Map> getDataBuffer() { + Map> ret = new HashMap<>(); + for (Map.Entry entry : buckets.entrySet()) { + List records = new ArrayList<>(); + Iterator rowItr = + new MutableIteratorWrapperIterator<>( + entry.getValue().getDataIterator(), + () -> new BinaryRowData(rowType.getFieldCount())); + while (rowItr.hasNext()) { + records.add( + recordConverter.convert(rowItr.next(), entry.getValue().getBucketInfo())); + } + ret.put(entry.getKey(), records); + } + return ret; + } + + // ------------------------------------------------------------------------- + // Inner Classes + // ------------------------------------------------------------------------- + + /** Write function to trigger the actual write action. */ + protected interface WriteFunction extends Serializable { + List write( + Iterator records, BucketInfo bucketInfo, String instant); + } +} 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..6f3ef1ff37d --- /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,590 @@ +/* + * 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.*; +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.CreateTableOperatorEvent; +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.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.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Collector; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +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.apache.hudi.util.ViewStorageProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Method; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Multi-table wrapper function that routes events to table-specific EventBucketStreamWriteFunction + * 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; + + private transient Map tableConfigurations; + + /** Schema evolution client to communicate with SchemaOperator */ + private transient SchemaEvolutionClient schemaEvolutionClient; + + /** 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 { + super.initializeState(context); + this.functionInitializationContext = context; + 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<>(); + this.schemaMaps = new HashMap<>(); + this.tableConfigurations = new HashMap<>(); + } + + @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); + } 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); + + initializedTables.computeIfAbsent( + tableId, + tId -> { + try { + // Send an explicit event to the coordinator so it can prepare + // resources *before* we attempt to write any data. + getOperatorEventGateway() + .sendEventToCoordinator( + new CreateTableOperatorEvent(createTableEvent)); + LOG.info( + "Sent CreateTableOperatorEvent to coordinator for new table: {}", + tId); + + // Now, create the physical dir for Hudi table. + Configuration tableConfig = createTableSpecificConfig(tId); + createHudiTablePath(tableConfig, tId); + } 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); + } + return true; // Mark as initialized for this function instance. + }); + // Ensure tableFunction is initialized + getOrCreateTableFunction(tableId); + } else if (event instanceof SchemaChangeEvent) { + SchemaChangeEvent schemaChangeEvent = event; + Schema existingSchema = schemaMaps.get(tableId); + if (existingSchema != null + && !SchemaUtils.isSchemaChangeEventRedundant( + existingSchema, schemaChangeEvent)) { + + LOG.info( + "Schema change event received for table {}: {}", + tableId, + schemaChangeEvent); + LOG.info( + "Existing schema for table {} has {} columns: {}", + tableId, + existingSchema.getColumnCount(), + existingSchema.getColumnNames()); + + Schema newSchema = + SchemaUtils.applySchemaChangeEvent(existingSchema, schemaChangeEvent); + + LOG.info( + "New schema for table {} has {} columns: {}", + tableId, + newSchema.getColumnCount(), + newSchema.getColumnNames()); + + schemaMaps.put(tableId, newSchema); + + // 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, flush buffers and update its rowType + EventBucketStreamWriteFunction tableFunction = tableFunctions.get(tableId); + if (tableFunction != null) { + LOG.info( + "Schema changed for table {}, flushing buffers with OLD schema and updating to NEW RowType", + tableId); + // NOTE: Capture the OLD RowType before any changes + // Buffered records were created with this schema + RowType oldRowType = convertSchemaToFlinkRowType(existingSchema); + + // Flush existing buffers using the OLD schema + // This ensures records buffered with N columns are read with N-column + // schema + tableFunction.flushAllBuckets(oldRowType); + + // Now safe to update to the NEW schema + // Future records will use this new schema + RowType newRowType = convertSchemaToFlinkRowType(newSchema); + tableFunction.updateRowType(newRowType); + + String newAvroSchema = + AvroSchemaConverter.convertToSchema(newRowType).toString(); + + LOG.info( + "Updating write client for table: {} with new schema: {}", + tableId, + newAvroSchema); + + // Update write client's source avro schema with new schema + tableFunction.updateWriteClientWithNewSchema(newAvroSchema); + + LOG.info("Successfully handled schema change for table: {}", tableId); + } + + LOG.debug("Updated schema for table: {}", tableId); + } + } + + // Forward the event to tableFunction so that schemaMap for serializer is updated + tableFunctions.get(event.tableId()).processSchemaChange(event); + } 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 (ChangeEvent) for writing. This triggers the actual Hudi write + * operations as side effects by delegating to table-specific functions. + * + *

Implements {@link EventProcessorFunction#processDataChange(DataChangeEvent)}. + */ + @Override + public void processDataChange(DataChangeEvent event) throws Exception { + TableId tableId = event.tableId(); + try { + LOG.debug("Processing change event for table: {}", tableId); + + // Get or create table-specific function to handle this event + EventBucketStreamWriteFunction tableFunction = getOrCreateTableFunction(tableId); + + // Use the table function to process the change event + // This will convert the event to HoodieFlinkInternalRow and buffer it for writing + tableFunction.processDataChange(event); + + 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); + } + } + + public static void createHudiTablePath(Configuration config, TableId tableId) + throws IOException { + String tablePath = config.get(FlinkOptions.PATH); + Path path = Paths.get(tablePath); + if (!Files.exists(path)) { + Files.createDirectories(path); + } + } + + /** + * Processes a flush event for a specific table function. This simulates the FlushEvent + * processing that would normally happen in EventStreamWriteFunction.processElement. + */ + private void processFlushForTableFunction( + EventBucketStreamWriteFunction tableFunction, Event flushEvent) { + try { + // Use reflection to access the protected flushRemaining method + // This is the same logic as in EventStreamWriteFunction when it processes FlushEvent + Method flushMethod = + tableFunction + .getClass() + .getSuperclass() + .getDeclaredMethod("flushRemaining", boolean.class); + flushMethod.setAccessible(true); + flushMethod.invoke(tableFunction, false); + } catch (Exception e) { + LOG.error( + "Failed to flush table function using reflection, falling back to endInput", e); + // Fallback: use endInput() which is public and also triggers flushing + tableFunction.endInput(); + } + } + + /** + * 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()) { + processFlushForTableFunction(entry.getValue(), event); + LOG.debug("Flushed table function for: {}", entry.getKey()); + } + } else { + LOG.info("Received flush event for {} specific tables", tableIds.size()); + for (TableId tableId : tableIds) { + EventBucketStreamWriteFunction tableFunction = tableFunctions.get(tableId); + if (tableFunction != null) { + processFlushForTableFunction(tableFunction, event); + 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 EventBucketStreamWriteFunction getOrCreateTableFunction(TableId tableId) { + EventBucketStreamWriteFunction existingFunction = tableFunctions.get(tableId); + if (existingFunction != null) { + return existingFunction; + } + + LOG.info("Creating new EventBucketStreamWriteFunction for table: {}", tableId); + try { + EventBucketStreamWriteFunction tableFunction = createTableFunction(tableId); + tableFunctions.put(tableId, tableFunction); + initializedTables.put(tableId, true); + 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 EventBucketStreamWriteFunction 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); + + EventBucketStreamWriteFunction tableFunction = + new EventBucketStreamWriteFunction(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); + tableFunction.setTableId(tableId); + + // This is the key change: 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); + 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); + + if (tableFunction.getRecordSerializer() instanceof HudiRecordEventSerializer) { + HudiRecordEventSerializer serializer = + (HudiRecordEventSerializer) tableFunction.getRecordSerializer(); + serializer.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).split("/coordinator")[0]; + if (rootPath != null) { + String tableBasePath = + String.format( + "%s/%s/%s", + rootPath, tableId.getSchemaName(), tableId.getTableName()); + localTableConfig.set(FlinkOptions.PATH, tableBasePath); + } + + // Modify ViewStorageProperties to point to coordinator table + FileSystemViewStorageConfig viewStorageConfig = + ViewStorageProperties.loadFromProperties( + this.config.get(FlinkOptions.PATH), localTableConfig); + localTableConfig.setString( + FileSystemViewStorageConfig.VIEW_TYPE.key(), + viewStorageConfig.getStorageType().name()); + localTableConfig.setString( + FileSystemViewStorageConfig.REMOTE_HOST_NAME.key(), + viewStorageConfig.getRemoteViewServerHost()); + localTableConfig.setString( + FileSystemViewStorageConfig.REMOTE_PORT_NUM.key(), + viewStorageConfig.getRemoteViewServerPort() + ""); + 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); + for (Map.Entry entry : tableFunctions.entrySet()) { + try { + LOG.debug("Delegating snapshotState for table: {}", entry.getKey()); + entry.getValue().snapshotState(); + } 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); + } + } + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + snapshotState(); + } + + protected void flushRemaining(boolean endInput) { + boolean hasData = !tableFunctions.isEmpty(); + this.currentInstant = instantToWrite(hasData); + + if (this.currentInstant == null) { + if (hasData) { + throw new RuntimeException( + "No inflight instant when flushing data for multi-table function!"); + } + } + + LOG.debug( + "Multi-table function requested instant: {} for {} table functions", + this.currentInstant, + tableFunctions.size()); + + // This method is intentionally overridden to be a no-op. + // The MultiTableEventStreamWriteFunction is a dispatcher and does not have its own + // data buffers to flush. Flushing is handled by the individual, table-specific + // write functions it manages. Calling the parent's flushRemaining would cause + // an erroneous, non-table-specific instant request to be sent to the coordinator, + // resulting in the NullPointerException. + } + + @Override + public void close() throws Exception { + for (EventBucketStreamWriteFunction func : tableFunctions.values()) { + try { + func.close(); + } catch (Exception e) { + LOG.error("Failed to close table function", e); + } + } + super.close(); + } + + public void endInput() { + super.endInput(); + flushRemaining(true); + for (EventBucketStreamWriteFunction func : tableFunctions.values()) { + try { + func.endInput(); + } catch (Exception e) { + LOG.error("Failed to complete endInput for table function", e); + } + } + } + + /** + * 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/model/BucketAssignmentIndex.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/model/BucketAssignmentIndex.java new file mode 100644 index 00000000000..a443f8a404c --- /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/model/BucketAssignmentIndex.java @@ -0,0 +1,82 @@ +package org.apache.flink.cdc.connectors.hudi.sink.model; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +/** + * A class to manage the bucket assignment index which maps partitions and bucket numbers to file + * IDs for a single table. Each EventBucketStreamWriteFunction instance handles one table, so this + * class only needs to track partitions within that table. The structure is: {@code PartitionPath -> + * BucketId -> FileId}. + */ +public class BucketAssignmentIndex implements Serializable { + + private static final long serialVersionUID = 1L; + + /** Index mapping partition paths to bucket-to-fileId mappings for a single table */ + private final Map> index; + + public BucketAssignmentIndex() { + this.index = new HashMap<>(); + } + + /** + * Retrieves the File ID for a given partition and bucket number. + * + * @param partition The partition path. + * @param bucketNum The bucket number. + * @return An Optional containing the file ID if it exists, otherwise an empty Optional. + */ + public Optional getFileId(String partition, int bucketNum) { + return Optional.ofNullable(index.get(partition)).map(bucketMap -> bucketMap.get(bucketNum)); + } + + /** + * Associates the specified file ID with the specified partition and bucket number. + * + * @param partition The partition path. + * @param bucketNum The bucket number. + * @param fileId The file ID to associate with the bucket. + */ + public void putFileId(String partition, int bucketNum, String fileId) { + index.computeIfAbsent(partition, k -> new HashMap<>()).put(bucketNum, fileId); + } + + /** + * Checks if the index contains mappings for the specified partition. + * + * @param partition The partition path. + * @return true if the index contains a mapping for the partition, false otherwise. + */ + public boolean containsPartition(String partition) { + return index.containsKey(partition); + } + + /** + * Bootstraps the index for a new partition with a pre-loaded map of bucket numbers to file IDs. + * + * @param partition The partition path. + * @param bucketToFileIDMap The map of bucket numbers to file IDs for the partition. + */ + public void bootstrapPartition(String partition, Map bucketToFileIDMap) { + index.put(partition, bucketToFileIDMap); + } + + /** + * Gets the map from bucket number to file ID for a given partition. Creates and returns an + * empty map if one does not exist. + * + * @param partition The partition path. + * @return The map of bucket numbers to file IDs. + */ + public Map getBucketToFileIdMap(String partition) { + return index.computeIfAbsent(partition, k -> new HashMap<>()); + } + + /** Clears the entire index. */ + public void clear() { + index.clear(); + } +} 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..1683896d358 --- /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,161 @@ +/* + * 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.configuration.FlinkOptions; +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(config, schemaOperatorUid, new MultiTableEventStreamWriteFunction(config)); + } + + private MultiTableWriteOperator( + Configuration config, + 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().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) { + // Create coordinator-specific configuration with dummy table settings + // This satisfies the coordinator's requirement for table initialization + Configuration coordinatorConfig = createCoordinatorConfig(conf); + + LOG.info("Creating multi-table write operator factory with extended coordinator support"); + return MultiTableWriteOperatorFactory.instance( + coordinatorConfig, + new MultiTableWriteOperator(coordinatorConfig, schemaOperatorUid)); + } + + /** + * Creates a coordinator-specific configuration with dummy table settings. This satisfies the + * coordinator's requirement for table initialization while actual table routing happens + * dynamically based on incoming events. Uses deterministic naming to allow reuse of existing + * coordinator tables. + */ + private static Configuration createCoordinatorConfig(Configuration originalConfig) { + Configuration coordinatorConfig = new Configuration(); + coordinatorConfig.addAll(originalConfig); + + // Create deterministic dummy table name based on base path hash for reusability + String originalPath = coordinatorConfig.get(FlinkOptions.PATH, "default"); + String pathHash = String.valueOf(Math.abs(originalPath.hashCode())); + String dummyTableName = "coordinator_" + pathHash; + + coordinatorConfig.set(FlinkOptions.TABLE_NAME, dummyTableName); + coordinatorConfig.set(FlinkOptions.DATABASE_NAME, "coordinator_db"); + + // Set deterministic path for coordinator table (allows reuse) + String coordinatorPath = originalPath + "/coordinator/" + dummyTableName; + coordinatorConfig.set(FlinkOptions.PATH, coordinatorPath); + + // Set dummy Avro schema with a simple structure (id: int) + String dummyAvroSchema = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"coordinator_record\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"id\",\n" + + " \"type\": \"int\"\n" + + " }\n" + + " ]\n" + + "}"; + coordinatorConfig.set(FlinkOptions.SOURCE_AVRO_SCHEMA, dummyAvroSchema); + + LOG.info( + "Created coordinator config with reusable dummy table: coordinator_db.{} at path: {}", + dummyTableName, + coordinatorPath); + return coordinatorConfig; + } +} 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..fd5ab899388 --- /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,103 @@ +/* + * 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.*; +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..670f1a83bb9 --- /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,472 @@ +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.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.*; +import org.apache.flink.types.RowKind; + +import org.apache.hudi.client.model.HoodieFlinkInternalRow; + +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. + * + * @param dataChangeEvent The DataChangeEvent to convert + * @param schema Schema for the table + * @param zoneId Time zone for timestamp conversion + * @param recordKey The record key extracted from the event + * @param partitionPath The partition path extracted from the event + * @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, + String recordKey, + String partitionPath, + String fileId, + String instantTime) { + + // Convert DataChangeEvent to RowData using existing utility + List fieldGetters = createFieldGetters(schema, zoneId); + RowData rowData = convertDataChangeEventToRowData(dataChangeEvent, fieldGetters); + + // 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 + ); + } + + /** + * Convert a DataChangeEvent to a HoodieFlinkInternalRow with automatic record key and partition + * path extraction. + * + * @param dataChangeEvent The DataChangeEvent to convert + * @param schema Schema for the table + * @param zoneId Time zone for timestamp conversion + * @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, + String fileId, + String instantTime) { + + // Extract record key from primary key fields + String recordKey = extractRecordKeyFromDataChangeEvent(dataChangeEvent, schema); + + // Default partition path - in real implementation this would be based on configured + // partition fields + String partitionPath = "default"; + + return convertDataChangeEventToHoodieFlinkInternalRow( + dataChangeEvent, schema, zoneId, recordKey, partitionPath, fileId, instantTime); + } + + /** 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. */ + private 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 recordKeyValues = 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"); + } + + recordKeyValues.add(fieldValue.toString()); + } + + return String.join(",", recordKeyValues); + } + + /** 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()); + } + } +} 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..a3833bc3a0b --- /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,136 @@ +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.table.types.logical.RowType; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.ZoneId; + +public class HudiSink implements Sink, WithPreWriteTopology { + + private static final Logger LOG = LoggerFactory.getLogger(HudiSink.class); + + private final Configuration conf; + // TODO: Check if these are used, remove them if unused + private final RowType rowType; + private final boolean overwrite; + private final boolean isBounded; + + private final String schemaOperatorUid; + + public HudiSink( + Configuration conf, + RowType rowType, + boolean overwrite, + boolean isBounded, + String schemaOperatorUid, + ZoneId zoneId) { + LOG.info("Creating Hoodie sink with conf: {}", conf); + this.conf = conf; + this.rowType = rowType; + this.overwrite = overwrite; + this.isBounded = isBounded; + 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..20d11a57d18 --- /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,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink.v2; + +import static java.nio.charset.StandardCharsets.UTF_8; + +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.shaded.guava31.com.google.common.hash.Hashing; + +/** 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 95389510306..d7ef9afb1c1 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-SNAPSHOT @@ -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..4584f657768 --- /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,517 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.pipeline.tests; + +import org.apache.flink.cdc.common.test.utils.TestUtils; +import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; +import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; +import org.apache.flink.cdc.pipeline.tests.utils.TarballFetcher; +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.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.*; +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"; + + protected final UniqueDatabase inventoryDatabase = + new UniqueDatabase(MYSQL, "hudi_inventory", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + + private String warehouse; + + @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..."); + jobManagerConsumer = new ToStringConsumer(); + // Using FixedHost instead of GenericContainer to ensure that ports are fixed for easier + // debugging during dev + jobManager = + new FixedHostPortGenericContainer<>(getFlinkDockerImageTag()) + .withCommand("jobmanager") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_JM_ALIAS) + .withExposedPorts(JOB_MANAGER_REST_PORT) + .withFixedExposedPort(8081, 8081) + .withFixedExposedPort(9005, 9005) + .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) + .withEnv( + "FLINK_ENV_JAVA_OPTS", + "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:9005") + .withCreateContainerCmdModifier(cmd -> cmd.withVolumes(sharedVolume)) + .withLogConsumer(jobManagerConsumer); + Startables.deepStart(Stream.of(jobManager)).join(); + runInContainerAsRoot(jobManager, "chmod", "0777", "-R", sharedVolume.toString()); + + taskManagerConsumer = new ToStringConsumer(); + taskManager = + new FixedHostPortGenericContainer<>(getFlinkDockerImageTag()) + .withCommand("taskmanager") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_TM_ALIAS) + .withFixedExposedPort(9006, 9006) + .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) + .withEnv( + "FLINK_ENV_JAVA_OPTS", + "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:9006") + .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: 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", + 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"); + submitPipelineJob( + pipelineJob, + hudiCdcConnector, + hudiHadoopCommonJar, + hadoopJar, + hadoopCompatibilityJar, + dropMetricsJar, + flinkParquet); + waitUntilJobRunning(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); + } + + /** + * 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: + * + *

    + *
  1. 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. + *
  2. 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 java.sql.Statement} object used to execute the SQL commands. + * @return A {@link java.util.List} of strings, where each string is a CSV representation of an + * inserted row, reflecting the expected state in the database. + * @throws java.sql.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"; + } + + 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); + } + + 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"); + } +} diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java index c63c54ad627..f0002ddfa88 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java @@ -23,7 +23,6 @@ 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.hadoop.conf.Configuration; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.Table; @@ -243,6 +242,7 @@ public void testSyncWholeDatabase() throws Exception { LOG.error("Update table for CDC failed.", e); throw e; } + List recordsInSnapshotPhase = new ArrayList<>( Arrays.asList( @@ -261,6 +261,7 @@ public void testSyncWholeDatabase() throws Exception { recordsInSnapshotPhase = recordsInSnapshotPhase.stream().sorted().collect(Collectors.toList()); validateSinkResult(warehouse, database, "products", recordsInSnapshotPhase); + Thread.sleep(3600000L); } /** diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java index 914278bcbcc..24877b18fba 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java @@ -17,6 +17,12 @@ package org.apache.flink.cdc.pipeline.tests.utils; +import static org.apache.flink.util.Preconditions.checkState; + +import com.github.dockerjava.api.DockerClient; +import com.github.dockerjava.api.command.ExecCreateCmdResponse; +import com.github.dockerjava.api.model.Volume; + import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Deadline; @@ -30,10 +36,6 @@ import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.table.api.ValidationException; import org.apache.flink.util.TestLogger; - -import com.github.dockerjava.api.DockerClient; -import com.github.dockerjava.api.command.ExecCreateCmdResponse; -import com.github.dockerjava.api.model.Volume; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.slf4j.Logger; @@ -53,8 +55,6 @@ import org.testcontainers.lifecycle.Startables; import org.testcontainers.utility.MountableFile; -import javax.annotation.Nullable; - import java.io.File; import java.io.IOException; import java.net.URL; @@ -72,7 +72,7 @@ import java.util.function.Function; import java.util.stream.Stream; -import static org.apache.flink.util.Preconditions.checkState; +import javax.annotation.Nullable; /** Test environment running pipeline job on Flink containers. */ @Testcontainers @@ -135,7 +135,7 @@ private int getParallelism() { "blob.server.port: 6124", "taskmanager.numberOfTaskSlots: 10", "parallelism.default: 4", - "execution.checkpointing.interval: 300", + "execution.checkpointing.interval: 30s", "state.backend.type: hashmap", "env.java.opts.all: -Doracle.jdbc.timezoneAsRegion=false", "execution.checkpointing.savepoint-dir: file:///opt/flink", @@ -389,6 +389,11 @@ public void waitUntilJobState(Duration timeout, JobStatus expectedStatus) { JobStatusMessage message = jobStatusMessages.iterator().next(); JobStatus jobStatus = message.getJobState(); if (!expectedStatus.isTerminalState() && jobStatus.isTerminalState()) { + try { + Thread.sleep(50000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } throw new ValidationException( String.format( "Job has been terminated! JobName: %s, JobID: %s, Status: %s", 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 diff --git a/pom.xml b/pom.xml index 0d9b53d7c95..f9592e5a47c 100644 --- a/pom.xml +++ b/pom.xml @@ -58,6 +58,14 @@ limitations under the License. scm:git:https://gitbox.apache.org/repos/asf/flink-cdc.git + + + local-repo + ${user.home}/.m2/repository + + + + 3.6-SNAPSHOT 2.12 From dac5e0163221e101179c3a208e21f87580f19016 Mon Sep 17 00:00:00 2001 From: voon Date: Mon, 13 Oct 2025 20:50:48 +0800 Subject: [PATCH 02/33] Checkpoint 34 - Fix checkstyle and RAT --- .../cdc/connectors/hudi/sink/HudiConfig.java | 18 +++++ .../connectors/hudi/sink/HudiDataSink.java | 17 +++++ .../hudi/sink/HudiDataSinkFactory.java | 24 ++++++- .../hudi/sink/HudiMetadataApplier.java | 18 +++++ .../sink/bucket/BucketAssignOperator.java | 70 ++++++++++++------- .../hudi/sink/bucket/BucketWrapper.java | 6 +- .../sink/bucket/BucketWrapperFlushEvent.java | 2 +- .../bucket/FlushEventAlignmentOperator.java | 5 +- ...tiTableStreamWriteOperatorCoordinator.java | 1 + .../sink/event/HudiRecordEventSerializer.java | 1 + .../sink/event/TableAwareCorrespondent.java | 1 + .../EventBucketStreamWriteFunction.java | 18 +++++ .../sink/function/EventProcessorFunction.java | 4 +- .../function/EventStreamWriteFunction.java | 6 +- .../MultiTableEventStreamWriteFunction.java | 1 + .../sink/model/BucketAssignmentIndex.java | 17 +++++ .../operator/MultiTableWriteOperator.java | 1 + .../hudi/sink/util/RowDataUtils.java | 17 +++++ .../cdc/connectors/hudi/sink/v2/HudiSink.java | 18 +++++ .../hudi/sink/v2/OperatorIDGenerator.java | 5 +- .../pipeline/tests/MySqlToHudiE2eITCase.java | 1 + .../tests/MySqlToIcebergE2eITCase.java | 1 + .../tests/utils/PipelineTestEnvironment.java | 14 ++-- 23 files changed, 217 insertions(+), 49 deletions(-) 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/HudiConfig.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiConfig.java index 1dd2d31649b..0a7ef8aadc8 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiConfig.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiConfig.java @@ -1,8 +1,26 @@ +/* + * 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.ConfigOption; import org.apache.flink.cdc.common.configuration.ConfigOptions; import org.apache.flink.configuration.description.Description; + import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.configuration.FlinkOptions; 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/HudiDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSink.java index e58d7115381..7a8b9e169e2 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSink.java @@ -1,3 +1,20 @@ +/* + * 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; 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/HudiDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactory.java index c9461fb6a1d..41fc6eef3db 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactory.java @@ -1,19 +1,37 @@ -package org.apache.flink.cdc.connectors.hudi.sink; +/* + * 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. + */ -import static org.apache.flink.cdc.connectors.hudi.sink.HudiDataSinkOptions.PREFIX_CATALOG_PROPERTIES; -import static org.apache.flink.cdc.connectors.hudi.sink.HudiDataSinkOptions.PREFIX_TABLE_PROPERTIES; +package org.apache.flink.cdc.connectors.hudi.sink; import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.factories.DataSinkFactory; import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.pipeline.PipelineOptions; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.HashSet; import java.util.Set; +import static org.apache.flink.cdc.connectors.hudi.sink.HudiDataSinkOptions.PREFIX_CATALOG_PROPERTIES; +import static org.apache.flink.cdc.connectors.hudi.sink.HudiDataSinkOptions.PREFIX_TABLE_PROPERTIES; + /** * Factory for creating {@link HudiDataSink}. This class defines the configuration options and * instantiates the sink by delegating option definitions to {@link HudiConfig}. 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 index b7ebd4166e5..5ad028658be 100644 --- 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 @@ -1,3 +1,20 @@ +/* + * 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; @@ -18,6 +35,7 @@ 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.common.config.HoodieMetadataConfig; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.configuration.FlinkOptions; 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 index 776df24b3be..4d961af0b91 100644 --- 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 @@ -18,7 +18,6 @@ package org.apache.flink.cdc.connectors.hudi.sink.bucket; import org.apache.flink.cdc.common.data.RecordData; -import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.FlushEvent; @@ -55,8 +54,10 @@ * 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 + *
  • 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 *
@@ -108,7 +109,10 @@ public void open() throws Exception { super.open(); this.totalTasksNumber = getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks(); this.currentTaskNumber = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); - LOG.info("BucketAssignOperator opened with {} buckets and {} tasks", numBuckets, totalTasksNumber); + LOG.info( + "BucketAssignOperator opened with {} buckets and {} tasks", + numBuckets, + totalTasksNumber); } @Override @@ -178,12 +182,15 @@ private int calculateBucket(DataChangeEvent event) { schemaCache.put(tableId, schema); } else { throw new IllegalStateException( - "No schema available for table " + tableId + " in bucket assignment. " + - "Could not find schema from SchemaOperator coordinator."); + "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); + "Failed to retrieve schema for table " + tableId + " from SchemaOperator", + e); } } @@ -191,7 +198,8 @@ private int calculateBucket(DataChangeEvent event) { final Schema finalSchema = schema; // Get or cache primary keys - List primaryKeys = primaryKeyCache.computeIfAbsent(tableId, k -> finalSchema.primaryKeys()); + List primaryKeys = + primaryKeyCache.computeIfAbsent(tableId, k -> finalSchema.primaryKeys()); if (primaryKeys.isEmpty()) { throw new IllegalStateException( @@ -202,20 +210,28 @@ private int calculateBucket(DataChangeEvent event) { final List finalPrimaryKeys = primaryKeys; // Get or cache field getters - List fieldGetters = fieldGetterCache.computeIfAbsent(tableId, k -> { - List getters = new ArrayList<>(finalPrimaryKeys.size()); - for (String primaryKeyField : finalPrimaryKeys) { - int fieldIndex = finalSchema.getColumnNames().indexOf(primaryKeyField); - if (fieldIndex == -1) { - throw new IllegalStateException( - "Primary key field '" + primaryKeyField + - "' not found in schema for table " + tableId); - } - DataType fieldType = finalSchema.getColumns().get(fieldIndex).getType(); - getters.add(RecordData.createFieldGetter(fieldType, fieldIndex)); - } - return getters; - }); + List fieldGetters = + fieldGetterCache.computeIfAbsent( + tableId, + k -> { + List getters = + new ArrayList<>(finalPrimaryKeys.size()); + for (String primaryKeyField : finalPrimaryKeys) { + int fieldIndex = + finalSchema.getColumnNames().indexOf(primaryKeyField); + if (fieldIndex == -1) { + throw new IllegalStateException( + "Primary key field '" + + primaryKeyField + + "' not found in schema for table " + + tableId); + } + DataType fieldType = + finalSchema.getColumns().get(fieldIndex).getType(); + getters.add(RecordData.createFieldGetter(fieldType, fieldIndex)); + } + return getters; + }); // Extract record key String recordKey = extractRecordKey(event, primaryKeys, fieldGetters); @@ -225,12 +241,12 @@ private int calculateBucket(DataChangeEvent event) { return BucketIdentifier.getBucketId(recordKey, tableIndexKeyFields, numBuckets); } - private String extractRecordKey(DataChangeEvent event, List primaryKeys, - List fieldGetters) { + private String extractRecordKey( + DataChangeEvent event, + List primaryKeys, + List fieldGetters) { // For DELETE, use 'before' data; for INSERT/UPDATE, use 'after' data - RecordData recordData = event.op() == OperationType.DELETE - ? event.before() - : event.after(); + RecordData recordData = event.op() == OperationType.DELETE ? event.before() : event.after(); if (recordData == null) { throw new IllegalStateException( @@ -253,4 +269,4 @@ private String extractRecordKey(DataChangeEvent event, List primaryKeys, return String.join(",", recordKeyPairs); } -} \ No newline at end of file +} 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 index 75f12e9693c..794f80a66b9 100644 --- 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 @@ -22,8 +22,8 @@ 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. + * 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. @@ -47,4 +47,4 @@ public int getBucket() { public Event getEvent() { return event; } -} \ No newline at end of file +} 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 index 391005b57f3..0aa53996bdb 100644 --- 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 @@ -81,4 +81,4 @@ public String toString() { + bucket + '}'; } -} \ No newline at end of file +} 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 index 67575358c96..615959916b8 100644 --- 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 @@ -79,7 +79,8 @@ public void processElement(StreamRecord streamRecord) { new FlushEvent( sourceSubTaskId, bucketWrapperFlushEvent.getTableIds(), - bucketWrapperFlushEvent.getSchemaChangeEventType())))); + bucketWrapperFlushEvent + .getSchemaChangeEventType())))); sourceTaskIdToAssignBucketSubTaskIds.remove(sourceSubTaskId); } else { LOG.info( @@ -93,4 +94,4 @@ public void processElement(StreamRecord streamRecord) { output.collect(streamRecord); } } -} \ No newline at end of file +} 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 index d9f2ba3b27c..62985667d65 100644 --- 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 @@ -30,6 +30,7 @@ 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.config.HoodieMetadataConfig; 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 index b8118ab79ce..8a5cfa7ba1c 100644 --- 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 @@ -26,6 +26,7 @@ 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 java.time.ZoneId; 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 index c1f95e99b47..71ecb3a020d 100644 --- 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 @@ -15,6 +15,7 @@ * 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; 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/EventBucketStreamWriteFunction.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/EventBucketStreamWriteFunction.java index a9266db3264..66c1423cd3b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventBucketStreamWriteFunction.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/EventBucketStreamWriteFunction.java @@ -1,3 +1,20 @@ +/* + * 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.data.RecordData; @@ -13,6 +30,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.table.types.logical.RowType; + import org.apache.hudi.client.model.HoodieFlinkInternalRow; import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.hash.BucketIndexUtil; 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 index 7da23baba4f..3d0458112a3 100644 --- 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 @@ -30,8 +30,8 @@ * *

    *
  • {@link #processDataChange(DataChangeEvent)} - Handles INSERT/UPDATE/DELETE operations - *
  • {@link #processSchemaChange(SchemaChangeEvent)} - Handles DDL operations (CREATE TABLE, - * ADD COLUMN, etc.) + *
  • {@link #processSchemaChange(SchemaChangeEvent)} - Handles DDL operations (CREATE TABLE, ADD + * COLUMN, etc.) *
  • {@link #processFlush(FlushEvent)} - Handles coordinated flushing of buffered data *
*/ 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/EventStreamWriteFunction.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/EventStreamWriteFunction.java index 668a4ace2cf..9c3fe4bcb77 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventStreamWriteFunction.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/EventStreamWriteFunction.java @@ -17,7 +17,6 @@ package org.apache.flink.cdc.connectors.hudi.sink.function; -import org.apache.avro.Schema; import org.apache.flink.cdc.common.event.*; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.MetricGroup; @@ -28,6 +27,8 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.RowKind; import org.apache.flink.util.Collector; + +import org.apache.avro.Schema; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.model.HoodieFlinkInternalRow; import org.apache.hudi.common.engine.HoodieReaderContext; @@ -36,6 +37,7 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.read.BufferedRecordMerger; import org.apache.hudi.common.table.read.BufferedRecordMergerFactory; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.common.util.collection.MappingIterator; @@ -623,7 +625,7 @@ private void logActiveTimeline() { try { if (metaClient != null) { metaClient.reloadActiveTimeline(); - var activeTimeline = metaClient.getActiveTimeline(); + HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); LOG.info("Active timeline state for table {}:", tableId); LOG.info( 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 index 6f3ef1ff37d..c6867a3fe65 100644 --- 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 @@ -34,6 +34,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; + import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.sink.common.AbstractStreamWriteFunction; 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/model/BucketAssignmentIndex.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/model/BucketAssignmentIndex.java index a443f8a404c..3f7b6bf4014 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/model/BucketAssignmentIndex.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/model/BucketAssignmentIndex.java @@ -1,3 +1,20 @@ +/* + * 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.model; import java.io.Serializable; 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 index 1683896d358..2cff14ae143 100644 --- 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 @@ -24,6 +24,7 @@ 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.configuration.FlinkOptions; import org.apache.hudi.sink.common.AbstractWriteOperator; import org.slf4j.Logger; 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 index 670f1a83bb9..151f04978ab 100644 --- 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 @@ -1,3 +1,20 @@ +/* + * 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; 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 index a3833bc3a0b..25c3abac901 100644 --- 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 @@ -1,3 +1,20 @@ +/* + * 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; @@ -16,6 +33,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 index 20d11a57d18..c00aa751608 100644 --- 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 @@ -17,11 +17,12 @@ package org.apache.flink.cdc.connectors.hudi.sink.v2; -import static java.nio.charset.StandardCharsets.UTF_8; - 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 { 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 index 4584f657768..1752550ae67 100644 --- 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 @@ -21,6 +21,7 @@ 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.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java index f0002ddfa88..e3f2510693b 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java @@ -23,6 +23,7 @@ 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.hadoop.conf.Configuration; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.Table; diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java index 24877b18fba..a631fd02873 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java @@ -17,12 +17,6 @@ package org.apache.flink.cdc.pipeline.tests.utils; -import static org.apache.flink.util.Preconditions.checkState; - -import com.github.dockerjava.api.DockerClient; -import com.github.dockerjava.api.command.ExecCreateCmdResponse; -import com.github.dockerjava.api.model.Volume; - import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Deadline; @@ -36,6 +30,10 @@ import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.table.api.ValidationException; import org.apache.flink.util.TestLogger; + +import com.github.dockerjava.api.DockerClient; +import com.github.dockerjava.api.command.ExecCreateCmdResponse; +import com.github.dockerjava.api.model.Volume; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.slf4j.Logger; @@ -55,6 +53,8 @@ import org.testcontainers.lifecycle.Startables; import org.testcontainers.utility.MountableFile; +import javax.annotation.Nullable; + import java.io.File; import java.io.IOException; import java.net.URL; @@ -72,7 +72,7 @@ import java.util.function.Function; import java.util.stream.Stream; -import javax.annotation.Nullable; +import static org.apache.flink.util.Preconditions.checkState; /** Test environment running pipeline job on Flink containers. */ @Testcontainers From 46d90fd16c17038b4a0280579477178ed3604839 Mon Sep 17 00:00:00 2001 From: voon Date: Mon, 13 Oct 2025 22:37:01 +0800 Subject: [PATCH 03/33] Checkpoint 35 - Remove unused code --- .../hudi/sink/HudiMetadataApplier.java | 83 ------------------- 1 file changed, 83 deletions(-) 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 index 5ad028658be..3919ec12cc1 100644 --- 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 @@ -35,14 +35,8 @@ 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.common.config.HoodieMetadataConfig; -import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.configuration.FlinkOptions; -import org.apache.hudi.storage.HoodieStorage; -import org.apache.hudi.storage.StorageConfiguration; -import org.apache.hudi.storage.StoragePath; -import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; import org.apache.hudi.table.catalog.CatalogOptions; import org.apache.hudi.table.catalog.HoodieCatalog; import org.slf4j.Logger; @@ -492,79 +486,6 @@ private ResolvedCatalogTable createUpdatedCatalogTable( return new ResolvedCatalogTable(catalogTable, newSchema); } - /** - * Updates the actual Hudi table schema stored in hoodie.properties. This is necessary because - * catalog.alterTable() only updates table_option.properties, not the actual Hudi table schema. - */ - private void updateHudiTableSchema(ObjectPath objectPath, ResolvedSchema newSchema) - throws Exception { - // Get table path - String basePath = catalogConfig.getString(CatalogOptions.CATALOG_PATH.key(), null); - String tablePath = - String.format( - "%s/%s/%s", - basePath, objectPath.getDatabaseName(), objectPath.getObjectName()); - - LOG.info("Updating Hudi table schema at path: {}", tablePath); - - // Convert Flink schema to Avro schema - org.apache.avro.Schema avroSchema = convertFlinkSchemaToAvro(newSchema); - - // Update the table schema using HoodieTableMetaClient - org.apache.hadoop.conf.Configuration hadoopConf = - new org.apache.hadoop.conf.Configuration(); - StorageConfiguration storageConf = new HadoopStorageConfiguration(hadoopConf); - - org.apache.hudi.common.table.HoodieTableMetaClient metaClient = - org.apache.hudi.common.table.HoodieTableMetaClient.builder() - .setConf(storageConf) - .setBasePath(tablePath) - .build(); - - // Commit the schema change by updating table config - HoodieTableConfig tableConfig = metaClient.getTableConfig(); - HoodieStorage storage = metaClient.getStorage(); - - // Write the updated schema to hoodie.properties - // Hudi stores the schema under "hoodie.table.schema" which is what HoodieCatalog reads - Properties props = new Properties(); - props.putAll(tableConfig.getProps()); - - String schemaStr = avroSchema.toString(); - // Update the create schema (historical record) - props.setProperty(HoodieTableConfig.CREATE_SCHEMA.key(), schemaStr); - // Also update the current schema property that the catalog reads - props.setProperty("hoodie.table.schema", schemaStr); - - StoragePath propsPath = - new StoragePath(metaClient.getMetaPath(), HoodieTableConfig.HOODIE_PROPERTIES_FILE); - - try (java.io.OutputStream outputStream = storage.create(propsPath, true)) { - props.store(outputStream, "Updated by Flink CDC HudiMetadataApplier"); - } - - LOG.info("Successfully updated Hudi table schema"); - } - - /** Converts Flink ResolvedSchema to Avro Schema for Hudi. */ - private org.apache.avro.Schema convertFlinkSchemaToAvro(ResolvedSchema flinkSchema) { - List avroFields = new ArrayList<>(); - - for (org.apache.flink.table.catalog.Column column : flinkSchema.getColumns()) { - org.apache.avro.Schema fieldSchema = convertFlinkTypeToAvro(column.getDataType()); - org.apache.avro.Schema.Field field = - new org.apache.avro.Schema.Field( - column.getName(), fieldSchema, null, (Object) null); - avroFields.add(field); - } - - org.apache.avro.Schema avroSchema = - org.apache.avro.Schema.createRecord( - "HudiSchema", null, "org.apache.hudi", false, avroFields); - - return avroSchema; - } - /** 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(); @@ -612,10 +533,6 @@ private org.apache.avro.Schema convertFlinkTypeToAvro(DataType flinkType) { private void updateAndVerifyTableChange( TableId tableId, ObjectPath objectPath, ResolvedSchema newSchema) throws Exception { - // TODO: Check if this is necessary - // updateHudiTableSchema(objectPath, newSchema); - LOG.info("Successfully modified schema for table: {}", tableId); - ResolvedCatalogTable verifyTable = getResolvedCatalogTable(objectPath); LOG.info( "Verified - Table {} now has columns: {}", From c236bb4a183462b5a764e0f56673a005dbf11dd8 Mon Sep 17 00:00:00 2001 From: voon Date: Wed, 22 Oct 2025 08:52:04 +0800 Subject: [PATCH 04/33] Checkpoint 36 - Add restore with checkpoint test --- .../pipeline/tests/MySqlToHudiE2eITCase.java | 474 ++++++++++++++++-- 1 file changed, 439 insertions(+), 35 deletions(-) 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 index 1752550ae67..c85e8f2e15b 100644 --- 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 @@ -17,11 +17,12 @@ package org.apache.flink.cdc.pipeline.tests; +import org.apache.flink.api.common.JobStatus; 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.runtime.client.JobStatusMessage; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -32,6 +33,7 @@ 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; @@ -44,6 +46,7 @@ import java.sql.Statement; import java.time.Duration; import java.util.*; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -63,6 +66,8 @@ public class MySqlToHudiE2eITCase extends PipelineTestEnvironment { private String warehouse; + private final boolean DEBUG = false; + @BeforeAll public static void initializeContainers() { LOG.info("Starting containers..."); @@ -84,40 +89,52 @@ public static void initializeContainers() { @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(); - // Using FixedHost instead of GenericContainer to ensure that ports are fixed for easier - // debugging during dev - jobManager = - new FixedHostPortGenericContainer<>(getFlinkDockerImageTag()) - .withCommand("jobmanager") - .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_JM_ALIAS) - .withExposedPorts(JOB_MANAGER_REST_PORT) - .withFixedExposedPort(8081, 8081) - .withFixedExposedPort(9005, 9005) - .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) - .withEnv( - "FLINK_ENV_JAVA_OPTS", - "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:9005") - .withCreateContainerCmdModifier(cmd -> cmd.withVolumes(sharedVolume)) - .withLogConsumer(jobManagerConsumer); + 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 = - new FixedHostPortGenericContainer<>(getFlinkDockerImageTag()) - .withCommand("taskmanager") - .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_TM_ALIAS) - .withFixedExposedPort(9006, 9006) - .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) - .withEnv( - "FLINK_ENV_JAVA_OPTS", - "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:9006") - .dependsOn(jobManager) - .withVolumesFrom(jobManager, BindMode.READ_WRITE) - .withLogConsumer(taskManagerConsumer); + taskManager + .withCommand("taskmanager") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_TM_ALIAS) + .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) + .dependsOn(jobManager) + .withVolumesFrom(jobManager, BindMode.READ_WRITE) + .withLogConsumer(taskManagerConsumer); Startables.deepStart(Stream.of(taskManager)).join(); runInContainerAsRoot(taskManager, "chmod", "0777", "-R", sharedVolume.toString()); @@ -271,11 +288,10 @@ public void testSyncWholeDatabase() throws Exception { * represents the expected data for each inserted row in a comma-separated format, which can be * used for validation. * - * @param stat The JDBC {@link java.sql.Statement} object used to execute the SQL commands. - * @return A {@link java.util.List} of strings, where each string is a CSV representation of an - * inserted row, reflecting the expected state in the database. - * @throws java.sql.SQLException if a database access error occurs or the executed SQL is - * invalid. + * @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<>(); @@ -414,6 +430,30 @@ protected String getHudiSQLConnectorResourceName() { return "hudi-sql-connector.jar"; } + @Override + public String stopJobWithSavepoint(org.apache.flink.api.common.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, + org.apache.flink.core.execution.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 { @@ -515,4 +555,368 @@ private static List getCustomersExpectedSinkResults() { "103, user_3, Shanghai, 123567891234", "104, user_4, Shanghai, 123567891234"); } + + @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" + + "# Enable checkpointing for savepoint support\n" + + "execution:\n" + + " checkpointing:\n" + + " interval: 3000\n" + + " timeout: 60000\n" + + " min-pause: 1000", + 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"); + submitPipelineJob( + pipelineJob, + hudiCdcConnector, + hudiHadoopCommonJar, + hadoopJar, + hadoopCompatibilityJar, + dropMetricsJar, + flinkParquet); + waitUntilJobRunning(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"); + submitPipelineJob( + pipelineJob, + savepointPath, + false, + hudiCdcConnector, + hudiHadoopCommonJar, + hadoopJar, + hadoopCompatibilityJar, + dropMetricsJar, + flinkParquet); + waitUntilJobRunning(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"); + } + + // @Test + // public void testFailoverWithMultipleTables() throws Exception { + // warehouse = sharedVolume.toString() + "/hudi_warehouse_failover_" + 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 for failover recovery + // 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: 5500-5504\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" + // + "# Enable checkpointing for failover recovery\n" + // + "execution:\n" + // + " checkpointing:\n" + // + " interval: 3000\n" + // + " timeout: 60000\n" + // + " min-pause: 1000", + // 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"); + // + // submitPipelineJob( + // pipelineJob, + // hudiCdcConnector, + // hudiHadoopCommonJar, + // hadoopJar, + // hadoopCompatibilityJar, + // dropMetricsJar, + // flinkParquet); + // waitUntilJobRunning(Duration.ofSeconds(60)); + // LOG.info("Pipeline job is running"); + // + // // Phase 1: 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 failover + // 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-Failover + // Product','Description',1.23, null, null, null);"); + // stat.execute( + // "INSERT INTO customers VALUES (105, 'user_pre_failover', 'Beijing', + // '987654321');"); + // LOG.info("Phase 2: Incremental data inserted before failover"); + // + // // Wait for data to be checkpointed + // Thread.sleep(5000); + // } + // + // // Phase 3: Trigger failover by killing and restarting TaskManager + // LOG.info("Phase 3: Triggering failover by restarting TaskManager"); + // taskManager.stop(); + // LOG.info("TaskManager stopped"); + // + // // Wait a bit to ensure failure is detected + // Thread.sleep(5000); + // + // // Restart TaskManager + // taskManager = + // new FixedHostPortGenericContainer<>(getFlinkDockerImageTag()) + // .withCommand("taskmanager") + // .withNetwork(NETWORK) + // .withNetworkAliases(INTER_CONTAINER_TM_ALIAS) + // // .withFixedExposedPort(9006, 9006) + // .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) + // .withEnv( + // "FLINK_ENV_JAVA_OPTS", + // + // "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:9006") + // .dependsOn(jobManager) + // .withVolumesFrom(jobManager, BindMode.READ_WRITE) + // .withLogConsumer(taskManagerConsumer); + // Startables.deepStart(Stream.of(taskManager)).join(); + // runInContainerAsRoot(taskManager, "chmod", "0777", "-R", sharedVolume.toString()); + // LOG.info("TaskManager restarted"); + // + // // Wait for job to recover from checkpoint + // waitUntilJobRunning(Duration.ofSeconds(120)); + // LOG.info("Job recovered from checkpoint after failover"); + // + // // Wait additional time for Hudi to stabilize after recovery + // // This ensures that any in-flight transactions are committed and tables are queryable + // Thread.sleep(10000); + // LOG.info("Waited for Hudi tables to stabilize after recovery"); + // + // // Phase 4: Validate data consistency after recovery + // // The pre-failover data should be present (checkpoint-based recovery ensures this) + // List expectedProductsAfterFailover = + // new ArrayList<>(getProductsExpectedSinkResults()); + // expectedProductsAfterFailover.add( + // "110, Pre-Failover Product, Description, 1.23, null, null, null"); + // + // List expectedCustomersAfterFailover = + // new ArrayList<>(getCustomersExpectedSinkResults()); + // expectedCustomersAfterFailover.add("105, user_pre_failover, Beijing, 987654321"); + // + // validateSinkResult(warehouse, database, "products", expectedProductsAfterFailover); + // validateSinkResult(warehouse, database, "customers", expectedCustomersAfterFailover); + // LOG.info("Phase 4: Data consistency validated after failover - rollback mechanism + // worked"); + // + // // Phase 5: Continue with post-failover 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-Failover 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_failover', 'Guangzhou', + // '111222333');"); + // stat.execute("DELETE FROM customers WHERE id=101;"); + // LOG.info("Phase 5: Post-failover data changes applied"); + // } + // + // // Phase 6: Final validation + // List expectedProductsFinal = new ArrayList<>(expectedProductsAfterFailover); + // // Update the pre-failover product description + // expectedProductsFinal.removeIf(row -> row.startsWith("110,")); + // expectedProductsFinal.add( + // "110, Pre-Failover Product, Updated Description, 1.23, null, null, null"); + // expectedProductsFinal.add( + // "111, Post-Failover Product, New Description, 4.56, null, null, null"); + // + // List expectedCustomersFinal = new ArrayList<>(expectedCustomersAfterFailover); + // // Remove deleted customer + // expectedCustomersFinal.removeIf(row -> row.startsWith("101,")); + // expectedCustomersFinal.add("106, user_post_failover, Guangzhou, 111222333"); + // + // validateSinkResult(warehouse, database, "products", expectedProductsFinal); + // validateSinkResult(warehouse, database, "customers", expectedCustomersFinal); + // LOG.info( + // "Phase 6: Final validation successful - failover and rollback mechanism + // working correctly for multiple tables"); + // } } From 43f4844c936ac11294382050d795ab39a004d1d0 Mon Sep 17 00:00:00 2001 From: voon Date: Wed, 22 Oct 2025 11:23:04 +0800 Subject: [PATCH 05/33] Checkpoint 36 - Fix spotless --- .../flink/cdc/connectors/hudi/sink/HudiMetadataApplier.java | 1 + 1 file changed, 1 insertion(+) 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 index 3919ec12cc1..090b4b6cb04 100644 --- 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 @@ -35,6 +35,7 @@ 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.common.config.HoodieMetadataConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.table.catalog.CatalogOptions; From d5cdb75f8a3be2fafa6f700d4aa6e673bca406c6 Mon Sep 17 00:00:00 2001 From: voon Date: Wed, 22 Oct 2025 16:36:01 +0800 Subject: [PATCH 06/33] Checkpoint 37 - Fix spotless and import errors --- .../connectors/hudi/sink/HudiDataSink.java | 10 +- .../hudi/sink/HudiDataSinkOptions.java | 1 + .../hudi/sink/HudiMetadataApplier.java | 16 +- .../sink/bucket/BucketAssignOperator.java | 8 +- .../sink/event/HudiRecordEventSerializer.java | 6 +- .../EventBucketStreamWriteFunction.java | 11 +- .../function/EventStreamWriteFunction.java | 7 +- .../MultiTableEventStreamWriteFunction.java | 71 +++- .../sink/model/BucketAssignmentIndex.java | 2 +- .../MultiTableWriteOperatorFactory.java | 6 +- .../hudi/sink/util/RowDataUtils.java | 18 +- .../cdc/connectors/hudi/sink/v2/HudiSink.java | 17 +- .../pipeline/tests/MySqlToHudiE2eITCase.java | 320 ++++++------------ 13 files changed, 226 insertions(+), 267 deletions(-) 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/HudiDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSink.java index 7a8b9e169e2..f7f7e20c463 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSink.java @@ -23,7 +23,6 @@ import org.apache.flink.cdc.common.sink.FlinkSinkProvider; import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.cdc.connectors.hudi.sink.v2.HudiSink; -import org.apache.flink.table.types.logical.RowType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -67,14 +66,7 @@ public EventSinkProvider getEventSinkProvider() { // Create the HudiSink with multi-table support via wrapper pattern // Use empty RowType since tables are created dynamically - HudiSink hudiSink = - new HudiSink( - flinkConfig, - RowType.of(), // Empty row type for dynamic multi-table support - overwrite, - isBounded, - schemaOperatorUid, - ZoneId.systemDefault()); + HudiSink hudiSink = new HudiSink(flinkConfig, schemaOperatorUid, ZoneId.systemDefault()); return FlinkSinkProvider.of(hudiSink); } 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 index d42650ba958..9b2c76bea4d 100644 --- 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 @@ -17,6 +17,7 @@ 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."; 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 index 090b4b6cb04..7ef46eda335 100644 --- 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 @@ -30,7 +30,15 @@ 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.*; +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; @@ -43,7 +51,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; +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; /** 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 index 4d961af0b91..11751336039 100644 --- 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 @@ -72,16 +72,16 @@ public class BucketAssignOperator extends AbstractStreamOperator private int totalTasksNumber; private int currentTaskNumber; - /** Schema evolution client to query schemas from SchemaOperator coordinator */ + /** Schema evolution client to query schemas from SchemaOperator coordinator. */ private transient SchemaEvolutionClient schemaEvolutionClient; - /** Cache of schemas per table for bucket calculation */ + /** Cache of schemas per table for bucket calculation. */ private final Map schemaCache = new HashMap<>(); - /** Cache of primary key fields per table */ + /** Cache of primary key fields per table. */ private final Map> primaryKeyCache = new HashMap<>(); - /** Cache of field getters per table */ + /** Cache of field getters per table. */ private final Map> fieldGetterCache = new HashMap<>(); public BucketAssignOperator(Configuration conf, String 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/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 index 8a5cfa7ba1c..80623f1682a 100644 --- 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 @@ -51,13 +51,13 @@ */ public class HudiRecordEventSerializer implements HudiRecordSerializer { - /** Schema cache per table - populated from CreateTableEvent and SchemaChangeEvent */ + /** Schema cache per table - populated from CreateTableEvent and SchemaChangeEvent. */ private final Map schemaMaps; - /** Field getter cache per table for efficient conversion */ + /** Field getter cache per table for efficient conversion. */ private final Map> fieldGetterCache; - /** Zone ID for timestamp conversion */ + /** Zone ID for timestamp conversion. */ private final ZoneId zoneId; public HudiRecordEventSerializer(ZoneId zoneId) { 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/EventBucketStreamWriteFunction.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/EventBucketStreamWriteFunction.java index 66c1423cd3b..f96ed5a09cc 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventBucketStreamWriteFunction.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/EventBucketStreamWriteFunction.java @@ -51,6 +51,7 @@ import java.util.Map; import java.util.Set; +/** Extension of EventStreamWriteFunction to handle bucketing. */ public class EventBucketStreamWriteFunction extends EventStreamWriteFunction { private static final Logger LOG = LoggerFactory.getLogger(EventBucketStreamWriteFunction.class); @@ -69,7 +70,7 @@ public class EventBucketStreamWriteFunction extends EventStreamWriteFunction { */ private Set incBucketIndexes; - /** Serializer for converting Events to HoodieFlinkInternalRow for single table */ + /** Serializer for converting Events to HoodieFlinkInternalRow for single table. */ private HudiRecordEventSerializer recordSerializer; /** Function for calculating the task partition to dispatch. */ @@ -78,16 +79,16 @@ public class EventBucketStreamWriteFunction extends EventStreamWriteFunction { /** Function to calculate num buckets per partition. */ private NumBucketsFunction numBucketsFunction; - /** Cached primary key fields for this table */ + /** Cached primary key fields for this table. */ private transient List primaryKeyFields; - /** Cached field getters for primary key fields */ + /** Cached field getters for primary key fields. */ private transient List primaryKeyFieldGetters; - /** Cached schema for this table */ + /** Cached schema for this table. */ private transient Schema cachedSchema; - /** Number of buckets for this function */ + /** Number of buckets for this function. */ private int numBuckets; /** 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/EventStreamWriteFunction.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/EventStreamWriteFunction.java index 9c3fe4bcb77..da5f816613a 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventStreamWriteFunction.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/EventStreamWriteFunction.java @@ -17,7 +17,11 @@ package org.apache.flink.cdc.connectors.hudi.sink.function; -import org.apache.flink.cdc.common.event.*; +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.configuration.Configuration; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.streaming.api.functions.ProcessFunction; @@ -74,6 +78,7 @@ import java.util.Map; import java.util.NoSuchElementException; +/** Base infrastructures for streaming writer function to handle Events. */ public abstract class EventStreamWriteFunction extends AbstractStreamWriteFunction implements EventProcessorFunction { 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 index c6867a3fe65..74c6f9553a0 100644 --- 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 @@ -17,7 +17,16 @@ package org.apache.flink.cdc.connectors.hudi.sink.function; -import org.apache.flink.cdc.common.event.*; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +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.CreateTableOperatorEvent; @@ -26,6 +35,8 @@ 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; @@ -64,21 +75,24 @@ public class MultiTableEventStreamWriteFunction extends AbstractStreamWriteFunct private static final Logger LOG = LoggerFactory.getLogger(MultiTableEventStreamWriteFunction.class); - /** Table-specific write functions created dynamically when new tables are encountered */ + /** 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 */ + /** Track tables that have been initialized to avoid duplicate initialization. */ private transient Map initializedTables; - /** Cache of schemas per table for RowType generation */ + /** 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 */ + /** Schema evolution client to communicate with SchemaOperator. */ private transient SchemaEvolutionClient schemaEvolutionClient; - /** Store the function initialization context for table functions */ + /** Store the function initialization context for table functions. */ private transient FunctionInitializationContext functionInitializationContext; public MultiTableEventStreamWriteFunction(Configuration config) { @@ -89,6 +103,35 @@ public MultiTableEventStreamWriteFunction(Configuration config) { public void initializeState(FunctionInitializationContext context) throws Exception { super.initializeState(context); 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 org.apache.flink.api.common.typeutils.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"); } @@ -106,7 +149,10 @@ public void open(Configuration parameters) throws Exception { super.open(parameters); this.tableFunctions = new HashMap<>(); this.initializedTables = new HashMap<>(); - this.schemaMaps = 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<>(); } @@ -510,6 +556,17 @@ public void snapshotState() { @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()); + } + + // Delegate snapshot to table functions snapshotState(); } 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/model/BucketAssignmentIndex.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/model/BucketAssignmentIndex.java index 3f7b6bf4014..63aad99f8c0 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/model/BucketAssignmentIndex.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/model/BucketAssignmentIndex.java @@ -32,7 +32,7 @@ public class BucketAssignmentIndex implements Serializable { private static final long serialVersionUID = 1L; - /** Index mapping partition paths to bucket-to-fileId mappings for a single table */ + /** Index mapping partition paths to bucket-to-fileId mappings for a single table. */ private final Map> index; public BucketAssignmentIndex() { 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 index fd5ab899388..93973bdf0c1 100644 --- 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 @@ -23,7 +23,11 @@ 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.*; +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; 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 index 151f04978ab..9c501f2c7fc 100644 --- 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 @@ -30,7 +30,23 @@ 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.*; +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; 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 index 25c3abac901..753e3af9592 100644 --- 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 @@ -31,7 +31,6 @@ 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.table.types.logical.RowType; import org.apache.flink.util.Collector; import org.slf4j.Logger; @@ -40,30 +39,18 @@ 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; - // TODO: Check if these are used, remove them if unused - private final RowType rowType; - private final boolean overwrite; - private final boolean isBounded; private final String schemaOperatorUid; - public HudiSink( - Configuration conf, - RowType rowType, - boolean overwrite, - boolean isBounded, - String schemaOperatorUid, - ZoneId zoneId) { + public HudiSink(Configuration conf, String schemaOperatorUid, ZoneId zoneId) { LOG.info("Creating Hoodie sink with conf: {}", conf); this.conf = conf; - this.rowType = rowType; - this.overwrite = overwrite; - this.isBounded = isBounded; this.schemaOperatorUid = schemaOperatorUid; } 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 index c85e8f2e15b..3536a1734b1 100644 --- 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 @@ -17,12 +17,17 @@ 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.runtime.client.JobStatusMessage; +import org.apache.flink.table.api.ValidationException; + import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -45,7 +50,13 @@ import java.sql.SQLException; import java.sql.Statement; import java.time.Duration; -import java.util.*; +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; @@ -66,7 +77,7 @@ public class MySqlToHudiE2eITCase extends PipelineTestEnvironment { private String warehouse; - private final boolean DEBUG = false; + private final boolean debug = true; @BeforeAll public static void initializeContainers() { @@ -80,7 +91,7 @@ public static void initializeContainers() { * 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. @@ -91,7 +102,7 @@ public void before() throws Exception { LOG.info("Starting containers..."); // 2. Instantiate the correct class and apply class-specific methods - if (DEBUG) { + if (debug) { // Use FixedHost instead of GenericContainer to ensure that ports are fixed for easier // debugging during dev jobManager = @@ -103,10 +114,11 @@ public void before() throws Exception { "-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"); + .withFixedExposedPort(9006, 9006); + // .withEnv( + // "FLINK_ENV_JAVA_OPTS", + // + // "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:9006"); } else { jobManager = new GenericContainer<>(getFlinkDockerImageTag()) @@ -210,15 +222,16 @@ public void testSyncWholeDatabase() throws Exception { Path hadoopCompatibilityJar = TestUtils.getResource("flink-hadoop-compatibility.jar"); Path dropMetricsJar = TestUtils.getResource("flink-metrics-dropwizard.jar"); Path flinkParquet = TestUtils.getResource("flink-parquet.jar"); - submitPipelineJob( - pipelineJob, - hudiCdcConnector, - hudiHadoopCommonJar, - hadoopJar, - hadoopCompatibilityJar, - dropMetricsJar, - flinkParquet); - waitUntilJobRunning(Duration.ofSeconds(60)); + JobID pipelineJobID = + submitPipelineJob( + pipelineJob, + hudiCdcConnector, + hudiHadoopCommonJar, + 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 @@ -284,7 +297,7 @@ public void testSyncWholeDatabase() throws Exception { * size from {@code VARCHAR(10)} to {@code VARCHAR(19)}. * * - * Throughout this process, the method constructs and returns a list of strings. Each string + *

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. * @@ -608,15 +621,16 @@ public void testStopAndRestartFromSavepoint() throws Exception { // Start the pipeline job LOG.info("Phase 1: Starting initial pipeline job"); - submitPipelineJob( - pipelineJob, - hudiCdcConnector, - hudiHadoopCommonJar, - hadoopJar, - hadoopCompatibilityJar, - dropMetricsJar, - flinkParquet); - waitUntilJobRunning(Duration.ofSeconds(60)); + 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 = @@ -683,17 +697,18 @@ public void testStopAndRestartFromSavepoint() throws Exception { // Phase 4: Restart from savepoint LOG.info("Phase 4: Restarting job from savepoint"); - submitPipelineJob( - pipelineJob, - savepointPath, - false, - hudiCdcConnector, - hudiHadoopCommonJar, - hadoopJar, - hadoopCompatibilityJar, - dropMetricsJar, - flinkParquet); - waitUntilJobRunning(Duration.ofSeconds(60)); + 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 @@ -739,184 +754,53 @@ public void testStopAndRestartFromSavepoint() throws Exception { "Phase 6: Final validation successful - stop/restart with savepoint working correctly for multiple tables"); } - // @Test - // public void testFailoverWithMultipleTables() throws Exception { - // warehouse = sharedVolume.toString() + "/hudi_warehouse_failover_" + 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 for failover recovery - // 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: 5500-5504\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" - // + "# Enable checkpointing for failover recovery\n" - // + "execution:\n" - // + " checkpointing:\n" - // + " interval: 3000\n" - // + " timeout: 60000\n" - // + " min-pause: 1000", - // 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"); - // - // submitPipelineJob( - // pipelineJob, - // hudiCdcConnector, - // hudiHadoopCommonJar, - // hadoopJar, - // hadoopCompatibilityJar, - // dropMetricsJar, - // flinkParquet); - // waitUntilJobRunning(Duration.ofSeconds(60)); - // LOG.info("Pipeline job is running"); - // - // // Phase 1: 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 failover - // 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-Failover - // Product','Description',1.23, null, null, null);"); - // stat.execute( - // "INSERT INTO customers VALUES (105, 'user_pre_failover', 'Beijing', - // '987654321');"); - // LOG.info("Phase 2: Incremental data inserted before failover"); - // - // // Wait for data to be checkpointed - // Thread.sleep(5000); - // } - // - // // Phase 3: Trigger failover by killing and restarting TaskManager - // LOG.info("Phase 3: Triggering failover by restarting TaskManager"); - // taskManager.stop(); - // LOG.info("TaskManager stopped"); - // - // // Wait a bit to ensure failure is detected - // Thread.sleep(5000); - // - // // Restart TaskManager - // taskManager = - // new FixedHostPortGenericContainer<>(getFlinkDockerImageTag()) - // .withCommand("taskmanager") - // .withNetwork(NETWORK) - // .withNetworkAliases(INTER_CONTAINER_TM_ALIAS) - // // .withFixedExposedPort(9006, 9006) - // .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) - // .withEnv( - // "FLINK_ENV_JAVA_OPTS", - // - // "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:9006") - // .dependsOn(jobManager) - // .withVolumesFrom(jobManager, BindMode.READ_WRITE) - // .withLogConsumer(taskManagerConsumer); - // Startables.deepStart(Stream.of(taskManager)).join(); - // runInContainerAsRoot(taskManager, "chmod", "0777", "-R", sharedVolume.toString()); - // LOG.info("TaskManager restarted"); - // - // // Wait for job to recover from checkpoint - // waitUntilJobRunning(Duration.ofSeconds(120)); - // LOG.info("Job recovered from checkpoint after failover"); - // - // // Wait additional time for Hudi to stabilize after recovery - // // This ensures that any in-flight transactions are committed and tables are queryable - // Thread.sleep(10000); - // LOG.info("Waited for Hudi tables to stabilize after recovery"); - // - // // Phase 4: Validate data consistency after recovery - // // The pre-failover data should be present (checkpoint-based recovery ensures this) - // List expectedProductsAfterFailover = - // new ArrayList<>(getProductsExpectedSinkResults()); - // expectedProductsAfterFailover.add( - // "110, Pre-Failover Product, Description, 1.23, null, null, null"); - // - // List expectedCustomersAfterFailover = - // new ArrayList<>(getCustomersExpectedSinkResults()); - // expectedCustomersAfterFailover.add("105, user_pre_failover, Beijing, 987654321"); - // - // validateSinkResult(warehouse, database, "products", expectedProductsAfterFailover); - // validateSinkResult(warehouse, database, "customers", expectedCustomersAfterFailover); - // LOG.info("Phase 4: Data consistency validated after failover - rollback mechanism - // worked"); - // - // // Phase 5: Continue with post-failover 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-Failover 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_failover', 'Guangzhou', - // '111222333');"); - // stat.execute("DELETE FROM customers WHERE id=101;"); - // LOG.info("Phase 5: Post-failover data changes applied"); - // } - // - // // Phase 6: Final validation - // List expectedProductsFinal = new ArrayList<>(expectedProductsAfterFailover); - // // Update the pre-failover product description - // expectedProductsFinal.removeIf(row -> row.startsWith("110,")); - // expectedProductsFinal.add( - // "110, Pre-Failover Product, Updated Description, 1.23, null, null, null"); - // expectedProductsFinal.add( - // "111, Post-Failover Product, New Description, 4.56, null, null, null"); - // - // List expectedCustomersFinal = new ArrayList<>(expectedCustomersAfterFailover); - // // Remove deleted customer - // expectedCustomersFinal.removeIf(row -> row.startsWith("101,")); - // expectedCustomersFinal.add("106, user_post_failover, Guangzhou, 111222333"); - // - // validateSinkResult(warehouse, database, "products", expectedProductsFinal); - // validateSinkResult(warehouse, database, "customers", expectedCustomersFinal); - // LOG.info( - // "Phase 6: Final validation successful - failover and rollback mechanism - // working correctly for multiple tables"); - // } + 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.isEmpty()) { + 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()) { + try { + Thread.sleep(50000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + 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; + } + } + } } From 76722f700fdb20bdf1388b12615379e76fb43333 Mon Sep 17 00:00:00 2001 From: voon Date: Wed, 22 Oct 2025 22:31:11 +0800 Subject: [PATCH 07/33] Checkpoint 38 - Start-stop-checkpoint fix --- ...tiTableStreamWriteOperatorCoordinator.java | 28 +++- .../pipeline/tests/MySqlToHudiE2eITCase.java | 121 ++++++++++-------- 2 files changed, 90 insertions(+), 59 deletions(-) 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 index 62985667d65..290971b6357 100644 --- 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 @@ -474,11 +474,16 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r tableContext.eventBuffers.getAllCompletedEvents()); }); - byte[] serializedState = - SerializationUtils.serialize((Serializable) allStates); + // 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 for checkpoint {}", + "Successfully checkpointed coordinator state with {} schemas for checkpoint {}", + tableSchemas.size(), checkpointId); } catch (Throwable t) { LOG.error( @@ -500,8 +505,23 @@ public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { return; } try { + Map checkpointState = SerializationUtils.deserialize(checkpointData); Map>> allStates = - SerializationUtils.deserialize(checkpointData); + (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. 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 index 3536a1734b1..d1c91331103 100644 --- 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 @@ -72,6 +72,17 @@ public class MySqlToHudiE2eITCase extends PipelineTestEnvironment { private static final String PEEK_SQL_FILE = "peek-hudi.sql"; + // 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: 1024m" + + "\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); @@ -143,7 +154,7 @@ public void before() throws Exception { .withCommand("taskmanager") .withNetwork(NETWORK) .withNetworkAliases(INTER_CONTAINER_TM_ALIAS) - .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) + .withEnv("FLINK_PROPERTIES", HUDI_FLINK_PROPERTIES) .dependsOn(jobManager) .withVolumesFrom(jobManager, BindMode.READ_WRITE) .withLogConsumer(taskManagerConsumer); @@ -515,60 +526,6 @@ private void validateSinkResult( Assertions.assertThat(results).containsExactlyInAnyOrderElementsOf(expected); } - 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"); - } - @Test public void testStopAndRestartFromSavepoint() throws Exception { warehouse = sharedVolume.toString() + "/hudi_warehouse_savepoint_" + UUID.randomUUID(); @@ -754,6 +711,60 @@ public void testStopAndRestartFromSavepoint() throws Exception { "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); } From aeb4f3b07c4c50a15b543eb28aeea225cece85cb Mon Sep 17 00:00:00 2001 From: voon Date: Thu, 23 Oct 2025 11:07:35 +0800 Subject: [PATCH 08/33] Checkpoint 39 - Enable MDT --- .../flink-cdc-pipeline-connector-hudi/pom.xml | 1 - .../flink/cdc/connectors/hudi/sink/HudiMetadataApplier.java | 2 +- .../MultiTableStreamWriteOperatorCoordinator.java | 2 +- .../flink/cdc/pipeline/tests/MySqlToHudiE2eITCase.java | 5 ++--- 4 files changed, 4 insertions(+), 6 deletions(-) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml index a8926185213..5ebd0f36b2b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml @@ -102,7 +102,6 @@ limitations under the License. org.apache.hudi hudi-hadoop-common ${hudi.version} - provided 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 index 7ef46eda335..d0a4d984909 100644 --- 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 @@ -567,7 +567,7 @@ private org.apache.flink.configuration.Configuration convertToCatalogConfig( // TODO: enable this in the future // Disable metadata table usage in catalog commit - flinkConfig.setString(HoodieMetadataConfig.ENABLE.key(), "false"); + flinkConfig.setString(HoodieMetadataConfig.ENABLE.key(), "true"); // Set mode to DFS (filesystem-based) // TODO: make this configurable 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 index 290971b6357..071f2335b74 100644 --- 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 @@ -724,7 +724,7 @@ private Configuration createTableSpecificConfig(TableId tableId) { // The central coordinator manages the only timeline server. tableConfig.setBoolean(HoodieWriteConfig.EMBEDDED_TIMELINE_SERVER_ENABLE.key(), false); // For simplicity, let's keep it consistent. - tableConfig.setBoolean(HoodieMetadataConfig.ENABLE.key(), false); + tableConfig.setBoolean(HoodieMetadataConfig.ENABLE.key(), true); // Use memory-based file system view since each client is lightweight. tableConfig.setString(FileSystemViewStorageConfig.VIEW_TYPE.key(), "MEMORY"); 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 index d1c91331103..468997a7c16 100644 --- 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 @@ -228,7 +228,7 @@ public void testSyncWholeDatabase() throws Exception { + " 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 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"); @@ -237,7 +237,6 @@ public void testSyncWholeDatabase() throws Exception { submitPipelineJob( pipelineJob, hudiCdcConnector, - hudiHadoopCommonJar, hadoopJar, hadoopCompatibilityJar, dropMetricsJar, @@ -455,7 +454,7 @@ protected String getHudiSQLConnectorResourceName() { } @Override - public String stopJobWithSavepoint(org.apache.flink.api.common.JobID jobID) { + public String stopJobWithSavepoint(JobID jobID) { String savepointPath = "/opt/flink/"; try { // Use REST API to stop with savepoint to avoid CLI classpath conflicts From b3f38bcbe18c3e3f2eecaf0d43c648834dd92511 Mon Sep 17 00:00:00 2001 From: voon Date: Thu, 23 Oct 2025 17:12:03 +0800 Subject: [PATCH 09/33] Checkpoint 40 - Remove MDT configs --- .../flink/cdc/connectors/hudi/sink/HudiMetadataApplier.java | 5 ----- .../MultiTableStreamWriteOperatorCoordinator.java | 3 --- 2 files changed, 8 deletions(-) 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 index d0a4d984909..9d1177440f2 100644 --- 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 @@ -44,7 +44,6 @@ import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; import org.apache.flink.table.types.DataType; -import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.table.catalog.CatalogOptions; import org.apache.hudi.table.catalog.HoodieCatalog; @@ -565,10 +564,6 @@ private org.apache.flink.configuration.Configuration convertToCatalogConfig( String basePath = cdcConfig.get(HudiConfig.PATH); flinkConfig.setString(CatalogOptions.CATALOG_PATH.key(), basePath); - // TODO: enable this in the future - // Disable metadata table usage in catalog commit - flinkConfig.setString(HoodieMetadataConfig.ENABLE.key(), "true"); - // Set mode to DFS (filesystem-based) // TODO: make this configurable flinkConfig.setString(CatalogOptions.MODE.key(), "dfs"); 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 index 071f2335b74..f9d00a5ca0f 100644 --- 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 @@ -33,7 +33,6 @@ import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -723,8 +722,6 @@ private Configuration createTableSpecificConfig(TableId tableId) { // Disable both embedded timeline server and metadata table for per-table clients. // The central coordinator manages the only timeline server. tableConfig.setBoolean(HoodieWriteConfig.EMBEDDED_TIMELINE_SERVER_ENABLE.key(), false); - // For simplicity, let's keep it consistent. - tableConfig.setBoolean(HoodieMetadataConfig.ENABLE.key(), true); // Use memory-based file system view since each client is lightweight. tableConfig.setString(FileSystemViewStorageConfig.VIEW_TYPE.key(), "MEMORY"); From be3f4d2b258408df7d005bad4ef668b9b5fc5575 Mon Sep 17 00:00:00 2001 From: voon Date: Thu, 23 Oct 2025 18:05:36 +0800 Subject: [PATCH 10/33] Checkpoint 41 - Fix testSyncWholeDb --- ...tiTableStreamWriteOperatorCoordinator.java | 5 ++ .../MultiTableEventStreamWriteFunction.java | 62 +++++++++++++++---- 2 files changed, 56 insertions(+), 11 deletions(-) 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 index f9d00a5ca0f..fba5842bb50 100644 --- 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 @@ -437,6 +437,11 @@ private void handleEnhancedWriteMetadataEvent(EnhancedWriteMetadataEvent enhance 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( 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 index 74c6f9553a0..83ea7fe9c4d 100644 --- 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 @@ -56,6 +56,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.lang.reflect.Field; import java.lang.reflect.Method; import java.nio.file.Files; import java.nio.file.Path; @@ -542,16 +543,10 @@ public void snapshotState() { // snapshotting. The direct call to flushRemaining() is removed to prevent sending // an invalid, generic instant request to the coordinator. // flushRemaining(false); - for (Map.Entry entry : tableFunctions.entrySet()) { - try { - LOG.debug("Delegating snapshotState for table: {}", entry.getKey()); - entry.getValue().snapshotState(); - } 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); - } - } + + // 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 @@ -567,7 +562,52 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { } // Delegate snapshot to table functions - snapshotState(); + // Child functions are composition objects, not Flink operators, so they shouldn't + // go through the full snapshotState(FunctionSnapshotContext) lifecycle which + // includes state reloading. Instead, we: + // 1. Call their abstract snapshotState() to flush buffers + // 2. Manually update their checkpointId for instant requests + long checkpointId = context.getCheckpointId(); + for (Map.Entry entry : tableFunctions.entrySet()) { + try { + EventBucketStreamWriteFunction tableFunction = entry.getValue(); + LOG.debug( + "Delegating snapshotState for table: {} with checkpointId: {}", + entry.getKey(), + checkpointId); + + // Call abstract snapshotState() to flush buffers + tableFunction.snapshotState(); + + // Update the child function's checkpointId using reflection + // This is necessary because child functions need the current checkpointId + // when requesting instants from the coordinator + setCheckpointId(tableFunction, checkpointId); + + 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); + } + } + } + + /** + * Sets the checkpointId field on a child AbstractStreamWriteFunction using reflection. This is + * necessary because checkpointId is protected and child functions are composition objects that + * need the current checkpoint ID for coordinator communication. + */ + private void setCheckpointId(AbstractStreamWriteFunction function, long checkpointId) { + try { + Field checkpointIdField = + AbstractStreamWriteFunction.class.getDeclaredField("checkpointId"); + checkpointIdField.setAccessible(true); + checkpointIdField.setLong(function, checkpointId); + } catch (Exception e) { + LOG.error("Failed to set checkpointId on child function using reflection", e); + throw new RuntimeException("Failed to set checkpointId on child function", e); + } } protected void flushRemaining(boolean endInput) { From de2f698dbbeed61463bd40f3066924d5627765f1 Mon Sep 17 00:00:00 2001 From: voon Date: Fri, 24 Oct 2025 11:02:37 +0800 Subject: [PATCH 11/33] Checkpoint 42 - Add compaction scheduling support --- ...tiTableStreamWriteOperatorCoordinator.java | 49 +++++++++ .../pipeline/tests/MySqlToHudiE2eITCase.java | 102 ++++++++++++++++-- 2 files changed, 145 insertions(+), 6 deletions(-) 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 index fba5842bb50..0fad097c27d 100644 --- 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 @@ -53,6 +53,8 @@ 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; @@ -150,6 +152,9 @@ private static class TableState implements Serializable { final String commitAction; final boolean isOverwrite; final WriteOperationType operationType; + final boolean scheduleCompaction; + final boolean scheduleClustering; + final boolean isDeltaTimeCompaction; TableState(Configuration conf) { this.operationType = @@ -160,6 +165,9 @@ private static class TableState implements Serializable { 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); } } @@ -662,6 +670,9 @@ private void doCommit( if (success) { tableContext.eventBuffers.reset(checkpointId); LOG.info("Successfully committed instant [{}] for table [{}]", instant, tableId); + + // Schedule table services (compaction, clustering) after successful commit + scheduleTableServices(tableId, tableContext, true); } else { LOG.error("Failed to commit instant [{}] for table [{}]", instant, tableId); MultiTableStreamWriteOperatorCoordinator.this.context.failJob( @@ -671,6 +682,44 @@ private void doCommit( } } + /** + * Schedules table services (compaction and clustering) for a specific table if enabled. This + * mirrors the logic in StreamWriteOperatorCoordinator.scheduleTableServices(). + * + * @param tableId The table identifier + * @param tableContext The table's context containing write client and state + * @param committed Whether a commit was just successfully completed + */ + private void scheduleTableServices( + TableId tableId, TableContext tableContext, boolean committed) { + TableState state = tableContext.tableState; + + // Schedule compaction if enabled for this table + if (state.scheduleCompaction) { + try { + CompactionUtil.scheduleCompaction( + tableContext.writeClient, state.isDeltaTimeCompaction, committed); + LOG.info("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 + } + } + + // Schedule clustering if enabled for this table + if (state.scheduleClustering) { + try { + // Create table-specific config for clustering + Configuration tableConfig = createTableSpecificConfig(tableId); + ClusteringUtil.scheduleClustering(tableConfig, tableContext.writeClient, 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 (timelineServerClient != null) { 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 index 468997a7c16..00d07885ba7 100644 --- 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 @@ -25,9 +25,11 @@ 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; @@ -72,6 +74,8 @@ public class MySqlToHudiE2eITCase extends PipelineTestEnvironment { 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 = @@ -218,7 +222,9 @@ public void testSyncWholeDatabase() throws Exception { + " type: hudi\n" + " path: %s\n" + " hoodie.datasource.write.recordkey.field: id\n" - + " hoodie.table.type: MERGE_ON_READ\n" + + " hoodie.table.type: " + + TABLE_TYPE + + " \n" + " hoodie.schema.on.read.enable: true\n" + " write.bucket_assign.tasks: 2\n" + " write.tasks: 2\n" @@ -226,7 +232,11 @@ public void testSyncWholeDatabase() throws Exception { + "pipeline:\n" + " schema.change.behavior: evolve\n" + " parallelism: %s", - MYSQL_TEST_USER, MYSQL_TEST_PASSWORD, database, warehouse, parallelism); + 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"); @@ -288,6 +298,13 @@ public void testSyncWholeDatabase() throws Exception { 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")); + } + LOG.info("Compaction scheduling verified successfully"); } /** @@ -464,10 +481,7 @@ public String stopJobWithSavepoint(JobID jobID) { String savepointLocation = getRestClusterClient() .stopWithSavepoint( - jobID, - false, - savepointPath, - org.apache.flink.core.execution.SavepointFormatType.CANONICAL) + jobID, false, savepointPath, SavepointFormatType.CANONICAL) .get(60, java.util.concurrent.TimeUnit.SECONDS); LOG.info("Savepoint completed at: {}", savepointLocation); @@ -813,4 +827,80 @@ public void waitUntilJobState(JobID jobId, Duration timeout, JobStatus expectedS } } } + + /** + * 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!"); + } + } } From a62aeb5303e50f012c813274fbff85dd4181d8b2 Mon Sep 17 00:00:00 2001 From: voon Date: Tue, 28 Oct 2025 18:23:16 +0800 Subject: [PATCH 12/33] Checkpoint 43 - Change compaction to be event driven --- .../cdc/connectors/hudi/sink/HudiConfig.java | 7 + .../hudi/sink/HudiDataSinkFactory.java | 3 + ...tiTableStreamWriteOperatorCoordinator.java | 135 ++++++++++++++++-- .../pipeline/tests/MySqlToHudiE2eITCase.java | 2 +- 4 files changed, 132 insertions(+), 15 deletions(-) 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/HudiConfig.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiConfig.java index 0a7ef8aadc8..42fd8b64fc5 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiConfig.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiConfig.java @@ -155,4 +155,11 @@ private static ConfigOption booleanOption( HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.key(), false, Description.builder().build()); + + public static final ConfigOption COMPACTION_DELTA_COMMITS = + ConfigOptions.key("compaction.delta_commits") + .intType() + .defaultValue(5) + .withDescription( + "Max delta commits needed to trigger compaction, default 5 commits"); } 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/HudiDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactory.java index 41fc6eef3db..22efcdf5fb7 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactory.java @@ -89,6 +89,9 @@ public Set> optionalOptions() { 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/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 index 0fad097c27d..0bb1e3d7e4c 100644 --- 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 @@ -34,8 +34,10 @@ 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.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; @@ -72,6 +74,8 @@ 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. * @@ -156,6 +160,13 @@ private static class TableState implements Serializable { 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)); @@ -168,6 +179,73 @@ private static class TableState implements Serializable { 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; } } @@ -671,8 +749,17 @@ private void doCommit( tableContext.eventBuffers.reset(checkpointId); LOG.info("Successfully committed instant [{}] for table [{}]", instant, tableId); - // Schedule table services (compaction, clustering) after successful commit - scheduleTableServices(tableId, tableContext, true); + // 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( @@ -683,35 +770,55 @@ private void doCommit( } /** - * Schedules table services (compaction and clustering) for a specific table if enabled. This - * mirrors the logic in StreamWriteOperatorCoordinator.scheduleTableServices(). + * 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 - * @param committed Whether a commit was just successfully completed */ - private void scheduleTableServices( - TableId tableId, TableContext tableContext, boolean committed) { + private void scheduleTableServicesIfNeeded(TableId tableId, TableContext tableContext) { TableState state = tableContext.tableState; - // Schedule compaction if enabled for this table - if (state.scheduleCompaction) { + // 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, committed); - LOG.info("Scheduled compaction for table [{}]", tableId); + 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)); } - // Schedule clustering if enabled for this table + // Clustering can remain on every commit or use similar metrics if (state.scheduleClustering) { try { - // Create table-specific config for clustering Configuration tableConfig = createTableSpecificConfig(tableId); - ClusteringUtil.scheduleClustering(tableConfig, tableContext.writeClient, committed); + 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); 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 index 00d07885ba7..738afa12f3e 100644 --- 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 @@ -228,6 +228,7 @@ public void testSyncWholeDatabase() throws Exception { + " 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" @@ -304,7 +305,6 @@ public void testSyncWholeDatabase() throws Exception { if (TABLE_TYPE.equals(HoodieTableType.MERGE_ON_READ.name())) { assertCompactionScheduled(warehouse, database, Arrays.asList("products", "customers")); } - LOG.info("Compaction scheduling verified successfully"); } /** From 410be89b2d70c975dc6b18d55c2fd43f88f66800 Mon Sep 17 00:00:00 2001 From: voon Date: Mon, 3 Nov 2025 12:11:19 +0800 Subject: [PATCH 13/33] Checkpoint 44 - Remove reflection call --- .../MultiTableEventStreamWriteFunction.java | 30 ++----------------- 1 file changed, 2 insertions(+), 28 deletions(-) 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 index 83ea7fe9c4d..51648dcc236 100644 --- 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 @@ -45,7 +45,6 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; - import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.sink.common.AbstractStreamWriteFunction; @@ -57,7 +56,6 @@ import java.io.IOException; import java.lang.reflect.Field; -import java.lang.reflect.Method; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -334,30 +332,6 @@ public static void createHudiTablePath(Configuration config, TableId tableId) } } - /** - * Processes a flush event for a specific table function. This simulates the FlushEvent - * processing that would normally happen in EventStreamWriteFunction.processElement. - */ - private void processFlushForTableFunction( - EventBucketStreamWriteFunction tableFunction, Event flushEvent) { - try { - // Use reflection to access the protected flushRemaining method - // This is the same logic as in EventStreamWriteFunction when it processes FlushEvent - Method flushMethod = - tableFunction - .getClass() - .getSuperclass() - .getDeclaredMethod("flushRemaining", boolean.class); - flushMethod.setAccessible(true); - flushMethod.invoke(tableFunction, false); - } catch (Exception e) { - LOG.error( - "Failed to flush table function using reflection, falling back to endInput", e); - // Fallback: use endInput() which is public and also triggers flushing - tableFunction.endInput(); - } - } - /** * Processes flush events for coordinated flushing across table functions. This handles both * table-specific and global flush operations. @@ -374,7 +348,7 @@ public void processFlush(FlushEvent event) throws Exception { tableFunctions.size()); for (Map.Entry entry : tableFunctions.entrySet()) { - processFlushForTableFunction(entry.getValue(), event); + entry.getValue().flushRemaining(false); LOG.debug("Flushed table function for: {}", entry.getKey()); } } else { @@ -382,7 +356,7 @@ public void processFlush(FlushEvent event) throws Exception { for (TableId tableId : tableIds) { EventBucketStreamWriteFunction tableFunction = tableFunctions.get(tableId); if (tableFunction != null) { - processFlushForTableFunction(tableFunction, event); + tableFunction.flushRemaining(false); LOG.debug("Flushed table function for: {}", tableId); } } From e6460d8f2f75e04b22ef5fa895fd1816f12bb63f Mon Sep 17 00:00:00 2001 From: voon Date: Mon, 3 Nov 2025 16:24:10 +0800 Subject: [PATCH 14/33] Checkpoint 45 - Change partitioning logic to avoid skew --- .../sink/bucket/BucketAssignOperator.java | 35 ++++++++++++++----- 1 file changed, 27 insertions(+), 8 deletions(-) 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 index 11751336039..b4372a67319 100644 --- 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 @@ -38,7 +38,8 @@ 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.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.slf4j.Logger; @@ -72,6 +73,9 @@ public class BucketAssignOperator extends AbstractStreamOperator 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; @@ -87,7 +91,6 @@ public class BucketAssignOperator extends AbstractStreamOperator public BucketAssignOperator(Configuration conf, String schemaOperatorUid) { this.numBuckets = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS); this.schemaOperatorUid = schemaOperatorUid; - // Use ALWAYS like Paimon does - allows chaining with both upstream and downstream this.chainingStrategy = ChainingStrategy.ALWAYS; } @@ -109,6 +112,7 @@ 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, @@ -158,8 +162,8 @@ public void processElement(StreamRecord streamRecord) throws Exception { // Calculate bucket for DataChangeEvent and route to specific task if (event instanceof DataChangeEvent) { DataChangeEvent dataEvent = (DataChangeEvent) event; - int bucket = calculateBucket(dataEvent); - output.collect(new StreamRecord<>(new BucketWrapper(bucket, event))); + int taskIndex = calculateTaskIndex(dataEvent); + output.collect(new StreamRecord<>(new BucketWrapper(taskIndex, event))); return; } @@ -169,10 +173,18 @@ public void processElement(StreamRecord streamRecord) throws Exception { } } - private int calculateBucket(DataChangeEvent 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 + // Get or cache schema, query from SchemaOperator coordinator if not cached Schema schema = schemaCache.get(tableId); if (schema == null) { try { @@ -236,9 +248,16 @@ private int calculateBucket(DataChangeEvent event) { // Extract record key String recordKey = extractRecordKey(event, primaryKeys, fieldGetters); - // Calculate bucket using Hudi's logic + // Calculate bucket using Hudi's logic (0 to numBuckets-1) String tableIndexKeyFields = String.join(",", primaryKeys); - return BucketIdentifier.getBucketId(recordKey, tableIndexKeyFields, numBuckets); + int bucketNumber = BucketIdentifier.getBucketId(recordKey, tableIndexKeyFields, numBuckets); + + // Use partition function to map bucket to task index for balanced distribution + // partition is "default" since we're not using Hudi partition fields in this context + String partition = "default"; + int taskIndex = partitionIndexFunc.apply(numBuckets, partition, bucketNumber); + + return taskIndex; } private String extractRecordKey( From 70600b2295ed6ee7bcdf96c005375dc32a2982c6 Mon Sep 17 00:00:00 2001 From: voon Date: Mon, 3 Nov 2025 22:30:15 +0800 Subject: [PATCH 15/33] Checkpoint 45 - Update naming convention to reduce confusion. --- .../EventBucketStreamWriteFunction.java | 41 +++++++++---------- 1 file changed, 20 insertions(+), 21 deletions(-) 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/EventBucketStreamWriteFunction.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/EventBucketStreamWriteFunction.java index f96ed5a09cc..677d3f5fcc3 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventBucketStreamWriteFunction.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/EventBucketStreamWriteFunction.java @@ -30,7 +30,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.table.types.logical.RowType; - import org.apache.hudi.client.model.HoodieFlinkInternalRow; import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.hash.BucketIndexUtil; @@ -73,8 +72,8 @@ public class EventBucketStreamWriteFunction extends EventStreamWriteFunction { /** Serializer for converting Events to HoodieFlinkInternalRow for single table. */ private HudiRecordEventSerializer recordSerializer; - /** Function for calculating the task partition to dispatch. */ - private Functions.Function3 partitionIndexFunc; + /** Function for calculating which task should handle a given bucket. */ + private Functions.Function3 taskAssignmentFunc; /** Function to calculate num buckets per partition. */ private NumBucketsFunction numBucketsFunction; @@ -109,7 +108,7 @@ public void open(Configuration parameters) throws IOException { this.parallelism = RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()); this.bucketAssignmentIndex = new BucketAssignmentIndex(); this.incBucketIndexes = new HashSet<>(); - this.partitionIndexFunc = BucketIndexUtil.getPartitionIndexFunc(parallelism); + this.taskAssignmentFunc = BucketIndexUtil.getPartitionIndexFunc(parallelism); this.numBucketsFunction = new NumBucketsFunction( config.get(FlinkOptions.BUCKET_INDEX_PARTITION_EXPRESSIONS), @@ -175,47 +174,47 @@ public void processSchemaChange(SchemaChangeEvent event) throws Exception { recordSerializer.serialize(event); } - private void defineRecordLocation(int bucketNum, HoodieFlinkInternalRow record) { + private void defineRecordLocation(int bucketId, HoodieFlinkInternalRow record) { final String partition = record.getPartitionPath(); // Check if this task should handle this bucket - if (!isBucketToLoad(bucketNum, partition)) { + if (!shouldTaskHandleBucket(bucketId, partition)) { throw new IllegalStateException( String.format( "Task %d received record for bucket %d which should not be handled by this task. " + "This indicates a partitioning problem - records must be routed to the correct task.", - taskID, bucketNum)); + taskID, bucketId)); } bootstrapIndexIfNeed(partition); Map bucketToFileId = bucketAssignmentIndex.getBucketToFileIdMap(partition); - final String bucketId = partition + "/" + bucketNum; + final String bucketKey = partition + "/" + bucketId; - if (incBucketIndexes.contains(bucketId)) { + if (incBucketIndexes.contains(bucketKey)) { record.setInstantTime("I"); - record.setFileId(bucketToFileId.get(bucketNum)); - } else if (bucketToFileId.containsKey(bucketNum)) { + record.setFileId(bucketToFileId.get(bucketId)); + } else if (bucketToFileId.containsKey(bucketId)) { record.setInstantTime("U"); - record.setFileId(bucketToFileId.get(bucketNum)); + record.setFileId(bucketToFileId.get(bucketId)); } else { String newFileId = isNonBlockingConcurrencyControl - ? BucketIdentifier.newBucketFileIdForNBCC(bucketNum) - : BucketIdentifier.newBucketFileIdPrefix(bucketNum); + ? BucketIdentifier.newBucketFileIdForNBCC(bucketId) + : BucketIdentifier.newBucketFileIdPrefix(bucketId); record.setInstantTime("I"); record.setFileId(newFileId); - bucketToFileId.put(bucketNum, newFileId); - incBucketIndexes.add(bucketId); + bucketToFileId.put(bucketId, newFileId); + incBucketIndexes.add(bucketKey); } } /** - * Determine whether the current fileID belongs to the current task. partitionIndex == this - * taskID belongs to this task. + * Determine whether this task should handle the given bucket. Returns true if the bucket is + * assigned to this task based on the task assignment function. */ - public boolean isBucketToLoad(int bucketNumber, String partition) { + public boolean shouldTaskHandleBucket(int bucketNumber, String partition) { int numBuckets = numBucketsFunction.getNumBuckets(partition); - return partitionIndexFunc.apply(numBuckets, partition, bucketNumber) == taskID; + return taskAssignmentFunc.apply(numBuckets, partition, bucketNumber) == taskID; } /** @@ -242,7 +241,7 @@ private void bootstrapIndexIfNeed(String partition) { fileSlice -> { String fileId = fileSlice.getFileId(); int bucketNumber = BucketIdentifier.bucketIdFromFileId(fileId); - if (isBucketToLoad(bucketNumber, partition)) { + if (shouldTaskHandleBucket(bucketNumber, partition)) { LOG.info( String.format( "Should load this partition bucket %s with fileId %s", From ed733aa7bba8a204e900013ea9eea7e4d3565d98 Mon Sep 17 00:00:00 2001 From: voon Date: Mon, 3 Nov 2025 22:57:20 +0800 Subject: [PATCH 16/33] Checkpoint 46 - Add partition path extractor --- .../sink/bucket/BucketAssignOperator.java | 131 +++++++++++++++++- .../EventBucketStreamWriteFunction.java | 1 + .../MultiTableEventStreamWriteFunction.java | 1 + .../hudi/sink/util/RowDataUtils.java | 83 ++++++++++- 4 files changed, 208 insertions(+), 8 deletions(-) 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 index b4372a67319..e2e5ae00bdb 100644 --- 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 @@ -38,6 +38,7 @@ 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.hudi.common.util.Functions; import org.apache.hudi.common.util.hash.BucketIndexUtil; import org.apache.hudi.configuration.FlinkOptions; @@ -174,9 +175,9 @@ public void processElement(StreamRecord streamRecord) throws Exception { } /** - * 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 + * 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 @@ -252,9 +253,10 @@ private int calculateTaskIndex(DataChangeEvent event) { String tableIndexKeyFields = String.join(",", primaryKeys); int bucketNumber = BucketIdentifier.getBucketId(recordKey, tableIndexKeyFields, numBuckets); + // Extract partition path from the event + String partition = extractPartitionPath(event, finalSchema, fieldGetters); + // Use partition function to map bucket to task index for balanced distribution - // partition is "default" since we're not using Hudi partition fields in this context - String partition = "default"; int taskIndex = partitionIndexFunc.apply(numBuckets, partition, bucketNumber); return taskIndex; @@ -288,4 +290,123 @@ private String extractRecordKey( return String.join(",", recordKeyPairs); } + + /** + * Extract partition path from the DataChangeEvent based on schema partition keys. + * + *

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 "default". + * + * @param event The DataChangeEvent to extract partition from + * @param schema The table schema containing partition key definitions + * @param fieldGetters Field getters for extracting values (not used currently, may be needed + * for optimization) + * @return The partition path string + */ + private String extractPartitionPath( + DataChangeEvent event, Schema schema, List fieldGetters) { + + // Check if schema has partition keys defined + List partitionKeys = schema.partitionKeys(); + if (partitionKeys == null || partitionKeys.isEmpty()) { + return "default"; + } + + // Get the record data to extract from (after for INSERT/UPDATE/REPLACE, before for DELETE) + RecordData recordData; + switch (event.op()) { + case INSERT: + case UPDATE: + case REPLACE: + recordData = event.after(); + break; + case DELETE: + recordData = event.before(); + break; + default: + throw new IllegalArgumentException("Unsupported operation: " + event.op()); + } + + if (recordData == null) { + throw new IllegalStateException( + "Cannot extract partition path: " + event.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 " + + event.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); + } + + /** + * Extract field value from RecordData based on field type. This is a simplified version - + * complex types may need additional handling. + */ + private Object getFieldValue(RecordData recordData, int fieldIndex, DataType fieldType) { + switch (fieldType.getTypeRoot()) { + case CHAR: + case VARCHAR: + return recordData.getString(fieldIndex).toString(); + case BOOLEAN: + return recordData.getBoolean(fieldIndex); + case TINYINT: + return recordData.getByte(fieldIndex); + case SMALLINT: + return recordData.getShort(fieldIndex); + case INTEGER: + case DATE: + 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, + org.apache.flink.cdc.common.types.DataTypeChecks.getPrecision(fieldType)); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return recordData.getLocalZonedTimestampData( + fieldIndex, + org.apache.flink.cdc.common.types.DataTypeChecks.getPrecision(fieldType)); + default: + // For other types, create a field getter and use it + RecordData.FieldGetter fieldGetter = + RecordData.createFieldGetter(fieldType, fieldIndex); + return fieldGetter.getFieldOrNull(recordData); + } + } } 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/EventBucketStreamWriteFunction.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/EventBucketStreamWriteFunction.java index 677d3f5fcc3..4ab4bc9d19f 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventBucketStreamWriteFunction.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/EventBucketStreamWriteFunction.java @@ -30,6 +30,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.table.types.logical.RowType; + import org.apache.hudi.client.model.HoodieFlinkInternalRow; import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.hash.BucketIndexUtil; 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 index 51648dcc236..c6da42720e6 100644 --- 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 @@ -45,6 +45,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; + import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.sink.common.AbstractStreamWriteFunction; 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 index 9c501f2c7fc..32840aef123 100644 --- 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 @@ -262,9 +262,8 @@ public static HoodieFlinkInternalRow convertDataChangeEventToHoodieFlinkInternal // Extract record key from primary key fields String recordKey = extractRecordKeyFromDataChangeEvent(dataChangeEvent, schema); - // Default partition path - in real implementation this would be based on configured - // partition fields - String partitionPath = "default"; + // Extract partition path from partition key fields + String partitionPath = extractPartitionPathFromDataChangeEvent(dataChangeEvent, schema); return convertDataChangeEventToHoodieFlinkInternalRow( dataChangeEvent, schema, zoneId, recordKey, partitionPath, fileId, instantTime); @@ -345,6 +344,84 @@ private static String extractRecordKeyFromDataChangeEvent( return String.join(",", recordKeyValues); } + /** + * Extract partition path from DataChangeEvent based on partition key fields in schema. + * + *

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 "default". + * + * @param dataChangeEvent The DataChangeEvent to extract partition from + * @param schema The table schema containing partition key definitions + * @return The partition path string + */ + private static String extractPartitionPathFromDataChangeEvent( + DataChangeEvent dataChangeEvent, Schema schema) { + List partitionKeys = schema.partitionKeys(); + if (partitionKeys == null || partitionKeys.isEmpty()) { + return "default"; + } + + // 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()) { From 6679b9e51d836faa80dfd2540c6b89cebba28bcb Mon Sep 17 00:00:00 2001 From: voon Date: Tue, 4 Nov 2025 00:04:50 +0800 Subject: [PATCH 17/33] Checkpoint 47 - Use HoodieFlinkInternalRow --- .../EventBucketStreamWriteFunction.java | 120 ++++++------------ .../hudi/sink/util/RowDataUtils.java | 7 +- 2 files changed, 40 insertions(+), 87 deletions(-) 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/EventBucketStreamWriteFunction.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/EventBucketStreamWriteFunction.java index 4ab4bc9d19f..316051d178b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventBucketStreamWriteFunction.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/EventBucketStreamWriteFunction.java @@ -17,13 +17,10 @@ package org.apache.flink.cdc.connectors.hudi.sink.function; -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.OperationType; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.types.DataType; import org.apache.flink.cdc.connectors.hudi.sink.event.HudiRecordEventSerializer; import org.apache.flink.cdc.connectors.hudi.sink.event.HudiRecordSerializer; import org.apache.flink.cdc.connectors.hudi.sink.model.BucketAssignmentIndex; @@ -82,12 +79,6 @@ public class EventBucketStreamWriteFunction extends EventStreamWriteFunction { /** Cached primary key fields for this table. */ private transient List primaryKeyFields; - /** Cached field getters for primary key fields. */ - private transient List primaryKeyFieldGetters; - - /** Cached schema for this table. */ - private transient Schema cachedSchema; - /** Number of buckets for this function. */ private int numBuckets; @@ -147,8 +138,8 @@ public void processDataChange(DataChangeEvent event) throws Exception { } HoodieFlinkInternalRow hoodieFlinkInternalRow = recordSerializer.serialize(event); - // Calculate bucket from event data for bucket assignment - int bucket = calculateBucketFromEvent(event); + // Calculate bucket from the serialized Hudi record + int bucket = calculateBucketFromRecord(hoodieFlinkInternalRow); // Define record location (file ID, instant time) based on bucket assignment defineRecordLocation(bucket, hoodieFlinkInternalRow); @@ -173,6 +164,18 @@ public void processSchemaChange(SchemaChangeEvent event) throws Exception { // Handle schema events (CreateTableEvent, SchemaChangeEvent) - they don't produce records // null will be returned from serialize recordSerializer.serialize(event); + + // Cache the schema's primary keys for bucket calculation + Schema schema = recordSerializer.getSchema(event.tableId()); + if (schema != null) { + primaryKeyFields = schema.primaryKeys(); + if (primaryKeyFields == null || primaryKeyFields.isEmpty()) { + throw new IllegalStateException( + "Cannot initialize bucket calculation: table " + + event.tableId() + + " has no primary keys"); + } + } } private void defineRecordLocation(int bucketId, HoodieFlinkInternalRow record) { @@ -266,84 +269,33 @@ private void bootstrapIndexIfNeed(String partition) { bucketAssignmentIndex.bootstrapPartition(partition, bucketToFileIDMap); } - /** Calculate bucket from DataChangeEvent using primary key fields. */ - private int calculateBucketFromEvent(DataChangeEvent dataChangeEvent) { - // Initialize cache on first call - if (cachedSchema == null) { - cachedSchema = recordSerializer.getSchema(dataChangeEvent.tableId()); - if (cachedSchema == null) { - throw new IllegalStateException( - "No schema available for table " + dataChangeEvent.tableId()); - } - - // Cache primary key fields - primaryKeyFields = cachedSchema.primaryKeys(); - if (primaryKeyFields.isEmpty()) { - throw new IllegalStateException( - "Cannot calculate bucket: table " - + dataChangeEvent.tableId() - + " has no primary keys"); - } - - // Cache field getters for primary key fields - primaryKeyFieldGetters = new ArrayList<>(primaryKeyFields.size()); - for (String primaryKeyField : primaryKeyFields) { - int fieldIndex = cachedSchema.getColumnNames().indexOf(primaryKeyField); - if (fieldIndex == -1) { - throw new IllegalStateException( - "Primary key field '" - + primaryKeyField - + "' not found in schema for table " - + dataChangeEvent.tableId()); - } - DataType fieldType = cachedSchema.getColumns().get(fieldIndex).getType(); - primaryKeyFieldGetters.add(RecordData.createFieldGetter(fieldType, fieldIndex)); - } - } - - // Extract record key from event data using cached field getters - String recordKey = extractRecordKeyFromEvent(dataChangeEvent); - - // Calculate bucket using Hudi's bucket logic - return calculateBucketFromRecordKey(recordKey, primaryKeyFields); - } - /** - * Extract record key from CDC event data using cached field getters for optimal performance. + * Calculate bucket from HoodieFlinkInternalRow using the record key. The record key is already + * computed by the serializer during conversion from CDC event. */ - private String extractRecordKeyFromEvent(DataChangeEvent dataChangeEvent) { - // For DELETE operations, use 'before' data; for INSERT/UPDATE, use 'after' data - RecordData recordData = - dataChangeEvent.op() == OperationType.DELETE - ? dataChangeEvent.before() - : dataChangeEvent.after(); - - if (recordData == null) { - throw new IllegalStateException( - "Cannot extract record key: " + dataChangeEvent.op() + " event has null data"); - } - - // Use cached field getters for optimal performance - List recordKeyPairs = new ArrayList<>(primaryKeyFields.size()); - for (int i = 0; i < primaryKeyFields.size(); i++) { - RecordData.FieldGetter fieldGetter = primaryKeyFieldGetters.get(i); - Object fieldValue = fieldGetter.getFieldOrNull(recordData); - - if (fieldValue == null) { - throw new IllegalStateException( - "Primary key field '" + primaryKeyFields.get(i) + "' is null in record"); + private int calculateBucketFromRecord(HoodieFlinkInternalRow record) { + // Get record key directly from HoodieFlinkInternalRow + String recordKey = record.getRecordKey(); + + // Initialize primary key fields lazily if not already done + if (primaryKeyFields == null) { + // Parse the record key to extract field names + // Record key format: "fieldName1:value1,fieldName2:value2" + String[] keyPairs = recordKey.split(","); + primaryKeyFields = new ArrayList<>(keyPairs.length); + for (String keyPair : keyPairs) { + String[] parts = keyPair.split(":", 2); + if (parts.length == 2) { + primaryKeyFields.add(parts[0]); + } else { + throw new IllegalStateException( + "Invalid record key format: " + + recordKey + + ". Expected 'fieldName:value' pairs."); + } } - - // Format as "fieldName:value" - recordKeyPairs.add(primaryKeyFields.get(i) + ":" + fieldValue); } - // Join primary key pairs with comma (recordKey1:val1,recordKey2:val2) - return String.join(",", recordKeyPairs); - } - - /** Calculate bucket ID from record key using Hudi's bucket logic. */ - private int calculateBucketFromRecordKey(String recordKey, List primaryKeyFields) { // Convert primary key field list to comma-separated string for Hudi bucket calculation String tableIndexKeyFields = String.join(",", primaryKeyFields); return BucketIdentifier.getBucketId(recordKey, tableIndexKeyFields, numBuckets); 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 index 32840aef123..6df43da91e6 100644 --- 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 @@ -314,7 +314,7 @@ private static String extractRecordKeyFromDataChangeEvent( "Record data is null for operation: " + dataChangeEvent.op()); } - List recordKeyValues = new ArrayList<>(primaryKeyFields.size()); + List recordKeyPairs = new ArrayList<>(primaryKeyFields.size()); for (String primaryKeyField : primaryKeyFields) { int fieldIndex = schema.getColumnNames().indexOf(primaryKeyField); if (fieldIndex == -1) { @@ -338,10 +338,11 @@ private static String extractRecordKeyFromDataChangeEvent( "Primary key field '" + primaryKeyField + "' is null in record"); } - recordKeyValues.add(fieldValue.toString()); + // Format as "fieldName:value" to match BucketAssignOperator format + recordKeyPairs.add(primaryKeyField + ":" + fieldValue); } - return String.join(",", recordKeyValues); + return String.join(",", recordKeyPairs); } /** From bccb94705d9cef6edd0ba2c0666e8ad7343937e3 Mon Sep 17 00:00:00 2001 From: voon Date: Tue, 4 Nov 2025 10:11:51 +0800 Subject: [PATCH 18/33] Checkpoint 48 - Fix partitioning issue for non-partitioned tables --- .../connectors/hudi/sink/bucket/BucketAssignOperator.java | 7 ++++--- .../flink/cdc/connectors/hudi/sink/util/RowDataUtils.java | 7 ++++--- 2 files changed, 8 insertions(+), 6 deletions(-) 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 index e2e5ae00bdb..52a9b86577e 100644 --- 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 @@ -301,13 +301,13 @@ private String extractRecordKey( *

  • Formats them as "field1=value1/field2=value2" (Hive-style partitioning) * * - *

    If no partition keys are defined, returns "default". + *

    If no partition keys are defined, returns empty string (for unpartitioned tables). * * @param event The DataChangeEvent to extract partition from * @param schema The table schema containing partition key definitions * @param fieldGetters Field getters for extracting values (not used currently, may be needed * for optimization) - * @return The partition path string + * @return The partition path string (empty string for unpartitioned tables) */ private String extractPartitionPath( DataChangeEvent event, Schema schema, List fieldGetters) { @@ -315,7 +315,8 @@ private String extractPartitionPath( // Check if schema has partition keys defined List partitionKeys = schema.partitionKeys(); if (partitionKeys == null || partitionKeys.isEmpty()) { - return "default"; + // 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) 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 index 6df43da91e6..52c519d3825 100644 --- 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 @@ -355,17 +355,18 @@ private static String extractRecordKeyFromDataChangeEvent( *

  • Formats them as "field1=value1/field2=value2" (Hive-style partitioning) * * - *

    If no partition keys are defined, returns "default". + *

    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 + * @return The partition path string (empty string for unpartitioned tables) */ private static String extractPartitionPathFromDataChangeEvent( DataChangeEvent dataChangeEvent, Schema schema) { List partitionKeys = schema.partitionKeys(); if (partitionKeys == null || partitionKeys.isEmpty()) { - return "default"; + // 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) From 26dadf0316b845350f25087279f89b0ff304b870 Mon Sep 17 00:00:00 2001 From: voon Date: Tue, 4 Nov 2025 11:06:13 +0800 Subject: [PATCH 19/33] Checkpoint 49 - Use RowDataKeyGen --- .../sink/bucket/BucketAssignOperator.java | 195 +----------------- .../hudi/sink/util/RowDataUtils.java | 62 +++++- 2 files changed, 62 insertions(+), 195 deletions(-) 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 index 52a9b86577e..17a8e01ef49 100644 --- 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 @@ -17,16 +17,14 @@ 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.OperationType; 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.types.DataType; 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; @@ -46,7 +44,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -86,9 +83,6 @@ public class BucketAssignOperator extends AbstractStreamOperator /** Cache of primary key fields per table. */ private final Map> primaryKeyCache = new HashMap<>(); - /** Cache of field getters per table. */ - private final Map> fieldGetterCache = new HashMap<>(); - public BucketAssignOperator(Configuration conf, String schemaOperatorUid) { this.numBuckets = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS); this.schemaOperatorUid = schemaOperatorUid; @@ -131,8 +125,7 @@ public void processElement(StreamRecord streamRecord) throws Exception { Schema newSchema = SchemaUtils.applySchemaChangeEvent(existingSchema, schemaEvent); schemaCache.put(schemaEvent.tableId(), newSchema); - // Clear caches when schema changes - fieldGetterCache.remove(schemaEvent.tableId()); + // Clear primary key cache when schema changes primaryKeyCache.remove(schemaEvent.tableId()); // Broadcast to all tasks @@ -219,195 +212,17 @@ private int calculateTaskIndex(DataChangeEvent event) { "Cannot calculate bucket: table " + tableId + " has no primary keys"); } - // Create final references for use in lambda - final List finalPrimaryKeys = primaryKeys; - - // Get or cache field getters - List fieldGetters = - fieldGetterCache.computeIfAbsent( - tableId, - k -> { - List getters = - new ArrayList<>(finalPrimaryKeys.size()); - for (String primaryKeyField : finalPrimaryKeys) { - int fieldIndex = - finalSchema.getColumnNames().indexOf(primaryKeyField); - if (fieldIndex == -1) { - throw new IllegalStateException( - "Primary key field '" - + primaryKeyField - + "' not found in schema for table " - + tableId); - } - DataType fieldType = - finalSchema.getColumns().get(fieldIndex).getType(); - getters.add(RecordData.createFieldGetter(fieldType, fieldIndex)); - } - return getters; - }); - - // Extract record key - String recordKey = extractRecordKey(event, primaryKeys, fieldGetters); + // Use RowDataUtils to extract record key and partition path + String recordKey = RowDataUtils.extractRecordKeyFromDataChangeEvent(event, finalSchema); + String partition = RowDataUtils.extractPartitionPathFromDataChangeEvent(event, finalSchema); // Calculate bucket using Hudi's logic (0 to numBuckets-1) String tableIndexKeyFields = String.join(",", primaryKeys); int bucketNumber = BucketIdentifier.getBucketId(recordKey, tableIndexKeyFields, numBuckets); - // Extract partition path from the event - String partition = extractPartitionPath(event, finalSchema, fieldGetters); - // Use partition function to map bucket to task index for balanced distribution int taskIndex = partitionIndexFunc.apply(numBuckets, partition, bucketNumber); return taskIndex; } - - private String extractRecordKey( - DataChangeEvent event, - List primaryKeys, - List fieldGetters) { - // For DELETE, use 'before' data; for INSERT/UPDATE, use 'after' data - RecordData recordData = event.op() == OperationType.DELETE ? event.before() : event.after(); - - if (recordData == null) { - throw new IllegalStateException( - "Cannot extract record key: " + event.op() + " event has null data"); - } - - List recordKeyPairs = new ArrayList<>(primaryKeys.size()); - for (int i = 0; i < primaryKeys.size(); i++) { - RecordData.FieldGetter fieldGetter = fieldGetters.get(i); - Object fieldValue = fieldGetter.getFieldOrNull(recordData); - - if (fieldValue == null) { - throw new IllegalStateException( - "Primary key field '" + primaryKeys.get(i) + "' is null in record"); - } - - // Format as "fieldName:value" - recordKeyPairs.add(primaryKeys.get(i) + ":" + fieldValue); - } - - return String.join(",", recordKeyPairs); - } - - /** - * Extract partition path from the DataChangeEvent based on schema partition keys. - * - *

    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 event The DataChangeEvent to extract partition from - * @param schema The table schema containing partition key definitions - * @param fieldGetters Field getters for extracting values (not used currently, may be needed - * for optimization) - * @return The partition path string (empty string for unpartitioned tables) - */ - private String extractPartitionPath( - DataChangeEvent event, Schema schema, List fieldGetters) { - - // Check if schema has partition keys defined - 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 (event.op()) { - case INSERT: - case UPDATE: - case REPLACE: - recordData = event.after(); - break; - case DELETE: - recordData = event.before(); - break; - default: - throw new IllegalArgumentException("Unsupported operation: " + event.op()); - } - - if (recordData == null) { - throw new IllegalStateException( - "Cannot extract partition path: " + event.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 " - + event.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); - } - - /** - * Extract field value from RecordData based on field type. This is a simplified version - - * complex types may need additional handling. - */ - private Object getFieldValue(RecordData recordData, int fieldIndex, DataType fieldType) { - switch (fieldType.getTypeRoot()) { - case CHAR: - case VARCHAR: - return recordData.getString(fieldIndex).toString(); - case BOOLEAN: - return recordData.getBoolean(fieldIndex); - case TINYINT: - return recordData.getByte(fieldIndex); - case SMALLINT: - return recordData.getShort(fieldIndex); - case INTEGER: - case DATE: - 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, - org.apache.flink.cdc.common.types.DataTypeChecks.getPrecision(fieldType)); - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return recordData.getLocalZonedTimestampData( - fieldIndex, - org.apache.flink.cdc.common.types.DataTypeChecks.getPrecision(fieldType)); - default: - // For other types, create a field getter and use it - RecordData.FieldGetter fieldGetter = - RecordData.createFieldGetter(fieldType, fieldIndex); - return fieldGetter.getFieldOrNull(recordData); - } - } } 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 index 52c519d3825..faddfd8fa2a 100644 --- 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 @@ -243,7 +243,51 @@ public static HoodieFlinkInternalRow convertDataChangeEventToHoodieFlinkInternal /** * Convert a DataChangeEvent to a HoodieFlinkInternalRow with automatic record key and partition - * path extraction. + * 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 + ); + } + + /** + * Convert a DataChangeEvent to a HoodieFlinkInternalRow with automatic record key and partition + * path extraction. Falls back to manual extraction when RowDataKeyGen is not available. * * @param dataChangeEvent The DataChangeEvent to convert * @param schema Schema for the table @@ -284,8 +328,15 @@ private static String mapCdcOperationToHudiOperation(OperationType cdcOp) { } } - /** Extract record key from DataChangeEvent based on primary key fields in schema. */ - private static String extractRecordKeyFromDataChangeEvent( + /** + * 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()) { @@ -346,7 +397,8 @@ private static String extractRecordKeyFromDataChangeEvent( } /** - * Extract partition path from DataChangeEvent based on partition key fields in schema. + * 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: * @@ -361,7 +413,7 @@ private static String extractRecordKeyFromDataChangeEvent( * @param schema The table schema containing partition key definitions * @return The partition path string (empty string for unpartitioned tables) */ - private static String extractPartitionPathFromDataChangeEvent( + public static String extractPartitionPathFromDataChangeEvent( DataChangeEvent dataChangeEvent, Schema schema) { List partitionKeys = schema.partitionKeys(); if (partitionKeys == null || partitionKeys.isEmpty()) { From 41aa9164b0a6ca76cd8a4f3ac6325eaf235611bf Mon Sep 17 00:00:00 2001 From: voon Date: Tue, 4 Nov 2025 11:36:57 +0800 Subject: [PATCH 20/33] Checkpoint 50 - Remove code duplication via overloading --- .../sink/event/HudiRecordEventSerializer.java | 60 +------------------ .../MultiTableEventStreamWriteFunction.java | 4 +- 2 files changed, 3 insertions(+), 61 deletions(-) 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 index 80623f1682a..77b67d3bfe6 100644 --- 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 @@ -134,49 +134,7 @@ public HoodieFlinkInternalRow serialize(Event event, String fileId, String insta */ @Override public HoodieFlinkInternalRow serialize(Event event) { - if (event instanceof CreateTableEvent) { - CreateTableEvent createTableEvent = (CreateTableEvent) event; - schemaMaps.put(createTableEvent.tableId(), createTableEvent.getSchema()); - // Clear field getter cache for this table since schema changed - fieldGetterCache.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 field getter cache for this table since schema changed - fieldGetterCache.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."); - } - - // Convert DataChangeEvent to HoodieFlinkInternalRow using utility function - // Use temporary values that will be overridden later - return RowDataUtils.convertDataChangeEventToHoodieFlinkInternalRow( - dataChangeEvent, schema, zoneId, "temp", "temp"); - } else { - throw new IllegalArgumentException( - "Unsupported event type for Hudi serialization: " - + event.getClass().getSimpleName()); - } + return serialize(event, "temp", "temp"); } /** @@ -199,22 +157,6 @@ public boolean hasSchema(TableId tableId) { return schemaMaps.containsKey(tableId); } - /** - * Get cached field getters for a table, creating them if needed. - * - * @param tableId The table identifier - * @return List of field getters or null if schema not available - */ - public List getFieldGetters(TableId tableId) { - Schema schema = schemaMaps.get(tableId); - if (schema == null) { - return null; - } - - return fieldGetterCache.computeIfAbsent( - tableId, k -> RowDataUtils.createFieldGetters(schema, zoneId)); - } - /** * Set schema for a table. Used to initialize table-specific serializers with schema. * 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 index c6da42720e6..577916ea134 100644 --- 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 @@ -436,8 +436,8 @@ private EventBucketStreamWriteFunction createTableFunction(TableId tableId) thro tableFunction.setCorrespondent(tableCorrespondent); tableFunction.setTableId(tableId); - // This is the key change: instead of passing the raw gateway, we pass a proxy - // that intercepts and enhances events with the table path. + // 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)); From 8a96e2371f41271a693a20ad8bdfd8e50a44ec89 Mon Sep 17 00:00:00 2001 From: voon Date: Wed, 5 Nov 2025 14:51:43 +0800 Subject: [PATCH 21/33] Checkpoint 51 - Refactor and remove Event*Functions --- .idea/vcs.xml | 4 +- .../flink-cdc-pipeline-connector-hudi/pom.xml | 2 +- ...tiTableStreamWriteOperatorCoordinator.java | 68 ++ .../sink/event/SchemaChangeOperatorEvent.java | 77 ++ .../EventBucketStreamWriteFunction.java | 308 -------- .../function/EventStreamWriteFunction.java | 709 ------------------ .../MultiTableEventStreamWriteFunction.java | 212 ++++-- .../flink-cdc-pipeline-e2e-tests/pom.xml | 2 +- .../pipeline/tests/MySqlToHudiE2eITCase.java | 15 +- 9 files changed, 308 insertions(+), 1089 deletions(-) create mode 100644 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 delete mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventBucketStreamWriteFunction.java delete mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventStreamWriteFunction.java diff --git a/.idea/vcs.xml b/.idea/vcs.xml index 08d52800435..818d136a83f 100644 --- a/.idea/vcs.xml +++ b/.idea/vcs.xml @@ -20,8 +20,6 @@ - - - \ No newline at end of file + diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml index 5ebd0f36b2b..d92348af784 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml @@ -31,7 +31,7 @@ limitations under the License. flink-cdc-pipeline-connector-hudi - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT 2.10.2 3.4.6 4.1.1 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 index 0bb1e3d7e4c..268c408554b 100644 --- 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 @@ -22,6 +22,7 @@ import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.connectors.hudi.sink.event.CreateTableOperatorEvent; import org.apache.flink.cdc.connectors.hudi.sink.event.EnhancedWriteMetadataEvent; +import org.apache.flink.cdc.connectors.hudi.sink.event.SchemaChangeOperatorEvent; import org.apache.flink.cdc.connectors.hudi.sink.util.RowDataUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -453,6 +454,8 @@ public void handleEventFromOperator( () -> { if (operatorEvent instanceof CreateTableOperatorEvent) { handleCreateTableEvent((CreateTableOperatorEvent) operatorEvent); + } else if (operatorEvent instanceof SchemaChangeOperatorEvent) { + handleSchemaChangeEvent((SchemaChangeOperatorEvent) operatorEvent); } else if (operatorEvent instanceof EnhancedWriteMetadataEvent) { handleEnhancedWriteMetadataEvent( (EnhancedWriteMetadataEvent) operatorEvent); @@ -507,6 +510,71 @@ private void handleCreateTableEvent(CreateTableOperatorEvent createTableOperator }); } + /** + * 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 event The schema change event containing the table ID and new schema + */ + private void handleSchemaChangeEvent(SchemaChangeOperatorEvent event) { + TableId tableId = event.getTableId(); + Schema newSchema = event.getNewSchema(); + + LOG.info( + "Received schema change event for table {}: {} columns", + tableId, + newSchema.getColumnCount()); + + // Update the cached schema + tableSchemas.put(tableId, newSchema); + LOG.info("Updated coordinator's schema cache for table: {}", tableId); + + // 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; + } + + 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); + } 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)); + } + } + private void handleEnhancedWriteMetadataEvent(EnhancedWriteMetadataEvent enhancedEvent) { String tablePath = enhancedEvent.getTablePath(); WriteMetadataEvent event = enhancedEvent.getOriginalEvent(); 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/function/EventBucketStreamWriteFunction.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/EventBucketStreamWriteFunction.java deleted file mode 100644 index 316051d178b..00000000000 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventBucketStreamWriteFunction.java +++ /dev/null @@ -1,308 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.cdc.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.SchemaChangeEvent; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.connectors.hudi.sink.event.HudiRecordEventSerializer; -import org.apache.flink.cdc.connectors.hudi.sink.event.HudiRecordSerializer; -import org.apache.flink.cdc.connectors.hudi.sink.model.BucketAssignmentIndex; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.table.types.logical.RowType; - -import org.apache.hudi.client.model.HoodieFlinkInternalRow; -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.configuration.OptionsResolver; -import org.apache.hudi.index.bucket.BucketIdentifier; -import org.apache.hudi.index.bucket.partition.NumBucketsFunction; -import org.apache.hudi.utils.RuntimeContextUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.time.ZoneId; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** Extension of EventStreamWriteFunction to handle bucketing. */ -public class EventBucketStreamWriteFunction extends EventStreamWriteFunction { - - private static final Logger LOG = LoggerFactory.getLogger(EventBucketStreamWriteFunction.class); - - private int parallelism; - - private boolean isNonBlockingConcurrencyControl; - - /** BucketID to file group mapping in each partition of a tableId. */ - private BucketAssignmentIndex bucketAssignmentIndex; - - /** - * Incremental bucket index of the current checkpoint interval, it is needed because the bucket - * type('I' or 'U') should be decided based on the committed files view, all the records in one - * bucket should have the same bucket type. - */ - private Set incBucketIndexes; - - /** Serializer for converting Events to HoodieFlinkInternalRow for single table. */ - private HudiRecordEventSerializer recordSerializer; - - /** Function for calculating which task should handle a given bucket. */ - private Functions.Function3 taskAssignmentFunc; - - /** Function to calculate num buckets per partition. */ - private NumBucketsFunction numBucketsFunction; - - /** Cached primary key fields for this table. */ - private transient List primaryKeyFields; - - /** Number of buckets for this function. */ - private int numBuckets; - - /** - * Constructs a BucketStreamWriteFunction. - * - * @param config The config options - */ - public EventBucketStreamWriteFunction(Configuration config, RowType rowType) { - super(config, rowType); - } - - @Override - public void open(Configuration parameters) throws IOException { - super.open(parameters); - this.isNonBlockingConcurrencyControl = - OptionsResolver.isNonBlockingConcurrencyControl(config); - this.taskID = RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); - this.parallelism = RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()); - this.bucketAssignmentIndex = new BucketAssignmentIndex(); - this.incBucketIndexes = new HashSet<>(); - this.taskAssignmentFunc = BucketIndexUtil.getPartitionIndexFunc(parallelism); - this.numBucketsFunction = - new NumBucketsFunction( - config.get(FlinkOptions.BUCKET_INDEX_PARTITION_EXPRESSIONS), - config.get(FlinkOptions.BUCKET_INDEX_PARTITION_RULE), - config.get(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS)); - - this.numBuckets = config.get(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS); - - // Initialize record serializer with system default zone ID - this.recordSerializer = new HudiRecordEventSerializer(ZoneId.systemDefault()); - } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - super.initializeState(context); - // Bootstrap will reload file groups from Hudi on startup - } - - @Override - public void snapshotState() { - LOG.info("Triggering snapshotState"); - super.snapshotState(); - this.incBucketIndexes.clear(); - } - - @Override - public void processDataChange(DataChangeEvent event) throws Exception { - // 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); - // Calculate bucket from the serialized Hudi record - int bucket = calculateBucketFromRecord(hoodieFlinkInternalRow); - - // Define record location (file ID, instant time) based on bucket assignment - defineRecordLocation(bucket, hoodieFlinkInternalRow); - - // Buffer the record for writing - bufferRecord(hoodieFlinkInternalRow); - - LOG.debug( - "Processed DataChangeEvent for table {}: partition={}, fileId={}, instantTime={}", - event.tableId(), - hoodieFlinkInternalRow.getPartitionPath(), - hoodieFlinkInternalRow.getFileId(), - hoodieFlinkInternalRow.getInstantTime()); - } - - @Override - public void processSchemaChange(SchemaChangeEvent event) throws Exception { - // Single-table functions typically receive schema via serializer setup - // This is called when CreateTableEvent arrives - LOG.info("Schema change event received: {}", event); - - // Handle schema events (CreateTableEvent, SchemaChangeEvent) - they don't produce records - // null will be returned from serialize - recordSerializer.serialize(event); - - // Cache the schema's primary keys for bucket calculation - Schema schema = recordSerializer.getSchema(event.tableId()); - if (schema != null) { - primaryKeyFields = schema.primaryKeys(); - if (primaryKeyFields == null || primaryKeyFields.isEmpty()) { - throw new IllegalStateException( - "Cannot initialize bucket calculation: table " - + event.tableId() - + " has no primary keys"); - } - } - } - - private void defineRecordLocation(int bucketId, HoodieFlinkInternalRow record) { - final String partition = record.getPartitionPath(); - - // Check if this task should handle this bucket - if (!shouldTaskHandleBucket(bucketId, partition)) { - throw new IllegalStateException( - String.format( - "Task %d received record for bucket %d which should not be handled by this task. " - + "This indicates a partitioning problem - records must be routed to the correct task.", - taskID, bucketId)); - } - - bootstrapIndexIfNeed(partition); - Map bucketToFileId = bucketAssignmentIndex.getBucketToFileIdMap(partition); - final String bucketKey = partition + "/" + bucketId; - - if (incBucketIndexes.contains(bucketKey)) { - record.setInstantTime("I"); - record.setFileId(bucketToFileId.get(bucketId)); - } else if (bucketToFileId.containsKey(bucketId)) { - record.setInstantTime("U"); - record.setFileId(bucketToFileId.get(bucketId)); - } else { - String newFileId = - isNonBlockingConcurrencyControl - ? BucketIdentifier.newBucketFileIdForNBCC(bucketId) - : BucketIdentifier.newBucketFileIdPrefix(bucketId); - record.setInstantTime("I"); - record.setFileId(newFileId); - bucketToFileId.put(bucketId, newFileId); - incBucketIndexes.add(bucketKey); - } - } - - /** - * Determine whether this task should handle the given bucket. Returns true if the bucket is - * assigned to this task based on the task assignment function. - */ - public boolean shouldTaskHandleBucket(int bucketNumber, String partition) { - int numBuckets = numBucketsFunction.getNumBuckets(partition); - return taskAssignmentFunc.apply(numBuckets, partition, bucketNumber) == taskID; - } - - /** - * Get partition_bucket -> fileID mapping from the existing hudi table. This is a required - * operation for each restart to avoid having duplicate file ids for one bucket. - */ - private void bootstrapIndexIfNeed(String partition) { - if (bucketAssignmentIndex.containsPartition(partition)) { - return; - } - LOG.info( - "Loading Hoodie Table {}, with path {}/{}", - this.metaClient.getTableConfig().getTableName(), - this.metaClient.getBasePath(), - partition); - - // Load existing fileID belongs to this task - Map bucketToFileIDMap = new HashMap<>(); - this.writeClient - .getHoodieTable() - .getHoodieView() - .getLatestFileSlices(partition) - .forEach( - fileSlice -> { - String fileId = fileSlice.getFileId(); - int bucketNumber = BucketIdentifier.bucketIdFromFileId(fileId); - if (shouldTaskHandleBucket(bucketNumber, partition)) { - LOG.info( - String.format( - "Should load this partition bucket %s with fileId %s", - bucketNumber, fileId)); - // Validate that one bucketId has only ONE fileId - if (bucketToFileIDMap.containsKey(bucketNumber)) { - throw new RuntimeException( - String.format( - "Duplicate fileId %s from bucket %s of partition %s found " - + "during the BucketStreamWriteFunction index bootstrap.", - fileId, bucketNumber, partition)); - } else { - LOG.info( - String.format( - "Adding fileId %s to the bucket %s of partition %s.", - fileId, bucketNumber, partition)); - bucketToFileIDMap.put(bucketNumber, fileId); - } - } - }); - bucketAssignmentIndex.bootstrapPartition(partition, bucketToFileIDMap); - } - - /** - * Calculate bucket from HoodieFlinkInternalRow using the record key. The record key is already - * computed by the serializer during conversion from CDC event. - */ - private int calculateBucketFromRecord(HoodieFlinkInternalRow record) { - // Get record key directly from HoodieFlinkInternalRow - String recordKey = record.getRecordKey(); - - // Initialize primary key fields lazily if not already done - if (primaryKeyFields == null) { - // Parse the record key to extract field names - // Record key format: "fieldName1:value1,fieldName2:value2" - String[] keyPairs = recordKey.split(","); - primaryKeyFields = new ArrayList<>(keyPairs.length); - for (String keyPair : keyPairs) { - String[] parts = keyPair.split(":", 2); - if (parts.length == 2) { - primaryKeyFields.add(parts[0]); - } else { - throw new IllegalStateException( - "Invalid record key format: " - + recordKey - + ". Expected 'fieldName:value' pairs."); - } - } - } - - // Convert primary key field list to comma-separated string for Hudi bucket calculation - String tableIndexKeyFields = String.join(",", primaryKeyFields); - return BucketIdentifier.getBucketId(recordKey, tableIndexKeyFields, numBuckets); - } - - /** Get the record serializer for schema setup. */ - public HudiRecordSerializer getRecordSerializer() { - return recordSerializer; - } -} 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/EventStreamWriteFunction.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/EventStreamWriteFunction.java deleted file mode 100644 index da5f816613a..00000000000 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/function/EventStreamWriteFunction.java +++ /dev/null @@ -1,709 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.cdc.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.cdc.common.event.TableId; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.streaming.api.functions.ProcessFunction; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.util.MemorySegmentPool; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.Collector; - -import org.apache.avro.Schema; -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.model.HoodieFlinkInternalRow; -import org.apache.hudi.common.engine.HoodieReaderContext; -import org.apache.hudi.common.model.HoodieOperation; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.read.BufferedRecordMerger; -import org.apache.hudi.common.table.read.BufferedRecordMergerFactory; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.common.util.VisibleForTesting; -import org.apache.hudi.common.util.collection.MappingIterator; -import org.apache.hudi.configuration.FlinkOptions; -import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.metrics.FlinkStreamWriteMetrics; -import org.apache.hudi.sink.buffer.MemorySegmentPoolFactory; -import org.apache.hudi.sink.buffer.RowDataBucket; -import org.apache.hudi.sink.buffer.TotalSizeTracer; -import org.apache.hudi.sink.bulk.RowDataKeyGen; -import org.apache.hudi.sink.common.AbstractStreamWriteFunction; -import org.apache.hudi.sink.event.WriteMetadataEvent; -import org.apache.hudi.sink.exception.MemoryPagesExhaustedException; -import org.apache.hudi.sink.transform.RecordConverter; -import org.apache.hudi.sink.utils.BufferUtils; -import org.apache.hudi.table.action.commit.BucketInfo; -import org.apache.hudi.table.action.commit.BucketType; -import org.apache.hudi.table.action.commit.FlinkWriteHelper; -import org.apache.hudi.util.FlinkWriteClients; -import org.apache.hudi.util.MutableIteratorWrapperIterator; -import org.apache.hudi.util.StreamerUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; - -/** Base infrastructures for streaming writer function to handle Events. */ -public abstract class EventStreamWriteFunction extends AbstractStreamWriteFunction - implements EventProcessorFunction { - - private static final long serialVersionUID = 1L; - - private static final Logger LOG = LoggerFactory.getLogger(EventStreamWriteFunction.class); - - /** Write buffer as buckets for a checkpoint. The key is bucket ID (partition path + fileID). */ - protected transient Map buckets; - - /** Write function to trigger the actual write action. */ - protected transient WriteFunction writeFunction; - - private transient BufferedRecordMerger recordMerger; - private transient HoodieReaderContext readerContext; - private transient List orderingFieldNames; - - protected RowType rowType; - - protected final RowDataKeyGen keyGen; - - /** Total size tracer. */ - private transient TotalSizeTracer tracer; - - /** Metrics for flink stream write. */ - protected transient FlinkStreamWriteMetrics writeMetrics; - - /** Table ID for table-specific coordination requests. */ - protected TableId tableId; - - protected transient MemorySegmentPool memorySegmentPool; - - protected transient RecordConverter recordConverter; - - /** - * Constructs an EventStreamWriteFunction. - * - * @param config The config options - */ - public EventStreamWriteFunction(Configuration config, RowType rowType) { - super(config); - this.rowType = rowType; - this.keyGen = RowDataKeyGen.instance(config, rowType); - } - - /** - * Sets the table ID for this function. This is used for table-specific coordination requests. - * - * @param tableId The table ID - */ - public void setTableId(TableId tableId) { - this.tableId = tableId; - } - - @Override - public void open(Configuration parameters) throws IOException { - this.tracer = new TotalSizeTracer(this.config); - initBuffer(); - initWriteFunction(); - initMergeClass(); - initRecordConverter(); - initWriteMetrics(); - } - - @Override - public void snapshotState() { - // Based on the fact that the coordinator starts the checkpoint first, - // it would check the validity. - // wait for the buffer data flush out and request a new instant - LOG.info("Triggered snapshotState"); - flushRemaining(false); - } - - @Override - public final void processElement( - Event event, ProcessFunction.Context ctx, Collector out) - throws Exception { - // Route event to appropriate handler based on type - if (event instanceof DataChangeEvent) { - processDataChange((DataChangeEvent) event); - } 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()); - } - } - - /** - * Process data change events (INSERT/UPDATE/DELETE). This is where actual data is buffered and - * written. - * - *

    Implements {@link EventProcessorFunction#processDataChange(DataChangeEvent)}. - * - * @param event The data change event - */ - @Override - public abstract void processDataChange(DataChangeEvent event) throws Exception; - - /** - * Process schema change events (CREATE TABLE, ADD COLUMN, etc.). Default: No-op. Override if - * schema evolution is needed. - * - *

    Implements {@link EventProcessorFunction#processSchemaChange(SchemaChangeEvent)}. - * - * @param event The schema change event - */ - @Override - public void processSchemaChange(SchemaChangeEvent event) throws Exception { - LOG.debug("Schema change event not handled by {}: {}", getClass().getSimpleName(), event); - } - - /** - * Process flush events for coordinated flushing. Default: Flush all buffered data. - * - *

    Implements {@link EventProcessorFunction#processFlush(FlushEvent)}. - * - * @param event The flush event - */ - @Override - public void processFlush(FlushEvent event) throws Exception { - LOG.info("Received a flush event, flushing all remaining data."); - flushRemaining(false); - } - - @Override - public void close() { - if (this.writeClient != null) { - this.writeClient.close(); - } - } - - /** End input action for batch source. */ - public void endInput() { - super.endInput(); - flushRemaining(true); - this.writeClient.cleanHandles(); - this.writeStatuses.clear(); - } - - // ------------------------------------------------------------------------- - // Utilities - // ------------------------------------------------------------------------- - - private void initBuffer() { - this.buckets = new LinkedHashMap<>(); - this.memorySegmentPool = MemorySegmentPoolFactory.createMemorySegmentPool(config); - } - - private void initWriteFunction() { - final String writeOperation = this.config.get(FlinkOptions.OPERATION); - switch (WriteOperationType.fromValue(writeOperation)) { - case INSERT: - this.writeFunction = - (records, bucketInfo, instantTime) -> - this.writeClient.insert(records, bucketInfo, instantTime); - break; - case UPSERT: - case DELETE: // shares the code path with UPSERT - case DELETE_PREPPED: - this.writeFunction = - (records, bucketInfo, instantTime) -> - this.writeClient.upsert(records, bucketInfo, instantTime); - break; - case INSERT_OVERWRITE: - this.writeFunction = - (records, bucketInfo, instantTime) -> - this.writeClient.insertOverwrite(records, bucketInfo, instantTime); - break; - case INSERT_OVERWRITE_TABLE: - this.writeFunction = - (records, bucketInfo, instantTime) -> - this.writeClient.insertOverwriteTable( - records, bucketInfo, instantTime); - break; - default: - throw new RuntimeException("Unsupported write operation : " + writeOperation); - } - } - - private void initWriteMetrics() { - MetricGroup metrics = getRuntimeContext().getMetricGroup(); - this.writeMetrics = new FlinkStreamWriteMetrics(metrics); - this.writeMetrics.registerMetrics(); - } - - private void initRecordConverter() { - this.recordConverter = RecordConverter.getInstance(keyGen); - } - - private void initMergeClass() { - readerContext = - writeClient - .getEngineContext() - .getReaderContextFactory(metaClient) - .getContext(); - readerContext.initRecordMergerForIngestion(writeClient.getConfig().getProps()); - - recordMerger = - BufferedRecordMergerFactory.create( - readerContext, - readerContext.getMergeMode(), - false, - readerContext.getRecordMerger(), - new Schema.Parser().parse(writeClient.getConfig().getSchema()), - readerContext.getPayloadClasses(writeClient.getConfig().getProps()), - writeClient.getConfig().getProps(), - metaClient.getTableConfig().getPartialUpdateMode()); - LOG.info("init hoodie merge with class [{}]", recordMerger.getClass().getName()); - } - - private boolean doBufferRecord(String bucketID, HoodieFlinkInternalRow record) - throws IOException { - try { - RowDataBucket bucket = - this.buckets.computeIfAbsent( - bucketID, - k -> - new RowDataBucket( - bucketID, - BufferUtils.createBuffer(rowType, memorySegmentPool), - getBucketInfo(record), - this.config.get(FlinkOptions.WRITE_BATCH_SIZE))); - - return bucket.writeRow(record.getRowData()); - } catch (MemoryPagesExhaustedException e) { - LOG.info( - "There is no enough free pages in memory pool to create buffer, need flushing first.", - e); - return false; - } - } - - /** - * Buffers the given record. - * - *

    Flush the data bucket first if the bucket records size is greater than the configured - * value {@link FlinkOptions#WRITE_BATCH_SIZE}. - * - *

    Flush the max size data bucket if the total buffer size exceeds the configured threshold - * {@link FlinkOptions#WRITE_TASK_MAX_SIZE}. - * - * @param record HoodieFlinkInternalRow - */ - protected void bufferRecord(HoodieFlinkInternalRow record) throws IOException { - writeMetrics.markRecordIn(); - // set operation type into rowkind of row. - record.getRowData() - .setRowKind( - RowKind.fromByteValue( - HoodieOperation.fromName(record.getOperationType()).getValue())); - final String bucketID = getBucketID(record.getPartitionPath(), record.getFileId()); - - // 1. try buffer the record into the memory pool - boolean success = doBufferRecord(bucketID, record); - if (!success) { - // 2. flushes the bucket if the memory pool is full - RowDataBucket bucketToFlush = - this.buckets.values().stream() - .max(Comparator.comparingLong(RowDataBucket::getBufferSize)) - .orElseThrow(NoSuchElementException::new); - if (flushBucket(bucketToFlush)) { - // 2.1 flushes the data bucket with maximum size - this.tracer.countDown(bucketToFlush.getBufferSize()); - disposeBucket(bucketToFlush.getBucketId()); - } else { - LOG.warn( - "The buffer size hits the threshold {}, but still flush the max size data bucket failed!", - this.tracer.maxBufferSize); - } - // 2.2 try to write row again - success = doBufferRecord(bucketID, record); - if (!success) { - throw new RuntimeException("Buffer is too small to hold a single record."); - } - } - RowDataBucket bucket = this.buckets.get(bucketID); - this.tracer.trace(bucket.getLastRecordSize()); - // 3. flushes the bucket if it is full - if (bucket.isFull()) { - if (flushBucket(bucket)) { - this.tracer.countDown(bucket.getBufferSize()); - disposeBucket(bucket.getBucketId()); - } - } - // update buffer metrics after tracing buffer size - writeMetrics.setWriteBufferedSize(this.tracer.bufferSize); - } - - private void disposeBucket(String bucketID) { - RowDataBucket bucket = this.buckets.remove(bucketID); - if (bucket != null) { - bucket.dispose(); - } - } - - private String getBucketID(String partitionPath, String fileId) { - return StreamerUtil.generateBucketKey(partitionPath, fileId); - } - - private static BucketInfo getBucketInfo(HoodieFlinkInternalRow internalRow) { - BucketType bucketType; - switch (internalRow.getInstantTime()) { - case "I": - bucketType = BucketType.INSERT; - break; - case "U": - bucketType = BucketType.UPDATE; - break; - default: - throw new HoodieException( - "Unexpected bucket type: " + internalRow.getInstantTime()); - } - return new BucketInfo(bucketType, internalRow.getFileId(), internalRow.getPartitionPath()); - } - - private boolean hasData() { - return !this.buckets.isEmpty() - && this.buckets.values().stream().anyMatch(bucket -> !bucket.isEmpty()); - } - - private boolean flushBucket(RowDataBucket bucket) { - return flushBucket(bucket, this.rowType); - } - - private boolean flushBucket(RowDataBucket bucket, RowType schemaToUse) { - String instant = instantToWriteForTable(true); - - if (instant == null) { - LOG.info("No inflight instant when flushing data, skip."); - return false; - } - - ValidationUtils.checkState( - !bucket.isEmpty(), "Data bucket to flush has no buffering records"); - final List writeStatus = writeRecords(instant, bucket, schemaToUse); - final WriteMetadataEvent event = - WriteMetadataEvent.builder() - .taskID(taskID) - .checkpointId(this.checkpointId) - .instantTime(instant) - .writeStatus(writeStatus) - .lastBatch(false) - .endInput(false) - .build(); - - this.eventGateway.sendEventToCoordinator(event); - writeStatuses.addAll(writeStatus); - return true; - } - - protected void flushRemaining(boolean endInput) { - writeMetrics.startDataFlush(); - this.currentInstant = instantToWriteForTable(hasData()); - if (this.currentInstant == null) { - if (hasData()) { - throw new HoodieException("No inflight instant when flushing data!"); - } else { - LOG.info("No data to flush and no inflight instant, sending empty commit metadata"); - final WriteMetadataEvent event = - WriteMetadataEvent.builder() - .taskID(taskID) - .checkpointId(checkpointId) - .instantTime(instantToWrite(false)) - .writeStatus(Collections.emptyList()) - .lastBatch(true) - .endInput(endInput) - .build(); - this.eventGateway.sendEventToCoordinator(event); - return; - } - } - final List writeStatus; - if (!buckets.isEmpty()) { - writeStatus = new ArrayList<>(); - // Create a snapshot of bucket IDs to avoid issues with disposed buckets - List bucketIds = new ArrayList<>(buckets.keySet()); - for (String bucketId : bucketIds) { - RowDataBucket bucket = buckets.get(bucketId); - if (bucket != null && !bucket.isEmpty()) { - writeStatus.addAll(writeRecords(currentInstant, bucket)); - } - // Remove and dispose bucket immediately after writing - disposeBucket(bucketId); - } - } else { - LOG.info("No data to write in subtask [{}] for instant [{}]", taskID, currentInstant); - writeStatus = Collections.emptyList(); - } - final WriteMetadataEvent event = - WriteMetadataEvent.builder() - .taskID(taskID) - .checkpointId(checkpointId) - .instantTime(currentInstant) - .writeStatus(writeStatus) - .lastBatch(true) - .endInput(endInput) - .build(); - - this.eventGateway.sendEventToCoordinator(event); - this.buckets.clear(); - this.tracer.reset(); - this.writeClient.cleanHandles(); - this.writeStatuses.addAll(writeStatus); - - writeMetrics.endDataFlush(); - writeMetrics.resetAfterCommit(); - } - - protected List writeRecords(String instant, RowDataBucket rowDataBucket) { - return writeRecords(instant, rowDataBucket, this.rowType); - } - - protected List writeRecords( - String instant, RowDataBucket rowDataBucket, RowType schemaToUse) { - writeMetrics.startFileFlush(); - - Iterator rowItr = - new MutableIteratorWrapperIterator<>( - rowDataBucket.getDataIterator(), - () -> new BinaryRowData(schemaToUse.getFieldCount())); - Iterator recordItr = - new MappingIterator<>( - rowItr, - rowData -> recordConverter.convert(rowData, rowDataBucket.getBucketInfo())); - - List statuses = - writeFunction.write( - deduplicateRecordsIfNeeded(recordItr), - rowDataBucket.getBucketInfo(), - instant); - writeMetrics.endFileFlush(); - writeMetrics.increaseNumOfFilesWritten(); - return statuses; - } - - protected Iterator deduplicateRecordsIfNeeded(Iterator records) { - if (config.get(FlinkOptions.PRE_COMBINE)) { - return FlinkWriteHelper.newInstance() - .deduplicateRecords( - records, - null, - -1, - this.writeClient.getConfig().getSchema(), - this.writeClient.getConfig().getProps(), - recordMerger, - readerContext, - orderingFieldNames.toArray(new String[0])); - } else { - return records; - } - } - - /** - * Table-specific version of instantToWrite that delegates to the parent's instantToWrite - * method. The table information is passed through the TableAwareCorrespondent that was set by - * MultiTableEventStreamWriteFunction. - */ - protected String instantToWriteForTable(boolean hasData) { - if (!hasData) { - return null; - } - - if (tableId == null) { - throw new IllegalStateException( - "TableId must be set before requesting instant from coordinator"); - } - - // Use the parent's instant request mechanism - // The TableAwareCorrespondent handles sending the table-specific requests - return instantToWrite(hasData); - } - - /** - * Flush all buckets immediately. Called when schema changes to ensure no data with old schema - * remains in buffers. - * - * @param schemaToUse The RowType schema to use for flushing (should be the OLD schema before - * the change) - */ - public void flushAllBuckets(RowType schemaToUse) { - LOG.info( - "Flushing all {} buckets with schema containing {} fields due to schema change", - buckets.size(), - schemaToUse.getFieldCount()); - if (buckets.isEmpty()) { - LOG.debug("No buckets to flush"); - return; - } - - // Create a snapshot of bucket IDs to avoid concurrent modification - List bucketIds = new ArrayList<>(buckets.keySet()); - - // Flush and dispose all buckets using the provided schema - for (String bucketId : bucketIds) { - RowDataBucket bucket = buckets.get(bucketId); - if (bucket != null && !bucket.isEmpty()) { - try { - flushBucket(bucket, schemaToUse); - } catch (Exception e) { - LOG.error("Failed to flush bucket {} during schema change", bucketId, e); - // Continue flushing other buckets even if one fails - } - } - // Dispose and remove bucket immediately to prevent access to disposed buckets - disposeBucket(bucketId); - } - - tracer.reset(); - LOG.info("All buckets flushed and cleared"); - } - - /** - * Update the rowType when schema evolves. This ensures new buffers are created with the correct - * schema. Note: keyGen is not updated since primary keys cannot change during schema evolution. - * - * @param newRowType The new RowType after schema evolution - */ - public void updateRowType(RowType newRowType) { - LOG.info( - "Updating RowType from {} fields to {} fields", - rowType.getFieldCount(), - newRowType.getFieldCount()); - this.rowType = newRowType; - - // Note: We do NOT call initMergeClass() here because: - // 1. We just flushed buffered data with OLD schema to parquet files - // 2. If we reinit merge components now, Hudi will expect NEW schema - // 3. During the next checkpoint, Hudi may need to read those files for merging - // 4. Reading old files with new converters may cause IndexOutOfBoundsException - // - // The merge components will use the old Avro schema until the next open() or - // until we explicitly update Hudi's table schema metadata via HudiMetadataApplier - - // Log active timeline state for debugging - logActiveTimeline(); - - LOG.info("RowType updated successfully"); - } - - public void updateWriteClientWithNewSchema(String newAvroSchema) { - this.config.setString(FlinkOptions.SOURCE_AVRO_SCHEMA.key(), newAvroSchema); - this.writeClient = FlinkWriteClients.createWriteClient(this.config, getRuntimeContext()); - } - - /** Logs the current state of the active timeline for debugging purposes. */ - private void logActiveTimeline() { - try { - if (metaClient != null) { - metaClient.reloadActiveTimeline(); - HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); - - LOG.info("Active timeline state for table {}:", tableId); - LOG.info( - " - Completed commits: {}", - activeTimeline - .getCommitsTimeline() - .filterCompletedInstants() - .countInstants()); - LOG.info( - " - Pending commits: {}", - activeTimeline - .getCommitsTimeline() - .filterPendingExcludingCompaction() - .countInstants()); - - List instantsInfo = new ArrayList<>(); - activeTimeline - .getInstants() - .forEach( - instant -> - instantsInfo.add( - instant.requestedTime() - + "(" - + instant.getState() - + ")")); - LOG.info(" - All instants: {}", instantsInfo); - - LOG.info( - " - Latest completed commit: {}", - activeTimeline - .getCommitsTimeline() - .filterCompletedInstants() - .lastInstant() - .map(instant -> instant.requestedTime()) - .orElse("None")); - } - } catch (Exception e) { - LOG.warn("Failed to log active timeline state", e); - } - } - - // metrics are now created per table via getOrCreateWriteMetrics(TableId) when needed - - // ------------------------------------------------------------------------- - // Getter/Setter - // ------------------------------------------------------------------------- - - @VisibleForTesting - @SuppressWarnings("rawtypes") - public Map> getDataBuffer() { - Map> ret = new HashMap<>(); - for (Map.Entry entry : buckets.entrySet()) { - List records = new ArrayList<>(); - Iterator rowItr = - new MutableIteratorWrapperIterator<>( - entry.getValue().getDataIterator(), - () -> new BinaryRowData(rowType.getFieldCount())); - while (rowItr.hasNext()) { - records.add( - recordConverter.convert(rowItr.next(), entry.getValue().getBucketInfo())); - } - ret.put(entry.getKey(), records); - } - return ret; - } - - // ------------------------------------------------------------------------- - // Inner Classes - // ------------------------------------------------------------------------- - - /** Write function to trigger the actual write action. */ - protected interface WriteFunction extends Serializable { - List write( - Iterator records, BucketInfo bucketInfo, String instant); - } -} 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 index 577916ea134..1a8f029de44 100644 --- 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 @@ -19,6 +19,7 @@ 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; @@ -29,10 +30,7 @@ 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.CreateTableOperatorEvent; -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.event.*; 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; @@ -42,12 +40,17 @@ 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.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.sink.bucket.BucketStreamWriteFunction; import org.apache.hudi.sink.common.AbstractStreamWriteFunction; import org.apache.hudi.sink.event.WriteMetadataEvent; import org.apache.hudi.util.AvroSchemaConverter; @@ -60,6 +63,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.time.ZoneId; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -76,7 +80,7 @@ public class MultiTableEventStreamWriteFunction extends AbstractStreamWriteFunct LoggerFactory.getLogger(MultiTableEventStreamWriteFunction.class); /** Table-specific write functions created dynamically when new tables are encountered. */ - private transient Map tableFunctions; + private transient Map tableFunctions; /** Track tables that have been initialized to avoid duplicate initialization. */ private transient Map initializedTables; @@ -92,6 +96,9 @@ public class MultiTableEventStreamWriteFunction extends AbstractStreamWriteFunct /** 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; @@ -115,7 +122,7 @@ public void initializeState(FunctionInitializationContext context) throws Except TupleSerializer> tupleSerializer = new TupleSerializer( Tuple2.class, - new org.apache.flink.api.common.typeutils.TypeSerializer[] { + new TypeSerializer[] { TableIdSerializer.INSTANCE, SchemaSerializer.INSTANCE }); ListStateDescriptor> schemaStateDescriptor = @@ -154,6 +161,18 @@ public void open(Configuration parameters) throws Exception { 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 @@ -162,7 +181,7 @@ public void processElement(Event event, Context ctx, Collector out) thr // Route event to appropriate handler based on type if (event instanceof DataChangeEvent) { - processDataChange((DataChangeEvent) event); + processDataChange((DataChangeEvent) event, ctx, out); } else if (event instanceof SchemaChangeEvent) { processSchemaChange((SchemaChangeEvent) event); } else if (event instanceof FlushEvent) { @@ -242,6 +261,11 @@ public void processSchemaChange(SchemaChangeEvent event) throws Exception { 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 @@ -251,46 +275,59 @@ public void processSchemaChange(SchemaChangeEvent event) throws Exception { "Invalidated cached table configuration for {} to pick up new schema", tableId); - // If table function exists, flush buffers and update its rowType - EventBucketStreamWriteFunction tableFunction = tableFunctions.get(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 + BucketStreamWriteFunction tableFunction = tableFunctions.get(tableId); if (tableFunction != null) { LOG.info( - "Schema changed for table {}, flushing buffers with OLD schema and updating to NEW RowType", + "Schema changed for table {}, closing and removing old table function", tableId); - // NOTE: Capture the OLD RowType before any changes - // Buffered records were created with this schema - RowType oldRowType = convertSchemaToFlinkRowType(existingSchema); - // Flush existing buffers using the OLD schema - // This ensures records buffered with N columns are read with N-column - // schema - tableFunction.flushAllBuckets(oldRowType); - - // Now safe to update to the NEW schema - // Future records will use this new schema - RowType newRowType = convertSchemaToFlinkRowType(newSchema); - tableFunction.updateRowType(newRowType); - - String newAvroSchema = - AvroSchemaConverter.convertToSchema(newRowType).toString(); + // 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); + } + // Notify coordinator about schema change so it can update its write client + try { + getOperatorEventGateway() + .sendEventToCoordinator( + new SchemaChangeOperatorEvent(tableId, newSchema)); LOG.info( - "Updating write client for table: {} with new schema: {}", + "Sent SchemaChangeOperatorEvent to coordinator for table: {}", + tableId); + } catch (Exception e) { + LOG.error( + "Failed to send SchemaChangeOperatorEvent to coordinator for table: {}", tableId, - newAvroSchema); - - // Update write client's source avro schema with new schema - tableFunction.updateWriteClientWithNewSchema(newAvroSchema); - - LOG.info("Successfully handled schema change 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); } } - // Forward the event to tableFunction so that schemaMap for serializer is updated - tableFunctions.get(event.tableId()).processSchemaChange(event); + // Single-table functions typically receive schema via serializer setup + // This is called when CreateTableEvent arrives + LOG.info("Schema change event received: {}", event); } 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); @@ -305,16 +342,37 @@ public void processSchemaChange(SchemaChangeEvent event) throws Exception { */ @Override public void processDataChange(DataChangeEvent event) throws Exception { + throw new UnsupportedOperationException( + "This method should not be called directly. Use processDataChange(DataChangeEvent, Context, Collector) instead."); + } + + /** + * 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. + */ + public void processDataChange(DataChangeEvent event, 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 - EventBucketStreamWriteFunction tableFunction = getOrCreateTableFunction(tableId); + BucketStreamWriteFunction tableFunction = getOrCreateTableFunction(tableId); + + // Create context adapter to convert Event context to HoodieFlinkInternalRow context + ProcessFunction.Context adaptedContext = + new ContextAdapter(ctx); - // Use the table function to process the change event - // This will convert the event to HoodieFlinkInternalRow and buffer it for writing - tableFunction.processDataChange(event); + tableFunction.processElement(hoodieFlinkInternalRow, adaptedContext, out); LOG.debug("Successfully processed change event for table: {}", tableId); @@ -347,15 +405,19 @@ public void processFlush(FlushEvent event) throws Exception { LOG.info( "Received global flush event, flushing all {} table functions", tableFunctions.size()); - for (Map.Entry entry : + 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", tableIds.size()); + LOG.info("Received flush event {} for {} specific tables", event, tableIds.size()); for (TableId tableId : tableIds) { - EventBucketStreamWriteFunction tableFunction = tableFunctions.get(tableId); + LOG.info( + "Flushing table {} with schema: {}", + tableId, + recordSerializer.getSchema(tableId)); + BucketStreamWriteFunction tableFunction = tableFunctions.get(tableId); if (tableFunction != null) { tableFunction.flushRemaining(false); LOG.debug("Flushed table function for: {}", tableId); @@ -387,15 +449,15 @@ public void processFlush(FlushEvent event) throws Exception { } } - private EventBucketStreamWriteFunction getOrCreateTableFunction(TableId tableId) { - EventBucketStreamWriteFunction existingFunction = tableFunctions.get(tableId); + private BucketStreamWriteFunction getOrCreateTableFunction(TableId tableId) { + BucketStreamWriteFunction existingFunction = tableFunctions.get(tableId); if (existingFunction != null) { return existingFunction; } - LOG.info("Creating new EventBucketStreamWriteFunction for table: {}", tableId); + LOG.info("Creating new BucketStreamWriteFunction for table: {}", tableId); try { - EventBucketStreamWriteFunction tableFunction = createTableFunction(tableId); + BucketStreamWriteFunction tableFunction = createTableFunction(tableId); tableFunctions.put(tableId, tableFunction); initializedTables.put(tableId, true); LOG.info("Successfully created and cached table function for: {}", tableId); @@ -406,7 +468,7 @@ private EventBucketStreamWriteFunction getOrCreateTableFunction(TableId tableId) } } - private EventBucketStreamWriteFunction createTableFunction(TableId tableId) throws Exception { + private BucketStreamWriteFunction createTableFunction(TableId tableId) throws Exception { Schema schema = schemaMaps.get(tableId); if (schema == null) { throw new IllegalStateException( @@ -424,8 +486,16 @@ private EventBucketStreamWriteFunction createTableFunction(TableId tableId) thro Configuration tableConfig = createTableSpecificConfig(tableId); RowType rowType = convertSchemaToFlinkRowType(schema); - EventBucketStreamWriteFunction tableFunction = - new EventBucketStreamWriteFunction(tableConfig, rowType); + // 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); + + BucketStreamWriteFunction tableFunction = + new BucketStreamWriteFunction(tableConfig, rowType); tableFunction.setRuntimeContext(getRuntimeContext()); @@ -434,7 +504,6 @@ private EventBucketStreamWriteFunction createTableFunction(TableId tableId) thro TableAwareCorrespondent tableCorrespondent = TableAwareCorrespondent.getInstance(correspondent, tableId); tableFunction.setCorrespondent(tableCorrespondent); - tableFunction.setTableId(tableId); // Instead of passing the raw gateway, we pass a proxy that intercepts and enhances events // with the table path @@ -453,12 +522,8 @@ private EventBucketStreamWriteFunction createTableFunction(TableId tableId) thro tableFunction.open(tableConfig); - if (tableFunction.getRecordSerializer() instanceof HudiRecordEventSerializer) { - HudiRecordEventSerializer serializer = - (HudiRecordEventSerializer) tableFunction.getRecordSerializer(); - serializer.setSchema(tableId, schema); - LOG.debug("Set schema for table function serializer: {}", tableId); - } + recordSerializer.setSchema(tableId, schema); + LOG.debug("Set schema for table function serializer: {}", tableId); LOG.debug("Successfully created table function for: {}", tableId); return tableFunction; @@ -543,9 +608,9 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { // 1. Call their abstract snapshotState() to flush buffers // 2. Manually update their checkpointId for instant requests long checkpointId = context.getCheckpointId(); - for (Map.Entry entry : tableFunctions.entrySet()) { + for (Map.Entry entry : tableFunctions.entrySet()) { try { - EventBucketStreamWriteFunction tableFunction = entry.getValue(); + BucketStreamWriteFunction tableFunction = entry.getValue(); LOG.debug( "Delegating snapshotState for table: {} with checkpointId: {}", entry.getKey(), @@ -611,7 +676,7 @@ protected void flushRemaining(boolean endInput) { @Override public void close() throws Exception { - for (EventBucketStreamWriteFunction func : tableFunctions.values()) { + for (BucketStreamWriteFunction func : tableFunctions.values()) { try { func.close(); } catch (Exception e) { @@ -624,7 +689,7 @@ public void close() throws Exception { public void endInput() { super.endInput(); flushRemaining(true); - for (EventBucketStreamWriteFunction func : tableFunctions.values()) { + for (BucketStreamWriteFunction func : tableFunctions.values()) { try { func.endInput(); } catch (Exception e) { @@ -633,6 +698,35 @@ public void endInput() { } } + /** + * Adapter to convert ProcessFunction Event RowData Context to ProcessFunction + * HoodieFlinkInternalRow RowData Context. This allows us to call + * BucketStreamWriteFunction.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, 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 d7ef9afb1c1..922009892b5 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,7 +43,7 @@ limitations under the License. 1.6.1 2.3.9 0.7.0 - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT 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 index 738afa12f3e..0e85373109d 100644 --- 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 @@ -81,9 +81,9 @@ public class MySqlToHudiE2eITCase extends PipelineTestEnvironment { private static final String HUDI_FLINK_PROPERTIES = FLINK_PROPERTIES + "\n" - + "taskmanager.memory.jvm-metaspace.size: 1024m" + + "taskmanager.memory.jvm-metaspace.size: 512M" + "\n" - + "taskmanager.memory.task.heap.size: 1024m" + + "taskmanager.memory.task.heap.size: 1024M" + "\n" + "taskmanager.memory.process.size: 4GB"; @@ -92,7 +92,7 @@ public class MySqlToHudiE2eITCase extends PipelineTestEnvironment { private String warehouse; - private final boolean debug = true; + private final boolean debug = false; @BeforeAll public static void initializeContainers() { @@ -129,11 +129,10 @@ public void before() throws Exception { "-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"); + .withFixedExposedPort(9006, 9006) + .withEnv( + "FLINK_ENV_JAVA_OPTS", + "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:9006"); } else { jobManager = new GenericContainer<>(getFlinkDockerImageTag()) From 713237031d96e7b8f2453c2e001815dde289ecfa Mon Sep 17 00:00:00 2001 From: voon Date: Wed, 5 Nov 2025 15:53:08 +0800 Subject: [PATCH 22/33] Checkpoint 52 - Use RowDataKeyGen implementations of RowDataUtils helper functions --- .../sink/bucket/BucketAssignOperator.java | 25 ++++- .../sink/event/HudiRecordEventSerializer.java | 25 ++++- .../hudi/sink/util/RowDataUtils.java | 104 ++++++------------ 3 files changed, 77 insertions(+), 77 deletions(-) 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 index 17a8e01ef49..9b5e19e0378 100644 --- 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 @@ -36,14 +36,17 @@ 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; @@ -83,6 +86,9 @@ public class BucketAssignOperator extends AbstractStreamOperator /** Cache of primary key fields per table. */ private final Map> primaryKeyCache = new HashMap<>(); + /** RowDataKeyGen cache per table for key and partition extraction. */ + private final Map keyGenCache = new HashMap<>(); + public BucketAssignOperator(Configuration conf, String schemaOperatorUid) { this.numBuckets = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS); this.schemaOperatorUid = schemaOperatorUid; @@ -125,8 +131,9 @@ public void processElement(StreamRecord streamRecord) throws Exception { Schema newSchema = SchemaUtils.applySchemaChangeEvent(existingSchema, schemaEvent); schemaCache.put(schemaEvent.tableId(), newSchema); - // Clear primary key cache when schema changes + // Clear caches when schema changes primaryKeyCache.remove(schemaEvent.tableId()); + keyGenCache.remove(schemaEvent.tableId()); // Broadcast to all tasks for (int i = 0; i < totalTasksNumber; i++) { @@ -212,9 +219,19 @@ private int calculateTaskIndex(DataChangeEvent event) { "Cannot calculate bucket: table " + tableId + " has no primary keys"); } - // Use RowDataUtils to extract record key and partition path - String recordKey = RowDataUtils.extractRecordKeyFromDataChangeEvent(event, finalSchema); - String partition = RowDataUtils.extractPartitionPathFromDataChangeEvent(event, finalSchema); + // Get or create RowDataKeyGen for this table + RowDataKeyGen keyGen = + keyGenCache.computeIfAbsent(tableId, k -> RowDataUtils.createKeyGen(finalSchema)); + + // Convert DataChangeEvent to RowData for key extraction + RowData rowData = + RowDataUtils.convertDataChangeEventToRowData( + event, + RowDataUtils.createFieldGetters(finalSchema, ZoneId.systemDefault())); + + // 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); 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 index 77b67d3bfe6..48b1a3dd437 100644 --- 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 @@ -28,6 +28,7 @@ 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; @@ -43,6 +44,7 @@ *

      *
    • 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 *
    * @@ -57,12 +59,16 @@ public class HudiRecordEventSerializer implements HudiRecordSerializer { /** Field getter cache per table for efficient conversion. */ private final Map> fieldGetterCache; + /** 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.fieldGetterCache = new HashMap<>(); + this.keyGenCache = new HashMap<>(); this.zoneId = zoneId; } @@ -81,8 +87,9 @@ public HoodieFlinkInternalRow serialize(Event event, String fileId, String insta if (event instanceof CreateTableEvent) { CreateTableEvent createTableEvent = (CreateTableEvent) event; schemaMaps.put(createTableEvent.tableId(), createTableEvent.getSchema()); - // Clear field getter cache for this table since schema changed + // Clear caches for this table since schema changed fieldGetterCache.remove(createTableEvent.tableId()); + keyGenCache.remove(createTableEvent.tableId()); // Schema events don't produce records return null; @@ -95,8 +102,9 @@ public HoodieFlinkInternalRow serialize(Event event, String fileId, String insta Schema newSchema = SchemaUtils.applySchemaChangeEvent(existingSchema, schemaChangeEvent); schemaMaps.put(schemaChangeEvent.tableId(), newSchema); - // Clear field getter cache for this table since schema changed + // Clear caches for this table since schema changed fieldGetterCache.remove(schemaChangeEvent.tableId()); + keyGenCache.remove(schemaChangeEvent.tableId()); } // Schema events don't produce records return null; @@ -112,9 +120,14 @@ public HoodieFlinkInternalRow serialize(Event event, String fileId, String insta + ". CreateTableEvent should arrive before DataChangeEvent."); } - // Convert DataChangeEvent to HoodieFlinkInternalRow using utility function + // 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, fileId, instantTime); + dataChangeEvent, schema, zoneId, keyGen, fileId, instantTime); } else { throw new IllegalArgumentException( @@ -165,7 +178,9 @@ public boolean hasSchema(TableId tableId) { */ public void setSchema(TableId tableId, Schema schema) { schemaMaps.put(tableId, schema); - // Clear cached field getters for this table so they get recreated with the new schema + // Clear cached field getters and key gens for this table so they get recreated with the new + // schema fieldGetterCache.remove(tableId); + 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/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 index faddfd8fa2a..336d584249a 100644 --- 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 @@ -27,6 +27,7 @@ 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; @@ -50,6 +51,8 @@ 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; @@ -201,46 +204,6 @@ public static FieldGetter createFieldGetter(DataType fieldType, int fieldPos, Zo }; } - /** - * Convert a DataChangeEvent to a HoodieFlinkInternalRow. - * - * @param dataChangeEvent The DataChangeEvent to convert - * @param schema Schema for the table - * @param zoneId Time zone for timestamp conversion - * @param recordKey The record key extracted from the event - * @param partitionPath The partition path extracted from the event - * @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, - String recordKey, - String partitionPath, - String fileId, - String instantTime) { - - // Convert DataChangeEvent to RowData using existing utility - List fieldGetters = createFieldGetters(schema, zoneId); - RowData rowData = convertDataChangeEventToRowData(dataChangeEvent, fieldGetters); - - // 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 - ); - } - /** * 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 @@ -285,34 +248,6 @@ public static HoodieFlinkInternalRow convertDataChangeEventToHoodieFlinkInternal ); } - /** - * Convert a DataChangeEvent to a HoodieFlinkInternalRow with automatic record key and partition - * path extraction. Falls back to manual extraction when RowDataKeyGen is not available. - * - * @param dataChangeEvent The DataChangeEvent to convert - * @param schema Schema for the table - * @param zoneId Time zone for timestamp conversion - * @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, - String fileId, - String instantTime) { - - // Extract record key from primary key fields - String recordKey = extractRecordKeyFromDataChangeEvent(dataChangeEvent, schema); - - // Extract partition path from partition key fields - String partitionPath = extractPartitionPathFromDataChangeEvent(dataChangeEvent, schema); - - return convertDataChangeEventToHoodieFlinkInternalRow( - dataChangeEvent, schema, zoneId, recordKey, partitionPath, fileId, instantTime); - } - /** Map CDC operation type to Hudi operation type string. */ private static String mapCdcOperationToHudiOperation(OperationType cdcOp) { switch (cdcOp) { @@ -633,4 +568,37 @@ public static LogicalType toLogicalType(DataType cdcType) { "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); + } } From 19568cf053e2a90d4bd4517315c46122ca4fc00e Mon Sep 17 00:00:00 2001 From: voon Date: Wed, 5 Nov 2025 16:01:30 +0800 Subject: [PATCH 23/33] Checkpoint 53 - Fix checkstyle issues --- .../sink/function/MultiTableEventStreamWriteFunction.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 index 1a8f029de44..24c72df80ac 100644 --- 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 @@ -30,7 +30,11 @@ 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.*; +import org.apache.flink.cdc.connectors.hudi.sink.event.CreateTableOperatorEvent; +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.SchemaChangeOperatorEvent; +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; From dc04d2741b8bf1f8972709e7cfde4db5d7deeba9 Mon Sep 17 00:00:00 2001 From: voon Date: Wed, 5 Nov 2025 20:45:25 +0800 Subject: [PATCH 24/33] Checkpoint 54 - Remove reflection usage --- .../ExtendedBucketStreamWriteFunction.java | 41 +++++++++++ .../MultiTableEventStreamWriteFunction.java | 68 +++++++------------ 2 files changed, 67 insertions(+), 42 deletions(-) create mode 100644 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 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..0982a5b4064 --- /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,41 @@ +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 index 24c72df80ac..47113241da0 100644 --- 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 @@ -54,7 +54,6 @@ import org.apache.hudi.client.model.HoodieFlinkInternalRow; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.configuration.FlinkOptions; -import org.apache.hudi.sink.bucket.BucketStreamWriteFunction; import org.apache.hudi.sink.common.AbstractStreamWriteFunction; import org.apache.hudi.sink.event.WriteMetadataEvent; import org.apache.hudi.util.AvroSchemaConverter; @@ -63,7 +62,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.lang.reflect.Field; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -73,9 +71,10 @@ import java.util.Map; /** - * Multi-table wrapper function that routes events to table-specific EventBucketStreamWriteFunction - * instances. This approach maintains table isolation by creating dedicated function instances per - * table while keeping the core write functions single-table focused. + * 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 { @@ -84,7 +83,7 @@ public class MultiTableEventStreamWriteFunction extends AbstractStreamWriteFunct LoggerFactory.getLogger(MultiTableEventStreamWriteFunction.class); /** Table-specific write functions created dynamically when new tables are encountered. */ - private transient Map tableFunctions; + private transient Map tableFunctions; /** Track tables that have been initialized to avoid duplicate initialization. */ private transient Map initializedTables; @@ -286,7 +285,7 @@ public void processSchemaChange(SchemaChangeEvent event) throws Exception { // which would cause a schema mismatch error. // A new function with the updated schema will be created on the next // DataChangeEvent - BucketStreamWriteFunction tableFunction = tableFunctions.get(tableId); + ExtendedBucketStreamWriteFunction tableFunction = tableFunctions.get(tableId); if (tableFunction != null) { LOG.info( "Schema changed for table {}, closing and removing old table function", @@ -370,7 +369,7 @@ public void processDataChange(DataChangeEvent event, Context ctx, Collector.Context adaptedContext = @@ -409,7 +408,7 @@ public void processFlush(FlushEvent event) throws Exception { LOG.info( "Received global flush event, flushing all {} table functions", tableFunctions.size()); - for (Map.Entry entry : + for (Map.Entry entry : tableFunctions.entrySet()) { entry.getValue().flushRemaining(false); LOG.debug("Flushed table function for: {}", entry.getKey()); @@ -421,7 +420,7 @@ public void processFlush(FlushEvent event) throws Exception { "Flushing table {} with schema: {}", tableId, recordSerializer.getSchema(tableId)); - BucketStreamWriteFunction tableFunction = tableFunctions.get(tableId); + ExtendedBucketStreamWriteFunction tableFunction = tableFunctions.get(tableId); if (tableFunction != null) { tableFunction.flushRemaining(false); LOG.debug("Flushed table function for: {}", tableId); @@ -453,15 +452,15 @@ public void processFlush(FlushEvent event) throws Exception { } } - private BucketStreamWriteFunction getOrCreateTableFunction(TableId tableId) { - BucketStreamWriteFunction existingFunction = tableFunctions.get(tableId); + private ExtendedBucketStreamWriteFunction getOrCreateTableFunction(TableId tableId) { + ExtendedBucketStreamWriteFunction existingFunction = tableFunctions.get(tableId); if (existingFunction != null) { return existingFunction; } - LOG.info("Creating new BucketStreamWriteFunction for table: {}", tableId); + LOG.info("Creating new ExtendedBucketStreamWriteFunction for table: {}", tableId); try { - BucketStreamWriteFunction tableFunction = createTableFunction(tableId); + ExtendedBucketStreamWriteFunction tableFunction = createTableFunction(tableId); tableFunctions.put(tableId, tableFunction); initializedTables.put(tableId, true); LOG.info("Successfully created and cached table function for: {}", tableId); @@ -472,7 +471,8 @@ private BucketStreamWriteFunction getOrCreateTableFunction(TableId tableId) { } } - private BucketStreamWriteFunction createTableFunction(TableId tableId) throws Exception { + private ExtendedBucketStreamWriteFunction createTableFunction(TableId tableId) + throws Exception { Schema schema = schemaMaps.get(tableId); if (schema == null) { throw new IllegalStateException( @@ -498,8 +498,8 @@ private BucketStreamWriteFunction createTableFunction(TableId tableId) throws Ex schema.getColumnCount(), avroSchemaInConfig); - BucketStreamWriteFunction tableFunction = - new BucketStreamWriteFunction(tableConfig, rowType); + ExtendedBucketStreamWriteFunction tableFunction = + new ExtendedBucketStreamWriteFunction(tableConfig, rowType); tableFunction.setRuntimeContext(getRuntimeContext()); @@ -612,9 +612,10 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { // 1. Call their abstract snapshotState() to flush buffers // 2. Manually update their checkpointId for instant requests long checkpointId = context.getCheckpointId(); - for (Map.Entry entry : tableFunctions.entrySet()) { + for (Map.Entry entry : + tableFunctions.entrySet()) { try { - BucketStreamWriteFunction tableFunction = entry.getValue(); + ExtendedBucketStreamWriteFunction tableFunction = entry.getValue(); LOG.debug( "Delegating snapshotState for table: {} with checkpointId: {}", entry.getKey(), @@ -623,10 +624,10 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { // Call abstract snapshotState() to flush buffers tableFunction.snapshotState(); - // Update the child function's checkpointId using reflection + // Update the child function's checkpointId // This is necessary because child functions need the current checkpointId // when requesting instants from the coordinator - setCheckpointId(tableFunction, checkpointId); + tableFunction.setCheckpointId(checkpointId); LOG.debug("Successfully snapshotted state for table: {}", entry.getKey()); } catch (Exception e) { @@ -637,23 +638,6 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { } } - /** - * Sets the checkpointId field on a child AbstractStreamWriteFunction using reflection. This is - * necessary because checkpointId is protected and child functions are composition objects that - * need the current checkpoint ID for coordinator communication. - */ - private void setCheckpointId(AbstractStreamWriteFunction function, long checkpointId) { - try { - Field checkpointIdField = - AbstractStreamWriteFunction.class.getDeclaredField("checkpointId"); - checkpointIdField.setAccessible(true); - checkpointIdField.setLong(function, checkpointId); - } catch (Exception e) { - LOG.error("Failed to set checkpointId on child function using reflection", e); - throw new RuntimeException("Failed to set checkpointId on child function", e); - } - } - protected void flushRemaining(boolean endInput) { boolean hasData = !tableFunctions.isEmpty(); this.currentInstant = instantToWrite(hasData); @@ -680,7 +664,7 @@ protected void flushRemaining(boolean endInput) { @Override public void close() throws Exception { - for (BucketStreamWriteFunction func : tableFunctions.values()) { + for (ExtendedBucketStreamWriteFunction func : tableFunctions.values()) { try { func.close(); } catch (Exception e) { @@ -693,7 +677,7 @@ public void close() throws Exception { public void endInput() { super.endInput(); flushRemaining(true); - for (BucketStreamWriteFunction func : tableFunctions.values()) { + for (ExtendedBucketStreamWriteFunction func : tableFunctions.values()) { try { func.endInput(); } catch (Exception e) { @@ -705,8 +689,8 @@ public void endInput() { /** * Adapter to convert ProcessFunction Event RowData Context to ProcessFunction * HoodieFlinkInternalRow RowData Context. This allows us to call - * BucketStreamWriteFunction.processElement with the correct context type without managing its - * internal state. + * ExtendedBucketStreamWriteFunction.processElement with the correct context type without + * managing its internal state. */ private class ContextAdapter extends ProcessFunction.Context { private final ProcessFunction.Context delegate; From b5400c0d7099c44adfb9b526b4ae5c69380b7937 Mon Sep 17 00:00:00 2001 From: voon Date: Thu, 6 Nov 2025 18:15:33 +0800 Subject: [PATCH 25/33] Checkpoint 55 - Address comments --- .../flink-cdc-pipeline-connector-hudi/pom.xml | 2 +- .../hudi/sink/HudiDataSinkFactory.java | 13 +- .../sink/bucket/BucketAssignOperator.java | 29 +-- ...tiTableStreamWriteOperatorCoordinator.java | 27 +++ .../sink/event/HudiRecordEventSerializer.java | 13 +- .../sink/function/EventProcessorFunction.java | 28 ++- .../MultiTableEventStreamWriteFunction.java | 215 ++++++++---------- .../sink/model/BucketAssignmentIndex.java | 99 -------- .../operator/MultiTableWriteOperator.java | 8 +- 9 files changed, 174 insertions(+), 260 deletions(-) delete mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/model/BucketAssignmentIndex.java diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml index d92348af784..39747d3da17 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml @@ -94,7 +94,7 @@ limitations under the License. org.apache.hudi - hudi-flink1.20.x + hudi-flink${flink.major.version}.x ${hudi.version} 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/HudiDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactory.java index 22efcdf5fb7..b1f1be7b12e 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactory.java @@ -57,6 +57,17 @@ public HudiDataSink createDataSink(Context context) { FactoryHelper.DefaultContext factoryContext = (FactoryHelper.DefaultContext) context; Configuration config = factoryContext.getFactoryConfiguration(); + // Validate that only BUCKET index type is used + String indexType = config.get(HudiConfig.INDEX_TYPE); + if (indexType != null && !indexType.equalsIgnoreCase("BUCKET")) { + throw new IllegalArgumentException( + String.format( + "Unsupported index type '%s'. Currently only 'BUCKET' index type is supported. " + + "Other index types (e.g., FLINK_STATE, BLOOM, SIMPLE) are not yet implemented " + + "for multi-table CDC pipelines.", + indexType)); + } + String schemaOperatorUid = context.getPipelineConfiguration() .get(PipelineOptions.PIPELINE_SCHEMA_OPERATOR_UID); @@ -69,8 +80,6 @@ public Set> requiredOptions() { Set> options = new HashSet<>(); options.add(HudiConfig.PATH); options.add(HudiConfig.RECORD_KEY_FIELD); - // options.add(HudiConfig.PRECOMBINE_FIELD); - // options.add(HudiConfig.BUCKET_INDEX_NUM_BUCKETS); 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/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 index 9b5e19e0378..ed0d61fd9f5 100644 --- 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 @@ -17,6 +17,7 @@ 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; @@ -83,12 +84,12 @@ public class BucketAssignOperator extends AbstractStreamOperator /** Cache of schemas per table for bucket calculation. */ private final Map schemaCache = new HashMap<>(); - /** Cache of primary key fields per table. */ - private final Map> primaryKeyCache = 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; @@ -132,8 +133,8 @@ public void processElement(StreamRecord streamRecord) throws Exception { schemaCache.put(schemaEvent.tableId(), newSchema); // Clear caches when schema changes - primaryKeyCache.remove(schemaEvent.tableId()); keyGenCache.remove(schemaEvent.tableId()); + fieldGetterCache.remove(schemaEvent.tableId()); // Broadcast to all tasks for (int i = 0; i < totalTasksNumber; i++) { @@ -211,8 +212,7 @@ private int calculateTaskIndex(DataChangeEvent event) { final Schema finalSchema = schema; // Get or cache primary keys - List primaryKeys = - primaryKeyCache.computeIfAbsent(tableId, k -> finalSchema.primaryKeys()); + List primaryKeys = finalSchema.primaryKeys(); if (primaryKeys.isEmpty()) { throw new IllegalStateException( @@ -223,11 +223,14 @@ private int calculateTaskIndex(DataChangeEvent event) { 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, - RowDataUtils.createFieldGetters(finalSchema, ZoneId.systemDefault())); + RowData rowData = RowDataUtils.convertDataChangeEventToRowData(event, fieldGetters); // Use RowDataKeyGen to extract record key and partition path String recordKey = keyGen.getRecordKey(rowData); @@ -237,9 +240,9 @@ private int calculateTaskIndex(DataChangeEvent event) { 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 - int taskIndex = partitionIndexFunc.apply(numBuckets, partition, bucketNumber); - - return taskIndex; + 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/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 index 268c408554b..a710618a899 100644 --- 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 @@ -63,7 +63,11 @@ 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; @@ -489,6 +493,9 @@ private void handleCreateTableEvent(CreateTableOperatorEvent createTableOperator 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); @@ -920,6 +927,26 @@ public void subtaskReady(int i, SubtaskGateway 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 coordinatorPath = baseConfig.getString(FlinkOptions.PATH); 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 index 48b1a3dd437..1b0eeaaf604 100644 --- 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 @@ -17,7 +17,6 @@ package org.apache.flink.cdc.connectors.hudi.sink.event; -import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; @@ -32,7 +31,6 @@ import java.time.ZoneId; import java.util.HashMap; -import java.util.List; import java.util.Map; /** @@ -56,9 +54,6 @@ public class HudiRecordEventSerializer implements HudiRecordSerializer { /** Schema cache per table - populated from CreateTableEvent and SchemaChangeEvent. */ private final Map schemaMaps; - /** Field getter cache per table for efficient conversion. */ - private final Map> fieldGetterCache; - /** RowDataKeyGen cache per table for key and partition extraction. */ private final Map keyGenCache; @@ -67,7 +62,6 @@ public class HudiRecordEventSerializer implements HudiRecordSerializer { public HudiRecordEventSerializer(ZoneId zoneId) { this.schemaMaps = new HashMap<>(); - this.fieldGetterCache = new HashMap<>(); this.keyGenCache = new HashMap<>(); this.zoneId = zoneId; } @@ -87,8 +81,7 @@ public HoodieFlinkInternalRow serialize(Event event, String fileId, String insta if (event instanceof CreateTableEvent) { CreateTableEvent createTableEvent = (CreateTableEvent) event; schemaMaps.put(createTableEvent.tableId(), createTableEvent.getSchema()); - // Clear caches for this table since schema changed - fieldGetterCache.remove(createTableEvent.tableId()); + // Clear keyGenCache for this table since schema changed keyGenCache.remove(createTableEvent.tableId()); // Schema events don't produce records return null; @@ -102,8 +95,7 @@ public HoodieFlinkInternalRow serialize(Event event, String fileId, String insta Schema newSchema = SchemaUtils.applySchemaChangeEvent(existingSchema, schemaChangeEvent); schemaMaps.put(schemaChangeEvent.tableId(), newSchema); - // Clear caches for this table since schema changed - fieldGetterCache.remove(schemaChangeEvent.tableId()); + // Clear keyGenCache for this table since schema changed keyGenCache.remove(schemaChangeEvent.tableId()); } // Schema events don't produce records @@ -180,7 +172,6 @@ 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 - fieldGetterCache.remove(tableId); 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/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 index 3d0458112a3..a07bf335212 100644 --- 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 @@ -18,8 +18,12 @@ 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 @@ -29,22 +33,34 @@ * clear separation of concerns: * *

      - *
    • {@link #processDataChange(DataChangeEvent)} - Handles INSERT/UPDATE/DELETE operations + *
    • {@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). This is where actual data is buffered and - * written. + * Process data change events (INSERT, UPDATE, DELETE operations). * - * @param event The data change event - * @throws Exception if processing fails + *

    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) throws Exception; + void processDataChange( + DataChangeEvent event, + ProcessFunction.Context ctx, + Collector out); /** * Process schema change events (CREATE TABLE, ADD COLUMN, etc.). 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 index 47113241da0..0e8f623e33b 100644 --- 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 @@ -61,10 +61,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; import java.time.ZoneId; import java.util.HashMap; import java.util.List; @@ -215,145 +211,129 @@ public void processSchemaChange(SchemaChangeEvent event) throws Exception { tId -> { try { // Send an explicit event to the coordinator so it can prepare - // resources *before* we attempt to write any data. + // resources (including creating physical directory) *before* we + // attempt to write any data. getOperatorEventGateway() .sendEventToCoordinator( new CreateTableOperatorEvent(createTableEvent)); LOG.info( "Sent CreateTableOperatorEvent to coordinator for new table: {}", tId); - - // Now, create the physical dir for Hudi table. - Configuration tableConfig = createTableSpecificConfig(tId); - createHudiTablePath(tableConfig, tId); } 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); } - return true; // Mark as initialized for this function instance. + // Mark as initialized for this function instance + return true; }); + // Ensure tableFunction is initialized getOrCreateTableFunction(tableId); - } else if (event instanceof SchemaChangeEvent) { - SchemaChangeEvent schemaChangeEvent = event; - Schema existingSchema = schemaMaps.get(tableId); - if (existingSchema != null - && !SchemaUtils.isSchemaChangeEventRedundant( - existingSchema, schemaChangeEvent)) { - - LOG.info( - "Schema change event received for table {}: {}", - tableId, - schemaChangeEvent); - LOG.info( - "Existing schema for table {} has {} columns: {}", - tableId, - existingSchema.getColumnCount(), - existingSchema.getColumnNames()); - - Schema newSchema = - SchemaUtils.applySchemaChangeEvent(existingSchema, schemaChangeEvent); - - LOG.info( - "New schema for table {} has {} columns: {}", - tableId, - newSchema.getColumnCount(), - newSchema.getColumnNames()); - - schemaMaps.put(tableId, newSchema); + return; + } - // 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); + LOG.info("Schema change event received: {}", event); + SchemaChangeEvent schemaChangeEvent = event; + Schema existingSchema = schemaMaps.get(tableId); + if (existingSchema == null + || SchemaUtils.isSchemaChangeEventRedundant( + existingSchema, schemaChangeEvent)) { + return; + } - // 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); - } + LOG.info("Schema change event received for table {}: {}", tableId, schemaChangeEvent); + LOG.info( + "Existing schema for table {} has {} columns: {}", + tableId, + existingSchema.getColumnCount(), + existingSchema.getColumnNames()); + + Schema newSchema = + SchemaUtils.applySchemaChangeEvent(existingSchema, schemaChangeEvent); + + 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 + } - // Notify coordinator about schema change so it can update its write client - try { - getOperatorEventGateway() - .sendEventToCoordinator( - new SchemaChangeOperatorEvent(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 - } + // 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); + } - LOG.debug("Updated schema for table: {}", tableId); - } + // Notify coordinator about schema change so it can update its write client + try { + getOperatorEventGateway() + .sendEventToCoordinator(new SchemaChangeOperatorEvent(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 } - // Single-table functions typically receive schema via serializer setup - // This is called when CreateTableEvent arrives - LOG.info("Schema change event received: {}", event); + 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 (ChangeEvent) for writing. This triggers the actual Hudi write - * operations as side effects by delegating to table-specific functions. - * - *

    Implements {@link EventProcessorFunction#processDataChange(DataChangeEvent)}. - */ - @Override - public void processDataChange(DataChangeEvent event) throws Exception { - throw new UnsupportedOperationException( - "This method should not be called directly. Use processDataChange(DataChangeEvent, Context, Collector) instead."); - } - /** * 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. */ - public void processDataChange(DataChangeEvent event, Context ctx, Collector out) { + @Override + public void processDataChange( + DataChangeEvent event, + ProcessFunction.Context ctx, + Collector out) { TableId tableId = event.tableId(); try { LOG.debug("Processing change event for table: {}", tableId); @@ -385,15 +365,6 @@ public void processDataChange(DataChangeEvent event, Context ctx, Collector - * BucketId -> FileId}. - */ -public class BucketAssignmentIndex implements Serializable { - - private static final long serialVersionUID = 1L; - - /** Index mapping partition paths to bucket-to-fileId mappings for a single table. */ - private final Map> index; - - public BucketAssignmentIndex() { - this.index = new HashMap<>(); - } - - /** - * Retrieves the File ID for a given partition and bucket number. - * - * @param partition The partition path. - * @param bucketNum The bucket number. - * @return An Optional containing the file ID if it exists, otherwise an empty Optional. - */ - public Optional getFileId(String partition, int bucketNum) { - return Optional.ofNullable(index.get(partition)).map(bucketMap -> bucketMap.get(bucketNum)); - } - - /** - * Associates the specified file ID with the specified partition and bucket number. - * - * @param partition The partition path. - * @param bucketNum The bucket number. - * @param fileId The file ID to associate with the bucket. - */ - public void putFileId(String partition, int bucketNum, String fileId) { - index.computeIfAbsent(partition, k -> new HashMap<>()).put(bucketNum, fileId); - } - - /** - * Checks if the index contains mappings for the specified partition. - * - * @param partition The partition path. - * @return true if the index contains a mapping for the partition, false otherwise. - */ - public boolean containsPartition(String partition) { - return index.containsKey(partition); - } - - /** - * Bootstraps the index for a new partition with a pre-loaded map of bucket numbers to file IDs. - * - * @param partition The partition path. - * @param bucketToFileIDMap The map of bucket numbers to file IDs for the partition. - */ - public void bootstrapPartition(String partition, Map bucketToFileIDMap) { - index.put(partition, bucketToFileIDMap); - } - - /** - * Gets the map from bucket number to file ID for a given partition. Creates and returns an - * empty map if one does not exist. - * - * @param partition The partition path. - * @return The map of bucket numbers to file IDs. - */ - public Map getBucketToFileIdMap(String partition) { - return index.computeIfAbsent(partition, k -> new HashMap<>()); - } - - /** Clears the entire index. */ - public void clear() { - index.clear(); - } -} 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 index 2cff14ae143..535bb3bac34 100644 --- 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 @@ -57,13 +57,11 @@ public class MultiTableWriteOperator extends AbstractWriteOperator { * @param config Configuration for the operator */ public MultiTableWriteOperator(Configuration config, String schemaOperatorUid) { - this(config, schemaOperatorUid, new MultiTableEventStreamWriteFunction(config)); + this(schemaOperatorUid, new MultiTableEventStreamWriteFunction(config)); } private MultiTableWriteOperator( - Configuration config, - String schemaOperatorUid, - MultiTableEventStreamWriteFunction writeFunction) { + String schemaOperatorUid, MultiTableEventStreamWriteFunction writeFunction) { super(writeFunction); this.schemaOperatorUid = schemaOperatorUid; this.multiTableWriteFunction = writeFunction; @@ -84,7 +82,7 @@ public void open() throws Exception { multiTableWriteFunction.setSchemaEvolutionClient(schemaEvolutionClient); // Register this sink subtask with the SchemaOperator - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + int subtaskIndex = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); try { schemaEvolutionClient.registerSubtask(subtaskIndex); LOG.info( From c527244eb167e8bfa2e00dad0da5ee1edf6d0720 Mon Sep 17 00:00:00 2001 From: voon Date: Thu, 6 Nov 2025 21:23:37 +0800 Subject: [PATCH 26/33] Checkpoint 56 - Address comments 2 --- .../coordinator/MultiTableStreamWriteOperatorCoordinator.java | 1 - 1 file changed, 1 deletion(-) 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 index a710618a899..34af7959ce7 100644 --- 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 @@ -286,7 +286,6 @@ boolean shouldTriggerCompaction() { public MultiTableStreamWriteOperatorCoordinator(Configuration conf, Context context) { super(conf, context); - conf.setString("fs.file.impl", "org.apache.hadoop.fs.LocalFileSystem"); this.baseConfig = conf; LOG.info( "MultiTableStreamWriteOperatorCoordinator initialized for operator: {} with config: {}", From 5559469ed8f2ce010572b63833d26b0fb5f27921 Mon Sep 17 00:00:00 2001 From: voon Date: Fri, 7 Nov 2025 11:24:13 +0800 Subject: [PATCH 27/33] Checkpoint 57 - Remove manual embedded timeline server management --- ...tiTableStreamWriteOperatorCoordinator.java | 36 +---------- .../MultiTableEventStreamWriteFunction.java | 60 ++++++++----------- .../operator/MultiTableWriteOperator.java | 51 +--------------- 3 files changed, 28 insertions(+), 119 deletions(-) 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 index 34af7959ce7..14e35267613 100644 --- 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 @@ -39,12 +39,10 @@ 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.table.view.FileSystemViewStorageConfig; 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.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieException; @@ -275,12 +273,6 @@ boolean shouldTriggerCompaction() { */ private transient SubtaskGateway[] gateways; - /** - * A dedicated write client whose only job is to run the embedded timeline server. This ensures - * there is only one timeline server for the entire job. - */ - private transient HoodieFlinkWriteClient timelineServerClient; - /** A single-thread executor to handle instant time requests, mimicking the parent behavior. */ private transient NonThrownExecutor instantRequestExecutor; @@ -328,20 +320,6 @@ public void start() throws Exception { // Initialize the gateways array to avoid NullPointerException when subtasks are ready. this.gateways = new SubtaskGateway[context.currentParallelism()]; - // Initialize a single write client for the coordinator path. - // Its primary role is to start and manage the embedded timeline server. - try { - // The baseConfig points to the dummy coordinator path. - // A .hoodie directory is required for the timeline server to start. - StreamerUtil.initTableIfNotExists(this.baseConfig); - this.timelineServerClient = FlinkWriteClients.createWriteClient(this.baseConfig); - LOG.info("Successfully started timeline server on coordinator."); - } catch (Exception e) { - LOG.error("Failed to start timeline server on coordinator.", e); - context.failJob(e); - return; - } - // 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. @@ -903,9 +881,6 @@ private void scheduleTableServicesIfNeeded(TableId tableId, TableContext tableCo @Override public void close() throws Exception { - if (timelineServerClient != null) { - timelineServerClient.close(); - } if (instantRequestExecutor != null) { instantRequestExecutor.close(); } @@ -948,9 +923,7 @@ private static void createHudiTablePath(Configuration config) throws IOException private Configuration createTableSpecificConfig(TableId tableId) { Configuration tableConfig = new Configuration(baseConfig); - String coordinatorPath = baseConfig.getString(FlinkOptions.PATH); - // Use the same logic as MultiTableEventStreamWriteFunction to strip "/coordinator" - String rootPath = coordinatorPath.split("/coordinator")[0]; + String rootPath = baseConfig.getString(FlinkOptions.PATH); String tablePath = String.format( "%s/%s/%s", rootPath, tableId.getSchemaName(), tableId.getTableName()); @@ -974,13 +947,6 @@ private Configuration createTableSpecificConfig(TableId tableId) { tableId); } - // Disable both embedded timeline server and metadata table for per-table clients. - // The central coordinator manages the only timeline server. - tableConfig.setBoolean(HoodieWriteConfig.EMBEDDED_TIMELINE_SERVER_ENABLE.key(), false); - - // Use memory-based file system view since each client is lightweight. - tableConfig.setString(FileSystemViewStorageConfig.VIEW_TYPE.key(), "MEMORY"); - return tableConfig; } 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 index 0e8f623e33b..79cfbd05e08 100644 --- 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 @@ -52,12 +52,10 @@ import org.apache.flink.util.OutputTag; import org.apache.hudi.client.model.HoodieFlinkInternalRow; -import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; 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.apache.hudi.util.ViewStorageProperties; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -107,7 +105,12 @@ public MultiTableEventStreamWriteFunction(Configuration config) { @Override public void initializeState(FunctionInitializationContext context) throws Exception { - super.initializeState(context); + // 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 @@ -523,28 +526,13 @@ private Configuration createTableSpecificConfig(TableId tableId) { AvroSchemaConverter.convertToSchema(rowType).toString(); localTableConfig.set(FlinkOptions.SOURCE_AVRO_SCHEMA, tableAvroSchema); - String rootPath = this.config.get(FlinkOptions.PATH).split("/coordinator")[0]; - if (rootPath != null) { - String tableBasePath = - String.format( - "%s/%s/%s", - rootPath, tableId.getSchemaName(), tableId.getTableName()); - localTableConfig.set(FlinkOptions.PATH, tableBasePath); - } + String rootPath = this.config.get(FlinkOptions.PATH); + String tableBasePath = + String.format( + "%s/%s/%s", + rootPath, tableId.getSchemaName(), tableId.getTableName()); + localTableConfig.set(FlinkOptions.PATH, tableBasePath); - // Modify ViewStorageProperties to point to coordinator table - FileSystemViewStorageConfig viewStorageConfig = - ViewStorageProperties.loadFromProperties( - this.config.get(FlinkOptions.PATH), localTableConfig); - localTableConfig.setString( - FileSystemViewStorageConfig.VIEW_TYPE.key(), - viewStorageConfig.getStorageType().name()); - localTableConfig.setString( - FileSystemViewStorageConfig.REMOTE_HOST_NAME.key(), - viewStorageConfig.getRemoteViewServerHost()); - localTableConfig.setString( - FileSystemViewStorageConfig.REMOTE_PORT_NUM.key(), - viewStorageConfig.getRemoteViewServerPort() + ""); return localTableConfig; }); } @@ -634,11 +622,13 @@ protected void flushRemaining(boolean endInput) { @Override public void close() throws Exception { - for (ExtendedBucketStreamWriteFunction func : tableFunctions.values()) { - try { - func.close(); - } catch (Exception e) { - LOG.error("Failed to close table function", e); + if (tableFunctions != null) { + for (ExtendedBucketStreamWriteFunction func : tableFunctions.values()) { + try { + func.close(); + } catch (Exception e) { + LOG.error("Failed to close table function", e); + } } } super.close(); @@ -646,11 +636,13 @@ public void close() throws Exception { public void endInput() { super.endInput(); - for (ExtendedBucketStreamWriteFunction func : tableFunctions.values()) { - try { - func.endInput(); - } catch (Exception e) { - LOG.error("Failed to complete endInput for table function", e); + if (tableFunctions != null) { + for (ExtendedBucketStreamWriteFunction func : tableFunctions.values()) { + try { + func.endInput(); + } catch (Exception e) { + LOG.error("Failed to complete endInput for table function", e); + } } } } 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 index 535bb3bac34..dda33b99d7e 100644 --- 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 @@ -25,7 +25,6 @@ import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; -import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.sink.common.AbstractWriteOperator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -105,56 +104,8 @@ public void open() throws Exception { */ public static MultiTableWriteOperatorFactory getFactory( Configuration conf, String schemaOperatorUid) { - // Create coordinator-specific configuration with dummy table settings - // This satisfies the coordinator's requirement for table initialization - Configuration coordinatorConfig = createCoordinatorConfig(conf); - LOG.info("Creating multi-table write operator factory with extended coordinator support"); return MultiTableWriteOperatorFactory.instance( - coordinatorConfig, - new MultiTableWriteOperator(coordinatorConfig, schemaOperatorUid)); - } - - /** - * Creates a coordinator-specific configuration with dummy table settings. This satisfies the - * coordinator's requirement for table initialization while actual table routing happens - * dynamically based on incoming events. Uses deterministic naming to allow reuse of existing - * coordinator tables. - */ - private static Configuration createCoordinatorConfig(Configuration originalConfig) { - Configuration coordinatorConfig = new Configuration(); - coordinatorConfig.addAll(originalConfig); - - // Create deterministic dummy table name based on base path hash for reusability - String originalPath = coordinatorConfig.get(FlinkOptions.PATH, "default"); - String pathHash = String.valueOf(Math.abs(originalPath.hashCode())); - String dummyTableName = "coordinator_" + pathHash; - - coordinatorConfig.set(FlinkOptions.TABLE_NAME, dummyTableName); - coordinatorConfig.set(FlinkOptions.DATABASE_NAME, "coordinator_db"); - - // Set deterministic path for coordinator table (allows reuse) - String coordinatorPath = originalPath + "/coordinator/" + dummyTableName; - coordinatorConfig.set(FlinkOptions.PATH, coordinatorPath); - - // Set dummy Avro schema with a simple structure (id: int) - String dummyAvroSchema = - "{\n" - + " \"type\": \"record\",\n" - + " \"name\": \"coordinator_record\",\n" - + " \"fields\": [\n" - + " {\n" - + " \"name\": \"id\",\n" - + " \"type\": \"int\"\n" - + " }\n" - + " ]\n" - + "}"; - coordinatorConfig.set(FlinkOptions.SOURCE_AVRO_SCHEMA, dummyAvroSchema); - - LOG.info( - "Created coordinator config with reusable dummy table: coordinator_db.{} at path: {}", - dummyTableName, - coordinatorPath); - return coordinatorConfig; + conf, new MultiTableWriteOperator(conf, schemaOperatorUid)); } } From 13a2921c44435eb65743aefbdea27480ca9fd860 Mon Sep 17 00:00:00 2001 From: voon Date: Fri, 7 Nov 2025 18:06:41 +0800 Subject: [PATCH 28/33] Checkpoint 58 - Remove unnecessary changes --- .../MultiTableStreamWriteOperatorCoordinator.java | 3 ++- .../flink/cdc/pipeline/tests/MySqlToHudiE2eITCase.java | 5 ----- .../flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java | 2 -- .../cdc/pipeline/tests/utils/PipelineTestEnvironment.java | 7 +------ 4 files changed, 3 insertions(+), 14 deletions(-) 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 index 14e35267613..c7b4581e2ac 100644 --- 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 @@ -565,7 +565,8 @@ private void handleEnhancedWriteMetadataEvent(EnhancedWriteMetadataEvent enhance TableId tableId = pathToTableId.get(tablePath); if (tableId == null) { - LOG.warn("No tableId found for path: {}. Cannot process event.", tablePath); + LOG.error("No tableId found for path: {}. Cannot process event.", tablePath); + context.failJob(new IllegalStateException("No tableId found for path: " + tablePath)); return; } 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 index 0e85373109d..7f4c88858af 100644 --- 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 @@ -812,11 +812,6 @@ public void waitUntilJobState(JobID jobId, Duration timeout, JobStatus expectedS JobStatusMessage message = optMessage.get(); JobStatus jobStatus = message.getJobState(); if (!expectedStatus.isTerminalState() && jobStatus.isTerminalState()) { - try { - Thread.sleep(50000); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } throw new ValidationException( String.format( "Job has been terminated! JobName: %s, JobID: %s, Status: %s", diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java index e3f2510693b..c63c54ad627 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToIcebergE2eITCase.java @@ -243,7 +243,6 @@ public void testSyncWholeDatabase() throws Exception { LOG.error("Update table for CDC failed.", e); throw e; } - List recordsInSnapshotPhase = new ArrayList<>( Arrays.asList( @@ -262,7 +261,6 @@ public void testSyncWholeDatabase() throws Exception { recordsInSnapshotPhase = recordsInSnapshotPhase.stream().sorted().collect(Collectors.toList()); validateSinkResult(warehouse, database, "products", recordsInSnapshotPhase); - Thread.sleep(3600000L); } /** diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java index a631fd02873..914278bcbcc 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java @@ -135,7 +135,7 @@ private int getParallelism() { "blob.server.port: 6124", "taskmanager.numberOfTaskSlots: 10", "parallelism.default: 4", - "execution.checkpointing.interval: 30s", + "execution.checkpointing.interval: 300", "state.backend.type: hashmap", "env.java.opts.all: -Doracle.jdbc.timezoneAsRegion=false", "execution.checkpointing.savepoint-dir: file:///opt/flink", @@ -389,11 +389,6 @@ public void waitUntilJobState(Duration timeout, JobStatus expectedStatus) { JobStatusMessage message = jobStatusMessages.iterator().next(); JobStatus jobStatus = message.getJobState(); if (!expectedStatus.isTerminalState() && jobStatus.isTerminalState()) { - try { - Thread.sleep(50000); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } throw new ValidationException( String.format( "Job has been terminated! JobName: %s, JobID: %s, Status: %s", From 2712125f0bd57841fc144fadea7224732b355038 Mon Sep 17 00:00:00 2001 From: voon Date: Fri, 21 Nov 2025 12:33:05 +0800 Subject: [PATCH 29/33] Checkpoint 59 - Bump hudi version to 1.1.0 --- .../flink-cdc-pipeline-connector-hudi/pom.xml | 7 +++---- .../ExtendedBucketStreamWriteFunction.java | 17 +++++++++++++++++ .../flink-cdc-pipeline-e2e-tests/pom.xml | 2 +- 3 files changed, 21 insertions(+), 5 deletions(-) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml index 39747d3da17..dfd846685fb 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml @@ -31,7 +31,7 @@ limitations under the License. flink-cdc-pipeline-connector-hudi - 1.2.0-SNAPSHOT + 1.1.0 2.10.2 3.4.6 4.1.1 @@ -314,9 +314,7 @@ limitations under the License. javax.servlet:javax.servlet-api org.apache.httpcomponents:* - org.apache.flink:flink-json - org.apache.flink:flink-metrics-dropwizard org.apache.hive:hive-common @@ -335,7 +333,6 @@ limitations under the License. commons-io:commons-io org.openjdk.jol:jol-core - com.fasterxml.jackson.module:jackson-module-afterburner io.airlift:aircompressor @@ -372,6 +369,8 @@ limitations under the License. META-INF/*.SF META-INF/*.DSA META-INF/*.RSA + + META-INF/maven/org.openjdk.jol/jol-core/pom.xml 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 index 0982a5b4064..32ec1ea52bc 100644 --- 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 @@ -1,3 +1,20 @@ +/* + * 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; 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 922009892b5..89ceffb62d2 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,7 +43,7 @@ limitations under the License. 1.6.1 2.3.9 0.7.0 - 1.2.0-SNAPSHOT + 1.1.0 From db6182167291c639069adb4e8142ded379c522fe Mon Sep 17 00:00:00 2001 From: voon Date: Fri, 21 Nov 2025 14:46:16 +0800 Subject: [PATCH 30/33] Checkpoint 60 - Fix java8 usage of Optional from isEmpty to !isPresent --- .../apache/flink/cdc/pipeline/tests/MySqlToHudiE2eITCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 7f4c88858af..b358abf306f 100644 --- 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 @@ -804,7 +804,7 @@ public void waitUntilJobState(JobID jobId, Duration timeout, JobStatus expectedS Optional optMessage = jobStatusMessages.stream().filter(j -> j.getJobId().equals(jobId)).findFirst(); - if (optMessage.isEmpty()) { + if (!optMessage.isPresent()) { LOG.warn("Job: {} not found, waiting for the next loop...", jobId); continue; } From e566dafd133e5138eb36446b76b70e0928b1f0c8 Mon Sep 17 00:00:00 2001 From: Shuo Cheng Date: Mon, 24 Nov 2025 11:46:54 +0800 Subject: [PATCH 31/33] Checkpoint 61 - fix table initialization --- ...tiTableStreamWriteOperatorCoordinator.java | 117 +++++++++--------- .../sink/event/CreateTableOperatorEvent.java | 59 --------- .../sink/event/TableAwareCorrespondent.java | 94 +++++++++++++- .../MultiTableEventStreamWriteFunction.java | 65 +++++----- 4 files changed, 182 insertions(+), 153 deletions(-) delete mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/CreateTableOperatorEvent.java 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 index c7b4581e2ac..29edd0e1076 100644 --- 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 @@ -20,9 +20,11 @@ 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.event.CreateTableOperatorEvent; import org.apache.flink.cdc.connectors.hudi.sink.event.EnhancedWriteMetadataEvent; import org.apache.flink.cdc.connectors.hudi.sink.event.SchemaChangeOperatorEvent; +import org.apache.flink.cdc.connectors.hudi.sink.event.TableAwareCorrespondent.CreateTableRequest; +import org.apache.flink.cdc.connectors.hudi.sink.event.TableAwareCorrespondent.CreateTableResponse; +import org.apache.flink.cdc.connectors.hudi.sink.event.TableAwareCorrespondent.MultiTableInstantTimeRequest; import org.apache.flink.cdc.connectors.hudi.sink.util.RowDataUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -96,29 +98,6 @@ public class MultiTableStreamWriteOperatorCoordinator extends StreamWriteOperato private static final Logger LOG = LoggerFactory.getLogger(MultiTableStreamWriteOperatorCoordinator.class); - /** - * A custom coordination request that includes the TableId to request an instant for a specific - * table. - */ - public static class MultiTableInstantTimeRequest implements CoordinationRequest, Serializable { - 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; - } - } - /** * 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. @@ -406,9 +385,20 @@ public CompletableFuture handleCoordinationRequest( Correspondent.InstantTimeResponse.getInstance( instantTime))); }, - "handling instant time request for checkpoint %d", + "Handling instant time request for checkpoint %d", ((MultiTableInstantTimeRequest) request).getCheckpointId()); return future; + } else if (request instanceof CreateTableRequest) { + CompletableFuture future = new CompletableFuture<>(); + executor.execute( + () -> { + boolean isSuccess = handleCreateTableEvent(((CreateTableRequest) request)); + future.complete( + CoordinationResponseSerDe.wrap(new CreateTableResponse(isSuccess))); + }, + "Handling create table request: ", + request); + return future; } else { LOG.warn("Received an unknown coordination request: {}", request.getClass().getName()); return super.handleCoordinationRequest(request); @@ -433,9 +423,7 @@ public void handleEventFromOperator( int subtask, int attemptNumber, OperatorEvent operatorEvent) { executor.execute( () -> { - if (operatorEvent instanceof CreateTableOperatorEvent) { - handleCreateTableEvent((CreateTableOperatorEvent) operatorEvent); - } else if (operatorEvent instanceof SchemaChangeOperatorEvent) { + if (operatorEvent instanceof SchemaChangeOperatorEvent) { handleSchemaChangeEvent((SchemaChangeOperatorEvent) operatorEvent); } else if (operatorEvent instanceof EnhancedWriteMetadataEvent) { handleEnhancedWriteMetadataEvent( @@ -450,8 +438,8 @@ public void handleEventFromOperator( operatorEvent); } - private void handleCreateTableEvent(CreateTableOperatorEvent createTableOperatorEvent) { - CreateTableEvent event = createTableOperatorEvent.getCreateTableEvent(); + private boolean handleCreateTableEvent(CreateTableRequest createTableRequest) { + CreateTableEvent event = createTableRequest.getCreateTableEvent(); TableId tableId = event.tableId(); // Store the schema for this table @@ -461,37 +449,44 @@ private void handleCreateTableEvent(CreateTableOperatorEvent createTableOperator tableId, event.getSchema().getColumnCount()); - 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; - } - }); + 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; } /** 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/CreateTableOperatorEvent.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/CreateTableOperatorEvent.java deleted file mode 100644 index 45043eb06dc..00000000000 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/event/CreateTableOperatorEvent.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.cdc.connectors.hudi.sink.event; - -import org.apache.flink.cdc.common.event.CreateTableEvent; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; - -/** - * An operator event that encapsulates a {@link CreateTableEvent}. - * - *

    This event 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 class CreateTableOperatorEvent implements OperatorEvent { - - private static final long serialVersionUID = 1L; - - private final CreateTableEvent createTableEvent; - - /** - * Constructs a new CreateTableOperatorEvent. - * - * @param createTableEvent The original CDC event that triggered this operator event. - */ - public CreateTableOperatorEvent(CreateTableEvent createTableEvent) { - this.createTableEvent = createTableEvent; - } - - /** - * Gets the encapsulated {@link CreateTableEvent}. - * - * @return The original create table event. - */ - public CreateTableEvent getCreateTableEvent() { - return createTableEvent; - } - - @Override - public String toString() { - return "CreateTableOperatorEvent{" + "tableId=" + createTableEvent.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/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 index 71ecb3a020d..b370baa5bcf 100644 --- 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 @@ -18,10 +18,13 @@ 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.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; @@ -66,10 +69,8 @@ public static TableAwareCorrespondent getInstance( @Override public String requestInstantTime(long checkpointId) { try { - MultiTableStreamWriteOperatorCoordinator.MultiTableInstantTimeRequest request = - new MultiTableStreamWriteOperatorCoordinator.MultiTableInstantTimeRequest( - checkpointId, tableId); - + MultiTableInstantTimeRequest request = + new MultiTableInstantTimeRequest(checkpointId, tableId); Correspondent.InstantTimeResponse response = CoordinationResponseSerDe.unwrap( this.gateway @@ -83,4 +84,89 @@ public String requestInstantTime(long checkpointId) { 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); + CreateTableResponse 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); + } + } + + /** + * An 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 implements CoordinationRequest { + private static final long serialVersionUID = 1L; + private final CreateTableEvent createTableEvent; + + public CreateTableRequest(CreateTableEvent createtableEvent) { + this.createTableEvent = createtableEvent; + } + + public CreateTableEvent getCreateTableEvent() { + return createTableEvent; + } + + @Override + public String toString() { + return "CreateTableRequest{" + "tableId=" + createTableEvent.tableId() + '}'; + } + } + + public static class CreateTableResponse implements CoordinationResponse { + private static final long serialVersionUID = 1L; + private final boolean success; + + public CreateTableResponse(boolean success) { + this.success = success; + } + + public boolean isSuccess() { + return 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/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 index 79cfbd05e08..ef08e2305ce 100644 --- 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 @@ -30,7 +30,6 @@ 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.CreateTableOperatorEvent; 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.SchemaChangeOperatorEvent; @@ -209,28 +208,35 @@ public void processSchemaChange(SchemaChangeEvent event) throws Exception { schemaMaps.put(tableId, createTableEvent.getSchema()); LOG.debug("Cached schema for new table: {}", tableId); - 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. - getOperatorEventGateway() - .sendEventToCoordinator( - new CreateTableOperatorEvent(createTableEvent)); - LOG.info( - "Sent CreateTableOperatorEvent to coordinator for new table: {}", - tId); - } 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); - } - // Mark as initialized for this function instance - return true; - }); + 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); @@ -238,23 +244,20 @@ public void processSchemaChange(SchemaChangeEvent event) throws Exception { } LOG.info("Schema change event received: {}", event); - SchemaChangeEvent schemaChangeEvent = event; Schema existingSchema = schemaMaps.get(tableId); if (existingSchema == null - || SchemaUtils.isSchemaChangeEventRedundant( - existingSchema, schemaChangeEvent)) { + || SchemaUtils.isSchemaChangeEventRedundant(existingSchema, event)) { return; } - LOG.info("Schema change event received for table {}: {}", tableId, schemaChangeEvent); + 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, schemaChangeEvent); + Schema newSchema = SchemaUtils.applySchemaChangeEvent(existingSchema, event); LOG.info( "New schema for table {} has {} columns: {}", @@ -647,6 +650,10 @@ public void endInput() { } } + 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 From 314b7ef073ed403e35007a2ccf2480f6ec48b42d Mon Sep 17 00:00:00 2001 From: Shuo Cheng Date: Mon, 24 Nov 2025 14:28:30 +0800 Subject: [PATCH 32/33] fix style --- .../connectors/hudi/sink/event/TableAwareCorrespondent.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 index b370baa5bcf..73ce0ead71b 100644 --- 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 @@ -132,7 +132,7 @@ public boolean requestCreatingTable(CreateTableEvent createTableEvent) { } /** - * An CoordinationRequest that encapsulates a {@link CreateTableEvent}. + * 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 @@ -157,6 +157,10 @@ public String toString() { } } + /** + * Response for a {@link CreateTableRequest}. This response is sent from writer coordinator to + * indicate whether the table is created successfully. + */ public static class CreateTableResponse implements CoordinationResponse { private static final long serialVersionUID = 1L; private final boolean success; From 3dadb7f8d4bd94d76a3805f7db7aad2b2bcc1a63 Mon Sep 17 00:00:00 2001 From: Shuo Cheng Date: Thu, 27 Nov 2025 14:58:11 +0800 Subject: [PATCH 33/33] Checkpoint 62 - fix flaky test --- ...tiTableStreamWriteOperatorCoordinator.java | 78 +++++++++++++----- .../sink/event/TableAwareCorrespondent.java | 81 ++++++++++++++++--- .../MultiTableEventStreamWriteFunction.java | 51 ++---------- 3 files changed, 133 insertions(+), 77 deletions(-) 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 index 29edd0e1076..160ba07996f 100644 --- 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 @@ -17,14 +17,13 @@ package org.apache.flink.cdc.connectors.hudi.sink.coordinator; -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.event.EnhancedWriteMetadataEvent; -import org.apache.flink.cdc.connectors.hudi.sink.event.SchemaChangeOperatorEvent; import org.apache.flink.cdc.connectors.hudi.sink.event.TableAwareCorrespondent.CreateTableRequest; -import org.apache.flink.cdc.connectors.hudi.sink.event.TableAwareCorrespondent.CreateTableResponse; 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; @@ -392,13 +391,30 @@ public CompletableFuture handleCoordinationRequest( CompletableFuture future = new CompletableFuture<>(); executor.execute( () -> { - boolean isSuccess = handleCreateTableEvent(((CreateTableRequest) request)); + CreateTableRequest createTableRequest = (CreateTableRequest) request; + boolean isSuccess = handleCreateTableRequest(createTableRequest); future.complete( - CoordinationResponseSerDe.wrap(new CreateTableResponse(isSuccess))); + 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); @@ -423,9 +439,7 @@ public void handleEventFromOperator( int subtask, int attemptNumber, OperatorEvent operatorEvent) { executor.execute( () -> { - if (operatorEvent instanceof SchemaChangeOperatorEvent) { - handleSchemaChangeEvent((SchemaChangeOperatorEvent) operatorEvent); - } else if (operatorEvent instanceof EnhancedWriteMetadataEvent) { + if (operatorEvent instanceof EnhancedWriteMetadataEvent) { handleEnhancedWriteMetadataEvent( (EnhancedWriteMetadataEvent) operatorEvent); } else { @@ -438,16 +452,14 @@ public void handleEventFromOperator( operatorEvent); } - private boolean handleCreateTableEvent(CreateTableRequest createTableRequest) { - CreateTableEvent event = createTableRequest.getCreateTableEvent(); - TableId tableId = event.tableId(); - + private boolean handleCreateTableRequest(CreateTableRequest createTableRequest) { + TableId tableId = createTableRequest.getTableId(); // Store the schema for this table - tableSchemas.put(tableId, event.getSchema()); + tableSchemas.put(tableId, createTableRequest.getSchema()); LOG.info( "Cached schema for table {}: {} columns", tableId, - event.getSchema().getColumnCount()); + createTableRequest.getSchema().getColumnCount()); TableContext tableContext = tableContexts.computeIfAbsent( @@ -493,20 +505,28 @@ private boolean handleCreateTableEvent(CreateTableRequest createTableRequest) { * 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 event The schema change event containing the table ID and new schema + * @param request The schema change request containing the table ID and new schema */ - private void handleSchemaChangeEvent(SchemaChangeOperatorEvent event) { - TableId tableId = event.getTableId(); - Schema newSchema = event.getNewSchema(); + 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: {}", tableId); + LOG.info( + "Updated coordinator's schema cache for table: {}, new schema: {}", + tableId, + newSchema); // Get the existing table context TableContext oldContext = tableContexts.get(tableId); @@ -514,7 +534,7 @@ private void handleSchemaChangeEvent(SchemaChangeOperatorEvent event) { LOG.warn( "Received schema change for unknown table: {}. Skipping write client update.", tableId); - return; + return true; } try { @@ -543,6 +563,7 @@ private void handleSchemaChangeEvent(SchemaChangeOperatorEvent event) { 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( @@ -551,6 +572,7 @@ private void handleSchemaChangeEvent(SchemaChangeOperatorEvent event) { + tableId + " after schema change", e)); + return false; } } @@ -736,7 +758,21 @@ private void commitInstantForTable( 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); 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 index 73ce0ead71b..1b572240986 100644 --- 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 @@ -20,6 +20,7 @@ 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; @@ -117,7 +118,31 @@ public TableId getTableId() { public boolean requestCreatingTable(CreateTableEvent createTableEvent) { try { CreateTableRequest request = new CreateTableRequest(createTableEvent); - CreateTableResponse response = + 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( @@ -139,38 +164,70 @@ public boolean requestCreatingTable(CreateTableEvent createTableEvent) { * 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 implements CoordinationRequest { + public static class CreateTableRequest extends SchemaChangeRequest { private static final long serialVersionUID = 1L; - private final CreateTableEvent createTableEvent; - public CreateTableRequest(CreateTableEvent createtableEvent) { - this.createTableEvent = createtableEvent; + public CreateTableRequest(CreateTableEvent createTableEvent) { + super(createTableEvent.tableId(), createTableEvent.getSchema()); } + } - public CreateTableEvent getCreateTableEvent() { - return createTableEvent; + /** + * 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 "CreateTableRequest{" + "tableId=" + createTableEvent.tableId() + '}'; + return "SchemaChangeRequest{" + "tableId=" + tableId + ", schema=" + schema + '}'; } } /** - * Response for a {@link CreateTableRequest}. This response is sent from writer coordinator to - * indicate whether the table is created successfully. + * 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 CreateTableResponse implements CoordinationResponse { + public static class SchemaChangeResponse implements CoordinationResponse { private static final long serialVersionUID = 1L; + private final TableId tableId; private final boolean success; - public CreateTableResponse(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/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 index ef08e2305ce..cc92bcc5810 100644 --- 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 @@ -32,7 +32,6 @@ 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.SchemaChangeOperatorEvent; 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; @@ -312,8 +311,7 @@ public void processSchemaChange(SchemaChangeEvent event) throws Exception { // Notify coordinator about schema change so it can update its write client try { - getOperatorEventGateway() - .sendEventToCoordinator(new SchemaChangeOperatorEvent(tableId, newSchema)); + getTableAwareCorrespondent(tableId).requestSchemaChange(tableId, newSchema); LOG.info("Sent SchemaChangeOperatorEvent to coordinator for table: {}", tableId); } catch (Exception e) { LOG.error( @@ -493,6 +491,9 @@ private ExtendedBucketStreamWriteFunction createTableFunction(TableId tableId) 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); @@ -566,13 +567,6 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { LOG.info("Persisted {} schemas to state", schemaMaps.size()); } - // Delegate snapshot to table functions - // Child functions are composition objects, not Flink operators, so they shouldn't - // go through the full snapshotState(FunctionSnapshotContext) lifecycle which - // includes state reloading. Instead, we: - // 1. Call their abstract snapshotState() to flush buffers - // 2. Manually update their checkpointId for instant requests - long checkpointId = context.getCheckpointId(); for (Map.Entry entry : tableFunctions.entrySet()) { try { @@ -580,16 +574,8 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { LOG.debug( "Delegating snapshotState for table: {} with checkpointId: {}", entry.getKey(), - checkpointId); - - // Call abstract snapshotState() to flush buffers - tableFunction.snapshotState(); - - // Update the child function's checkpointId - // This is necessary because child functions need the current checkpointId - // when requesting instants from the coordinator - tableFunction.setCheckpointId(checkpointId); - + 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); @@ -597,30 +583,7 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { "Failed to snapshot state for table: " + entry.getKey(), e); } } - } - - protected void flushRemaining(boolean endInput) { - boolean hasData = !tableFunctions.isEmpty(); - this.currentInstant = instantToWrite(hasData); - - if (this.currentInstant == null) { - if (hasData) { - throw new RuntimeException( - "No inflight instant when flushing data for multi-table function!"); - } - } - - LOG.debug( - "Multi-table function requested instant: {} for {} table functions", - this.currentInstant, - tableFunctions.size()); - - // This method is intentionally overridden to be a no-op. - // The MultiTableEventStreamWriteFunction is a dispatcher and does not have its own - // data buffers to flush. Flushing is handled by the individual, table-specific - // write functions it manages. Calling the parent's flushRemaining would cause - // an erroneous, non-table-specific instant request to be sent to the coordinator, - // resulting in the NullPointerException. + this.checkpointId = context.getCheckpointId(); } @Override