Ensure Compaction Operation compacts the data file as defined in the workload
This commit is contained in:
committed by
vinoth chandar
parent
2f8ce93030
commit
2e12c86d01
@@ -173,26 +173,26 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table");
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileLoc,
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileLoc);
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, recordItr);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileId);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileLoc,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords) throws IOException {
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, Optional<HoodieDataFile> dataFileOpt) throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, keyToNewRecords);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileLoc);
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, keyToNewRecords, dataFileOpt);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileId);
|
||||
}
|
||||
|
||||
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle,
|
||||
String commitTime, String fileLoc)
|
||||
String commitTime, String fileId)
|
||||
throws IOException {
|
||||
if (upsertHandle.getOldFilePath() == null) {
|
||||
throw new HoodieUpsertException(
|
||||
"Error in finding the old file path at commit " + commitTime + " at fileLoc: " + fileLoc);
|
||||
"Error in finding the old file path at commit " + commitTime + " for fileId: " + fileId);
|
||||
} else {
|
||||
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema());
|
||||
ParquetReader<IndexedRecord> reader = AvroParquetReader.builder(upsertHandle.getOldFilePath())
|
||||
@@ -222,14 +222,14 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
.iterator();
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc,
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc);
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileId);
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileLoc);
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, Optional<HoodieDataFile> dataFileToBeMerged) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileId, dataFileToBeMerged);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime,
|
||||
|
||||
Reference in New Issue
Block a user