1
0

[HUDI-2439] Remove SparkBoundedInMemoryExecutor (#4860)

This commit is contained in:
Raymond Xu
2022-02-26 05:02:12 -08:00
committed by GitHub
parent 1379300b5b
commit c77b2591d0
14 changed files with 99 additions and 113 deletions

View File

@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieRecord;
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.Option;
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
@@ -50,6 +51,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.function.Function;
/**
* Utilities for format.
@@ -193,8 +195,9 @@ public class FormatUtils {
HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes(new JobConf(hadoopConf)),
getParallelProducers(),
Option.empty(),
x -> x,
new DefaultSizeEstimator<>());
Function.identity(),
new DefaultSizeEstimator<>(),
Functions.noop());
// Consumer of this record reader
this.iterator = this.executor.getQueue().iterator();
this.scanner = FormatUtils.unMergedLogScanner(split, logSchema, hadoopConf,