1
0

Fixed HUDI-87 : Remove schemastr from BaseAvroPayload

This commit is contained in:
Vinoth Chandar
2019-03-27 15:47:49 -07:00
committed by vinoth chandar
parent 372fbc4733
commit e56c1612e4
3 changed files with 7 additions and 8 deletions

View File

@@ -50,7 +50,7 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
private long insertRecordsWritten = 0;
private long recordsDeleted = 0;
private Iterator<HoodieRecord<T>> recordIterator;
private boolean useWriterSchema;
private boolean useWriterSchema = false;
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
String partitionPath, String fileId) {
@@ -78,6 +78,9 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
logger.info("New InsertHandle for partition :" + partitionPath + " with fileId " + fileId);
}
/**
* Called by the compactor code path
*/
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordIterator) {
this(config, commitTime, hoodieTable, partitionPath, fileId);

View File

@@ -76,6 +76,9 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
.filter(dataFile -> dataFile.getFileId().equals(fileId)).findFirst());
}
/**
* Called by compactor code path
*/
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
Map<String, HoodieRecord<T>> keyToNewRecords, String fileId, Optional<HoodieDataFile> dataFileToBeMerged) {
super(config, commitTime, hoodieTable);

View File

@@ -30,17 +30,11 @@ import org.apache.avro.generic.GenericRecord;
*/
public abstract class BaseAvroPayload implements Serializable {
/**
* Avro data extracted from the source converted to bytes
*/
protected final byte [] recordBytes;
/**
* The schema of the Avro data
*/
protected final String schemaStr;
/**
* For purposes of preCombining
*/
@@ -53,7 +47,6 @@ public abstract class BaseAvroPayload implements Serializable {
public BaseAvroPayload(GenericRecord record, Comparable orderingVal) {
try {
this.recordBytes = HoodieAvroUtils.avroToBytes(record);
this.schemaStr = record.getSchema().toString();
} catch (IOException io) {
throw new HoodieIOException("Cannot convert GenericRecord to bytes", io);
}