[HUDI-2107] Support Read Log Only MOR Table For Spark (#3193)
This commit is contained in:
@@ -32,6 +32,7 @@ import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.BootstrapBaseFileSplit;
|
||||
@@ -49,11 +50,13 @@ import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@@ -137,16 +140,14 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
|
||||
}
|
||||
|
||||
// Return parquet file with a list of log files in the same file group.
|
||||
public static Map<HoodieBaseFile, List<String>> groupLogsByBaseFile(Configuration conf, List<HoodieBaseFile> fileStatuses) {
|
||||
Map<Path, List<HoodieBaseFile>> partitionsToParquetSplits =
|
||||
fileStatuses.stream().collect(Collectors.groupingBy(file -> file.getFileStatus().getPath().getParent()));
|
||||
public static List<Pair<Option<HoodieBaseFile>, List<String>>> groupLogsByBaseFile(Configuration conf, List<Path> partitionPaths) {
|
||||
Set<Path> partitionSet = new HashSet<>(partitionPaths);
|
||||
// TODO(vc): Should we handle also non-hoodie splits here?
|
||||
Map<Path, HoodieTableMetaClient> partitionsToMetaClient = getTableMetaClientByPartitionPath(conf, partitionsToParquetSplits.keySet());
|
||||
Map<Path, HoodieTableMetaClient> partitionsToMetaClient = getTableMetaClientByPartitionPath(conf, partitionSet);
|
||||
|
||||
// for all unique split parents, obtain all delta files based on delta commit timeline,
|
||||
// grouped on file id
|
||||
Map<HoodieBaseFile, List<String>> resultMap = new HashMap<>();
|
||||
partitionsToParquetSplits.keySet().forEach(partitionPath -> {
|
||||
// Get all the base file and it's log files pairs in required partition paths.
|
||||
List<Pair<Option<HoodieBaseFile>, List<String>>> baseAndLogsList = new ArrayList<>();
|
||||
partitionSet.forEach(partitionPath -> {
|
||||
// for each partition path obtain the data & log file groupings, then map back to inputsplits
|
||||
HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath);
|
||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline());
|
||||
@@ -161,28 +162,18 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
|
||||
.map(instant -> fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, instant.getTimestamp()))
|
||||
.orElse(Stream.empty());
|
||||
|
||||
// subgroup splits again by file id & match with log files.
|
||||
Map<String, List<HoodieBaseFile>> groupedInputSplits = partitionsToParquetSplits.get(partitionPath).stream()
|
||||
.collect(Collectors.groupingBy(file -> FSUtils.getFileId(file.getFileStatus().getPath().getName())));
|
||||
latestFileSlices.forEach(fileSlice -> {
|
||||
List<HoodieBaseFile> dataFileSplits = groupedInputSplits.get(fileSlice.getFileId());
|
||||
dataFileSplits.forEach(split -> {
|
||||
try {
|
||||
List<String> logFilePaths = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
|
||||
List<String> logFilePaths = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
|
||||
.map(logFile -> logFile.getPath().toString()).collect(Collectors.toList());
|
||||
resultMap.put(split, logFilePaths);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error creating hoodie real time split ", e);
|
||||
}
|
||||
});
|
||||
baseAndLogsList.add(Pair.of(fileSlice.getBaseFile(), logFilePaths));
|
||||
});
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error obtaining data file/log file grouping: " + partitionPath, e);
|
||||
}
|
||||
});
|
||||
return resultMap;
|
||||
return baseAndLogsList;
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Add a field to the existing fields projected.
|
||||
|
||||
Reference in New Issue
Block a user