1
0

[HUDI-2112] Support reading pure logs file group for flink batch reader after compaction (#3202)

This commit is contained in:
Danny Chan
2021-07-02 16:29:22 +08:00
committed by GitHub
parent 6403547431
commit 7462fdefc3
3 changed files with 40 additions and 36 deletions

View File

@@ -19,7 +19,7 @@
package org.apache.hudi.table;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.BaseFile;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -29,7 +29,6 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.hadoop.HoodieROTablePathFilter;
import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils;
import org.apache.hudi.source.StreamReadMonitoringFunction;
import org.apache.hudi.source.StreamReadOperator;
import org.apache.hudi.table.format.FilePathUtils;
@@ -272,39 +271,24 @@ public class HoodieTableSource implements
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants(), fileStatuses);
List<HoodieBaseFile> latestFiles = fsView.getLatestBaseFiles().collect(Collectors.toList());
String latestCommit = fsView.getLastInstant().get().getTimestamp();
final String mergeType = this.conf.getString(FlinkOptions.MERGE_TYPE);
final AtomicInteger cnt = new AtomicInteger(0);
if (latestFiles.size() > 0) {
Map<HoodieBaseFile, List<String>> fileGroup =
HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(hadoopConf, latestFiles);
return fileGroup.entrySet().stream().map(kv -> {
HoodieBaseFile baseFile = kv.getKey();
Option<List<String>> logPaths = kv.getValue().size() == 0
? Option.empty()
: Option.of(kv.getValue());
return new MergeOnReadInputSplit(cnt.getAndAdd(1),
baseFile.getPath(), logPaths, latestCommit,
metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, null);
}).collect(Collectors.toList());
} else {
// all the files are logs
return Arrays.stream(paths).map(partitionPath -> {
String relPartitionPath = FSUtils.getRelativePartitionPath(path, partitionPath);
return fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, latestCommit)
.map(fileSlice -> {
Option<List<String>> logPaths = Option.ofNullable(fileSlice.getLogFiles()
.sorted(HoodieLogFile.getLogFileComparator())
.map(logFile -> logFile.getPath().toString())
.collect(Collectors.toList()));
return new MergeOnReadInputSplit(cnt.getAndAdd(1),
null, logPaths, latestCommit,
metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, null);
}).collect(Collectors.toList()); })
.flatMap(Collection::stream)
.collect(Collectors.toList());
}
// generates one input split for each file group
return Arrays.stream(paths).map(partitionPath -> {
String relPartitionPath = FSUtils.getRelativePartitionPath(path, partitionPath);
return fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, latestCommit)
.map(fileSlice -> {
String basePath = fileSlice.getBaseFile().map(BaseFile::getPath).orElse(null);
Option<List<String>> logPaths = Option.ofNullable(fileSlice.getLogFiles()
.sorted(HoodieLogFile.getLogFileComparator())
.map(logFile -> logFile.getPath().toString())
.collect(Collectors.toList()));
return new MergeOnReadInputSplit(cnt.getAndAdd(1), basePath, logPaths, latestCommit,
metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, null);
}).collect(Collectors.toList()); })
.flatMap(Collection::stream)
.collect(Collectors.toList());
}
public InputFormat<RowData, ?> getInputFormat() {
@@ -431,11 +415,12 @@ public class HoodieTableSource implements
}
/**
* Reload the active timeline view.
* Reset the state of the table source.
*/
@VisibleForTesting
public void reloadActiveTimeline() {
public void reset() {
this.metaClient.reloadActiveTimeline();
this.requiredPartitions = null;
}
/**