Serializing the complete payload object instead of serializing just the GenericRecord
Removing Converter hierarchy as we now depend purely on JavaSerialization and require the payload to be java serializable
This commit is contained in:
committed by
vinoth chandar
parent
e83dde3b95
commit
7243ce40c9
@@ -18,7 +18,10 @@
|
||||
|
||||
package com.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
|
||||
@@ -29,9 +32,14 @@ public abstract class BaseAvroPayload implements Serializable {
|
||||
|
||||
|
||||
/**
|
||||
* Avro data extracted from the source
|
||||
* Avro data extracted from the source converted to bytes
|
||||
*/
|
||||
protected final GenericRecord record;
|
||||
protected final byte [] recordBytes;
|
||||
|
||||
/**
|
||||
* The schema of the Avro data
|
||||
*/
|
||||
protected final String schemaStr;
|
||||
|
||||
/**
|
||||
* For purposes of preCombining
|
||||
@@ -43,7 +51,12 @@ public abstract class BaseAvroPayload implements Serializable {
|
||||
* @param orderingVal
|
||||
*/
|
||||
public BaseAvroPayload(GenericRecord record, Comparable orderingVal) {
|
||||
this.record = record;
|
||||
try {
|
||||
this.recordBytes = HoodieAvroUtils.avroToBytes(record);
|
||||
this.schemaStr = record.getSchema().toString();
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("Cannot convert GenericRecord to bytes", io);
|
||||
}
|
||||
this.orderingVal = orderingVal;
|
||||
if (orderingVal == null) {
|
||||
throw new HoodieException("Ordering value is null for record: " + record);
|
||||
|
||||
@@ -66,6 +66,7 @@ public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements
|
||||
|
||||
@Override
|
||||
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
|
||||
return Optional.of(HoodieAvroUtils.rewriteRecord(record, schema));
|
||||
return Optional.of(HoodieAvroUtils.rewriteRecord(HoodieAvroUtils.bytesToAvro(recordBytes, Schema.parse(schemaStr)),
|
||||
schema));
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user