Fixing behavior of Merge/CreateHandle for invalid/wrong schema records
This commit is contained in:
committed by
vinoth chandar
parent
994d42d307
commit
7985eb72b5
@@ -26,7 +26,6 @@ import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.io.HoodieCreateHandle;
|
||||
import com.uber.hoodie.io.HoodieIOHandle;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
@@ -38,7 +37,6 @@ import java.util.function.Function;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.spark.TaskContext;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
|
||||
@@ -61,20 +59,30 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extend
|
||||
this.hoodieTable = hoodieTable;
|
||||
}
|
||||
|
||||
// Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread.
|
||||
static class HoodieInsertValueGenResult<T extends HoodieRecord> {
|
||||
public T record;
|
||||
public Optional<IndexedRecord> insertValue;
|
||||
// It caches the exception seen while fetching insert value.
|
||||
public Optional<Exception> exception = Optional.empty();
|
||||
|
||||
public HoodieInsertValueGenResult(T record, Schema schema) {
|
||||
this.record = record;
|
||||
try {
|
||||
this.insertValue = record.getData().getInsertValue(schema);
|
||||
} catch (Exception e) {
|
||||
this.exception = Optional.of(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some
|
||||
* expensive operations of transformation to the reader thread.
|
||||
*/
|
||||
static <T extends HoodieRecordPayload> Function<HoodieRecord<T>,
|
||||
Tuple2<HoodieRecord<T>, Optional<IndexedRecord>>> getTransformFunction(Schema schema) {
|
||||
return hoodieRecord -> {
|
||||
try {
|
||||
return new Tuple2<HoodieRecord<T>, Optional<IndexedRecord>>(hoodieRecord,
|
||||
hoodieRecord.getData().getInsertValue(schema));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
};
|
||||
HoodieInsertValueGenResult<HoodieRecord>> getTransformFunction(Schema schema) {
|
||||
return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -85,7 +93,7 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extend
|
||||
protected List<WriteStatus> computeNext() {
|
||||
// Executor service used for launching writer thread.
|
||||
BoundedInMemoryExecutor<HoodieRecord<T>,
|
||||
Tuple2<HoodieRecord<T>, Optional<IndexedRecord>>, List<WriteStatus>> bufferedIteratorExecutor = null;
|
||||
HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> bufferedIteratorExecutor = null;
|
||||
try {
|
||||
final Schema schema = HoodieIOHandle.createHoodieWriteSchema(hoodieConfig);
|
||||
bufferedIteratorExecutor =
|
||||
@@ -117,14 +125,14 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extend
|
||||
* writes to one or more create-handles
|
||||
*/
|
||||
protected class CopyOnWriteInsertHandler extends
|
||||
BoundedInMemoryQueueConsumer<Tuple2<HoodieRecord<T>, Optional<IndexedRecord>>, List<WriteStatus>> {
|
||||
BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> {
|
||||
|
||||
protected final List<WriteStatus> statuses = new ArrayList<>();
|
||||
protected HoodieIOHandle handle;
|
||||
|
||||
@Override
|
||||
protected void consumeOneRecord(Tuple2<HoodieRecord<T>, Optional<IndexedRecord>> payload) {
|
||||
final HoodieRecord insertPayload = payload._1();
|
||||
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
|
||||
@@ -141,16 +149,16 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extend
|
||||
.randomUUID().toString());
|
||||
}
|
||||
|
||||
if (handle.canWrite(payload._1())) {
|
||||
if (handle.canWrite(payload.record)) {
|
||||
// write the payload, if the handle has capacity
|
||||
handle.write(insertPayload, payload._2());
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception);
|
||||
} else {
|
||||
// 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.write(insertPayload, payload._2()); // we should be able to write 1 payload.
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception); // we should be able to write 1 payload.
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user