1
0

[HUDI-1678] Row level delete for Flink sink (#2659)

This commit is contained in:
Danny Chan
2021-03-11 19:44:06 +08:00
committed by GitHub
parent 2fdae6835c
commit 12ff562d2b
8 changed files with 169 additions and 83 deletions

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.operator.transform;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.WriteOperationType;
@@ -33,6 +34,7 @@ import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.types.RowKind;
import java.io.IOException;
@@ -100,9 +102,12 @@ public class RowDataToHoodieFunction<I extends RowData, O extends HoodieRecord<?
final String payloadClazz = this.config.getString(FlinkOptions.PAYLOAD_CLASS);
Comparable orderingVal = (Comparable) HoodieAvroUtils.getNestedFieldVal(gr,
this.config.getString(FlinkOptions.PRECOMBINE_FIELD), false);
final HoodieKey hoodieKey = keyGenerator.getKey(gr);
// nullify the payload insert data to mark the record as a DELETE
gr = record.getRowKind() == RowKind.DELETE ? null : gr;
HoodieRecordPayload payload = shouldCombine
? StreamerUtil.createPayload(payloadClazz, gr, orderingVal)
: StreamerUtil.createPayload(payloadClazz, gr);
return new HoodieRecord<>(keyGenerator.getKey(gr), payload);
return new HoodieRecord<>(hoodieKey, payload);
}
}