Parallelized read-write operations in Hoodie Merge phase
This commit is contained in:
committed by
vinoth chandar
parent
6c226ca21a
commit
720e42f52a
@@ -34,10 +34,16 @@ 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.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.io.HoodieCleanHelper;
|
||||
import com.uber.hoodie.io.HoodieMergeHandle;
|
||||
import java.io.IOException;
|
||||
@@ -52,6 +58,9 @@ 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;
|
||||
@@ -173,8 +182,19 @@ 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 {
|
||||
String commitTime, String fileLoc)
|
||||
throws IOException {
|
||||
if (upsertHandle.getOldFilePath() == null) {
|
||||
throw new HoodieUpsertException(
|
||||
"Error in finding the old file path at commit " + commitTime + " at fileLoc: " + fileLoc);
|
||||
@@ -182,32 +202,44 @@ 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);
|
||||
try {
|
||||
IndexedRecord record;
|
||||
while ((record = reader.read()) != null) {
|
||||
// Two types of writes here (new record, and old record).
|
||||
// We have already catch the exception during writing new records.
|
||||
// But for old records, we should fail if any exception happens.
|
||||
upsertHandle.write((GenericRecord) record);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to read record from " + upsertHandle.getOldFilePath() + " with new Schema "
|
||||
+ upsertHandle.getSchema(), e);
|
||||
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();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
} finally {
|
||||
reader.close();
|
||||
upsertHandle.close();
|
||||
writerService.shutdownNow();
|
||||
}
|
||||
}
|
||||
|
||||
//TODO(vc): This needs to be revisited
|
||||
if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
|
||||
logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
|
||||
+ upsertHandle.getWriteStatus());
|
||||
logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath()
|
||||
+ ", " + upsertHandle.getWriteStatus());
|
||||
}
|
||||
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus()))
|
||||
.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);
|
||||
@@ -792,4 +824,4 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user