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..5ebd0f36b2b --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/pom.xml @@ -0,0 +1,384 @@ + + + + 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} + + + + 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..42fd8b64fc5 --- /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,165 @@ +/* + * 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; + +/** + * 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()); + + 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/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..f7f7e20c463 --- /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,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink; + +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.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.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, 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..22efcdf5fb7 --- /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,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink; + +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}. + */ +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); + + // Compaction settings + options.add(HudiConfig.COMPACTION_DELTA_COMMITS); + return options; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkOptions.java new file mode 100644 index 00000000000..9b2c76bea4d --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkOptions.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink; + +/** HudiDataSink Options reference {@link HudiConfig}. */ +public class HudiDataSinkOptions { + // prefix for passing properties for table creation. + public static final String PREFIX_TABLE_PROPERTIES = "table.properties."; + + // prefix for passing properties for catalog creation. + public static final String PREFIX_CATALOG_PROPERTIES = "catalog.properties."; +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiMetadataApplier.java new file mode 100644 index 00000000000..9d1177440f2 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/HudiMetadataApplier.java @@ -0,0 +1,650 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink; + +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.visitor.SchemaChangeEventVisitor; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.sink.MetadataApplier; +import org.apache.flink.cdc.common.types.utils.DataTypeUtils; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.TableChange; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.types.DataType; + +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.table.catalog.CatalogOptions; +import org.apache.hudi.table.catalog.HoodieCatalog; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * A {@link MetadataApplier} that applies schema changes to Hudi tables. + * + *

This applier is responsible for initializing the Hudi table metadata in the file system if it + * does not already exist. + */ +public class HudiMetadataApplier implements MetadataApplier { + + private static final Logger LOG = LoggerFactory.getLogger(HudiMetadataApplier.class); + + private final Configuration config; + + // Catalog is unSerializable, similar to PaimonMetadataApplier + private transient HoodieCatalog catalog; + + private final org.apache.flink.configuration.Configuration catalogConfig; + + public HudiMetadataApplier(Configuration config) { + this.config = config; + this.catalogConfig = convertToCatalogConfig(config); + } + + @Override + public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { + LOG.info("Applying schema change event: {}", schemaChangeEvent); + // Initialize catalog if not already done + if (catalog == null) { + catalog = new HoodieCatalog("hoodie_catalog", catalogConfig); + try { + catalog.open(); + } catch (CatalogException e) { + throw new RuntimeException("Failed to open HoodieCatalog", e); + } + } + + try { + SchemaChangeEventVisitor.visit( + schemaChangeEvent, + addColumnEvent -> { + applyAddColumn(addColumnEvent); + return null; + }, + alterColumnTypeEvent -> { + applyAlterColumnType(alterColumnTypeEvent); + return null; + }, + createTableEvent -> { + applyCreateTable(createTableEvent); + return null; + }, + dropColumnEvent -> { + applyDropColumn(dropColumnEvent); + return null; + }, + dropTableEvent -> { + throw new UnsupportedOperationException("DropTableEvent is not supported"); + }, + renameColumnEvent -> { + applyRenameColumn(renameColumnEvent); + return null; + }, + truncateTableEvent -> { + throw new UnsupportedOperationException( + "TruncateTableEvent is not supported"); + }); + } catch (Exception e) { + LOG.error("Failed to apply schema change for table {}", schemaChangeEvent.tableId(), e); + throw new RuntimeException("Failed to apply schema change", e); + } + } + + @Override + public void close() throws Exception { + if (catalog != null) { + catalog.close(); + } + } + + private void applyCreateTable(CreateTableEvent event) { + try { + TableId tableId = event.tableId(); + String databaseName = tableId.getSchemaName(); + + // Create database if it doesn't exist + if (!catalog.databaseExists(databaseName)) { + CatalogDatabase database = new CatalogDatabaseImpl(new HashMap<>(), null); + catalog.createDatabase(databaseName, database, true); + LOG.info("Created database: {}", databaseName); + } + + // Convert CDC Schema to Flink ResolvedCatalogTable + ResolvedCatalogTable catalogTable = convertToCatalogTable(event.getSchema()); + ObjectPath objectPath = new ObjectPath(databaseName, tableId.getTableName()); + + // Create table using catalog + catalog.createTable(objectPath, catalogTable, true); + + LOG.info("Successfully created Hudi table {} via catalog", tableId); + } catch (DatabaseAlreadyExistException e) { + // Should not happen because ignoreIfExists=true + LOG.warn("Database already exists: {}", e.getMessage()); + } catch (TableAlreadyExistException e) { + // Should not happen because ignoreIfExists=true + LOG.warn("Table already exists: {}", e.getMessage()); + } catch (Exception e) { + throw new RuntimeException("Failed to create table via catalog", e); + } + } + + private void applyAddColumn(AddColumnEvent event) throws Exception { + TableId tableId = event.tableId(); + ObjectPath objectPath = new ObjectPath(tableId.getSchemaName(), tableId.getTableName()); + + // Get existing table and ensure it's resolved + ResolvedCatalogTable existingTable = getResolvedCatalogTable(objectPath); + ResolvedSchema existingSchema = existingTable.getResolvedSchema(); + + // Build new columns list with added columns + List newColumns = + new ArrayList<>(existingSchema.getColumns()); + List tableChanges = new ArrayList<>(); + for (AddColumnEvent.ColumnWithPosition columnWithPosition : event.getAddedColumns()) { + Column addColumn = columnWithPosition.getAddColumn(); + DataType flinkType = DataTypeUtils.toFlinkDataType(addColumn.getType()); + org.apache.flink.table.catalog.Column newColumn = + org.apache.flink.table.catalog.Column.physical(addColumn.getName(), flinkType); + + // Handle column position + switch (columnWithPosition.getPosition()) { + case FIRST: + newColumns.add(0, newColumn); + + tableChanges.add( + TableChange.add(newColumn, TableChange.ColumnPosition.first())); + break; + case LAST: + newColumns.add(newColumn); + tableChanges.add(TableChange.add(newColumn)); + break; + case BEFORE: + int beforeIndex = + findColumnIndex(newColumns, columnWithPosition.getExistedColumnName()); + newColumns.add(beforeIndex, newColumn); + tableChanges.add( + TableChange.add( + newColumn, + TableChange.ColumnPosition.after( + newColumns.get(beforeIndex).getName()))); + break; + case AFTER: + int afterIndex = + findColumnIndex(newColumns, columnWithPosition.getExistedColumnName()); + newColumns.add(afterIndex + 1, newColumn); + tableChanges.add( + TableChange.add( + newColumn, + TableChange.ColumnPosition.after( + newColumns.get(afterIndex + 1).getName()))); + break; + } + LOG.info( + "Adding column {} to table {} at position {}", + addColumn.getName(), + tableId, + columnWithPosition.getPosition()); + } + + // Create new resolved schema + ResolvedSchema newSchema = + new ResolvedSchema( + newColumns, + existingSchema.getWatermarkSpecs(), + existingSchema.getPrimaryKey().orElse(null)); + + // Create new catalog table + ResolvedCatalogTable newTable = createUpdatedCatalogTable(existingTable, newSchema); + + // Alter table + LOG.info("Committing add column changes {} to HoodieCatalog", tableChanges); + catalog.alterTable(objectPath, newTable, tableChanges, false); + + // Verify the change was persisted + updateAndVerifyTableChange(tableId, objectPath, newSchema); + } + + private void applyDropColumn(DropColumnEvent event) throws Exception { + TableId tableId = event.tableId(); + ObjectPath objectPath = new ObjectPath(tableId.getSchemaName(), tableId.getTableName()); + + // Get existing table and ensure it's resolved + ResolvedCatalogTable existingTable = getResolvedCatalogTable(objectPath); + ResolvedSchema existingSchema = existingTable.getResolvedSchema(); + + LOG.info( + "Before drop - Table {} has columns: {}", + tableId, + existingSchema.getColumns().stream() + .map(org.apache.flink.table.catalog.Column::getName) + .collect(Collectors.toList())); + + // Build new columns list without dropped columns + List newColumns = + new ArrayList<>(existingSchema.getColumns()); + newColumns.removeIf(col -> event.getDroppedColumnNames().contains(col.getName())); + + LOG.info("Dropping columns {} from table {}", event.getDroppedColumnNames(), tableId); + + // Create new resolved schema + ResolvedSchema newSchema = + new ResolvedSchema( + newColumns, + existingSchema.getWatermarkSpecs(), + existingSchema.getPrimaryKey().orElse(null)); + + LOG.info( + "After drop - Table {} should have columns: {}", + tableId, + newSchema.getColumns().stream() + .map(org.apache.flink.table.catalog.Column::getName) + .collect(Collectors.toList())); + + // Create new catalog table + ResolvedCatalogTable newTable = createUpdatedCatalogTable(existingTable, newSchema); + + // Build table changes + List tableChanges = + event.getDroppedColumnNames().stream() + .map(colName -> TableChange.dropColumn(colName)) + .collect(Collectors.toList()); + + // Alter table in using Hoodie's catalog and commit required metadata changes + LOG.info("Committing drop column changes {} to HoodieCatalog", tableChanges); + catalog.alterTable(objectPath, newTable, tableChanges, false); + + LOG.info("Successfully dropped columns from table {}", tableId); + + // Verify the change was persisted + updateAndVerifyTableChange(tableId, objectPath, newSchema); + } + + private void applyRenameColumn(RenameColumnEvent event) throws Exception { + TableId tableId = event.tableId(); + ObjectPath objectPath = new ObjectPath(tableId.getSchemaName(), tableId.getTableName()); + + // Get existing table and ensure it's resolved + ResolvedCatalogTable existingTable = getResolvedCatalogTable(objectPath); + ResolvedSchema existingSchema = existingTable.getResolvedSchema(); + + // Build new columns list with renamed columns + List newColumns = new ArrayList<>(); + List tableChanges = new ArrayList<>(); + for (org.apache.flink.table.catalog.Column oldCol : existingSchema.getColumns()) { + String newName = + event.getNameMapping().getOrDefault(oldCol.getName(), oldCol.getName()); + if (!newName.equals(oldCol.getName())) { + LOG.info( + "Renaming column {} to {} in table {}", oldCol.getName(), newName, tableId); + newColumns.add( + org.apache.flink.table.catalog.Column.physical( + newName, oldCol.getDataType())); + tableChanges.add(TableChange.modifyColumnName(oldCol, newName)); + } else { + // No name change + newColumns.add(oldCol); + } + } + + // Create new resolved schema + ResolvedSchema newSchema = + new ResolvedSchema( + newColumns, + existingSchema.getWatermarkSpecs(), + existingSchema.getPrimaryKey().orElse(null)); + + // Create new catalog table + ResolvedCatalogTable newTable = createUpdatedCatalogTable(existingTable, newSchema); + + // Alter table in using Hoodie's catalog and commit required metadata changes + catalog.alterTable(objectPath, newTable, tableChanges, false); + LOG.info("Successfully renamed columns in table {}", tableId); + + // Verify the change was persisted + updateAndVerifyTableChange(tableId, objectPath, newSchema); + } + + private void applyAlterColumnType(AlterColumnTypeEvent event) throws Exception { + TableId tableId = event.tableId(); + ObjectPath objectPath = new ObjectPath(tableId.getSchemaName(), tableId.getTableName()); + + // Get existing table and ensure it's resolved + ResolvedCatalogTable existingTable = getResolvedCatalogTable(objectPath); + ResolvedSchema existingSchema = existingTable.getResolvedSchema(); + + // Build new columns list with altered types + List newColumns = new ArrayList<>(); + List tableChanges = new ArrayList<>(); + for (org.apache.flink.table.catalog.Column oldCol : existingSchema.getColumns()) { + if (event.getTypeMapping().containsKey(oldCol.getName())) { + DataType newType = + DataTypeUtils.toFlinkDataType(event.getTypeMapping().get(oldCol.getName())); + LOG.info( + "Altering column {} type from {} to {} in table {}", + oldCol.getName(), + oldCol.getDataType(), + newType, + tableId); + newColumns.add( + org.apache.flink.table.catalog.Column.physical(oldCol.getName(), newType)); + tableChanges.add(TableChange.modifyPhysicalColumnType(oldCol, newType)); + } else { + // No type change + newColumns.add(oldCol); + } + } + + // Create new resolved schema + ResolvedSchema newSchema = + new ResolvedSchema( + newColumns, + existingSchema.getWatermarkSpecs(), + existingSchema.getPrimaryKey().orElse(null)); + + // Create new catalog table + ResolvedCatalogTable newTable = createUpdatedCatalogTable(existingTable, newSchema); + + // Alter table by passing in tableChanges + catalog.alterTable(objectPath, newTable, tableChanges, false); + LOG.info("Successfully altered column types in table {}", tableId); + + // Verify the change was persisted + updateAndVerifyTableChange(tableId, objectPath, newSchema); + } + + /** + * Gets a table from the catalog and ensures it's returned as a ResolvedCatalogTable. If the + * catalog returns a DefaultCatalogTable, it will be converted to ResolvedCatalogTable. + */ + private ResolvedCatalogTable getResolvedCatalogTable(ObjectPath objectPath) throws Exception { + CatalogBaseTable table = catalog.getTable(objectPath); + + if (table instanceof ResolvedCatalogTable) { + return (ResolvedCatalogTable) table; + } + + // If it's a CatalogTable (or DefaultCatalogTable), resolve it + if (table instanceof CatalogTable) { + CatalogTable catalogTable = (CatalogTable) table; + org.apache.flink.table.api.Schema schema = catalogTable.getUnresolvedSchema(); + + // Resolve the schema + List resolvedColumns = new ArrayList<>(); + for (org.apache.flink.table.api.Schema.UnresolvedColumn column : schema.getColumns()) { + if (column instanceof org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn) { + org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn physicalColumn = + (org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn) column; + // We need to get the data type - this is already resolved in the schema + // For now, we'll rebuild from the schema by resolving it + // This is a workaround since we don't have access to the type resolver + } + } + + // Alternative approach: rebuild the ResolvedCatalogTable from scratch + // Extract physical columns from the schema + ResolvedSchema resolvedSchema = resolveSchema(schema); + + return new ResolvedCatalogTable(catalogTable, resolvedSchema); + } + + throw new IllegalStateException( + "Unexpected catalog table type: " + table.getClass().getName()); + } + + /** + * Resolves an unresolved schema to a ResolvedSchema. This manually extracts column information + * from the schema. + */ + private ResolvedSchema resolveSchema(org.apache.flink.table.api.Schema unresolvedSchema) { + List columns = new ArrayList<>(); + + for (org.apache.flink.table.api.Schema.UnresolvedColumn unresolvedColumn : + unresolvedSchema.getColumns()) { + if (unresolvedColumn + instanceof org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn) { + org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn physicalColumn = + (org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn) + unresolvedColumn; + + // Get the column name + String columnName = physicalColumn.getName(); + + // Get the data type - cast from AbstractDataType to DataType + // This is safe because unresolved schemas from catalog tables contain DataType + DataType dataType = (DataType) physicalColumn.getDataType(); + + columns.add(org.apache.flink.table.catalog.Column.physical(columnName, dataType)); + } + } + + // Extract primary key if exists + UniqueConstraint primaryKey = null; + if (unresolvedSchema.getPrimaryKey().isPresent()) { + org.apache.flink.table.api.Schema.UnresolvedPrimaryKey unresolvedPrimaryKey = + unresolvedSchema.getPrimaryKey().get(); + primaryKey = + UniqueConstraint.primaryKey( + unresolvedPrimaryKey.getConstraintName(), + unresolvedPrimaryKey.getColumnNames()); + } + + return new ResolvedSchema(columns, new ArrayList<>(), primaryKey); + } + + private int findColumnIndex( + List columns, String columnName) { + for (int i = 0; i < columns.size(); i++) { + if (columns.get(i).getName().equals(columnName)) { + return i; + } + } + throw new IllegalArgumentException("Column not found: " + columnName); + } + + private ResolvedCatalogTable createUpdatedCatalogTable( + ResolvedCatalogTable existingTable, ResolvedSchema newSchema) { + // Build Flink Schema from resolved schema + org.apache.flink.table.api.Schema tableSchema = + org.apache.flink.table.api.Schema.newBuilder() + .fromResolvedSchema(newSchema) + .build(); + + // Create new CatalogTable with same options and comment + CatalogTable catalogTable = + CatalogTable.of( + tableSchema, + existingTable.getComment(), + existingTable.getPartitionKeys(), + existingTable.getOptions()); + + return new ResolvedCatalogTable(catalogTable, newSchema); + } + + /** Converts a Flink DataType to an Avro Schema. */ + private org.apache.avro.Schema convertFlinkTypeToAvro(DataType flinkType) { + org.apache.flink.table.types.logical.LogicalType logicalType = flinkType.getLogicalType(); + + switch (logicalType.getTypeRoot()) { + case BOOLEAN: + return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.BOOLEAN); + case TINYINT: + case SMALLINT: + case INTEGER: + return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT); + case BIGINT: + return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.LONG); + case FLOAT: + return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.FLOAT); + case DOUBLE: + return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.DOUBLE); + case VARCHAR: + case CHAR: + return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.STRING); + case VARBINARY: + case BINARY: + return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.BYTES); + case DECIMAL: + org.apache.flink.table.types.logical.DecimalType decimalType = + (org.apache.flink.table.types.logical.DecimalType) logicalType; + return org.apache.avro.LogicalTypes.decimal( + decimalType.getPrecision(), decimalType.getScale()) + .addToSchema( + org.apache.avro.Schema.create(org.apache.avro.Schema.Type.BYTES)); + case DATE: + return org.apache.avro.LogicalTypes.date() + .addToSchema( + org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT)); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return org.apache.avro.LogicalTypes.timestampMicros() + .addToSchema( + org.apache.avro.Schema.create(org.apache.avro.Schema.Type.LONG)); + default: + // Default to string for unsupported types + LOG.warn("Unsupported Flink type {}, defaulting to STRING", logicalType); + return org.apache.avro.Schema.create(org.apache.avro.Schema.Type.STRING); + } + } + + private void updateAndVerifyTableChange( + TableId tableId, ObjectPath objectPath, ResolvedSchema newSchema) throws Exception { + ResolvedCatalogTable verifyTable = getResolvedCatalogTable(objectPath); + LOG.info( + "Verified - Table {} now has columns: {}", + tableId, + verifyTable.getResolvedSchema().getColumns().stream() + .map(org.apache.flink.table.catalog.Column::getName) + .collect(Collectors.toList())); + } + + /** Converts CDC Configuration to Flink Configuration for HoodieCatalog. */ + private org.apache.flink.configuration.Configuration convertToCatalogConfig( + Configuration cdcConfig) { + org.apache.flink.configuration.Configuration flinkConfig = + new org.apache.flink.configuration.Configuration(); + + // Set catalog path (base path for all tables) + String basePath = cdcConfig.get(HudiConfig.PATH); + flinkConfig.setString(CatalogOptions.CATALOG_PATH.key(), basePath); + + // Set mode to DFS (filesystem-based) + // TODO: make this configurable + flinkConfig.setString(CatalogOptions.MODE.key(), "dfs"); + + // Set default database + flinkConfig.setString(CatalogOptions.DEFAULT_DATABASE.key(), "default"); + + return flinkConfig; + } + + /** Converts CDC Schema to Flink ResolvedCatalogTable. */ + private ResolvedCatalogTable convertToCatalogTable(Schema cdcSchema) { + // Build resolved columns + List resolvedColumns = new ArrayList<>(); + for (Column column : cdcSchema.getColumns()) { + DataType flinkType = DataTypeUtils.toFlinkDataType(column.getType()); + resolvedColumns.add( + org.apache.flink.table.catalog.Column.physical(column.getName(), flinkType)); + } + + // Build primary key constraint + List primaryKeys = cdcSchema.primaryKeys(); + UniqueConstraint primaryKeyConstraint = null; + if (primaryKeys != null && !primaryKeys.isEmpty()) { + primaryKeyConstraint = UniqueConstraint.primaryKey("pk", primaryKeys); + } + + // Build ResolvedSchema + ResolvedSchema resolvedSchema = + new ResolvedSchema( + resolvedColumns, + new ArrayList<>(), // No watermark specs + primaryKeyConstraint); + + // Build Flink Schema from resolved schema + org.apache.flink.table.api.Schema tableSchema = + org.apache.flink.table.api.Schema.newBuilder() + .fromResolvedSchema(resolvedSchema) + .build(); + + // Build table options (Hudi-specific configurations) + Map tableOptions = new HashMap<>(); + + // Add table type + String tableType = config.get(HudiConfig.TABLE_TYPE); + if (tableType != null) { + tableOptions.put(FlinkOptions.TABLE_TYPE.key(), tableType); + } + + // Add record key field + // TODO: Support multiple recordKeyFields + String recordKeyField = config.get(HudiConfig.RECORD_KEY_FIELD); + if (recordKeyField == null && primaryKeys != null && !primaryKeys.isEmpty()) { + // Use first primary key as default record key + recordKeyField = primaryKeys.get(0); + } + if (recordKeyField != null) { + tableOptions.put(FlinkOptions.RECORD_KEY_FIELD.key(), recordKeyField); + } + + // Add ordering fields if specified + String orderingFields = config.get(HudiConfig.ORDERING_FIELDS); + if (orderingFields != null) { + tableOptions.put(FlinkOptions.ORDERING_FIELDS.key(), orderingFields); + } + + // Add partition fields if specified + List partitionKeys = cdcSchema.partitionKeys(); + if (partitionKeys != null && !partitionKeys.isEmpty()) { + tableOptions.put( + FlinkOptions.PARTITION_PATH_FIELD.key(), String.join(",", partitionKeys)); + } + + // Create CatalogTable + CatalogTable catalogTable = + CatalogTable.of( + tableSchema, + cdcSchema.comment(), + partitionKeys != null ? partitionKeys : Collections.emptyList(), + tableOptions); + + return new ResolvedCatalogTable(catalogTable, resolvedSchema); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketAssignOperator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketAssignOperator.java new file mode 100644 index 00000000000..11751336039 --- /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,272 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink.bucket; + +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.FlushEvent; +import org.apache.flink.cdc.common.event.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); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketWrapper.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketWrapper.java new file mode 100644 index 00000000000..794f80a66b9 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketWrapper.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink.bucket; + +import org.apache.flink.cdc.common.event.Event; + +import java.io.Serializable; + +/** + * Wrapper class that implements Event and associates an event with a target bucket/task index. Used + * to enable bucket-based partitioning while allowing schema events to be broadcast. + * + *

By implementing Event, this wrapper can be transparently passed through the operator chain + * while maintaining bidirectional communication for FlushSuccessEvent. + */ +public class BucketWrapper implements Event, Serializable { + + private static final long serialVersionUID = 1L; + + private final int bucket; + private final Event event; + + public BucketWrapper(int bucket, Event event) { + this.bucket = bucket; + this.event = event; + } + + public int getBucket() { + return bucket; + } + + public Event getEvent() { + return event; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketWrapperFlushEvent.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketWrapperFlushEvent.java new file mode 100644 index 00000000000..0aa53996bdb --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketWrapperFlushEvent.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink.bucket; + +import org.apache.flink.cdc.common.event.FlushEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.TableId; + +import java.util.List; +import java.util.Objects; + +/** A wrapper class for {@link FlushEvent} to attach bucket id and task metadata. */ +public class BucketWrapperFlushEvent extends FlushEvent { + + private final int bucket; + private final int bucketAssignTaskId; + + public BucketWrapperFlushEvent( + int bucket, + int sourceSubTaskId, + int bucketAssignTaskId, + List tableIds, + SchemaChangeEventType schemaChangeEventType) { + super(sourceSubTaskId, tableIds, schemaChangeEventType); + this.bucket = bucket; + this.bucketAssignTaskId = bucketAssignTaskId; + } + + public int getBucket() { + return bucket; + } + + public int getBucketAssignTaskId() { + return bucketAssignTaskId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + BucketWrapperFlushEvent that = (BucketWrapperFlushEvent) o; + return bucket == that.bucket + && bucketAssignTaskId == that.bucketAssignTaskId + && getSourceSubTaskId() == that.getSourceSubTaskId(); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), bucket, bucketAssignTaskId); + } + + @Override + public String toString() { + return "BucketWrapperFlushEvent{subTaskId=" + + getSourceSubTaskId() + + ", bucketAssignTaskId=" + + bucketAssignTaskId + + ", bucket=" + + bucket + + '}'; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/FlushEventAlignmentOperator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/FlushEventAlignmentOperator.java new file mode 100644 index 00000000000..615959916b8 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/FlushEventAlignmentOperator.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink.bucket; + +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.FlushEvent; +import org.apache.flink.cdc.runtime.operators.schema.regular.SchemaOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** Align {@link FlushEvent}s broadcasted by {@link BucketAssignOperator}. */ +public class FlushEventAlignmentOperator extends AbstractStreamOperator + implements OneInputStreamOperator { + + private transient int totalTasksNumber; + + /** + * Key: subtask id of {@link SchemaOperator}, Value: subtask ids of {@link + * BucketAssignOperator}. + */ + private transient Map> sourceTaskIdToAssignBucketSubTaskIds; + + private transient int currentSubTaskId; + + public FlushEventAlignmentOperator() { + // It's necessary to avoid unpredictable outcomes of Event shuffling. + this.chainingStrategy = ChainingStrategy.ALWAYS; + } + + @Override + public void open() throws Exception { + super.open(); + this.totalTasksNumber = getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks(); + this.currentSubTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + sourceTaskIdToAssignBucketSubTaskIds = new HashMap<>(); + } + + @Override + public void processElement(StreamRecord streamRecord) { + BucketWrapper wrapper = streamRecord.getValue(); + Event event = wrapper.getEvent(); + + if (event instanceof BucketWrapperFlushEvent) { + BucketWrapperFlushEvent bucketWrapperFlushEvent = (BucketWrapperFlushEvent) event; + int sourceSubTaskId = bucketWrapperFlushEvent.getSourceSubTaskId(); + Set subTaskIds = + sourceTaskIdToAssignBucketSubTaskIds.getOrDefault( + sourceSubTaskId, new HashSet<>()); + int subtaskId = bucketWrapperFlushEvent.getBucketAssignTaskId(); + subTaskIds.add(subtaskId); + if (subTaskIds.size() == totalTasksNumber) { + LOG.info("{} send FlushEvent of {}", currentSubTaskId, sourceSubTaskId); + output.collect( + new StreamRecord<>( + new BucketWrapper( + wrapper.getBucket(), + new FlushEvent( + sourceSubTaskId, + bucketWrapperFlushEvent.getTableIds(), + bucketWrapperFlushEvent + .getSchemaChangeEventType())))); + sourceTaskIdToAssignBucketSubTaskIds.remove(sourceSubTaskId); + } else { + LOG.info( + "{} collect FlushEvent of {} with subtask {}", + currentSubTaskId, + sourceSubTaskId, + subtaskId); + sourceTaskIdToAssignBucketSubTaskIds.put(sourceSubTaskId, subTaskIds); + } + } else { + output.collect(streamRecord); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/coordinator/MultiTableStreamWriteOperatorCoordinator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/coordinator/MultiTableStreamWriteOperatorCoordinator.java new file mode 100644 index 00000000000..0bb1e3d7e4c --- /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,913 @@ +/* + * 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.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; +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.ClusteringUtil; +import org.apache.hudi.util.CompactionUtil; +import org.apache.hudi.util.FlinkWriteClients; +import org.apache.hudi.util.StreamerUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.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; + +import static org.apache.hudi.configuration.FlinkOptions.COMPACTION_DELTA_COMMITS; + +/** + * A custom OperatorCoordinator that manages Hudi writes for multiple tables. + * + *

This coordinator extends the default {@link StreamWriteOperatorCoordinator}. The parent class + * is designed for a single destination table, so its core logic (e.g., for commits and + * checkpointing) cannot be reused directly for a multi-table sink. + * + *

Therefore, this implementation overrides the essential lifecycle methods to manage a + * collection of per-table resources. It dynamically creates and manages a dedicated {@link + * HoodieFlinkWriteClient}, {@link EventBuffers}, and timeline for each table that appears in the + * upstream CDC data. + */ +public class MultiTableStreamWriteOperatorCoordinator extends StreamWriteOperatorCoordinator { + + private static final Logger LOG = + LoggerFactory.getLogger(MultiTableStreamWriteOperatorCoordinator.class); + + /** + * 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; + final boolean scheduleCompaction; + final boolean scheduleClustering; + final boolean isDeltaTimeCompaction; + + // Event-driven compaction tracking - tracks actual write activity + long commitsSinceLastCompaction = 0; + // For MOR tables, track log file growth + long totalLogBytesWritten = 0; + + final int commitsThreshold; + + TableState(Configuration conf) { + this.operationType = + WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION)); + this.commitAction = + CommitUtils.getCommitActionType( + this.operationType, + HoodieTableType.valueOf( + conf.getString(FlinkOptions.TABLE_TYPE).toUpperCase())); + this.isOverwrite = WriteOperationType.isOverwrite(this.operationType); + this.scheduleCompaction = OptionsResolver.needsScheduleCompaction(conf); + this.scheduleClustering = OptionsResolver.needsScheduleClustering(conf); + this.isDeltaTimeCompaction = OptionsResolver.isDeltaTimeCompaction(conf); + this.commitsThreshold = conf.get(COMPACTION_DELTA_COMMITS); + } + + /** + * Updates compaction metrics based on write statuses. Skips empty commits where no actual + * data was written. + * + * @param writeStatuses The write statuses from the latest commit + * @return true if this commit had actual writes, false if it was empty + */ + boolean updateCompactionMetrics(List writeStatuses) { + if (writeStatuses == null || writeStatuses.isEmpty()) { + LOG.debug("No write statuses - skipping compaction metric update"); + return false; + } + + // Check if any actual writes occurred (skip empty commits) + long totalWrites = + writeStatuses.stream() + .map(WriteStatus::getStat) + .filter(stat -> stat != null) + .mapToLong(HoodieWriteStat::getNumWrites) + .sum(); + + if (totalWrites == 0) { + LOG.debug( + "Empty commit detected (numWrites=0) - skipping compaction metric update"); + return false; + } + + // Track log file bytes written (for MOR tables) + long bytesWritten = + writeStatuses.stream() + .map(WriteStatus::getStat) + .filter(stat -> stat != null) + .mapToLong(HoodieWriteStat::getTotalWriteBytes) + .sum(); + + commitsSinceLastCompaction++; + totalLogBytesWritten += bytesWritten; + + LOG.debug( + "Updated compaction metrics: commits={}, bytes={}", + commitsSinceLastCompaction, + totalLogBytesWritten); + return true; + } + + /** Resets compaction metrics after compaction is scheduled. */ + void resetCompactionMetrics() { + commitsSinceLastCompaction = 0; + totalLogBytesWritten = 0; + } + + /** + * Determines if compaction should be triggered based on write activity. Only triggers for + * MOR tables with actual data writes. + * + * @return true if compaction should be scheduled + */ + boolean shouldTriggerCompaction() { + // Only trigger for MOR tables (DELTA_COMMIT means log files) + if (!commitAction.equals(HoodieTimeline.DELTA_COMMIT_ACTION)) { + return false; + } + + return commitsSinceLastCompaction >= commitsThreshold; + } + } + + /** The base Flink configuration. */ + private final Configuration baseConfig; + + /** + * A single, unified map holding the context for each managed table. The key is the {@link + * TableId}, providing a centralized place for all per-table resources. + */ + private final Map tableContexts = new ConcurrentHashMap<>(); + + /** A reverse lookup map from table path to TableId for efficient event routing. */ + private final Map pathToTableId = new ConcurrentHashMap<>(); + + /** Cache of schemas per table for config creation. */ + private final Map tableSchemas = new ConcurrentHashMap<>(); + + /** + * Gateways for sending events to sub-tasks. This field is necessary because the parent's + * `gateways` array is private and not initialized if we don't call super.start(). + */ + private transient SubtaskGateway[] gateways; + + /** + * A 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; + } + + LOG.info( + "Fetching instant, but got null pair for : {}", + tableContext.eventBuffers.getInstantAndEventBuffer(event.getCheckpointId())); + + // The instant should have been created by handleCoordinationRequest + if (tableContext.eventBuffers.getInstantAndEventBuffer(event.getCheckpointId()) == null) { + LOG.error( + "FATAL: Received WriteMetadataEvent for table {} at checkpoint {} before an instant was created. " + + "This should not happen. The sink function must request an instant before sending data.", + tableId, + event.getCheckpointId()); + context.failJob( + new IllegalStateException( + "Received data for table " + + tableId + + " at checkpoint " + + event.getCheckpointId() + + " without a valid Hudi instant.")); + return; + } + + LOG.debug( + "Buffering event for table: {}, checkpoint: {}", tableId, event.getCheckpointId()); + tableContext.eventBuffers.addEventToBuffer(event); + } + + @Override + public void checkpointCoordinator(long checkpointId, CompletableFuture result) { + executor.execute( + () -> { + try { + Map>> allStates = + new HashMap<>(); + tableContexts.forEach( + (tableId, tableContext) -> { + allStates.put( + tableId, + tableContext.eventBuffers.getAllCompletedEvents()); + }); + + // Create a wrapper that includes both event buffers AND schemas + Map checkpointState = new HashMap<>(); + checkpointState.put("eventBuffers", allStates); + checkpointState.put("schemas", new HashMap<>(tableSchemas)); + + byte[] serializedState = SerializationUtils.serialize(checkpointState); + result.complete(serializedState); + LOG.info( + "Successfully checkpointed coordinator state with {} schemas for checkpoint {}", + tableSchemas.size(), + checkpointId); + } catch (Throwable t) { + LOG.error( + "Failed to checkpoint coordinator state for checkpoint {}", + checkpointId, + t); + result.completeExceptionally(t); + } + }, + "checkpointing coordinator state %d", + checkpointId); + } + + @Override + @SuppressWarnings("unchecked") + public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { + if (checkpointData == null) { + LOG.info("No coordinator checkpoint data to restore for checkpoint {}.", checkpointId); + return; + } + try { + Map checkpointState = SerializationUtils.deserialize(checkpointData); + Map>> allStates = + (Map>>) + checkpointState.get("eventBuffers"); + Map restoredSchemas = + (Map) checkpointState.get("schemas"); + + // Restore schemas + if (restoredSchemas != null && !restoredSchemas.isEmpty()) { + tableSchemas.clear(); + tableSchemas.putAll(restoredSchemas); + LOG.info( + "Restored {} schemas from checkpoint: {}", + tableSchemas.size(), + tableSchemas.keySet()); + } + + allStates.forEach( + (tableId, completedEvents) -> { + // Lazily create table context if it doesn't exist. + // The actual write client is initialized in start(). + tableContexts.computeIfAbsent( + tableId, + tId -> { + Configuration tableConfig = createTableSpecificConfig(tId); + String tablePath = tableConfig.getString(FlinkOptions.PATH); + pathToTableId.put(tablePath, tId); + TableState tableState = new TableState(tableConfig); + EventBuffers eventBuffers = + EventBuffers.getInstance(tableConfig); + return new TableContext( + null, eventBuffers, tableState, tablePath); + }); + TableContext tableContext = tableContexts.get(tableId); + tableContext.eventBuffers.addEventsToBuffer(completedEvents); + }); + LOG.info("Successfully restored coordinator state from checkpoint {}", checkpointId); + } catch (Throwable t) { + LOG.error("Failed to restore coordinator state from checkpoint {}", checkpointId, t); + context.failJob(new RuntimeException("Failed to restore coordinator state", t)); + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + executor.execute( + () -> { + LOG.info( + "Checkpoint {} completed. Committing instants for all managed tables.", + checkpointId); + for (Map.Entry entry : tableContexts.entrySet()) { + TableId tableId = entry.getKey(); + TableContext tableContext = entry.getValue(); + + tableContext + .eventBuffers + .getEventBufferStream() + .filter(e -> e.getKey() < checkpointId) + .forEach( + bufferEntry -> { + long ckpId = bufferEntry.getKey(); + String instant = bufferEntry.getValue().getLeft(); + WriteMetadataEvent[] events = + bufferEntry.getValue().getRight(); + try { + commitInstantForTable( + tableId, + tableContext, + ckpId, + instant, + events); + } catch (Exception e) { + LOG.error( + "Exception while committing instant {} for table {}", + instant, + tableId, + e); + MultiTableStreamWriteOperatorCoordinator.this + .context.failJob(e); + } + }); + } + }, + "committing instants for checkpoint %d", + checkpointId); + } + + private void commitInstantForTable( + TableId tableId, + TableContext tableContext, + long checkpointId, + String instant, + WriteMetadataEvent[] eventBuffer) { + + if (Arrays.stream(eventBuffer).allMatch(Objects::isNull)) { + LOG.info("No events for instant {}, table {}. Resetting buffer.", instant, tableId); + tableContext.eventBuffers.reset(checkpointId); + // Even with no events, we must clean up the inflight instant. + // A simple rollback handles this. + tableContext.writeClient.rollback(instant); + return; + } + + List writeStatuses = + Arrays.stream(eventBuffer) + .filter(Objects::nonNull) + .map(WriteMetadataEvent::getWriteStatuses) + .flatMap(Collection::stream) + .collect(Collectors.toList()); + + if (writeStatuses.isEmpty() && !OptionsResolver.allowCommitOnEmptyBatch(baseConfig)) { + LOG.info( + "No data written for instant {}, table {}. Aborting commit and rolling back.", + instant, + tableId); + tableContext.eventBuffers.reset(checkpointId); + tableContext.writeClient.rollback(instant); + return; + } + + doCommit(tableId, tableContext, checkpointId, instant, writeStatuses); + } + + @SuppressWarnings("unchecked") + private void doCommit( + TableId tableId, + TableContext tableContext, + long checkpointId, + String instant, + List writeStatuses) { + + TableState state = tableContext.tableState; + final Map> partitionToReplacedFileIds = + state.isOverwrite + ? tableContext.writeClient.getPartitionToReplacedFileIds( + state.operationType, writeStatuses) + : Collections.emptyMap(); + + HashMap checkpointCommitMetadata = new HashMap<>(); + StreamerUtil.addFlinkCheckpointIdIntoMetaData( + baseConfig, checkpointCommitMetadata, checkpointId); + + boolean success = + tableContext.writeClient.commit( + instant, + writeStatuses, + Option.of(checkpointCommitMetadata), + state.commitAction, + partitionToReplacedFileIds); + + if (success) { + tableContext.eventBuffers.reset(checkpointId); + LOG.info("Successfully committed instant [{}] for table [{}]", instant, tableId); + + // Update compaction metrics based on actual write activity + boolean hasWrites = tableContext.tableState.updateCompactionMetrics(writeStatuses); + + // Event-driven table services scheduling - only if there were actual writes + if (hasWrites) { + scheduleTableServicesIfNeeded(tableId, tableContext); + } else { + LOG.debug( + "Skipping table services scheduling for table [{}] - empty commit", + tableId); + } + } else { + LOG.error("Failed to commit instant [{}] for table [{}]", instant, tableId); + MultiTableStreamWriteOperatorCoordinator.this.context.failJob( + new HoodieException( + String.format( + "Commit failed for instant %s, table %s", instant, tableId))); + } + } + + /** + * Event-driven table services scheduling. Only schedules compaction/clustering when certain + * thresholds are met based on write metrics. + * + * @param tableId The table identifier + * @param tableContext The table's context containing write client and state + */ + private void scheduleTableServicesIfNeeded(TableId tableId, TableContext tableContext) { + TableState state = tableContext.tableState; + + // Event-driven compaction scheduling + if (state.scheduleCompaction && state.shouldTriggerCompaction()) { + try { + LOG.info( + "Triggering compaction for table [{}] - threshold met: commits={}/{}, bytes={} MB", + tableId, + state.commitsSinceLastCompaction, + state.commitsThreshold, + state.totalLogBytesWritten / (1024 * 1024)); + + CompactionUtil.scheduleCompaction( + tableContext.writeClient, + state.isDeltaTimeCompaction, + true); // committed = true since we just committed + + // Reset metrics after scheduling + state.resetCompactionMetrics(); + + LOG.info("Successfully scheduled compaction for table [{}]", tableId); + } catch (Exception e) { + LOG.error("Failed to schedule compaction for table [{}]", tableId, e); + // Don't fail the job, just log the error + } + } else if (state.scheduleCompaction) { + LOG.debug( + "Compaction not triggered for table [{}] - commits={}/{}, bytes={} MB", + tableId, + state.commitsSinceLastCompaction, + state.commitsThreshold, + state.totalLogBytesWritten / (1024 * 1024)); + } + + // Clustering can remain on every commit or use similar metrics + if (state.scheduleClustering) { + try { + Configuration tableConfig = createTableSpecificConfig(tableId); + ClusteringUtil.scheduleClustering( + tableConfig, + tableContext.writeClient, + true); // committed = true since we just committed + LOG.info("Scheduled clustering for table [{}]", tableId); + } catch (Exception e) { + LOG.error("Failed to schedule clustering for table [{}]", tableId, e); + // Don't fail the job, just log the error + } + } + } + + @Override + public void close() throws Exception { + if (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); + + // 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..80623f1682a --- /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,229 @@ +/* + * 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..71ecb3a020d --- /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,86 @@ +/* + * 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..f96ed5a09cc --- /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,356 @@ +/* + * 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; +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; + +/** 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 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..3d0458112a3 --- /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..da5f816613a --- /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,709 @@ +/* + * 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 new file mode 100644 index 00000000000..83ea7fe9c4d --- /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,688 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink.function; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.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; +import org.apache.flink.cdc.connectors.hudi.sink.event.EnhancedWriteMetadataEvent; +import org.apache.flink.cdc.connectors.hudi.sink.event.HudiRecordEventSerializer; +import org.apache.flink.cdc.connectors.hudi.sink.event.TableAwareCorrespondent; +import org.apache.flink.cdc.connectors.hudi.sink.util.RowDataUtils; +import org.apache.flink.cdc.runtime.operators.sink.SchemaEvolutionClient; +import org.apache.flink.cdc.runtime.serializer.TableIdSerializer; +import org.apache.flink.cdc.runtime.serializer.schema.SchemaSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.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.Field; +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; + + /** Persistent state for schemas to survive checkpoints/savepoints. */ + private transient ListState> schemaState; + + private transient Map tableConfigurations; + + /** Schema evolution client to communicate with SchemaOperator. */ + private transient SchemaEvolutionClient schemaEvolutionClient; + + /** 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; + + // 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"); + } + + /** + * Sets the SchemaEvolutionClient from the operator level since functions don't have direct + * access to TaskOperatorEventGateway. + */ + public void setSchemaEvolutionClient(SchemaEvolutionClient schemaEvolutionClient) { + this.schemaEvolutionClient = schemaEvolutionClient; + LOG.info("SchemaEvolutionClient set for MultiTableEventStreamWriteFunction"); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + this.tableFunctions = new HashMap<>(); + this.initializedTables = new HashMap<>(); + // Don't reinitialize schemaMaps if it already has restored schemas from state + if (this.schemaMaps == null) { + this.schemaMaps = new HashMap<>(); + } + this.tableConfigurations = new HashMap<>(); + } + + @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); + + // NOTE: This abstract method is intentionally empty for multi-table function. + // The actual delegation happens in snapshotState(FunctionSnapshotContext) + // to ensure child functions receive the correct checkpointId. + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + // Persist schemas to state for recovery + if (schemaState != null && schemaMaps != null) { + schemaState.clear(); + for (Map.Entry entry : schemaMaps.entrySet()) { + schemaState.add(new Tuple2<>(entry.getKey(), entry.getValue())); + LOG.debug("Persisted schema for table: {}", entry.getKey()); + } + LOG.info("Persisted {} schemas to state", schemaMaps.size()); + } + + // 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 { + 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) { + 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..63aad99f8c0 --- /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,99 @@ +/* + * 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; +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..2cff14ae143 --- /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,162 @@ +/* + * 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..93973bdf0c1 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/operator/MultiTableWriteOperatorFactory.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink.operator; + +import org.apache.flink.cdc.connectors.hudi.sink.coordinator.MultiTableStreamWriteOperatorCoordinator; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEventDispatcher; +import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.SimpleUdfStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.table.data.RowData; + +import org.apache.hudi.sink.common.AbstractWriteOperator; +import org.apache.hudi.sink.event.Correspondent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Custom WriteOperatorFactory that creates our extended multi-table StreamWriteOperatorCoordinator + * instead of Hudi's original single-table coordinator. + * + *

This factory ensures that multi-table CDC scenarios use the proper coordinator with: + * + *

    + *
  • Per-table client management + *
  • Per-table instant tracking + *
  • Dynamic table registration + *
  • Table-aware event routing + *
+ * + * @param The input type for the write operator + */ +public class MultiTableWriteOperatorFactory extends SimpleUdfStreamOperatorFactory + implements CoordinatedOperatorFactory, OneInputStreamOperatorFactory { + + private static final Logger LOG = LoggerFactory.getLogger(MultiTableWriteOperatorFactory.class); + private static final long serialVersionUID = 1L; + + private final Configuration conf; + private final AbstractWriteOperator writeOperator; + + public MultiTableWriteOperatorFactory( + Configuration conf, AbstractWriteOperator writeOperator) { + super(writeOperator); + this.conf = conf; + this.writeOperator = writeOperator; + } + + public static MultiTableWriteOperatorFactory instance( + Configuration conf, AbstractWriteOperator writeOperator) { + return new MultiTableWriteOperatorFactory<>(conf, writeOperator); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + LOG.info("Creating multi-table write operator with extended coordinator support"); + + // necessary setting for the operator. + super.createStreamOperator(parameters); + + final OperatorID operatorID = parameters.getStreamConfig().getOperatorID(); + final OperatorEventDispatcher eventDispatcher = parameters.getOperatorEventDispatcher(); + + this.writeOperator.setCorrespondent( + Correspondent.getInstance( + operatorID, + parameters + .getContainingTask() + .getEnvironment() + .getOperatorCoordinatorEventGateway())); + this.writeOperator.setOperatorEventGateway( + eventDispatcher.getOperatorEventGateway(operatorID)); + eventDispatcher.registerEventHandler(operatorID, writeOperator); + return (T) writeOperator; + } + + @Override + public OperatorCoordinator.Provider getCoordinatorProvider( + String operatorName, OperatorID operatorID) { + LOG.info( + "Creating multi-table StreamWriteOperatorCoordinator provider for operator: {}", + operatorName); + return new MultiTableStreamWriteOperatorCoordinator.Provider(operatorID, conf); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/util/RowDataUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/util/RowDataUtils.java new file mode 100644 index 00000000000..9c501f2c7fc --- /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,505 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink.util; + +import org.apache.flink.cdc.common.data.DecimalData; +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.data.RecordData.FieldGetter; +import org.apache.flink.cdc.common.event.ChangeEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.OperationType; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypeChecks; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.types.RowKind; + +import org.apache.hudi.client.model.HoodieFlinkInternalRow; + +import 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..753e3af9592 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/v2/HudiSink.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink.v2; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.WriterInitContext; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.connectors.hudi.sink.bucket.BucketAssignOperator; +import org.apache.flink.cdc.connectors.hudi.sink.bucket.BucketWrapper; +import org.apache.flink.cdc.connectors.hudi.sink.bucket.FlushEventAlignmentOperator; +import org.apache.flink.cdc.connectors.hudi.sink.operator.MultiTableWriteOperator; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Collector; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.ZoneId; + +/** A {@link Sink} implementation for Apache Hudi. */ +public class HudiSink implements Sink, WithPreWriteTopology { + + private static final Logger LOG = LoggerFactory.getLogger(HudiSink.class); + + private final Configuration conf; + + private final String schemaOperatorUid; + + public HudiSink(Configuration conf, String schemaOperatorUid, ZoneId zoneId) { + LOG.info("Creating Hoodie sink with conf: {}", conf); + this.conf = conf; + this.schemaOperatorUid = schemaOperatorUid; + } + + @Override + public SinkWriter createWriter(InitContext context) throws IOException { + return DummySinkWriter.INSTANCE; + } + + @Override + public SinkWriter createWriter(WriterInitContext context) throws IOException { + return DummySinkWriter.INSTANCE; + } + + @Override + public DataStream addPreWriteTopology(DataStream dataStream) { + LOG.info("Building Hudi pre-write topology with bucket assignment and partitioning"); + + // Step 1: Bucket assignment operator + // - Calculates bucket for DataChangeEvents + // - Broadcasts schema events to all tasks + // - Wraps events in BucketWrapper for downstream partitioning + DataStream bucketAssignedStream = + dataStream + .transform( + "bucket_assign", + TypeInformation.of(BucketWrapper.class), + new BucketAssignOperator(conf, schemaOperatorUid)) + .uid("bucket_assign"); + + // Step 2: Partition by bucket index + // - Routes events to tasks based on bucket index + // - Schema events are broadcast (sent to all bucket indices) + // - Data events go to their specific bucket's task + DataStream partitionedStream = + bucketAssignedStream.partitionCustom( + (key, numPartitions) -> key % numPartitions, + (KeySelector) BucketWrapper::getBucket); + + // Step 3: Flush event alignment + // - Aligns FlushEvents from multiple BucketAssignOperator instances + // - Ensures each writer receives only one FlushEvent per source + DataStream alignedStream = + partitionedStream + .transform( + "flush_event_alignment", + TypeInformation.of(BucketWrapper.class), + new FlushEventAlignmentOperator()) + .uid("flush_event_alignment"); + + // Step 4: Unwrap and write to Hudi + // Use map to unwrap BucketWrapper before passing to MultiTableWriteOperator + DataStream unwrappedStream = + alignedStream.map(wrapper -> wrapper.getEvent(), TypeInformation.of(Event.class)); + + return unwrappedStream + .transform( + "multi_table_write", + TypeInformation.of(RowData.class), + MultiTableWriteOperator.getFactory(conf, schemaOperatorUid)) + .uid("multi_table_write") + .flatMap( + (RowData rowData, Collector out) -> { + // Write side effects are handled by the operator, no events emitted + // downstream + }) + .returns(TypeInformation.of(Event.class)); + } + + /** Dummy sink writer that does nothing. */ + private static class DummySinkWriter implements SinkWriter { + private static final SinkWriter INSTANCE = new DummySinkWriter(); + + @Override + public void write(Event element, Context context) { + // do nothing + } + + @Override + public void flush(boolean endOfInput) { + // do nothing + } + + @Override + public void close() { + // do nothing + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/v2/OperatorIDGenerator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/v2/OperatorIDGenerator.java new file mode 100644 index 00000000000..c00aa751608 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/v2/OperatorIDGenerator.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.connectors.hudi.sink.v2; + +import org.apache.flink.runtime.jobgraph.OperatorID; + +import org.apache.flink.shaded.guava31.com.google.common.hash.Hashing; + +import static java.nio.charset.StandardCharsets.UTF_8; + +/** Generator for creating deterministic OperatorIDs from UIDs. */ +public class OperatorIDGenerator { + + private final String transformationUid; + + public OperatorIDGenerator(String transformationUid) { + this.transformationUid = transformationUid; + } + + public OperatorID generate() { + byte[] hash = + Hashing.murmur3_128(0) + .newHasher() + .putString(transformationUid, UTF_8) + .hash() + .asBytes(); + return new OperatorID(hash); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory new file mode 100644 index 00000000000..c0b308d0b68 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.flink.cdc.connectors.hudi.sink.HudiDataSinkFactory diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml index 73869eec91a..725db6dbe2d 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml @@ -40,6 +40,7 @@ limitations under the License. flink-cdc-pipeline-connector-maxcompute flink-cdc-pipeline-connector-iceberg flink-cdc-pipeline-connector-fluss + flink-cdc-pipeline-connector-hudi diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml index 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..738afa12f3e --- /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,906 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.pipeline.tests; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.cdc.common.test.utils.TestUtils; +import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; +import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; +import org.apache.flink.cdc.pipeline.tests.utils.TarballFetcher; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.core.execution.SavepointFormatType; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.table.api.ValidationException; + +import org.apache.hudi.common.model.HoodieTableType; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.BindMode; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.FixedHostPortGenericContainer; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.ToStringConsumer; +import org.testcontainers.images.builder.Transferable; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.MountableFile; + +import java.nio.file.Path; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** End-to-end tests for mysql cdc to Iceberg pipeline job. */ +public class MySqlToHudiE2eITCase extends PipelineTestEnvironment { + + private static final Logger LOG = LoggerFactory.getLogger(MySqlToHudiE2eITCase.class); + + private static final Duration HUDI_TESTCASE_TIMEOUT = Duration.ofMinutes(20); + + private static final String FLINK_LIB_DIR = "/opt/flink/lib"; + + private static final String PEEK_SQL_FILE = "peek-hudi.sql"; + + private static final String TABLE_TYPE = HoodieTableType.MERGE_ON_READ.name(); + + // Custom Flink properties for Hudi tests with increased metaspace and heap for heavy + // dependencies + private static final String HUDI_FLINK_PROPERTIES = + FLINK_PROPERTIES + + "\n" + + "taskmanager.memory.jvm-metaspace.size: 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); + + private String warehouse; + + private final boolean debug = true; + + @BeforeAll + public static void initializeContainers() { + LOG.info("Starting containers..."); + Startables.deepStart(Stream.of(MYSQL)).join(); + LOG.info("Containers are started."); + } + + /* + * The Flink SQL Client requires certain core dependencies, like Hadoop's FileSystem, + * on its main classpath (`/lib`) to be discovered correctly by the ServiceLoader. + * Adding them as temporary session JARs via the `--jar` flag is unreliable for these + * low-level services. + *

+ * By copying these dependencies directly into the container's `/opt/flink/lib` + * directory, we ensure they are loaded by Flink's main classloader, which + * permanently resolves the `No FileSystem for scheme: file` error during validation. + */ + @BeforeEach + @Override + public void before() throws Exception { + LOG.info("Starting containers..."); + + // 2. Instantiate the correct class and apply class-specific methods + if (debug) { + // Use FixedHost instead of GenericContainer to ensure that ports are fixed for easier + // debugging during dev + jobManager = + new FixedHostPortGenericContainer<>(getFlinkDockerImageTag()) + .withFixedExposedPort(8081, JOB_MANAGER_REST_PORT) + .withFixedExposedPort(9005, 9005) + .withEnv( + "FLINK_ENV_JAVA_OPTS", + "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:9005"); + taskManager = + new FixedHostPortGenericContainer<>(getFlinkDockerImageTag()) + .withFixedExposedPort(9006, 9006); + // .withEnv( + // "FLINK_ENV_JAVA_OPTS", + // + // "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:9006"); + } else { + jobManager = + new GenericContainer<>(getFlinkDockerImageTag()) + // Expose ports for random mapping by Docker + .withExposedPorts(JOB_MANAGER_REST_PORT); + taskManager = new FixedHostPortGenericContainer<>(getFlinkDockerImageTag()); + } + + jobManagerConsumer = new ToStringConsumer(); + jobManager + .withCommand("jobmanager") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_JM_ALIAS) + .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) + .withCreateContainerCmdModifier(cmd -> cmd.withVolumes(sharedVolume)) + .withLogConsumer(jobManagerConsumer); + Startables.deepStart(Stream.of(jobManager)).join(); + runInContainerAsRoot(jobManager, "chmod", "0777", "-R", sharedVolume.toString()); + + taskManagerConsumer = new ToStringConsumer(); + taskManager + .withCommand("taskmanager") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_TM_ALIAS) + .withEnv("FLINK_PROPERTIES", HUDI_FLINK_PROPERTIES) + .dependsOn(jobManager) + .withVolumesFrom(jobManager, BindMode.READ_WRITE) + .withLogConsumer(taskManagerConsumer); + Startables.deepStart(Stream.of(taskManager)).join(); + runInContainerAsRoot(taskManager, "chmod", "0777", "-R", sharedVolume.toString()); + + TarballFetcher.fetchLatest(jobManager); + LOG.info("CDC executables deployed."); + + inventoryDatabase.createAndInitialize(); + + jobManager.copyFileToContainer( + MountableFile.forHostPath(TestUtils.getResource(getHudiSQLConnectorResourceName())), + FLINK_LIB_DIR + "/" + getHudiSQLConnectorResourceName()); + jobManager.copyFileToContainer( + MountableFile.forHostPath(TestUtils.getResource("flink-shade-hadoop.jar")), + FLINK_LIB_DIR + "/flink-shade-hadoop.jar"); + jobManager.copyFileToContainer( + MountableFile.forHostPath(TestUtils.getResource("hudi-hadoop-common.jar")), + FLINK_LIB_DIR + "/hudi-hadoop-common.jar"); + jobManager.copyFileToContainer( + MountableFile.forHostPath(TestUtils.getResource("flink-hadoop-compatibility.jar")), + FLINK_LIB_DIR + "/flink-hadoop-compatibility.jar"); + jobManager.copyFileToContainer( + MountableFile.forHostPath(TestUtils.getResource("flink-parquet.jar")), + FLINK_LIB_DIR + "/flink-parquet.jar"); + } + + @AfterEach + public void after() { + try { + super.after(); + inventoryDatabase.dropDatabase(); + } catch (Exception e) { + LOG.error("Failed to clean up resources", e); + } + } + + @Test + public void testSyncWholeDatabase() throws Exception { + warehouse = sharedVolume.toString() + "/hudi_warehouse_" + UUID.randomUUID(); + String database = inventoryDatabase.getDatabaseName(); + + LOG.info("Preparing Hudi warehouse directory: {}", warehouse); + runInContainerAsRoot(jobManager, "mkdir", "-p", warehouse); + runInContainerAsRoot(jobManager, "chmod", "-R", "0777", warehouse); + + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: mysql\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.\\.*\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: hudi\n" + + " path: %s\n" + + " hoodie.datasource.write.recordkey.field: id\n" + + " hoodie.table.type: " + + TABLE_TYPE + + " \n" + + " hoodie.schema.on.read.enable: true\n" + + " write.bucket_assign.tasks: 2\n" + + " write.tasks: 2\n" + + " compaction.delta_commits: 2\n" + + "\n" + + "pipeline:\n" + + " schema.change.behavior: evolve\n" + + " parallelism: %s", + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + database, + warehouse, + parallelism); + Path hudiCdcConnector = TestUtils.getResource("hudi-cdc-pipeline-connector.jar"); + // Path hudiHadoopCommonJar = TestUtils.getResource("hudi-hadoop-common.jar"); + Path hadoopJar = TestUtils.getResource("flink-shade-hadoop.jar"); + Path hadoopCompatibilityJar = TestUtils.getResource("flink-hadoop-compatibility.jar"); + Path dropMetricsJar = TestUtils.getResource("flink-metrics-dropwizard.jar"); + Path flinkParquet = TestUtils.getResource("flink-parquet.jar"); + JobID pipelineJobID = + submitPipelineJob( + pipelineJob, + hudiCdcConnector, + hadoopJar, + hadoopCompatibilityJar, + dropMetricsJar, + flinkParquet); + waitUntilJobRunning(pipelineJobID, Duration.ofSeconds(60)); + LOG.info("Pipeline job is running"); + + // Validate that source records from RDB have been initialized properly and landed in sink + validateSinkResult(warehouse, database, "products", getProductsExpectedSinkResults()); + validateSinkResult(warehouse, database, "customers", getCustomersExpectedSinkResults()); + + // Generate binlogs + LOG.info("Begin incremental reading stage."); + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), MYSQL.getDatabasePort(), database); + List recordsInIncrementalPhase; + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stat = conn.createStatement()) { + + stat.execute( + "INSERT INTO products VALUES (default,'Ten','Jukebox',0.2, null, null, null);"); // 110 + stat.execute("UPDATE products SET description='Fay' WHERE id=106;"); + stat.execute("UPDATE products SET weight='5.125' WHERE id=107;"); + + // modify table schema + stat.execute("ALTER TABLE products DROP COLUMN point_c;"); + stat.execute("DELETE FROM products WHERE id=101;"); + + stat.execute( + "INSERT INTO products VALUES (default,'Eleven','Kryo',5.18, null, null);"); // 111 + stat.execute( + "INSERT INTO products VALUES (default,'Twelve', 'Lily', 2.14, null, null);"); // 112 + + validateSinkResult( + warehouse, database, "products", getProductsExpectedAfterDropSinkResults()); + + recordsInIncrementalPhase = createChangesAndValidate(stat); + } catch (SQLException e) { + LOG.error("Update table for CDC failed.", e); + throw e; + } + + // Build expected results + List recordsInSnapshotPhase = getProductsExpectedAfterAddModSinkResults(); + recordsInSnapshotPhase.addAll(recordsInIncrementalPhase); + + validateSinkResult(warehouse, database, "products", recordsInSnapshotPhase); + + // Verify that compaction was scheduled for at least one table (only for MOR tables) + LOG.info("Verifying compaction scheduling for MOR tables..."); + if (TABLE_TYPE.equals(HoodieTableType.MERGE_ON_READ.name())) { + assertCompactionScheduled(warehouse, database, Arrays.asList("products", "customers")); + } + } + + /** + * Executes a series of DDL (Data Definition Language) and DML (Data Manipulation Language) + * operations on the {@code products} table to simulate schema evolution and data loading. + * + *

The method performs two primary phases: + * + *

    + *
  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 Statement} object used to execute the SQL commands. + * @return A {@link List} of strings, where each string is a CSV representation of an inserted + * row, reflecting the expected state in the database. + * @throws SQLException if a database access error occurs or the executed SQL is invalid. + */ + private List createChangesAndValidate(Statement stat) throws SQLException { + List result = new ArrayList<>(); + StringBuilder sqlFields = new StringBuilder(); + + // Auto-increment id will start from this + int currentId = 113; + final int statementBatchCount = 1000; + + // Step 1 - Add Column: Add 10 columns with VARCHAR(10) sequentially + for (int addColumnRepeat = 0; addColumnRepeat < 10; addColumnRepeat++) { + String addColAlterTableCmd = + String.format( + "ALTER TABLE products ADD COLUMN point_c_%s VARCHAR(10);", + addColumnRepeat); + stat.execute(addColAlterTableCmd); + LOG.info("Executed: {}", addColAlterTableCmd); + sqlFields.append(", '1'"); + StringBuilder resultFields = new StringBuilder(); + for (int addedFieldCount = 0; addedFieldCount < 10; addedFieldCount++) { + if (addedFieldCount <= addColumnRepeat) { + resultFields.append(", 1"); + } else { + resultFields.append(", null"); + } + } + + for (int statementCount = 0; statementCount < statementBatchCount; statementCount++) { + stat.addBatch( + String.format( + "INSERT INTO products VALUES (default,'finally', null, 2.14, null, null %s);", + sqlFields)); + result.add( + String.format( + "%s, finally, null, 2.14, null, null%s", currentId, resultFields)); + currentId++; + } + stat.executeBatch(); + } + + // Step 2 - Modify type for the columns added in Step 1, increasing the VARCHAR length + for (int modifyColumnRepeat = 0; modifyColumnRepeat < 10; modifyColumnRepeat++) { + // Perform 1000 inserts as a batch, continuing the ID sequence from Step 1 + for (int statementCount = 0; statementCount < statementBatchCount; statementCount++) { + stat.addBatch( + String.format( + "INSERT INTO products VALUES (default,'finally', null, 2.14, null, null %s);", + sqlFields)); + + result.add( + String.format( + "%s, finally, null, 2.14, null, null%s", + currentId, ", 1, 1, 1, 1, 1, 1, 1, 1, 1, 1")); + // Continue incrementing the counter for each insert + currentId++; + } + stat.executeBatch(); + + String modifyColTypeAlterCmd = + String.format( + "ALTER TABLE products MODIFY point_c_0 VARCHAR(%s);", + 10 + modifyColumnRepeat); + stat.execute(modifyColTypeAlterCmd); + LOG.info("Executed: {}", modifyColTypeAlterCmd); + } + + return result; + } + + private List fetchHudiTableRows(String warehouse, String databaseName, String tableName) + throws Exception { + String template = + readLines("docker/" + PEEK_SQL_FILE).stream() + .filter(line -> !line.startsWith("--")) + .collect(Collectors.joining("\n")); + String sql = String.format(template, warehouse, databaseName, tableName); + String containerSqlPath = sharedVolume.toString() + "/" + PEEK_SQL_FILE; + jobManager.copyFileToContainer(Transferable.of(sql), containerSqlPath); + LOG.info("Executing SQL client in container with Hudi connector and Hadoop JARs"); + + // Pass in empty FLINK_ENV_JAVA_OPTS so that we do not launch a new JVM (for SQL + // submission/parsing) inheriting environment variables which will cause it to bind to the + // same debug port, causing the port already in use error + String[] commandToExecute = { + "bash", + "-c", + "FLINK_ENV_JAVA_OPTS='' /opt/flink/bin/sql-client.sh" + + " --jar " + + FLINK_LIB_DIR + + "/" + + getHudiSQLConnectorResourceName() + + " --jar " + + FLINK_LIB_DIR + + "/flink-shade-hadoop.jar" + + " -f " + + containerSqlPath + }; + LOG.debug("Executing command: {}", String.join(" ", commandToExecute)); + Container.ExecResult result = jobManager.execInContainer(commandToExecute); + + LOG.debug("SQL client execution completed with exit code: {}", result.getExitCode()); + LOG.debug("SQL client stdout: {}", result.getStdout()); + LOG.debug("SQL client stderr: {}", result.getStderr()); + + if (result.getExitCode() != 0) { + LOG.error("SQL client execution failed!"); + LOG.error("Exit code: {}", result.getExitCode()); + LOG.error("Stdout: {}", result.getStdout()); + LOG.error("Stderr: {}", result.getStderr()); + throw new RuntimeException( + "Failed to execute Hudi peek script. Exit code: " + + result.getExitCode() + + ". Stdout: " + + result.getStdout() + + "; Stderr: " + + result.getStderr()); + } + + return Arrays.stream(result.getStdout().split("\n")) + .filter(line -> line.startsWith("|")) + .skip(1) + .map(MySqlToHudiE2eITCase::extractRow) + .map(row -> String.format("%s", String.join(", ", row))) + .collect(Collectors.toList()); + } + + private static String[] extractRow(String row) { + return Arrays.stream(row.split("\\|")) + .map(String::trim) + .filter(col -> !col.isEmpty()) + .map(col -> col.equals("") ? "null" : col) + .toArray(String[]::new); + } + + protected String getHudiSQLConnectorResourceName() { + return "hudi-sql-connector.jar"; + } + + @Override + public String stopJobWithSavepoint(JobID jobID) { + String savepointPath = "/opt/flink/"; + try { + // Use REST API to stop with savepoint to avoid CLI classpath conflicts + // (Hadoop/Hudi JARs in FLINK_LIB_DIR conflict with Flink's commons-cli) + LOG.info("Stopping job {} with savepoint to {}", jobID, savepointPath); + + String savepointLocation = + getRestClusterClient() + .stopWithSavepoint( + jobID, false, savepointPath, SavepointFormatType.CANONICAL) + .get(60, java.util.concurrent.TimeUnit.SECONDS); + + LOG.info("Savepoint completed at: {}", savepointLocation); + return savepointLocation; + } catch (Exception e) { + throw new RuntimeException("Failed to stop job with savepoint", e); + } + } + + private void validateSinkResult( + String warehouse, String database, String table, List expected) + throws InterruptedException { + LOG.info("Verifying Hudi {}::{}::{} results...", warehouse, database, table); + long deadline = System.currentTimeMillis() + HUDI_TESTCASE_TIMEOUT.toMillis(); + List results = Collections.emptyList(); + while (System.currentTimeMillis() < deadline) { + try { + results = fetchHudiTableRows(warehouse, database, table); + Assertions.assertThat(results).containsExactlyInAnyOrderElementsOf(expected); + LOG.info( + "Successfully verified {} records in {} seconds for {}::{}.", + expected.size(), + (System.currentTimeMillis() - deadline + HUDI_TESTCASE_TIMEOUT.toMillis()) + / 1000, + database, + table); + return; + } catch (Exception e) { + LOG.warn("Validate failed, waiting for the next loop...", e); + } catch (AssertionError ignored) { + // AssertionError contains way too much records and might flood the log output. + if (expected.size() == results.size()) { + // Size of rows match up, print the contents + final int rowsToPrint = 100; + LOG.warn( + "Result expected: {}, but got {}", + expected.stream() + .sorted() + .limit(rowsToPrint) + .collect(Collectors.toList()), + results.stream() + .sorted() + .limit(rowsToPrint) + .collect(Collectors.toList())); + } else { + LOG.warn( + "Results mismatch, expected {} records, but got {} actually. Waiting for the next loop...", + expected.size(), + results.size()); + } + } + + Thread.sleep(10000L); + } + Assertions.assertThat(results).containsExactlyInAnyOrderElementsOf(expected); + } + + @Test + public void testStopAndRestartFromSavepoint() throws Exception { + warehouse = sharedVolume.toString() + "/hudi_warehouse_savepoint_" + UUID.randomUUID(); + String database = inventoryDatabase.getDatabaseName(); + + LOG.info("Preparing Hudi warehouse directory: {}", warehouse); + runInContainerAsRoot(jobManager, "mkdir", "-p", warehouse); + runInContainerAsRoot(jobManager, "chmod", "-R", "0777", warehouse); + + // Configure pipeline with checkpointing + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: mysql\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.\\.*\n" + + " server-id: 5600-5604\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: hudi\n" + + " path: %s\n" + + " hoodie.datasource.write.recordkey.field: id\n" + + " hoodie.table.type: MERGE_ON_READ\n" + + " hoodie.schema.on.read.enable: true\n" + + " write.bucket_assign.tasks: 2\n" + + " write.tasks: 2\n" + + "\n" + + "pipeline:\n" + + " schema.change.behavior: evolve\n" + + " parallelism: %s\n" + + "\n" + + "# 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"); + JobID pipelineJobID1 = + submitPipelineJob( + pipelineJob, + hudiCdcConnector, + hudiHadoopCommonJar, + hadoopJar, + hadoopCompatibilityJar, + dropMetricsJar, + flinkParquet); + waitUntilJobRunning(pipelineJobID1, Duration.ofSeconds(60)); + + // Store the jobID of the submitted job, we will need it for stopping the job later + Collection jobs = + getRestClusterClient().listJobs().get(10, TimeUnit.SECONDS); + Assertions.assertThat(jobs).hasSize(1); + JobStatusMessage pipelineJobMessage = jobs.iterator().next(); + LOG.info( + "Pipeline job: ID={}, Name={}, Status={}", + pipelineJobMessage.getJobId(), + pipelineJobMessage.getJobName(), + pipelineJobMessage.getJobState()); + + // Validate initial snapshot data for both tables + validateSinkResult(warehouse, database, "products", getProductsExpectedSinkResults()); + validateSinkResult(warehouse, database, "customers", getCustomersExpectedSinkResults()); + LOG.info("Phase 1: Initial snapshot validated successfully"); + + // Phase 2: Insert incremental data before stopping + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), MYSQL.getDatabasePort(), database); + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stat = conn.createStatement()) { + + stat.execute( + "INSERT INTO products VALUES (default,'Pre-Stop Product','Description',1.23, null, null, null);"); + stat.execute( + "INSERT INTO customers VALUES (105, 'user_pre_stop', 'Beijing', '987654321');"); + LOG.info("Phase 2: Incremental data inserted before stop"); + + // Wait for data to be checkpointed + Thread.sleep(5000); + } + + // Validate data before stopping + List expectedProductsBeforeStop = new ArrayList<>(getProductsExpectedSinkResults()); + expectedProductsBeforeStop.add( + "110, Pre-Stop Product, Description, 1.23, null, null, null"); + + List expectedCustomersBeforeStop = + new ArrayList<>(getCustomersExpectedSinkResults()); + expectedCustomersBeforeStop.add("105, user_pre_stop, Beijing, 987654321"); + + validateSinkResult(warehouse, database, "products", expectedProductsBeforeStop); + validateSinkResult(warehouse, database, "customers", expectedCustomersBeforeStop); + LOG.info("Phase 2: Data validated before stop"); + + // Phase 3: Stop job with savepoint + LOG.info("Phase 3: Stopping job with savepoint"); + Collection runningJobs = + getRestClusterClient().listJobs().get(10, TimeUnit.SECONDS).stream() + .filter(j -> j.getJobState().equals(JobStatus.RUNNING)) + .collect(Collectors.toList()); + + if (runningJobs.isEmpty()) { + throw new RuntimeException("No running jobs found!"); + } + + String savepointPath = stopJobWithSavepoint(pipelineJobMessage.getJobId()); + LOG.info("Job stopped with savepoint at: {}", savepointPath); + + // Phase 4: Restart from savepoint + LOG.info("Phase 4: Restarting job from savepoint"); + JobID pipelineJobID2 = + submitPipelineJob( + pipelineJob, + savepointPath, + false, + hudiCdcConnector, + hudiHadoopCommonJar, + hadoopJar, + hadoopCompatibilityJar, + dropMetricsJar, + flinkParquet); + waitUntilJobRunning(pipelineJobID2, Duration.ofSeconds(60)); + LOG.info("Job restarted from savepoint"); + + // Wait for Hudi to stabilize after restart + Thread.sleep(5000); + + // Validate data after restart - should be the same as before stop + validateSinkResult(warehouse, database, "products", expectedProductsBeforeStop); + validateSinkResult(warehouse, database, "customers", expectedCustomersBeforeStop); + LOG.info("Phase 4: Data consistency validated after restart from savepoint"); + + // Phase 5: Continue with post-restart data to ensure pipeline still works + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stat = conn.createStatement()) { + + stat.execute( + "INSERT INTO products VALUES (default,'Post-Restart Product','New Description',4.56, null, null, null);"); + stat.execute("UPDATE products SET description='Updated Description' WHERE id=110;"); + stat.execute( + "INSERT INTO customers VALUES (106, 'user_post_restart', 'Guangzhou', '111222333');"); + stat.execute("DELETE FROM customers WHERE id=101;"); + LOG.info("Phase 5: Post-restart data changes applied"); + } + + // Phase 6: Final validation + List expectedProductsFinal = new ArrayList<>(expectedProductsBeforeStop); + // Update the pre-stop product description + expectedProductsFinal.removeIf(row -> row.startsWith("110,")); + expectedProductsFinal.add( + "110, Pre-Stop Product, Updated Description, 1.23, null, null, null"); + expectedProductsFinal.add( + "111, Post-Restart Product, New Description, 4.56, null, null, null"); + + List expectedCustomersFinal = new ArrayList<>(expectedCustomersBeforeStop); + // Remove deleted customer + expectedCustomersFinal.removeIf(row -> row.startsWith("101,")); + expectedCustomersFinal.add("106, user_post_restart, Guangzhou, 111222333"); + + validateSinkResult(warehouse, database, "products", expectedProductsFinal); + validateSinkResult(warehouse, database, "customers", expectedCustomersFinal); + LOG.info( + "Phase 6: Final validation successful - stop/restart with savepoint working correctly for multiple tables"); + } + + private static List getProductsExpectedSinkResults() { + return Arrays.asList( + "101, One, Alice, 3.202, red, {\"key1\": \"value1\"}, null", + "102, Two, Bob, 1.703, white, {\"key2\": \"value2\"}, null", + "103, Three, Cecily, 4.105, red, {\"key3\": \"value3\"}, null", + "104, Four, Derrida, 1.857, white, {\"key4\": \"value4\"}, null", + "105, Five, Evelyn, 5.211, red, {\"K\": \"V\", \"k\": \"v\"}, null", + "106, Six, Ferris, 9.813, null, null, null", + "107, Seven, Grace, 2.117, null, null, null", + "108, Eight, Hesse, 6.819, null, null, null", + "109, Nine, IINA, 5.223, null, null, null"); + } + + private static List getProductsExpectedAfterDropSinkResults() { + return Arrays.asList( + "102, Two, Bob, 1.703, white, {\"key2\": \"value2\"}", + "103, Three, Cecily, 4.105, red, {\"key3\": \"value3\"}", + "104, Four, Derrida, 1.857, white, {\"key4\": \"value4\"}", + "105, Five, Evelyn, 5.211, red, {\"K\": \"V\", \"k\": \"v\"}", + "106, Six, Fay, 9.813, null, null", + "107, Seven, Grace, 5.125, null, null", + "108, Eight, Hesse, 6.819, null, null", + "109, Nine, IINA, 5.223, null, null", + "110, Ten, Jukebox, 0.2, null, null", + "111, Eleven, Kryo, 5.18, null, null", + "112, Twelve, Lily, 2.14, null, null"); + } + + private static List getProductsExpectedAfterAddModSinkResults() { + // We need this list to be mutable, i.e. not fixed sized + // Arrays.asList returns a fixed size list which is not mutable + return new ArrayList<>( + Arrays.asList( + "102, Two, Bob, 1.703, white, {\"key2\": \"value2\"}, null, null, null, null, null, null, null, null, null, null", + "103, Three, Cecily, 4.105, red, {\"key3\": \"value3\"}, null, null, null, null, null, null, null, null, null, null", + "104, Four, Derrida, 1.857, white, {\"key4\": \"value4\"}, null, null, null, null, null, null, null, null, null, null", + "105, Five, Evelyn, 5.211, red, {\"K\": \"V\", \"k\": \"v\"}, null, null, null, null, null, null, null, null, null, null", + "106, Six, Fay, 9.813, null, null, null, null, null, null, null, null, null, null, null, null", + "107, Seven, Grace, 5.125, null, null, null, null, null, null, null, null, null, null, null, null", + "108, Eight, Hesse, 6.819, null, null, null, null, null, null, null, null, null, null, null, null", + "109, Nine, IINA, 5.223, null, null, null, null, null, null, null, null, null, null, null, null", + "110, Ten, Jukebox, 0.2, null, null, null, null, null, null, null, null, null, null, null, null", + "111, Eleven, Kryo, 5.18, null, null, null, null, null, null, null, null, null, null, null, null", + "112, Twelve, Lily, 2.14, null, null, null, null, null, null, null, null, null, null, null, null")); + } + + private static List getCustomersExpectedSinkResults() { + return Arrays.asList( + "101, user_1, Shanghai, 123567891234", + "102, user_2, Shanghai, 123567891234", + "103, user_3, Shanghai, 123567891234", + "104, user_4, Shanghai, 123567891234"); + } + + public void waitUntilJobRunning(JobID jobId, Duration timeout) { + waitUntilJobState(jobId, timeout, JobStatus.RUNNING); + } + + public void waitUntilJobFinished(JobID jobId, Duration timeout) { + waitUntilJobState(jobId, timeout, JobStatus.FINISHED); + } + + public void waitUntilJobState(JobID jobId, Duration timeout, JobStatus expectedStatus) { + RestClusterClient clusterClient = getRestClusterClient(); + Deadline deadline = Deadline.fromNow(timeout); + while (deadline.hasTimeLeft()) { + Collection jobStatusMessages; + try { + jobStatusMessages = clusterClient.listJobs().get(10, TimeUnit.SECONDS); + } catch (Exception e) { + LOG.warn("Error when fetching job status.", e); + continue; + } + + if (jobStatusMessages == null || jobStatusMessages.isEmpty()) { + continue; + } + + Optional optMessage = + jobStatusMessages.stream().filter(j -> j.getJobId().equals(jobId)).findFirst(); + + if (optMessage.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; + } + } + } + + /** + * Asserts that compaction was scheduled for the given tables by checking for + * .compaction.requested files in the Hudi timeline directory inside the container. + * + *

Should only be invoked for MERGE_ON_READ tables. + * + * @param warehouse The warehouse directory path + * @param database The database name + * @param tables List of table names to check + */ + private void assertCompactionScheduled(String warehouse, String database, List tables) + throws Exception { + boolean compactionFound = false; + StringBuilder debugInfo = new StringBuilder(); + + for (String table : tables) { + // This will exclude metadata table timeline results + String timelinePath = + String.format("%s/%s/%s/.hoodie/timeline", warehouse, database, table); + debugInfo.append( + String.format( + "\nChecking timeline for %s.%s at: %s", database, table, timelinePath)); + + // Check if timeline directory exists in container + Container.ExecResult lsResult = jobManager.execInContainer("ls", "-la", timelinePath); + if (lsResult.getExitCode() != 0) { + debugInfo.append( + String.format( + " - Timeline directory does not exist or cannot be accessed: %s", + lsResult.getStderr())); + continue; + } + + // Find .compaction.requested files + Container.ExecResult findResult = + jobManager.execInContainer( + "find", timelinePath, "-name", "*.compaction.requested"); + + if (findResult.getExitCode() == 0 && !findResult.getStdout().trim().isEmpty()) { + compactionFound = true; + String[] compactionFiles = findResult.getStdout().trim().split("\n"); + debugInfo.append( + String.format( + " - Found %d compaction file(s): %s", + compactionFiles.length, Arrays.toString(compactionFiles))); + LOG.info( + "Compaction scheduled for table {}.{}: {}", + database, + table, + Arrays.toString(compactionFiles)); + } else { + debugInfo.append(" - No compaction.requested files found"); + + // List all timeline files for debugging + Container.ExecResult allFilesResult = + jobManager.execInContainer("ls", "-1", timelinePath); + if (allFilesResult.getExitCode() == 0) { + debugInfo.append( + String.format( + "\n All timeline files: %s", + allFilesResult.getStdout().replace("\n", ", "))); + } + } + } + + if (!compactionFound) { + LOG.error("Compaction verification failed. Debug info:{}", debugInfo); + Assertions.fail( + "No compaction.requested files found in any table timeline. " + + "Expected at least one compaction to be scheduled." + + debugInfo); + } else { + LOG.info("Compaction verification successful!"); + } + } +} diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/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..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 @@ -243,6 +243,7 @@ public void testSyncWholeDatabase() throws Exception { LOG.error("Update table for CDC failed.", e); throw e; } + List recordsInSnapshotPhase = new ArrayList<>( Arrays.asList( @@ -261,6 +262,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 6e8bac9170e..bdd56192022 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: 300", + "execution.checkpointing.interval: 30s", "state.backend.type: hashmap", "env.java.opts.all: -Doracle.jdbc.timezoneAsRegion=false", "execution.checkpointing.savepoint-dir: file:///opt/flink", @@ -386,6 +386,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