-
Notifications
You must be signed in to change notification settings - Fork 2.5k
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
base: master
Are you sure you want to change the base?
Changes from 6 commits
6354f79
64a9756
6425371
8a46112
49c86fa
930e1e9
33c8c4a
ba9a9ef
5ca19a8
cde5b65
f25f5bc
d02d371
072c7dd
3df692e
28c2982
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -30,12 +30,14 @@ | |
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; | ||
|
@@ -51,6 +53,7 @@ | |
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; | ||
|
@@ -237,10 +240,14 @@ private List<HoodieProducer<HoodieRecord<?>>> getParallelProducers( | |
) { | ||
List<HoodieProducer<HoodieRecord<?>>> producers = new ArrayList<>(); | ||
producers.add(new FunctionBasedQueueProducer<>(queue -> { | ||
HoodieUnMergedLogRecordScanner scanner = | ||
scannerBuilder.withLogRecordScannerCallback(queue::insertRecord).build(); | ||
// Scan all the delta-log files, filling in the queue | ||
scanner.scan(); | ||
HoodieUnMergedLogRecordScanner scanner = scannerBuilder.build(); | ||
try (ClosableIterator<HoodieRecord<?>> logRecordIterator = scanner.iterator()) { | ||
while (logRecordIterator.hasNext()) { | ||
queue.insertRecord(logRecordIterator.next()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We can remove the whole producers and queue I think. |
||
} | ||
} catch (Exception e) { | ||
throw new HoodieException("Error converting the record to avro", e); | ||
} | ||
return null; | ||
})); | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,14 +19,17 @@ | |
package org.apache.hudi.hadoop.realtime; | ||
|
||
import org.apache.hudi.common.config.HoodieMemoryConfig; | ||
import org.apache.hudi.common.model.HoodieRecord; | ||
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.collection.ClosableIterator; | ||
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.common.util.queue.IteratorBasedQueueProducer; | ||
import org.apache.hudi.exception.HoodieException; | ||
import org.apache.hudi.hadoop.RecordReaderValueIterator; | ||
import org.apache.hudi.hadoop.SafeParquetRecordReaderWrapper; | ||
import org.apache.hudi.hadoop.fs.HadoopFSUtils; | ||
|
@@ -69,7 +72,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader | |
* @param realReader Parquet Reader | ||
*/ | ||
public RealtimeUnmergedRecordReader(RealtimeSplit split, JobConf job, | ||
RecordReader<NullWritable, ArrayWritable> realReader) { | ||
RecordReader<NullWritable, ArrayWritable> realReader) { | ||
super(split, job); | ||
this.parquetReader = new SafeParquetRecordReaderWrapper(realReader); | ||
// Iterator for consuming records from parquet file | ||
|
@@ -105,16 +108,18 @@ private List<HoodieProducer<ArrayWritable>> getParallelProducers( | |
) { | ||
return Arrays.asList( | ||
new FunctionBasedQueueProducer<>(queue -> { | ||
HoodieUnMergedLogRecordScanner scanner = | ||
scannerBuilder.withLogRecordScannerCallback(record -> { | ||
// convert Hoodie log record to Hadoop AvroWritable and buffer | ||
GenericRecord rec = (GenericRecord) record.toIndexedRecord(getReaderSchema(), payloadProps).get().getData(); | ||
ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(rec, getHiveSchema(), isSupportTimestamp()); | ||
queue.insertRecord(aWritable); | ||
}) | ||
.build(); | ||
// Scan all the delta-log files, filling in the queue | ||
scanner.scan(); | ||
HoodieUnMergedLogRecordScanner scanner = scannerBuilder.build(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nice refactoring, can we also fix the usage of There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Adjusted the usage within BoundedMemoryRecords in MergeOnReadInputFormat#getUnMergedLogFileIterator There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The |
||
try (ClosableIterator<HoodieRecord<?>> logRecordIterator = scanner.iterator()) { | ||
while (logRecordIterator.hasNext()) { | ||
HoodieRecord<?> record = logRecordIterator.next(); | ||
// convert Hoodie log record to Hadoop AvroWritable and buffer | ||
GenericRecord rec = (GenericRecord) record.toIndexedRecord(getReaderSchema(), payloadProps).get().getData(); | ||
ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(rec, getHiveSchema(), isSupportTimestamp()); | ||
queue.insertRecord(aWritable); | ||
} | ||
} catch (Exception e) { | ||
throw new HoodieException("Error converting the record to avro", e); | ||
} | ||
return null; | ||
}), | ||
new IteratorBasedQueueProducer<>(parquetRecordsIterator) | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe we remove this check because
hasNext
should always be invoked beforenext
.