Skip to content

Commit 8911aa2

Browse files
[HUDI-7576] Improve efficiency of getRelativePartitionPath, reduce computation of partitionPath in AbstractTableFileSystemView (apache#11001)
1 parent 5e2290e commit 8911aa2

File tree

6 files changed

+66
-83
lines changed

6 files changed

+66
-83
lines changed

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -467,7 +467,7 @@ private boolean hasPendingFiles(String partitionPath) {
467467
try {
468468
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(hoodieTable.getMetaClient(), hoodieTable.getActiveTimeline());
469469
StoragePath fullPartitionPath = new StoragePath(hoodieTable.getMetaClient().getBasePathV2(), partitionPath);
470-
fsView.addFilesToView(FSUtils.getAllDataFilesInPartition(
470+
fsView.addFilesToView(partitionPath, FSUtils.getAllDataFilesInPartition(
471471
hoodieTable.getMetaClient().getStorage(), fullPartitionPath));
472472
// use #getAllFileGroups(partitionPath) instead of #getAllFileGroups() to exclude the replaced file groups.
473473
return fsView.getAllFileGroups(partitionPath).findAny().isPresent();

hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -469,9 +469,9 @@ public void testUpsertPartitionerWithSmallFileHandlingPickingMultipleCandidates(
469469
assertEquals(3, partitioner.numPartitions());
470470
assertEquals(
471471
Arrays.asList(
472-
new BucketInfo(BucketType.UPDATE, "fg-1", partitionPath),
472+
new BucketInfo(BucketType.UPDATE, "fg-3", partitionPath),
473473
new BucketInfo(BucketType.UPDATE, "fg-2", partitionPath),
474-
new BucketInfo(BucketType.UPDATE, "fg-3", partitionPath)
474+
new BucketInfo(BucketType.UPDATE, "fg-1", partitionPath)
475475
),
476476
partitioner.getBucketInfos());
477477
}

hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java

+4-5
Original file line numberDiff line numberDiff line change
@@ -234,17 +234,16 @@ public static String getRelativePartitionPath(StoragePath basePath, StoragePath
234234
fullPartitionPath = getPathWithoutSchemeAndAuthority(fullPartitionPath);
235235

236236
String fullPartitionPathStr = fullPartitionPath.toString();
237+
String basePathString = basePath.toString();
237238

238-
if (!fullPartitionPathStr.startsWith(basePath.toString())) {
239+
if (!fullPartitionPathStr.startsWith(basePathString)) {
239240
throw new IllegalArgumentException("Partition path \"" + fullPartitionPathStr
240241
+ "\" does not belong to base-path \"" + basePath + "\"");
241242
}
242243

243-
int partitionStartIndex = fullPartitionPathStr.indexOf(basePath.getName(),
244-
basePath.getParent() == null ? 0 : basePath.getParent().toString().length());
245244
// Partition-Path could be empty for non-partitioned tables
246-
return partitionStartIndex + basePath.getName().length() == fullPartitionPathStr.length() ? ""
247-
: fullPartitionPathStr.substring(partitionStartIndex + basePath.getName().length() + 1);
245+
return fullPartitionPathStr.length() == basePathString.length() ? ""
246+
: fullPartitionPathStr.substring(basePathString.length() + 1);
248247
}
249248

250249
public static StoragePath getPathWithoutSchemeAndAuthority(StoragePath path) {

hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java

+40-43
Original file line numberDiff line numberDiff line change
@@ -109,10 +109,6 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
109109

110110
private BootstrapIndex bootstrapIndex;
111111

112-
private String getPartitionPathFor(HoodieBaseFile baseFile) {
113-
return FSUtils.getRelativePartitionPath(metaClient.getBasePathV2(), baseFile.getStoragePath().getParent());
114-
}
115-
116112
/**
117113
* Initialize the view.
118114
*/
@@ -158,10 +154,21 @@ public Option<String> getCompletionTime(String instantTime) {
158154

159155
/**
160156
* Adds the provided statuses into the file system view, and also caches it inside this object.
157+
* If the file statuses are limited to a single partition, use {@link #addFilesToView(String, List)} instead.
161158
*/
162159
public List<HoodieFileGroup> addFilesToView(List<StoragePathInfo> statuses) {
160+
Map<String, List<StoragePathInfo>> statusesByPartitionPath = statuses.stream()
161+
.collect(Collectors.groupingBy(fileStatus -> FSUtils.getRelativePartitionPath(metaClient.getBasePathV2(), fileStatus.getPath().getParent())));
162+
return statusesByPartitionPath.entrySet().stream().map(entry -> addFilesToView(entry.getKey(), entry.getValue()))
163+
.flatMap(List::stream).collect(Collectors.toList());
164+
}
165+
166+
/**
167+
* Adds the provided statuses into the file system view for a single partition, and also caches it inside this object.
168+
*/
169+
public List<HoodieFileGroup> addFilesToView(String partitionPath, List<StoragePathInfo> statuses) {
163170
HoodieTimer timer = HoodieTimer.start();
164-
List<HoodieFileGroup> fileGroups = buildFileGroups(statuses, visibleCommitsAndCompactionTimeline, true);
171+
List<HoodieFileGroup> fileGroups = buildFileGroups(partitionPath, statuses, visibleCommitsAndCompactionTimeline, true);
165172
long fgBuildTimeTakenMs = timer.endTimer();
166173
timer.startTimer();
167174
// Group by partition for efficient updates for both InMemory and DiskBased structures.
@@ -191,37 +198,28 @@ public List<HoodieFileGroup> addFilesToView(List<StoragePathInfo> statuses) {
191198
/**
192199
* Build FileGroups from passed in file-status.
193200
*/
194-
protected List<HoodieFileGroup> buildFileGroups(List<StoragePathInfo> statuses, HoodieTimeline timeline,
201+
protected List<HoodieFileGroup> buildFileGroups(String partition, List<StoragePathInfo> statuses, HoodieTimeline timeline,
195202
boolean addPendingCompactionFileSlice) {
196-
return buildFileGroups(convertFileStatusesToBaseFiles(statuses), convertFileStatusesToLogFiles(statuses),
203+
return buildFileGroups(partition, convertFileStatusesToBaseFiles(statuses), convertFileStatusesToLogFiles(statuses),
197204
timeline,
198205
addPendingCompactionFileSlice);
199206
}
200207

201-
protected List<HoodieFileGroup> buildFileGroups(Stream<HoodieBaseFile> baseFileStream,
208+
protected List<HoodieFileGroup> buildFileGroups(String partition, Stream<HoodieBaseFile> baseFileStream,
202209
Stream<HoodieLogFile> logFileStream, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) {
203-
Map<Pair<String, String>, List<HoodieBaseFile>> baseFiles =
204-
baseFileStream.collect(Collectors.groupingBy(baseFile -> {
205-
String partitionPathStr = getPartitionPathFor(baseFile);
206-
return Pair.of(partitionPathStr, baseFile.getFileId());
207-
}));
208-
209-
Map<Pair<String, String>, List<HoodieLogFile>> logFiles = logFileStream.collect(Collectors.groupingBy((logFile) -> {
210-
String partitionPathStr =
211-
FSUtils.getRelativePartitionPath(metaClient.getBasePathV2(), logFile.getPath().getParent());
212-
return Pair.of(partitionPathStr, logFile.getFileId());
213-
}));
214-
215-
Set<Pair<String, String>> fileIdSet = new HashSet<>(baseFiles.keySet());
210+
Map<String, List<HoodieBaseFile>> baseFiles =
211+
baseFileStream.collect(Collectors.groupingBy(HoodieBaseFile::getFileId));
212+
213+
Map<String, List<HoodieLogFile>> logFiles = logFileStream.collect(Collectors.groupingBy(HoodieLogFile::getFileId));
214+
215+
Set<String> fileIdSet = new HashSet<>(baseFiles.keySet());
216216
fileIdSet.addAll(logFiles.keySet());
217217

218-
List<HoodieFileGroup> fileGroups = new ArrayList<>();
219-
fileIdSet.forEach(pair -> {
220-
String fileId = pair.getValue();
221-
String partitionPath = pair.getKey();
222-
HoodieFileGroup group = new HoodieFileGroup(partitionPath, fileId, timeline);
223-
if (baseFiles.containsKey(pair)) {
224-
baseFiles.get(pair).forEach(group::addBaseFile);
218+
List<HoodieFileGroup> fileGroups = new ArrayList<>(fileIdSet.size());
219+
fileIdSet.forEach(fileId -> {
220+
HoodieFileGroup group = new HoodieFileGroup(partition, fileId, timeline);
221+
if (baseFiles.containsKey(fileId)) {
222+
baseFiles.get(fileId).forEach(group::addBaseFile);
225223
}
226224
if (addPendingCompactionFileSlice) {
227225
// pending compaction file slice must be added before log files so that
@@ -235,8 +233,8 @@ protected List<HoodieFileGroup> buildFileGroups(Stream<HoodieBaseFile> baseFileS
235233
group.addNewFileSliceAtInstant(pendingCompaction.get().getKey());
236234
}
237235
}
238-
if (logFiles.containsKey(pair)) {
239-
logFiles.get(pair).stream().sorted(HoodieLogFile.getLogFileComparator()).forEach(logFile -> group.addLogFile(completionTimeQueryView, logFile));
236+
if (logFiles.containsKey(fileId)) {
237+
logFiles.get(fileId).stream().sorted(HoodieLogFile.getLogFileComparator()).forEach(logFile -> group.addLogFile(completionTimeQueryView, logFile));
240238
}
241239
fileGroups.add(group);
242240
});
@@ -379,9 +377,9 @@ private void ensurePartitionsLoadedCorrectly(List<String> partitionList) {
379377
LOG.debug("Time taken to list partitions " + partitionSet + " =" + (endLsTs - beginLsTs));
380378
pathInfoMap.forEach((partitionPair, statuses) -> {
381379
String relativePartitionStr = partitionPair.getLeft();
382-
List<HoodieFileGroup> groups = addFilesToView(statuses);
380+
List<HoodieFileGroup> groups = addFilesToView(relativePartitionStr, statuses);
383381
if (groups.isEmpty()) {
384-
storePartitionView(relativePartitionStr, new ArrayList<>());
382+
storePartitionView(relativePartitionStr, Collections.emptyList());
385383
}
386384
LOG.debug("#files found in partition (" + relativePartitionStr + ") =" + statuses.size());
387385
});
@@ -469,7 +467,7 @@ protected void ensurePartitionLoadedCorrectly(String partition) {
469467
// Not loaded yet
470468
try {
471469
LOG.info("Building file system view for partition (" + partitionPathStr + ")");
472-
List<HoodieFileGroup> groups = addFilesToView(getAllFilesInPartition(partitionPathStr));
470+
List<HoodieFileGroup> groups = addFilesToView(partitionPathStr, getAllFilesInPartition(partitionPathStr));
473471
if (groups.isEmpty()) {
474472
storePartitionView(partitionPathStr, new ArrayList<>());
475473
}
@@ -541,11 +539,10 @@ private Stream<HoodieLogFile> convertFileStatusesToLogFiles(List<StoragePathInfo
541539
* With async compaction, it is possible to see partial/complete base-files due to inflight-compactions, Ignore those
542540
* base-files.
543541
*
542+
* @param partitionPath partition path for the base file
544543
* @param baseFile base File
545544
*/
546-
protected boolean isBaseFileDueToPendingCompaction(HoodieBaseFile baseFile) {
547-
final String partitionPath = getPartitionPathFor(baseFile);
548-
545+
protected boolean isBaseFileDueToPendingCompaction(String partitionPath, HoodieBaseFile baseFile) {
549546
Option<Pair<String, CompactionOperation>> compactionWithInstantTime =
550547
getPendingCompactionOperationWithInstant(new HoodieFileGroupId(partitionPath, baseFile.getFileId()));
551548
return (compactionWithInstantTime.isPresent()) && (null != compactionWithInstantTime.get().getKey())
@@ -745,7 +742,7 @@ private Stream<HoodieBaseFile> getLatestBaseFilesBeforeOrOnFromCache(String part
745742
.map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllBaseFiles()
746743
.filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, maxCommitTime
747744
))
748-
.filter(df -> !isBaseFileDueToPendingCompaction(df) && !isBaseFileDueToPendingClustering(df)).findFirst()))
745+
.filter(df -> !isBaseFileDueToPendingCompaction(partitionPath, df) && !isBaseFileDueToPendingClustering(df)).findFirst()))
749746
.filter(Option::isPresent).map(Option::get)
750747
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df));
751748
}
@@ -761,7 +758,7 @@ public final Option<HoodieBaseFile> getBaseFileOn(String partitionStr, String in
761758
} else {
762759
return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllBaseFiles()
763760
.filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.EQUALS,
764-
instantTime)).filter(df -> !isBaseFileDueToPendingCompaction(df) && !isBaseFileDueToPendingClustering(df)).findFirst().orElse(null))
761+
instantTime)).filter(df -> !isBaseFileDueToPendingCompaction(partitionPath, df) && !isBaseFileDueToPendingClustering(df)).findFirst().orElse(null))
765762
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df));
766763
}
767764
} finally {
@@ -797,7 +794,7 @@ public final Stream<HoodieBaseFile> getLatestBaseFilesInRange(List<String> commi
797794
.filter(fileGroup -> !isFileGroupReplacedBeforeAny(fileGroup.getFileGroupId(), commitsToReturn))
798795
.map(fileGroup -> Pair.of(fileGroup.getFileGroupId(), Option.fromJavaOptional(
799796
fileGroup.getAllBaseFiles().filter(baseFile -> commitsToReturn.contains(baseFile.getCommitTime())
800-
&& !isBaseFileDueToPendingCompaction(baseFile) && !isBaseFileDueToPendingClustering(baseFile)).findFirst()))).filter(p -> p.getValue().isPresent())
797+
&& !isBaseFileDueToPendingCompaction(fileGroup.getPartitionPath(), baseFile) && !isBaseFileDueToPendingClustering(baseFile)).findFirst()))).filter(p -> p.getValue().isPresent())
801798
.map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get()));
802799
} finally {
803800
readLock.unlock();
@@ -833,7 +830,7 @@ public final Stream<HoodieBaseFile> getAllBaseFiles(String partitionStr) {
833830
return fetchAllBaseFiles(partitionPath)
834831
.filter(df -> !isFileGroupReplaced(partitionPath, df.getFileId()))
835832
.filter(df -> visibleCommitsAndCompactionTimeline.containsOrBeforeTimelineStarts(df.getCommitTime()))
836-
.filter(df -> !isBaseFileDueToPendingCompaction(df) && !isBaseFileDueToPendingClustering(df))
833+
.filter(df -> !isBaseFileDueToPendingCompaction(partitionPath, df) && !isBaseFileDueToPendingClustering(df))
837834
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df));
838835
} finally {
839836
readLock.unlock();
@@ -875,7 +872,7 @@ public final Stream<FileSlice> getLatestFileSlicesStateless(String partitionStr)
875872
return getLatestFileSlices(partition);
876873
} else {
877874
try {
878-
Stream<FileSlice> fileSliceStream = buildFileGroups(getAllFilesInPartition(partition), visibleCommitsAndCompactionTimeline, true).stream()
875+
Stream<FileSlice> fileSliceStream = buildFileGroups(partition, getAllFilesInPartition(partition), visibleCommitsAndCompactionTimeline, true).stream()
879876
.filter(fg -> !isFileGroupReplaced(fg))
880877
.map(HoodieFileGroup::getLatestFileSlice)
881878
.filter(Option::isPresent).map(Option::get)
@@ -1090,7 +1087,7 @@ public final Stream<HoodieFileGroup> getAllFileGroupsStateless(String partitionS
10901087
return getAllFileGroups(partition);
10911088
} else {
10921089
try {
1093-
Stream<HoodieFileGroup> fileGroupStream = buildFileGroups(getAllFilesInPartition(partition), visibleCommitsAndCompactionTimeline, true).stream()
1090+
Stream<HoodieFileGroup> fileGroupStream = buildFileGroups(partition, getAllFilesInPartition(partition), visibleCommitsAndCompactionTimeline, true).stream()
10941091
.filter(fg -> !isFileGroupReplaced(fg));
10951092
if (bootstrapIndex.useIndex()) {
10961093
final Map<HoodieFileGroupId, BootstrapBaseFileMapping> bootstrapBaseFileMappings = getBootstrapBaseFileMappings(partition);
@@ -1430,7 +1427,7 @@ public Stream<HoodieBaseFile> fetchLatestBaseFiles(final String partitionPath) {
14301427

14311428
protected Option<HoodieBaseFile> getLatestBaseFile(HoodieFileGroup fileGroup) {
14321429
return Option
1433-
.fromJavaOptional(fileGroup.getAllBaseFiles().filter(df -> !isBaseFileDueToPendingCompaction(df) && !isBaseFileDueToPendingClustering(df)).findFirst());
1430+
.fromJavaOptional(fileGroup.getAllBaseFiles().filter(df -> !isBaseFileDueToPendingCompaction(fileGroup.getPartitionPath(), df) && !isBaseFileDueToPendingClustering(df)).findFirst());
14341431
}
14351432

14361433
/**

hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java

+3-3
Original file line numberDiff line numberDiff line change
@@ -273,7 +273,7 @@ private void updatePartitionWriteFileGroups(Map<String, List<HoodieWriteStat>> p
273273
p.getFileSizeInBytes(), false, (short) 0, 0, 0))
274274
.collect(Collectors.toList());
275275
List<HoodieFileGroup> fileGroups =
276-
buildFileGroups(pathInfoList, timeline.filterCompletedAndCompactionInstants(), false);
276+
buildFileGroups(partition, pathInfoList, timeline.filterCompletedAndCompactionInstants(), false);
277277
applyDeltaFileSlicesToPartitionView(partition, fileGroups, DeltaApplyMode.ADD);
278278
} else {
279279
LOG.warn("Skipping partition (" + partition + ") when syncing instant (" + instant + ") as it is not loaded");
@@ -382,7 +382,7 @@ private void removeFileSlicesForPartition(HoodieTimeline timeline, HoodieInstant
382382
.map(p -> new StoragePathInfo(new StoragePath(p), 0, false, (short) 0, 0, 0))
383383
.collect(Collectors.toList());
384384
List<HoodieFileGroup> fileGroups =
385-
buildFileGroups(pathInfoList, timeline.filterCompletedAndCompactionInstants(), false);
385+
buildFileGroups(partition, pathInfoList, timeline.filterCompletedAndCompactionInstants(), false);
386386
applyDeltaFileSlicesToPartitionView(partition, fileGroups, DeltaApplyMode.REMOVE);
387387
} else {
388388
LOG.warn("Skipping partition (" + partition + ") when syncing instant (" + instant + ") as it is not loaded");
@@ -451,7 +451,7 @@ protected void applyDeltaFileSlicesToPartitionView(String partition, List<Hoodie
451451

452452
HoodieTimeline timeline = deltaFileGroups.stream().map(df -> df.getTimeline()).findAny().get();
453453
List<HoodieFileGroup> fgs =
454-
buildFileGroups(viewDataFiles.values().stream(), viewLogFiles.values().stream(), timeline, true);
454+
buildFileGroups(partition, viewDataFiles.values().stream(), viewLogFiles.values().stream(), timeline, true);
455455
storePartitionView(partition, fgs);
456456
}
457457

hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java

+16-29
Original file line numberDiff line numberDiff line change
@@ -46,6 +46,8 @@
4646
import org.junit.jupiter.api.AfterEach;
4747
import org.junit.jupiter.api.BeforeEach;
4848
import org.junit.jupiter.api.Test;
49+
import org.junit.jupiter.params.ParameterizedTest;
50+
import org.junit.jupiter.params.provider.CsvSource;
4951

5052
import java.io.IOException;
5153
import java.nio.file.Files;
@@ -204,35 +206,20 @@ public void testGetRelativePartitionPath() {
204206
assertThrows(IllegalArgumentException.class, () -> FSUtils.getRelativePartitionPath(basePath, nonPartitionPath));
205207
}
206208

207-
@Test
208-
public void testGetRelativePartitionPathWithStoragePath() {
209-
StoragePath basePath = new StoragePath("/test/apache");
210-
StoragePath partitionPath = new StoragePath("/test/apache/hudi/sub");
211-
assertEquals("hudi/sub", FSUtils.getRelativePartitionPath(basePath, partitionPath));
212-
213-
StoragePath nonPartitionPath = new StoragePath("/test/something/else");
214-
assertThrows(IllegalArgumentException.class, () -> FSUtils.getRelativePartitionPath(basePath, nonPartitionPath));
215-
}
216-
217-
@Test
218-
public void testGetRelativePartitionPathSameFolder() {
219-
Path basePath = new Path("/test");
220-
Path partitionPath = new Path("/test");
221-
assertEquals("", FSUtils.getRelativePartitionPath(basePath, partitionPath));
222-
}
223-
224-
@Test
225-
public void testGetRelativePartitionPathRepeatedFolderNameBasePath() {
226-
Path basePath = new Path("/test/apache/apache");
227-
Path partitionPath = new Path("/test/apache/apache/hudi");
228-
assertEquals("hudi", FSUtils.getRelativePartitionPath(basePath, partitionPath));
229-
}
230-
231-
@Test
232-
public void testGetRelativePartitionPathRepeatedFolderNamePartitionPath() {
233-
Path basePath = new Path("/test/apache");
234-
Path partitionPath = new Path("/test/apache/apache/hudi");
235-
assertEquals("apache/hudi", FSUtils.getRelativePartitionPath(basePath, partitionPath));
209+
@ParameterizedTest
210+
@CsvSource({
211+
"/test,/test,",
212+
"s3://test,s3://test,",
213+
"s3://test/foo,s3://test/foo,",
214+
"/test/foo,/test/foo,",
215+
"/test/apache/apache,/test/apache/apache/hudi,hudi",
216+
"/test/apache,/test/apache/hudi,hudi",
217+
"s3://test/apache,s3://test/apache/apache/hudi,apache/hudi"})
218+
public void testGetRelativePartitionPath(String basePathStr, String partitionPathStr, String expected) {
219+
StoragePath basePath = new StoragePath(basePathStr);
220+
StoragePath partitionPath = new StoragePath(partitionPathStr);
221+
String result = FSUtils.getRelativePartitionPath(basePath, partitionPath);
222+
assertEquals(expected == null ? "" : expected, result);
236223
}
237224

238225
@Test

0 commit comments

Comments
 (0)