Skip to content
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -226,7 +226,7 @@ public Result inputSplits(
String basePath = fileSlice.getBaseFile().map(BaseFile::getPath).orElse(null);
return new MergeOnReadInputSplit(cnt.getAndAdd(1),
basePath, logPaths, endInstant,
metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, instantRange);
metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, instantRange, fileSlice.getFileId());
}).collect(Collectors.toList()))
Copy link
Contributor

Choose a reason for hiding this comment

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

instantRange,fileSlice.getFileId() -> instantRange, fileSlice.getFileId()

.flatMap(Collection::stream)
.collect(Collectors.toList());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.hudi.table;

import org.apache.flink.api.java.functions.KeySelector;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.model.BaseFile;
import org.apache.hudi.common.model.HoodieLogFile;
Expand Down Expand Up @@ -181,7 +182,8 @@ public DataStream<RowData> produceDataStream(StreamExecutionEnvironment execEnv)
OneInputStreamOperatorFactory<MergeOnReadInputSplit, RowData> factory = StreamReadOperator.factory((MergeOnReadInputFormat) inputFormat);
SingleOutputStreamOperator<RowData> source = execEnv.addSource(monitoringFunction, getSourceOperatorName("split_monitor"))
.setParallelism(1)
.transform("split_reader", typeInfo, factory)
.keyBy((KeySelector<MergeOnReadInputSplit, String>) mos -> mos.getFileId())
.transform("split_reader", typeInfo, factory)
Copy link
Contributor

Choose a reason for hiding this comment

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

Is the explicit cast necessary ?

Copy link
Contributor

Choose a reason for hiding this comment

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

Can the explicit KeySelector be removed ?

.setParallelism(conf.getInteger(FlinkOptions.READ_TASKS));
return new DataStreamSource<>(source);
} else {
Expand Down Expand Up @@ -316,7 +318,7 @@ private List<MergeOnReadInputSplit> buildFileIndex() {
.map(logFile -> logFile.getPath().toString())
.collect(Collectors.toList()));
return new MergeOnReadInputSplit(cnt.getAndAdd(1), basePath, logPaths, latestCommit,
metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, null);
metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, null, fileSlice.getFileId());
}).collect(Collectors.toList()))
Copy link
Contributor

Choose a reason for hiding this comment

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

null,fileSlice.getFileId() -> null, fileSlice.getFileId()

.flatMap(Collection::stream)
.collect(Collectors.toList());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,11 @@

package org.apache.hudi.table.format.mor;

import org.apache.flink.core.io.InputSplit;
import org.apache.hudi.common.table.log.InstantRange;
import org.apache.hudi.common.util.Option;

import org.apache.flink.core.io.InputSplit;

Copy link
Contributor

Choose a reason for hiding this comment

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

The changes for the import is not necessary.

import javax.annotation.Nullable;

import java.util.List;

/**
Expand All @@ -43,20 +41,25 @@ public class MergeOnReadInputSplit implements InputSplit {
private final long maxCompactionMemoryInBytes;
private final String mergeType;
private final Option<InstantRange> instantRange;
private String fileId;


// for streaming reader to record the consumed offset,
// which is the start of next round reading.
private long consumed = NUM_NO_CONSUMPTION;



public MergeOnReadInputSplit(
int splitNum,
@Nullable String basePath,
Option<List<String>> logPaths,
String latestCommit,
String tablePath,
long maxCompactionMemoryInBytes,
String mergeType,
@Nullable InstantRange instantRange) {
int splitNum,
@Nullable String basePath,
Option<List<String>> logPaths,
String latestCommit,
Copy link
Contributor

Choose a reason for hiding this comment

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

Fix the indentation

String tablePath,
long maxCompactionMemoryInBytes,
String mergeType,
@Nullable InstantRange instantRange,
String fileId) {
this.splitNum = splitNum;
this.basePath = Option.ofNullable(basePath);
this.logPaths = logPaths;
Expand All @@ -65,6 +68,15 @@ public MergeOnReadInputSplit(
this.maxCompactionMemoryInBytes = maxCompactionMemoryInBytes;
this.mergeType = mergeType;
this.instantRange = Option.ofNullable(instantRange);
this.fileId = fileId;
}

public String getFileId() {
return fileId;
}

public void setFileId(String fileId) {
this.fileId = fileId;
}

public Option<String> getBasePath() {
Expand Down