voonhous commented on code in PR #4164: URL: https://github.com/apache/flink-cdc/pull/4164#discussion_r2488427204
########## 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,583 @@ +/* + * 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<FieldGetter> 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<FieldGetter> createFieldGetters(Schema schema, ZoneId zoneId) { + List<Column> columns = schema.getColumns(); + List<FieldGetter> 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<FieldGetter> 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); + + // Extract partition path from partition key fields + String partitionPath = extractPartitionPathFromDataChangeEvent(dataChangeEvent, schema); + + return convertDataChangeEventToHoodieFlinkInternalRow( + dataChangeEvent, schema, zoneId, recordKey, partitionPath, fileId, instantTime); + } + + /** Map CDC operation type to Hudi operation type string. */ + private static String mapCdcOperationToHudiOperation(OperationType cdcOp) { + switch (cdcOp) { + 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<String> 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<String> recordKeyPairs = new ArrayList<>(primaryKeyFields.size()); + for (String primaryKeyField : primaryKeyFields) { + int fieldIndex = schema.getColumnNames().indexOf(primaryKeyField); + if (fieldIndex == -1) { + throw new IllegalStateException( + "Primary key field '" + + primaryKeyField + + "' not found in schema for table " + + dataChangeEvent.tableId()); + } + + Object fieldValue = + recordData.isNullAt(fieldIndex) + ? null + : getFieldValue( + recordData, + fieldIndex, + schema.getColumns().get(fieldIndex).getType()); + + if (fieldValue == null) { + throw new IllegalStateException( + "Primary key field '" + primaryKeyField + "' is null in record"); + } + + // Format as "fieldName:value" to match BucketAssignOperator format + recordKeyPairs.add(primaryKeyField + ":" + fieldValue); + } + + return String.join(",", recordKeyPairs); + } + + /** + * Extract partition path from DataChangeEvent based on partition key fields in schema. + * + * <p>If the schema has partition keys defined: + * + * <ul> + * <li>Extracts partition field values from the record data + * <li>Formats them as "field1=value1/field2=value2" (Hive-style partitioning) + * </ul> + * + * <p>If no partition keys are defined, returns "default". + * + * @param dataChangeEvent The DataChangeEvent to extract partition from + * @param schema The table schema containing partition key definitions + * @return The partition path string + */ + private static String extractPartitionPathFromDataChangeEvent( + DataChangeEvent dataChangeEvent, Schema schema) { + List<String> partitionKeys = schema.partitionKeys(); + if (partitionKeys == null || partitionKeys.isEmpty()) { + return "default"; Review Comment: Yeap, good catch, fixed. ########## 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<Event>, WithPreWriteTopology<Event> { + + 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<Event> createWriter(InitContext context) throws IOException { + return DummySinkWriter.INSTANCE; + } + + @Override + public SinkWriter<Event> createWriter(WriterInitContext context) throws IOException { + return DummySinkWriter.INSTANCE; + } + + @Override + public DataStream<Event> addPreWriteTopology(DataStream<Event> 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<BucketWrapper> 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<BucketWrapper> partitionedStream = + bucketAssignedStream.partitionCustom( + (key, numPartitions) -> key % numPartitions, Review Comment: Done -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
