1
0

[HUDI-4151] flink split_reader supports rocksdb (#5675)

* [HUDI-4151] flink split_reader supports rocksdb
This commit is contained in:
Bo Cui
2022-05-28 08:37:34 +08:00
committed by GitHub
parent 554caa3421
commit 93fe5a497e
2 changed files with 20 additions and 23 deletions

View File

@@ -30,6 +30,7 @@ import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer; import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer;
import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig; import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.util.StreamerUtil;
@@ -120,38 +121,34 @@ public class FormatUtils {
public static HoodieMergedLogRecordScanner logScanner( public static HoodieMergedLogRecordScanner logScanner(
MergeOnReadInputSplit split, MergeOnReadInputSplit split,
Schema logSchema, Schema logSchema,
Configuration config, org.apache.flink.configuration.Configuration flinkConf,
boolean withOperationField) { Configuration hadoopConf) {
FileSystem fs = FSUtils.getFs(split.getTablePath(), config); HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(flinkConf);
FileSystem fs = FSUtils.getFs(split.getTablePath(), hadoopConf);
return HoodieMergedLogRecordScanner.newBuilder() return HoodieMergedLogRecordScanner.newBuilder()
.withFileSystem(fs) .withFileSystem(fs)
.withBasePath(split.getTablePath()) .withBasePath(split.getTablePath())
.withLogFilePaths(split.getLogPaths().get()) .withLogFilePaths(split.getLogPaths().get())
.withReaderSchema(logSchema) .withReaderSchema(logSchema)
.withLatestInstantTime(split.getLatestCommit()) .withLatestInstantTime(split.getLatestCommit())
.withReadBlocksLazily( .withReadBlocksLazily(writeConfig.getCompactionLazyBlockReadEnabled())
string2Boolean(
config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)))
.withReverseReader(false) .withReverseReader(false)
.withBufferSize( .withBufferSize(writeConfig.getMaxDFSStreamBufferSize())
config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
.withMaxMemorySizeInBytes(split.getMaxCompactionMemoryInBytes()) .withMaxMemorySizeInBytes(split.getMaxCompactionMemoryInBytes())
.withSpillableMapBasePath( .withDiskMapType(writeConfig.getCommonConfig().getSpillableDiskMapType())
config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, .withSpillableMapBasePath(writeConfig.getSpillableMapBasePath())
HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH))
.withInstantRange(split.getInstantRange()) .withInstantRange(split.getInstantRange())
.withOperationField(withOperationField) .withOperationField(flinkConf.getBoolean(FlinkOptions.CHANGELOG_ENABLED))
.build(); .build();
} }
private static HoodieUnMergedLogRecordScanner unMergedLogScanner( private static HoodieUnMergedLogRecordScanner unMergedLogScanner(
MergeOnReadInputSplit split, MergeOnReadInputSplit split,
Schema logSchema, Schema logSchema,
Configuration config, org.apache.flink.configuration.Configuration flinkConf,
Configuration hadoopConf,
HoodieUnMergedLogRecordScanner.LogRecordScannerCallback callback) { HoodieUnMergedLogRecordScanner.LogRecordScannerCallback callback) {
FileSystem fs = FSUtils.getFs(split.getTablePath(), config); FileSystem fs = FSUtils.getFs(split.getTablePath(), hadoopConf);
return HoodieUnMergedLogRecordScanner.newBuilder() return HoodieUnMergedLogRecordScanner.newBuilder()
.withFileSystem(fs) .withFileSystem(fs)
.withBasePath(split.getTablePath()) .withBasePath(split.getTablePath())
@@ -160,11 +157,11 @@ public class FormatUtils {
.withLatestInstantTime(split.getLatestCommit()) .withLatestInstantTime(split.getLatestCommit())
.withReadBlocksLazily( .withReadBlocksLazily(
string2Boolean( string2Boolean(
config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, flinkConf.getString(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED))) HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)))
.withReverseReader(false) .withReverseReader(false)
.withBufferSize( .withBufferSize(
config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, flinkConf.getInteger(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
.withInstantRange(split.getInstantRange()) .withInstantRange(split.getInstantRange())
.withLogRecordScannerCallback(callback) .withLogRecordScannerCallback(callback)
@@ -198,7 +195,7 @@ public class FormatUtils {
Functions.noop()); Functions.noop());
// Consumer of this record reader // Consumer of this record reader
this.iterator = this.executor.getQueue().iterator(); this.iterator = this.executor.getQueue().iterator();
this.scanner = FormatUtils.unMergedLogScanner(split, logSchema, hadoopConf, this.scanner = FormatUtils.unMergedLogScanner(split, logSchema, flinkConf, hadoopConf,
record -> executor.getQueue().insertRecord(record)); record -> executor.getQueue().insertRecord(record));
// Start reading and buffering // Start reading and buffering
this.executor.startProducers(); this.executor.startProducers();

View File

@@ -192,6 +192,7 @@ public class MergeOnReadInputFormat
getLogFileIterator(split)); getLogFileIterator(split));
} else if (split.getMergeType().equals(FlinkOptions.REALTIME_PAYLOAD_COMBINE)) { } else if (split.getMergeType().equals(FlinkOptions.REALTIME_PAYLOAD_COMBINE)) {
this.iterator = new MergeIterator( this.iterator = new MergeIterator(
conf,
hadoopConf, hadoopConf,
split, split,
this.tableState.getRowType(), this.tableState.getRowType(),
@@ -200,7 +201,6 @@ public class MergeOnReadInputFormat
new Schema.Parser().parse(this.tableState.getRequiredAvroSchema()), new Schema.Parser().parse(this.tableState.getRequiredAvroSchema()),
this.requiredPos, this.requiredPos,
this.emitDelete, this.emitDelete,
this.conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED),
this.tableState.getOperationPos(), this.tableState.getOperationPos(),
getFullSchemaReader(split.getBasePath().get())); getFullSchemaReader(split.getBasePath().get()));
} else { } else {
@@ -323,7 +323,7 @@ public class MergeOnReadInputFormat
final GenericRecordBuilder recordBuilder = new GenericRecordBuilder(requiredSchema); final GenericRecordBuilder recordBuilder = new GenericRecordBuilder(requiredSchema);
final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter = final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter =
AvroToRowDataConverters.createRowConverter(tableState.getRequiredRowType()); AvroToRowDataConverters.createRowConverter(tableState.getRequiredRowType());
final HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(split, tableSchema, hadoopConf, conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED)); final HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(split, tableSchema, conf, hadoopConf);
final Iterator<String> logRecordsKeyIterator = scanner.getRecords().keySet().iterator(); final Iterator<String> logRecordsKeyIterator = scanner.getRecords().keySet().iterator();
final int[] pkOffset = tableState.getPkOffsetsInRequired(); final int[] pkOffset = tableState.getPkOffsetsInRequired();
// flag saying whether the pk semantics has been dropped by user specified // flag saying whether the pk semantics has been dropped by user specified
@@ -639,6 +639,7 @@ public class MergeOnReadInputFormat
private RowData currentRecord; private RowData currentRecord;
MergeIterator( MergeIterator(
Configuration finkConf,
org.apache.hadoop.conf.Configuration hadoopConf, org.apache.hadoop.conf.Configuration hadoopConf,
MergeOnReadInputSplit split, MergeOnReadInputSplit split,
RowType tableRowType, RowType tableRowType,
@@ -647,12 +648,11 @@ public class MergeOnReadInputFormat
Schema requiredSchema, Schema requiredSchema,
int[] requiredPos, int[] requiredPos,
boolean emitDelete, boolean emitDelete,
boolean withOperationField,
int operationPos, int operationPos,
ParquetColumnarRowSplitReader reader) { // the reader should be with full schema ParquetColumnarRowSplitReader reader) { // the reader should be with full schema
this.tableSchema = tableSchema; this.tableSchema = tableSchema;
this.reader = reader; this.reader = reader;
this.scanner = FormatUtils.logScanner(split, tableSchema, hadoopConf, withOperationField); this.scanner = FormatUtils.logScanner(split, tableSchema, finkConf, hadoopConf);
this.logKeysIterator = scanner.getRecords().keySet().iterator(); this.logKeysIterator = scanner.getRecords().keySet().iterator();
this.requiredSchema = requiredSchema; this.requiredSchema = requiredSchema;
this.requiredPos = requiredPos; this.requiredPos = requiredPos;