[HUDI-377] Adding Delete() support to DeltaStreamer (#1073)
- Provides ability to perform hard deletes by writing delete marker records into the source data - if the record contains a special field _hoodie_delete_marker set to true, deletes are performed
This commit is contained in:
committed by
vinoth chandar
parent
726ae47ce2
commit
7031445eb3
@@ -31,7 +31,6 @@ import java.io.Serializable;
|
||||
* Base class for all AVRO record based payloads, that can be ordered based on a field.
|
||||
*/
|
||||
public abstract class BaseAvroPayload implements Serializable {
|
||||
|
||||
/**
|
||||
* Avro data extracted from the source converted to bytes.
|
||||
*/
|
||||
@@ -43,8 +42,10 @@ public abstract class BaseAvroPayload implements Serializable {
|
||||
protected final Comparable orderingVal;
|
||||
|
||||
/**
|
||||
* @param record
|
||||
* @param orderingVal
|
||||
* Instantiate {@link BaseAvroPayload}.
|
||||
*
|
||||
* @param record Generic record for the payload.
|
||||
* @param orderingVal {@link Comparable} to be used in pre combine.
|
||||
*/
|
||||
public BaseAvroPayload(GenericRecord record, Comparable orderingVal) {
|
||||
try {
|
||||
|
||||
@@ -103,7 +103,7 @@ public class DataSourceUtils {
|
||||
|
||||
/**
|
||||
* Create a key generator class via reflection, passing in any configs needed.
|
||||
*
|
||||
* <p>
|
||||
* If the class name of key generator is configured through the properties file, i.e., {@code props}, use the
|
||||
* corresponding key generator class; otherwise, use the default key generator class specified in {@code
|
||||
* DataSourceWriteOptions}.
|
||||
@@ -125,7 +125,7 @@ public class DataSourceUtils {
|
||||
throws IOException {
|
||||
try {
|
||||
return (HoodieRecordPayload) ReflectionUtils.loadClass(payloadClass,
|
||||
new Class<?>[]{GenericRecord.class, Comparable.class}, record, orderingVal);
|
||||
new Class<?>[] {GenericRecord.class, Comparable.class}, record, orderingVal);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not create payload for class: " + payloadClass, e);
|
||||
}
|
||||
@@ -140,7 +140,7 @@ public class DataSourceUtils {
|
||||
}
|
||||
|
||||
public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
|
||||
String tblName, Map<String, String> parameters) {
|
||||
String tblName, Map<String, String> parameters) {
|
||||
|
||||
// inline compaction is on by default for MOR
|
||||
boolean inlineCompact = parameters.get(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY())
|
||||
@@ -162,7 +162,7 @@ public class DataSourceUtils {
|
||||
}
|
||||
|
||||
public static JavaRDD<WriteStatus> doWriteOperation(HoodieWriteClient client, JavaRDD<HoodieRecord> hoodieRecords,
|
||||
String commitTime, String operation) {
|
||||
String commitTime, String operation) {
|
||||
if (operation.equals(DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL())) {
|
||||
return client.bulkInsert(hoodieRecords, commitTime);
|
||||
} else if (operation.equals(DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())) {
|
||||
@@ -174,19 +174,19 @@ public class DataSourceUtils {
|
||||
}
|
||||
|
||||
public static JavaRDD<WriteStatus> doDeleteOperation(HoodieWriteClient client, JavaRDD<HoodieKey> hoodieKeys,
|
||||
String commitTime) {
|
||||
String commitTime) {
|
||||
return client.delete(hoodieKeys, commitTime);
|
||||
}
|
||||
|
||||
public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable orderingVal, HoodieKey hKey,
|
||||
String payloadClass) throws IOException {
|
||||
String payloadClass) throws IOException {
|
||||
HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal);
|
||||
return new HoodieRecord<>(hKey, payload);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc, JavaRDD<HoodieRecord> incomingHoodieRecords,
|
||||
HoodieWriteConfig writeConfig, Option<EmbeddedTimelineService> timelineService) {
|
||||
HoodieWriteConfig writeConfig, Option<EmbeddedTimelineService> timelineService) {
|
||||
HoodieReadClient client = null;
|
||||
try {
|
||||
client = new HoodieReadClient<>(jssc, writeConfig, timelineService);
|
||||
@@ -205,7 +205,7 @@ public class DataSourceUtils {
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc, JavaRDD<HoodieRecord> incomingHoodieRecords,
|
||||
Map<String, String> parameters, Option<EmbeddedTimelineService> timelineService) {
|
||||
Map<String, String> parameters, Option<EmbeddedTimelineService> timelineService) {
|
||||
HoodieWriteConfig writeConfig =
|
||||
HoodieWriteConfig.newBuilder().withPath(parameters.get("path")).withProps(parameters).build();
|
||||
return dropDuplicates(jssc, incomingHoodieRecords, writeConfig, timelineService);
|
||||
|
||||
@@ -38,8 +38,7 @@ public class OverwriteWithLatestAvroPayload extends BaseAvroPayload
|
||||
implements HoodieRecordPayload<OverwriteWithLatestAvroPayload> {
|
||||
|
||||
/**
|
||||
* @param record
|
||||
* @param orderingVal
|
||||
*
|
||||
*/
|
||||
public OverwriteWithLatestAvroPayload(GenericRecord record, Comparable orderingVal) {
|
||||
super(record, orderingVal);
|
||||
@@ -61,8 +60,15 @@ public class OverwriteWithLatestAvroPayload extends BaseAvroPayload
|
||||
|
||||
@Override
|
||||
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException {
|
||||
|
||||
GenericRecord genericRecord = (GenericRecord) getInsertValue(schema).get();
|
||||
// combining strategy here trivially ignores currentValue on disk and writes this record
|
||||
return getInsertValue(schema);
|
||||
Object deleteMarker = genericRecord.get("_hoodie_is_deleted");
|
||||
if (deleteMarker instanceof Boolean && (boolean) deleteMarker) {
|
||||
return Option.empty();
|
||||
} else {
|
||||
return Option.of(genericRecord);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
Reference in New Issue
Block a user