|
|
|
|
@@ -18,6 +18,7 @@
|
|
|
|
|
|
|
|
|
|
package org.apache.hudi.source;
|
|
|
|
|
|
|
|
|
|
import org.apache.hudi.common.fs.FSUtils;
|
|
|
|
|
import org.apache.hudi.common.model.BaseFile;
|
|
|
|
|
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
|
|
|
|
import org.apache.hudi.common.model.HoodieLogFile;
|
|
|
|
|
@@ -29,20 +30,24 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
|
|
|
|
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
|
|
|
|
import org.apache.hudi.common.util.Option;
|
|
|
|
|
import org.apache.hudi.configuration.FlinkOptions;
|
|
|
|
|
import org.apache.hudi.configuration.OptionsResolver;
|
|
|
|
|
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
|
|
|
|
import org.apache.hudi.sink.partitioner.profile.WriteProfiles;
|
|
|
|
|
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
|
|
|
|
import org.apache.hudi.util.StreamerUtil;
|
|
|
|
|
|
|
|
|
|
import org.apache.flink.configuration.Configuration;
|
|
|
|
|
import org.apache.flink.core.fs.Path;
|
|
|
|
|
import org.apache.flink.table.types.logical.RowType;
|
|
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
|
|
import org.slf4j.Logger;
|
|
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
|
|
|
|
import javax.annotation.Nullable;
|
|
|
|
|
|
|
|
|
|
import java.util.ArrayList;
|
|
|
|
|
import java.util.Arrays;
|
|
|
|
|
import java.util.Collection;
|
|
|
|
|
import java.util.Collections;
|
|
|
|
|
import java.util.HashSet;
|
|
|
|
|
@@ -117,7 +122,102 @@ public class IncrementalInputSplits implements Serializable {
|
|
|
|
|
public Result inputSplits(
|
|
|
|
|
HoodieTableMetaClient metaClient,
|
|
|
|
|
org.apache.hadoop.conf.Configuration hadoopConf) {
|
|
|
|
|
return inputSplits(metaClient, hadoopConf, null);
|
|
|
|
|
HoodieTimeline commitTimeline = metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants();
|
|
|
|
|
if (commitTimeline.empty()) {
|
|
|
|
|
LOG.warn("No splits found for the table under path " + path);
|
|
|
|
|
return Result.EMPTY;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
final String startCommit = this.conf.getString(FlinkOptions.READ_START_COMMIT);
|
|
|
|
|
final String endCommit = this.conf.getString(FlinkOptions.READ_END_COMMIT);
|
|
|
|
|
final boolean startFromEarliest = FlinkOptions.START_COMMIT_EARLIEST.equalsIgnoreCase(startCommit);
|
|
|
|
|
final boolean startOutOfRange = startCommit != null && commitTimeline.isBeforeTimelineStarts(startCommit);
|
|
|
|
|
final boolean endOutOfRange = endCommit != null && commitTimeline.isBeforeTimelineStarts(endCommit);
|
|
|
|
|
boolean fullTableScan = startFromEarliest || startOutOfRange || endOutOfRange;
|
|
|
|
|
|
|
|
|
|
// Step1: find out the files to read, tries to read the files from the commit metadata first,
|
|
|
|
|
// fallback to full table scan if any of the following conditions matches:
|
|
|
|
|
// 1. there are files in metadata be deleted;
|
|
|
|
|
// 2. read from earliest
|
|
|
|
|
// 3. the start commit is archived
|
|
|
|
|
// 4. the end commit is archived
|
|
|
|
|
Set<String> readPartitions;
|
|
|
|
|
final FileStatus[] fileStatuses;
|
|
|
|
|
List<HoodieInstant> instants = filterInstantsWithRange(commitTimeline, null);
|
|
|
|
|
if (fullTableScan) {
|
|
|
|
|
// scans the partitions and files directly.
|
|
|
|
|
FileIndex fileIndex = getFileIndex();
|
|
|
|
|
readPartitions = new HashSet<>(fileIndex.getOrBuildPartitionPaths());
|
|
|
|
|
if (readPartitions.size() == 0) {
|
|
|
|
|
LOG.warn("No partitions found for reading in user provided path.");
|
|
|
|
|
return Result.EMPTY;
|
|
|
|
|
}
|
|
|
|
|
fileStatuses = fileIndex.getFilesInPartitions();
|
|
|
|
|
} else {
|
|
|
|
|
if (instants.size() == 0) {
|
|
|
|
|
LOG.info("No new instant found for the table under path " + path + ", skip reading");
|
|
|
|
|
return Result.EMPTY;
|
|
|
|
|
}
|
|
|
|
|
String tableName = conf.getString(FlinkOptions.TABLE_NAME);
|
|
|
|
|
List<HoodieCommitMetadata> metadataList = instants.stream()
|
|
|
|
|
.map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, commitTimeline)).collect(Collectors.toList());
|
|
|
|
|
readPartitions = getReadPartitions(metadataList);
|
|
|
|
|
if (readPartitions.size() == 0) {
|
|
|
|
|
LOG.warn("No partitions found for reading in user provided path.");
|
|
|
|
|
return Result.EMPTY;
|
|
|
|
|
}
|
|
|
|
|
FileStatus[] files = WriteProfiles.getRawWritePathsOfInstants(path, hadoopConf, metadataList, metaClient.getTableType());
|
|
|
|
|
FileSystem fs = FSUtils.getFs(path.toString(), hadoopConf);
|
|
|
|
|
if (Arrays.stream(files).anyMatch(fileStatus -> !StreamerUtil.fileExists(fs, fileStatus.getPath()))) {
|
|
|
|
|
LOG.warn("Found deleted files in metadata, fall back to full table scan.");
|
|
|
|
|
// fallback to full table scan
|
|
|
|
|
fullTableScan = true;
|
|
|
|
|
// reading from the earliest, scans the partitions and files directly.
|
|
|
|
|
FileIndex fileIndex = getFileIndex();
|
|
|
|
|
readPartitions = new HashSet<>(fileIndex.getOrBuildPartitionPaths());
|
|
|
|
|
if (readPartitions.size() == 0) {
|
|
|
|
|
LOG.warn("No partitions found for reading in user provided path.");
|
|
|
|
|
return Result.EMPTY;
|
|
|
|
|
}
|
|
|
|
|
fileStatuses = fileIndex.getFilesInPartitions();
|
|
|
|
|
} else {
|
|
|
|
|
fileStatuses = files;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (fileStatuses.length == 0) {
|
|
|
|
|
LOG.warn("No files found for reading in user provided path.");
|
|
|
|
|
return Result.EMPTY;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Step2: generates the instant range
|
|
|
|
|
// if the specified end commit is archived, still uses the specified timestamp,
|
|
|
|
|
// else uses the latest filtered instant time
|
|
|
|
|
// (would be the latest instant time if the specified end commit is greater than the latest instant time)
|
|
|
|
|
final String rangeEnd = endOutOfRange ? endCommit : instants.get(instants.size() - 1).getTimestamp();
|
|
|
|
|
// keep the same semantics with streaming read, default start from the latest commit
|
|
|
|
|
final String rangeStart = startFromEarliest ? null : (startCommit == null ? rangeEnd : startCommit);
|
|
|
|
|
final InstantRange instantRange;
|
|
|
|
|
if (!fullTableScan) {
|
|
|
|
|
instantRange = InstantRange.builder().startInstant(rangeStart).endInstant(rangeEnd)
|
|
|
|
|
.rangeType(InstantRange.RangeType.CLOSE_CLOSE).build();
|
|
|
|
|
} else if (startFromEarliest && endCommit == null) {
|
|
|
|
|
// short-cut for snapshot read
|
|
|
|
|
instantRange = null;
|
|
|
|
|
} else {
|
|
|
|
|
instantRange = InstantRange.builder().startInstant(rangeStart).endInstant(rangeEnd)
|
|
|
|
|
.rangeType(InstantRange.RangeType.CLOSE_CLOSE).nullableBoundary(true).build();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Step3: decides the read end commit
|
|
|
|
|
final String endInstant = fullTableScan
|
|
|
|
|
? commitTimeline.lastInstant().get().getTimestamp()
|
|
|
|
|
: instants.get(instants.size() - 1).getTimestamp();
|
|
|
|
|
|
|
|
|
|
List<MergeOnReadInputSplit> inputSplits = getInputSplits(metaClient, commitTimeline,
|
|
|
|
|
fileStatuses, readPartitions, endInstant, instantRange);
|
|
|
|
|
|
|
|
|
|
return Result.instance(inputSplits, endInstant);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
@@ -146,18 +246,19 @@ public class IncrementalInputSplits implements Serializable {
|
|
|
|
|
if (issuedInstant != null) {
|
|
|
|
|
// the streaming reader may record the last issued instant, if the issued instant is present,
|
|
|
|
|
// the instant range should be: (issued instant, the latest instant].
|
|
|
|
|
instantRange = InstantRange.getInstance(issuedInstant, instantToIssue.getTimestamp(),
|
|
|
|
|
InstantRange.RangeType.OPEN_CLOSE);
|
|
|
|
|
instantRange = InstantRange.builder().startInstant(issuedInstant).endInstant(instantToIssue.getTimestamp())
|
|
|
|
|
.rangeType(InstantRange.RangeType.OPEN_CLOSE).build();
|
|
|
|
|
} else if (this.conf.getOptional(FlinkOptions.READ_START_COMMIT).isPresent()) {
|
|
|
|
|
// first time consume and has a start commit
|
|
|
|
|
final String startCommit = this.conf.getString(FlinkOptions.READ_START_COMMIT);
|
|
|
|
|
instantRange = startCommit.equalsIgnoreCase(FlinkOptions.START_COMMIT_EARLIEST)
|
|
|
|
|
? null
|
|
|
|
|
: InstantRange.getInstance(startCommit, instantToIssue.getTimestamp(), InstantRange.RangeType.CLOSE_CLOSE);
|
|
|
|
|
: InstantRange.builder().startInstant(startCommit).endInstant(instantToIssue.getTimestamp())
|
|
|
|
|
.rangeType(InstantRange.RangeType.CLOSE_CLOSE).build();
|
|
|
|
|
} else {
|
|
|
|
|
// first time consume and no start commit, consumes the latest incremental data set.
|
|
|
|
|
instantRange = InstantRange.getInstance(instantToIssue.getTimestamp(), instantToIssue.getTimestamp(),
|
|
|
|
|
InstantRange.RangeType.CLOSE_CLOSE);
|
|
|
|
|
instantRange = InstantRange.builder().startInstant(instantToIssue.getTimestamp()).endInstant(instantToIssue.getTimestamp())
|
|
|
|
|
.rangeType(InstantRange.RangeType.CLOSE_CLOSE).build();
|
|
|
|
|
}
|
|
|
|
|
} else {
|
|
|
|
|
LOG.info("No new instant found for the table under path " + path + ", skip reading");
|
|
|
|
|
@@ -166,18 +267,14 @@ public class IncrementalInputSplits implements Serializable {
|
|
|
|
|
|
|
|
|
|
String tableName = conf.getString(FlinkOptions.TABLE_NAME);
|
|
|
|
|
|
|
|
|
|
Set<String> writePartitions;
|
|
|
|
|
Set<String> readPartitions;
|
|
|
|
|
final FileStatus[] fileStatuses;
|
|
|
|
|
|
|
|
|
|
if (instantRange == null) {
|
|
|
|
|
// reading from the earliest, scans the partitions and files directly.
|
|
|
|
|
FileIndex fileIndex = FileIndex.instance(new org.apache.hadoop.fs.Path(path.toUri()), conf, rowType);
|
|
|
|
|
if (this.requiredPartitions != null) {
|
|
|
|
|
// apply partition push down
|
|
|
|
|
fileIndex.setPartitionPaths(this.requiredPartitions);
|
|
|
|
|
}
|
|
|
|
|
writePartitions = new HashSet<>(fileIndex.getOrBuildPartitionPaths());
|
|
|
|
|
if (writePartitions.size() == 0) {
|
|
|
|
|
FileIndex fileIndex = getFileIndex();
|
|
|
|
|
readPartitions = new HashSet<>(fileIndex.getOrBuildPartitionPaths());
|
|
|
|
|
if (readPartitions.size() == 0) {
|
|
|
|
|
LOG.warn("No partitions found for reading in user provided path.");
|
|
|
|
|
return Result.EMPTY;
|
|
|
|
|
}
|
|
|
|
|
@@ -198,13 +295,8 @@ public class IncrementalInputSplits implements Serializable {
|
|
|
|
|
? mergeList(archivedMetadataList, activeMetadataList)
|
|
|
|
|
: activeMetadataList;
|
|
|
|
|
|
|
|
|
|
writePartitions = HoodieInputFormatUtils.getWritePartitionPaths(metadataList);
|
|
|
|
|
// apply partition push down
|
|
|
|
|
if (this.requiredPartitions != null) {
|
|
|
|
|
writePartitions = writePartitions.stream()
|
|
|
|
|
.filter(this.requiredPartitions::contains).collect(Collectors.toSet());
|
|
|
|
|
}
|
|
|
|
|
if (writePartitions.size() == 0) {
|
|
|
|
|
readPartitions = getReadPartitions(metadataList);
|
|
|
|
|
if (readPartitions.size() == 0) {
|
|
|
|
|
LOG.warn("No partitions found for reading in user provided path.");
|
|
|
|
|
return Result.EMPTY;
|
|
|
|
|
}
|
|
|
|
|
@@ -216,11 +308,24 @@ public class IncrementalInputSplits implements Serializable {
|
|
|
|
|
return Result.EMPTY;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, commitTimeline, fileStatuses);
|
|
|
|
|
final String endInstant = instantToIssue.getTimestamp();
|
|
|
|
|
List<MergeOnReadInputSplit> inputSplits = getInputSplits(metaClient, commitTimeline,
|
|
|
|
|
fileStatuses, readPartitions, endInstant, instantRange);
|
|
|
|
|
|
|
|
|
|
return Result.instance(inputSplits, endInstant);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private List<MergeOnReadInputSplit> getInputSplits(
|
|
|
|
|
HoodieTableMetaClient metaClient,
|
|
|
|
|
HoodieTimeline commitTimeline,
|
|
|
|
|
FileStatus[] fileStatuses,
|
|
|
|
|
Set<String> readPartitions,
|
|
|
|
|
String endInstant,
|
|
|
|
|
InstantRange instantRange) {
|
|
|
|
|
final HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, commitTimeline, fileStatuses);
|
|
|
|
|
final AtomicInteger cnt = new AtomicInteger(0);
|
|
|
|
|
final String mergeType = this.conf.getString(FlinkOptions.MERGE_TYPE);
|
|
|
|
|
List<MergeOnReadInputSplit> inputSplits = writePartitions.stream()
|
|
|
|
|
return readPartitions.stream()
|
|
|
|
|
.map(relPartitionPath -> fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, endInstant)
|
|
|
|
|
.map(fileSlice -> {
|
|
|
|
|
Option<List<String>> logPaths = Option.ofNullable(fileSlice.getLogFiles()
|
|
|
|
|
@@ -234,7 +339,32 @@ public class IncrementalInputSplits implements Serializable {
|
|
|
|
|
}).collect(Collectors.toList()))
|
|
|
|
|
.flatMap(Collection::stream)
|
|
|
|
|
.collect(Collectors.toList());
|
|
|
|
|
return Result.instance(inputSplits, endInstant);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private FileIndex getFileIndex() {
|
|
|
|
|
FileIndex fileIndex = FileIndex.instance(new org.apache.hadoop.fs.Path(path.toUri()), conf, rowType);
|
|
|
|
|
if (this.requiredPartitions != null) {
|
|
|
|
|
// apply partition push down
|
|
|
|
|
fileIndex.setPartitionPaths(this.requiredPartitions);
|
|
|
|
|
}
|
|
|
|
|
return fileIndex;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Returns the partitions to read with given metadata list.
|
|
|
|
|
* The partitions would be filtered by the pushed down required partitions.
|
|
|
|
|
*
|
|
|
|
|
* @param metadataList The metadata list
|
|
|
|
|
* @return the set of read partitions
|
|
|
|
|
*/
|
|
|
|
|
private Set<String> getReadPartitions(List<HoodieCommitMetadata> metadataList) {
|
|
|
|
|
Set<String> partitions = HoodieInputFormatUtils.getWritePartitionPaths(metadataList);
|
|
|
|
|
// apply partition push down
|
|
|
|
|
if (this.requiredPartitions != null) {
|
|
|
|
|
return partitions.stream()
|
|
|
|
|
.filter(this.requiredPartitions::contains).collect(Collectors.toSet());
|
|
|
|
|
}
|
|
|
|
|
return partitions;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
@@ -287,8 +417,7 @@ public class IncrementalInputSplits implements Serializable {
|
|
|
|
|
|
|
|
|
|
Stream<HoodieInstant> instantStream = completedTimeline.getInstants();
|
|
|
|
|
|
|
|
|
|
if (this.conf.getOptional(FlinkOptions.READ_START_COMMIT).isPresent()
|
|
|
|
|
&& !this.conf.get(FlinkOptions.READ_START_COMMIT).equalsIgnoreCase(FlinkOptions.START_COMMIT_EARLIEST)) {
|
|
|
|
|
if (OptionsResolver.isSpecificStartCommit(this.conf)) {
|
|
|
|
|
final String startCommit = this.conf.get(FlinkOptions.READ_START_COMMIT);
|
|
|
|
|
instantStream = instantStream
|
|
|
|
|
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN_OR_EQUALS, startCommit));
|
|
|
|
|
|