1
0

Fixing source schema and writer schema distinction in payloads

This commit is contained in:
Nishith Agarwal
2019-03-22 16:27:51 -07:00
committed by Balaji Varadarajan
parent 395806fc68
commit 3d9041e216
10 changed files with 114 additions and 30 deletions

View File

@@ -64,6 +64,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
private long recordsDeleted = 0;
private long updatedRecordsWritten = 0;
private long insertRecordsWritten = 0;
private boolean useWriterSchema;
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
Iterator<HoodieRecord<T>> recordItr, String fileId) {
@@ -80,6 +81,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
super(config, commitTime, hoodieTable);
this.fileSystemView = hoodieTable.getROFileSystemView();
this.keyToNewRecords = keyToNewRecords;
this.useWriterSchema = true;
init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get())
.getPartitionPath(), dataFileToBeMerged);
}
@@ -125,7 +127,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
writeStatus.getStat().setPaths(new Path(config.getBasePath()), newFilePath, tempPath);
// Create the writer for writing the new version file
storageWriter = HoodieStorageWriterFactory
.getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema);
.getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, writerSchema);
} catch (IOException io) {
logger.error("Error in update task at commit " + commitTime, io);
writeStatus.setGlobalError(io);
@@ -143,7 +145,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
// Load the new records in a map
logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge());
this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(),
config.getSpillableMapBasePath(), new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema));
config.getSpillableMapBasePath(), new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(originalSchema));
} catch (IOException io) {
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
}
@@ -177,7 +179,9 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
Optional recordMetadata = hoodieRecord.getData().getMetadata();
try {
if (indexedRecord.isPresent()) {
storageWriter.writeAvroWithMetadata(indexedRecord.get(), hoodieRecord);
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
IndexedRecord recordWithMetadataInSchema = rewriteRecord((GenericRecord) indexedRecord.get());
storageWriter.writeAvroWithMetadata(recordWithMetadataInSchema, hoodieRecord);
recordsWritten++;
} else {
recordsDeleted++;
@@ -209,7 +213,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key));
try {
Optional<IndexedRecord> combinedAvroRecord = hoodieRecord.getData()
.combineAndGetUpdateValue(oldRecord, schema);
.combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchema : originalSchema);
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
/* ONLY WHEN
* 1) we have an update for this key AND
@@ -235,7 +239,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
storageWriter.writeAvro(key, oldRecord);
} catch (ClassCastException e) {
logger.error("Schema mismatch when rewriting old record " + oldRecord + " from file "
+ getOldFilePath() + " to file " + getStorageWriterPath() + " with schema " + schema
+ getOldFilePath() + " to file " + getStorageWriterPath() + " with writerSchema " + writerSchema
.toString(true));
throw new HoodieUpsertException(errMsg, e);
} catch (IOException e) {
@@ -254,7 +258,11 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
for (String key : keyToNewRecords.keySet()) {
if (!writtenRecordKeys.contains(key)) {
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
if (useWriterSchema) {
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
} else {
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(originalSchema));
}
insertRecordsWritten++;
}
}
@@ -293,4 +301,4 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
public WriteStatus getWriteStatus() {
return writeStatus;
}
}
}