1
0

Support union mode in HoodieRealtimeRecordReader for pure insert workloads

Also Replace BufferedIteratorPayload abstraction with function passing
This commit is contained in:
Balaji Varadarajan
2018-04-26 10:18:05 -07:00
committed by vinoth chandar
parent 93f345a032
commit dfc0c61eb7
44 changed files with 2545 additions and 1179 deletions

View File

@@ -33,17 +33,16 @@ import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor;
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.BufferedIterator;
import com.uber.hoodie.func.BufferedIteratorExecutor;
import com.uber.hoodie.func.LazyInsertIterable;
import com.uber.hoodie.func.ParquetReaderIterator;
import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload;
import com.uber.hoodie.func.payload.GenericRecordBufferedIteratorPayload;
import com.uber.hoodie.func.SparkBoundedInMemoryExecutor;
import com.uber.hoodie.io.HoodieCleanHelper;
import com.uber.hoodie.io.HoodieMergeHandle;
import java.io.IOException;
@@ -58,9 +57,6 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.stream.Collectors;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -182,16 +178,6 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
return handleUpdateInternal(upsertHandle, commitTime, fileLoc);
}
/**
* Transformer function to help transform a GenericRecord. This transformer is used by BufferedIterator to offload
* some expensive operations of transformation to the reader thread.
*
*/
public static java.util.function.Function<GenericRecord, AbstractBufferedIteratorPayload>
bufferedItrPayloadTransform() {
return (genericRecord) -> new GenericRecordBufferedIteratorPayload(genericRecord);
}
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle,
String commitTime, String fileLoc)
throws IOException {
@@ -202,23 +188,19 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema());
ParquetReader<IndexedRecord> reader = AvroParquetReader.builder(upsertHandle.getOldFilePath())
.withConf(getHadoopConf()).build();
final ExecutorService writerService = Executors.newFixedThreadPool(1);
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
try {
java.util.function.Function<BufferedIterator, Void> runnableFunction = (bufferedIterator) -> {
handleWrite(bufferedIterator, upsertHandle);
return null;
};
BufferedIteratorExecutor<GenericRecord, AbstractBufferedIteratorPayload, Void> wrapper =
new BufferedIteratorExecutor(config, new ParquetReaderIterator(reader), bufferedItrPayloadTransform(),
writerService);
Future writerResult = wrapper.start(runnableFunction);
writerResult.get();
wrapper = new SparkBoundedInMemoryExecutor(config, new ParquetReaderIterator(reader),
new UpdateHandler(upsertHandle), x -> x);
wrapper.execute();
} catch (Exception e) {
throw new HoodieException(e);
} finally {
reader.close();
upsertHandle.close();
writerService.shutdownNow();
if (null != wrapper) {
wrapper.shutdownNow();
}
}
}
@@ -231,15 +213,6 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
.iterator();
}
private void handleWrite(final BufferedIterator<GenericRecord, GenericRecord> bufferedIterator,
final HoodieMergeHandle upsertHandle) {
while (bufferedIterator.hasNext()) {
final GenericRecordBufferedIteratorPayload payload = (GenericRecordBufferedIteratorPayload) bufferedIterator
.next();
upsertHandle.write(payload.getOutputPayload());
}
}
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc,
Iterator<HoodieRecord<T>> recordItr) {
return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc);
@@ -493,6 +466,32 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
UPDATE, INSERT
}
/**
* Consumer that dequeues records from queue and sends to Merge Handle
*/
private static class UpdateHandler extends BoundedInMemoryQueueConsumer<GenericRecord, Void> {
private final HoodieMergeHandle upsertHandle;
private UpdateHandler(HoodieMergeHandle upsertHandle) {
this.upsertHandle = upsertHandle;
}
@Override
protected void consumeOneRecord(GenericRecord record) {
upsertHandle.write(record);
}
@Override
protected void finish() {
}
@Override
protected Void getResult() {
return null;
}
}
private static class PartitionCleanStat implements Serializable {
private final String partitionPath;