1
0

[HUDI-2550] Expand File-Group candidates list for appending for MOR tables (#3986)

This commit is contained in:
Alexey Kudinkin
2021-11-22 16:19:59 -08:00
committed by GitHub
parent fe57e9beea
commit 3bdab01a49
4 changed files with 124 additions and 57 deletions

View File

@@ -288,6 +288,10 @@ public class UpsertPartitioner<T extends HoodieRecordPayload<T>> extends Partiti
return smallFileLocations;
}
public List<BucketInfo> getBucketInfos() {
return Collections.unmodifiableList(new ArrayList<>(bucketInfoMap.values()));
}
public BucketInfo getBucketInfo(int bucketNumber) {
return bucketInfoMap.get(bucketNumber);
}

View File

@@ -26,15 +26,16 @@ import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.WorkloadProfile;
import org.apache.hudi.table.action.commit.SmallFile;
import org.apache.hudi.table.action.commit.UpsertPartitioner;
import javax.annotation.Nonnull;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.stream.Collectors;
@@ -51,68 +52,68 @@ public class SparkUpsertDeltaCommitPartitioner<T extends HoodieRecordPayload<T>>
@Override
protected List<SmallFile> getSmallFiles(String partitionPath) {
// smallFiles only for partitionPath
List<SmallFile> smallFileLocations = new ArrayList<>();
// Init here since this class (and member variables) might not have been initialized
HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
// Find out all eligible small file slices
if (!commitTimeline.empty()) {
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
// find smallest file in partition and append to it
List<FileSlice> allSmallFileSlices = new ArrayList<>();
// If we cannot index log files, then we choose the smallest parquet file in the partition and add inserts to
// it. Doing this overtime for a partition, we ensure that we handle small file issues
if (!table.getIndex().canIndexLogFiles()) {
// TODO : choose last N small files since there can be multiple small files written to a single partition
// by different spark partitions in a single batch
Option<FileSlice> smallFileSlice = Option.fromJavaOptional(table.getSliceView()
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false)
.filter(
fileSlice -> fileSlice.getLogFiles().count() < 1 && fileSlice.getBaseFile().get().getFileSize() < config
.getParquetSmallFileLimit())
.min((FileSlice left, FileSlice right) ->
left.getBaseFile().get().getFileSize() < right.getBaseFile().get().getFileSize() ? -1 : 1));
if (smallFileSlice.isPresent()) {
allSmallFileSlices.add(smallFileSlice.get());
}
if (commitTimeline.empty()) {
return Collections.emptyList();
}
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
// Find out all eligible small file slices, looking for
// smallest file in the partition to append to
List<FileSlice> smallFileSlicesCandidates = getSmallFileCandidates(partitionPath, latestCommitTime);
List<SmallFile> smallFileLocations = new ArrayList<>();
// Create SmallFiles from the eligible file slices
for (FileSlice smallFileSlice : smallFileSlicesCandidates) {
SmallFile sf = new SmallFile();
if (smallFileSlice.getBaseFile().isPresent()) {
// TODO : Move logic of file name, file id, base commit time handling inside file slice
String filename = smallFileSlice.getBaseFile().get().getFileName();
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename));
sf.sizeBytes = getTotalFileSize(smallFileSlice);
smallFileLocations.add(sf);
} else {
// If we can index log files, we can add more inserts to log files for fileIds NOT including those under
// pending compaction
List<FileSlice> allFileSlices =
table.getSliceView().getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false)
.collect(Collectors.toList());
for (FileSlice fileSlice : allFileSlices) {
if (isSmallFile(fileSlice)) {
allSmallFileSlices.add(fileSlice);
}
}
}
// Create SmallFiles from the eligible file slices
for (FileSlice smallFileSlice : allSmallFileSlices) {
SmallFile sf = new SmallFile();
if (smallFileSlice.getBaseFile().isPresent()) {
// TODO : Move logic of file name, file id, base commit time handling inside file slice
String filename = smallFileSlice.getBaseFile().get().getFileName();
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename));
sf.sizeBytes = getTotalFileSize(smallFileSlice);
smallFileLocations.add(sf);
} else {
HoodieLogFile logFile = smallFileSlice.getLogFiles().findFirst().get();
sf.location = new HoodieRecordLocation(FSUtils.getBaseCommitTimeFromLogPath(logFile.getPath()),
FSUtils.getFileIdFromLogPath(logFile.getPath()));
sf.sizeBytes = getTotalFileSize(smallFileSlice);
smallFileLocations.add(sf);
}
HoodieLogFile logFile = smallFileSlice.getLogFiles().findFirst().get();
sf.location = new HoodieRecordLocation(FSUtils.getBaseCommitTimeFromLogPath(logFile.getPath()),
FSUtils.getFileIdFromLogPath(logFile.getPath()));
sf.sizeBytes = getTotalFileSize(smallFileSlice);
smallFileLocations.add(sf);
}
}
return smallFileLocations;
}
@Nonnull
private List<FileSlice> getSmallFileCandidates(String partitionPath, HoodieInstant latestCommitInstant) {
// If we can index log files, we can add more inserts to log files for fileIds NOT including those under
// pending compaction
if (table.getIndex().canIndexLogFiles()) {
return table.getSliceView()
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitInstant.getTimestamp(), false)
.filter(this::isSmallFile)
.collect(Collectors.toList());
}
// If we cannot index log files, then we choose the smallest parquet file in the partition and add inserts to
// it. Doing this overtime for a partition, we ensure that we handle small file issues
return table.getSliceView()
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitInstant.getTimestamp(), false)
.filter(
fileSlice ->
// NOTE: We can not pad slices with existing log-files w/o compacting these,
// hence skipping
fileSlice.getLogFiles().count() < 1
&& fileSlice.getBaseFile().get().getFileSize() < config.getParquetSmallFileLimit())
.sorted(Comparator.comparing(fileSlice -> fileSlice.getBaseFile().get().getFileSize()))
.limit(config.getSmallFileGroupCandidatesLimit())
.collect(Collectors.toList());
}
public List<String> getSmallFileIds() {
return (List<String>) smallFiles.stream().map(smallFile -> ((SmallFile) smallFile).location.getFileId())
return smallFiles.stream().map(smallFile -> smallFile.location.getFileId())
.collect(Collectors.toList());
}
@@ -132,8 +133,12 @@ public class SparkUpsertDeltaCommitPartitioner<T extends HoodieRecordPayload<T>>
// TODO (NA) : Make this static part of utility
public long convertLogFilesSizeToExpectedParquetSize(List<HoodieLogFile> hoodieLogFiles) {
long totalSizeOfLogFiles = hoodieLogFiles.stream().map(HoodieLogFile::getFileSize)
.filter(size -> size > 0).reduce(Long::sum).orElse(0L);
long totalSizeOfLogFiles =
hoodieLogFiles.stream()
.map(HoodieLogFile::getFileSize)
.filter(size -> size > 0)
.reduce(Long::sum)
.orElse(0L);
// Here we assume that if there is no base parquet file, all log files contain only inserts.
// We can then just get the parquet equivalent size of these log files, compare that with
// {@link config.getParquetMaxFileSize()} and decide if there is scope to insert more rows