1
0

Adding ability for inserts to be written to log files

This commit is contained in:
Nishith Agarwal
2018-05-13 16:25:11 -07:00
committed by vinoth chandar
parent 34827d50e1
commit 3da063f83b
52 changed files with 1061 additions and 519 deletions

View File

@@ -243,7 +243,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
}
}
public void close() {
@Override
public WriteStatus close() {
try {
// write out any pending records (this can happen when inserts are turned into updates)
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
@@ -269,6 +270,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
RuntimeStats runtimeStats = new RuntimeStats();
runtimeStats.setTotalUpsertTime(timer.endTimer());
writeStatus.getStat().setRuntimeStats(runtimeStats);
return writeStatus;
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
}
@@ -283,6 +285,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
return (this.tempPath == null) ? this.newFilePath : this.tempPath;
}
@Override
public WriteStatus getWriteStatus() {
return writeStatus;
}