ForinMH commented on issue #13534:
URL: https://github.com/apache/hudi/issues/13534#issuecomment-3055408961
List<HoodieCompactionOperation> operations = engineContext
.flatMap(partitionPaths, partitionPath -> fileSystemView
.getLatestFileSlicesStateless(partitionPath)
.filter(slice -> filterFileSlice(slice,
lastCompletedInstantTime, fgIdsInPendingCompactionAndClustering, instantRange))
.flatMap(s -> {
List<HoodieLogFile> logFiles = s.getLogFiles()
//
==============================================================
// IMPORTANT
//
==============================================================
// Currently, our filesystem view could
return a file slice with pending log files there,
// these files should be excluded from
the plan, let's say we have such a sequence of actions
// t10: a delta commit starts,
// t20: the compaction is scheduled and
the t10 delta commit is still pending, and the "fg_10.log" is included in the
plan
// t25: the delta commit 10 finishes,
// t30: the compaction execution starts,
now the reader considers the log file "fg_10.log" as valid.
// for both OCC and NB-CC, this is
in-correct.
.filter(logFile ->
completionTimeQueryView.isCompletedBefore(compactionInstant,
logFile.getDeltaCommitTime()))
.sorted(HoodieLogFile.getLogFileComparator())
.collect(toList());
Option<HoodieBaseFile> dataFile =
s.getBaseFile();
// FIX: skip empty logs
if (logFiles.isEmpty()) {
return Stream.empty();
}
totalLogFiles.add(logFiles.size());
totalFileSlices.add(1L);
// Avro generated classes are not inheriting
Serializable. Using CompactionOperation POJO
// for Map operations and collecting them
finally in Avro generated classes for storing
// into meta files.
return Stream.of(new
CompactionOperation(dataFile, partitionPath, logFiles,
writeConfig.getCompactionStrategy().captureMetrics(writeConfig, s)));
}), partitionPaths.size())
.stream()
.map(CompactionUtils::buildHoodieCompactionOperation)
.collect(toList());
> Thanks, you are right, it seems a bug, the MDT compaction triggers with
empty log files there, will fire a fix for it.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]