Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-8163] Refactor UnMergedLogHandler with iterators #12608

Open
wants to merge 15 commits into
base: master
Choose a base branch
from
Original file line number Diff line number Diff line change
Expand Up @@ -193,7 +193,7 @@ private static Iterator<HoodieRecord<?>> getUnmergedLogFileRecords(List<String>
scanner.scan(false);
return scanner.iterator();
}

private static Iterator<HoodieRecord> getBaseFileRecords(HoodieBaseFile baseFile, HoodieTableMetaClient metaClient, Schema readerSchema) {
HoodieRecordMerger recordMerger =
HoodieRecordUtils.createRecordMerger(metaClient.getBasePath().toString(), EngineType.SPARK, Collections.emptyList(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -146,9 +146,6 @@ public boolean hasNext() {

@Override
public HoodieRecord<?> next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
return recordIterator.next();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,42 +27,33 @@
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner;
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.Functions;
import org.apache.hudi.common.util.HoodieRecordUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ClosableIterator;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer;
import org.apache.hudi.common.util.queue.HoodieProducer;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.storage.HoodieStorage;
import org.apache.hudi.storage.HoodieStorageUtils;
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
import org.apache.hudi.util.FlinkWriteClients;
import org.apache.hudi.util.StreamerUtil;

import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.GenericRecordBuilder;
import org.apache.avro.generic.IndexedRecord;
import org.apache.flink.table.data.RowData;
import org.apache.flink.types.RowKind;
import org.apache.flink.util.CloseableIterator;
import org.apache.hadoop.conf.Configuration;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.function.Function;
import java.util.stream.Collectors;

/**
Expand Down Expand Up @@ -181,9 +172,6 @@ public static HoodieMergedLogRecordScanner logScanner(
* Utility to read and buffer the records in the unMerged log record scanner.
*/
public static class BoundedMemoryRecords {
// Executor that runs the above producers in parallel
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BoundedMemoryRecords can be eliminated too I guess.

private final BoundedInMemoryExecutor<HoodieRecord<?>, HoodieRecord<?>, ?> executor;

// Iterator for the buffer consumer
private final Iterator<HoodieRecord<?>> iterator;

Expand All @@ -199,63 +187,34 @@ public BoundedMemoryRecords(
.collect(Collectors.toList());
HoodieRecordMerger merger = HoodieRecordUtils.createRecordMerger(
split.getTablePath(), EngineType.FLINK, mergers, flinkConf.getString(FlinkOptions.RECORD_MERGER_STRATEGY_ID));
HoodieUnMergedLogRecordScanner.Builder scannerBuilder =
HoodieUnMergedLogRecordScanner scanner =
HoodieUnMergedLogRecordScanner.newBuilder()
.withStorage(HoodieStorageUtils.getStorage(
split.getTablePath(), HadoopFSUtils.getStorageConf(hadoopConf)))
.withBasePath(split.getTablePath())
.withLogFilePaths(split.getLogPaths().get())
.withReaderSchema(logSchema)
.withInternalSchema(internalSchema)
.withLatestInstantTime(split.getLatestCommit())
.withReverseReader(false)
.withBufferSize(
flinkConf.getInteger(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE.key(),
HoodieMemoryConfig.DEFAULT_MR_MAX_DFS_STREAM_BUFFER_SIZE))
.withInstantRange(split.getInstantRange())
.withRecordMerger(merger);

this.executor = new BoundedInMemoryExecutor<>(
StreamerUtil.getMaxCompactionMemoryInBytes(flinkConf),
getParallelProducers(scannerBuilder),
Option.empty(),
Function.identity(),
new DefaultSizeEstimator<>(),
Functions.noop());
this.iterator = this.executor.getRecordIterator();
.withBasePath(split.getTablePath())
.withLogFilePaths(split.getLogPaths().get())
.withReaderSchema(logSchema)
.withInternalSchema(internalSchema)
.withLatestInstantTime(split.getLatestCommit())
.withReverseReader(false)
.withBufferSize(
flinkConf.getInteger(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE.key(),
HoodieMemoryConfig.DEFAULT_MR_MAX_DFS_STREAM_BUFFER_SIZE))
.withInstantRange(split.getInstantRange())
.withRecordMerger(merger)
.build();

// Start reading and buffering
this.executor.startProducingAsync();
this.iterator = scanner.iterator();
}

public Iterator<HoodieRecord<?>> getRecordsIterator() {
return this.iterator;
}

/**
* Setup log and parquet reading in parallel. Both write to central buffer.
*/
private List<HoodieProducer<HoodieRecord<?>>> getParallelProducers(
HoodieUnMergedLogRecordScanner.Builder scannerBuilder
) {
List<HoodieProducer<HoodieRecord<?>>> producers = new ArrayList<>();
producers.add(new FunctionBasedQueueProducer<>(queue -> {
HoodieUnMergedLogRecordScanner scanner = scannerBuilder.build();
try (ClosableIterator<HoodieRecord<?>> logRecordIterator = scanner.iterator()) {
while (logRecordIterator.hasNext()) {
queue.insertRecord(logRecordIterator.next());
}
} catch (Exception e) {
throw new HoodieException("Error converting the record to avro", e);
}
return null;
}));

return producers;
}

public void close() {
this.executor.shutdownNow();
if (this.iterator instanceof ClosableIterator) {
((ClosableIterator<HoodieRecord<?>>) this.iterator).close();
}
}
}

Expand Down