1
0

[HUDI-1296] Support Metadata Table in Spark Datasource (#4789)

* Bootstrapping initial support for Metadata Table in Spark Datasource

- Consolidated Avro/Row conversion utilities to center around Spark's AvroDeserializer ; removed duplication
- Bootstrapped HoodieBaseRelation
- Updated HoodieMergeOnReadRDD to be able to handle Metadata Table
- Modified MOR relations to be able to read different Base File formats (Parquet, HFile)
This commit is contained in:
Alexey Kudinkin
2022-02-24 13:23:13 -08:00
committed by GitHub
parent 521338b4d9
commit 85e8a5c4de
56 changed files with 1634 additions and 1010 deletions

View File

@@ -68,13 +68,13 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
}
// Return parquet file with a list of log files in the same file group.
public static List<Pair<Option<HoodieBaseFile>, List<String>>> groupLogsByBaseFile(Configuration conf, List<Path> partitionPaths) {
public static List<Pair<Option<HoodieBaseFile>, List<HoodieLogFile>>> 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, partitionSet);
// Get all the base file and it's log files pairs in required partition paths.
List<Pair<Option<HoodieBaseFile>, List<String>>> baseAndLogsList = new ArrayList<>();
List<Pair<Option<HoodieBaseFile>, List<HoodieLogFile>>> 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);
@@ -91,8 +91,7 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
.orElse(Stream.empty());
latestFileSlices.forEach(fileSlice -> {
List<String> logFilePaths = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
.map(logFile -> logFile.getPath().toString()).collect(Collectors.toList());
List<HoodieLogFile> logFilePaths = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
baseAndLogsList.add(Pair.of(fileSlice.getBaseFile(), logFilePaths));
});
} catch (Exception e) {