1
0

refactor classes to accept Map passed by RealtimeCompactor to avoid multiple map creations in HoodieMergeHandle

This commit is contained in:
Nishith Agarwal
2018-01-24 13:34:14 -08:00
committed by vinoth chandar
parent 30049383f5
commit 7076c2e9f0
4 changed files with 100 additions and 60 deletions

View File

@@ -416,10 +416,23 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileLoc,
Iterator<HoodieRecord<T>> recordItr)
Iterator<HoodieRecord<T>> recordItr)
throws IOException {
// these are updates
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr);
return handleUpdateInternal(upsertHandle, commitTime, fileLoc);
}
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileLoc,
Map<String, HoodieRecord<T>> keyToNewRecords)
throws IOException {
// these are updates
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, keyToNewRecords);
return handleUpdateInternal(upsertHandle, commitTime, fileLoc);
}
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle, String commitTime, String fileLoc)
throws IOException {
if (upsertHandle.getOldFilePath() == null) {
throw new HoodieUpsertException("Error in finding the old file path at commit " +
commitTime + " at fileLoc: " + fileLoc);
@@ -459,6 +472,11 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc);
}
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc,
Map<String, HoodieRecord<T>> keyToNewRecords) {
return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileLoc);
}
public Iterator<List<WriteStatus>> handleInsert(String commitTime,
Iterator<HoodieRecord<T>> recordItr) throws Exception {
return new LazyInsertIterable<>(recordItr, config, commitTime, this);