Fix for updating duplicate records in same/different files in same parition
This commit is contained in:
committed by
vinoth chandar
parent
fa73a911cc
commit
4d1fba24c9
@@ -394,6 +394,13 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
if (v1._2().isPresent()) {
|
||||
String filename = v1._2().get();
|
||||
if (filename != null && !filename.isEmpty()) {
|
||||
// When you have a record in multiple files in the same partition, then rowKeyRecordPairRDD will have 2
|
||||
// entries with the same exact in memory copy of the HoodieRecord and the 2 separate filenames that the
|
||||
// record is found in. This will result in setting currentLocation 2 times and it will fail the second time.
|
||||
// This check will create a new in memory copy of the hoodie record.
|
||||
if (record.getCurrentLocation() != null) {
|
||||
record = new HoodieRecord<T>(record.getKey(), record.getData());
|
||||
}
|
||||
record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename),
|
||||
FSUtils.getFileId(filename)));
|
||||
}
|
||||
|
||||
@@ -35,9 +35,11 @@ import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -52,6 +54,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
|
||||
private WriteStatus writeStatus;
|
||||
private Map<String, HoodieRecord<T>> keyToNewRecords;
|
||||
private Set<String> writtenRecordKeys;
|
||||
private HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||
private TableFileSystemView.ReadOptimizedView fileSystemView;
|
||||
private Path newFilePath;
|
||||
@@ -81,6 +84,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
* Extract old file path, initialize StorageWriter and WriteStatus
|
||||
*/
|
||||
private void init(String fileId, String partitionPath) {
|
||||
this.writtenRecordKeys = new HashSet<>();
|
||||
|
||||
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
@@ -205,7 +210,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
*/
|
||||
copyOldRecord = false;
|
||||
}
|
||||
keyToNewRecords.remove(key);
|
||||
writtenRecordKeys.add(key);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to combine/merge new record with old value in storage, for new record {"
|
||||
@@ -239,10 +244,13 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
|
||||
while (pendingRecordsItr.hasNext()) {
|
||||
String key = pendingRecordsItr.next();
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
writeUpdateRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
|
||||
if (!writtenRecordKeys.contains(key)) {
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
writeUpdateRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
|
||||
}
|
||||
}
|
||||
keyToNewRecords.clear();
|
||||
writtenRecordKeys.clear();
|
||||
|
||||
if (storageWriter != null) {
|
||||
storageWriter.close();
|
||||
|
||||
Reference in New Issue
Block a user