Spark Stage retry handling
This commit is contained in:
committed by
vinoth chandar
parent
3fd2fd6e9d
commit
145034c5fa
@@ -27,16 +27,12 @@ import com.uber.hoodie.io.HoodieCreateHandle;
|
||||
import com.uber.hoodie.io.HoodieIOHandle;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.function.Function;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
/**
|
||||
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
|
||||
@@ -48,15 +44,17 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extend
|
||||
protected final HoodieWriteConfig hoodieConfig;
|
||||
protected final String commitTime;
|
||||
protected final HoodieTable<T> hoodieTable;
|
||||
protected Set<String> partitionsCleaned;
|
||||
protected final String idPrefix;
|
||||
protected int numFilesWritten;
|
||||
|
||||
public CopyOnWriteLazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
|
||||
String commitTime, HoodieTable<T> hoodieTable) {
|
||||
String commitTime, HoodieTable<T> hoodieTable, String idPrefix) {
|
||||
super(sortedRecordItr);
|
||||
this.partitionsCleaned = new HashSet<>();
|
||||
this.hoodieConfig = config;
|
||||
this.commitTime = commitTime;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.idPrefix = idPrefix;
|
||||
this.numFilesWritten = 0;
|
||||
}
|
||||
|
||||
// Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread.
|
||||
@@ -113,7 +111,10 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extend
|
||||
|
||||
@Override
|
||||
protected void end() {
|
||||
}
|
||||
|
||||
protected String getNextFileId(String idPfx) {
|
||||
return String.format("%s-%d", idPfx, numFilesWritten++);
|
||||
}
|
||||
|
||||
protected CopyOnWriteInsertHandler getInsertHandler() {
|
||||
@@ -133,20 +134,11 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extend
|
||||
@Override
|
||||
protected void consumeOneRecord(HoodieInsertValueGenResult<HoodieRecord> payload) {
|
||||
final HoodieRecord insertPayload = payload.record;
|
||||
// clean up any partial failures
|
||||
if (!partitionsCleaned.contains(insertPayload.getPartitionPath())) {
|
||||
// This insert task could fail multiple times, but Spark will faithfully retry with
|
||||
// the same data again. Thus, before we open any files under a given partition, we
|
||||
// first delete any files in the same partitionPath written by same Spark partition
|
||||
HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig, commitTime, insertPayload.getPartitionPath(),
|
||||
TaskContext.getPartitionId(), hoodieTable);
|
||||
partitionsCleaned.add(insertPayload.getPartitionPath());
|
||||
}
|
||||
|
||||
// lazily initialize the handle, for the first time
|
||||
if (handle == null) {
|
||||
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(), UUID
|
||||
.randomUUID().toString());
|
||||
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(),
|
||||
getNextFileId(idPrefix));
|
||||
}
|
||||
|
||||
if (handle.canWrite(payload.record)) {
|
||||
@@ -156,8 +148,8 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extend
|
||||
// handle is full.
|
||||
statuses.add(handle.close());
|
||||
// Need to handle the rejected payload & open new handle
|
||||
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(), UUID
|
||||
.randomUUID().toString());
|
||||
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(),
|
||||
getNextFileId(idPrefix));
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception); // we should be able to write 1 payload.
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user