-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-2107] Support Read Log Only MOR Table For Spark #3193
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -32,6 +32,7 @@ | |
| 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.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 static InputSplit[] getRealtimeSplits(Configuration conf, Stream<FileSpli | |
| } | ||
|
|
||
| // 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 static Map<HoodieBaseFile, List<String>> groupLogsByBaseFile(Configuratio | |
| .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. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,7 +19,6 @@ | |
| package org.apache.hudi | ||
|
|
||
| import org.apache.avro.Schema | ||
| import org.apache.hudi.common.model.HoodieBaseFile | ||
| import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} | ||
| import org.apache.hudi.common.table.view.HoodieTableFileSystemView | ||
| import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils | ||
|
|
@@ -137,12 +136,15 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext, | |
| } | ||
|
|
||
| def buildFileIndex(filters: Array[Filter]): List[HoodieMergeOnReadFileSplit] = { | ||
|
|
||
| val fileStatuses = if (globPaths.isDefined) { | ||
| // Get all partition paths | ||
| val partitionPaths = if (globPaths.isDefined) { | ||
| // Load files from the global paths if it has defined to be compatible with the original mode | ||
| val inMemoryFileIndex = HoodieSparkUtils.createInMemoryFileIndex(sqlContext.sparkSession, globPaths.get) | ||
| inMemoryFileIndex.allFiles() | ||
| } else { // Load files by the HoodieFileIndex. | ||
| val fsView = new HoodieTableFileSystemView(metaClient, | ||
| metaClient.getActiveTimeline.getCommitsTimeline | ||
| .filterCompletedInstants, inMemoryFileIndex.allFiles().toArray) | ||
| fsView.getLatestBaseFiles.iterator().asScala.toList.map(_.getFileStatus.getPath.getParent) | ||
|
||
| } else { // Load partition path by the HoodieFileIndex. | ||
| val hoodieFileIndex = HoodieFileIndex(sqlContext.sparkSession, metaClient, | ||
| Some(tableStructSchema), optParams, FileStatusCache.getOrCreate(sqlContext.sparkSession)) | ||
|
|
||
|
|
@@ -152,34 +154,35 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext, | |
| val partitionFilterExpression = | ||
| HoodieSparkUtils.convertToCatalystExpressions(partitionFilters, tableStructSchema) | ||
|
|
||
| // if convert success to catalyst expression, use the partition prune | ||
| if (partitionFilterExpression.isDefined) { | ||
| hoodieFileIndex.listFiles(Seq(partitionFilterExpression.get), Seq.empty).flatMap(_.files) | ||
| } else { | ||
| hoodieFileIndex.allFiles | ||
| } | ||
| val allPartitionPaths = hoodieFileIndex.getAllQueryPartitionPaths | ||
| // If convert success to catalyst expression, use the partition prune | ||
| hoodieFileIndex.prunePartition(allPartitionPaths, partitionFilterExpression.map(Seq(_)).getOrElse(Seq.empty)) | ||
| .map(_.fullPartitionPath(metaClient.getBasePath)) | ||
| } | ||
|
|
||
| if (fileStatuses.isEmpty) { // If this an empty table, return an empty split list. | ||
| if (partitionPaths.isEmpty) { // If this an empty table, return an empty split list. | ||
| List.empty[HoodieMergeOnReadFileSplit] | ||
| } else { | ||
| val fsView = new HoodieTableFileSystemView(metaClient, | ||
| metaClient.getActiveTimeline.getCommitsTimeline | ||
| .filterCompletedInstants, fileStatuses.toArray) | ||
| val latestFiles: List[HoodieBaseFile] = fsView.getLatestBaseFiles.iterator().asScala.toList | ||
|
|
||
| if (!fsView.getLastInstant.isPresent) { // Return empty list if the table has no commit | ||
| val lastInstant = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants.lastInstant() | ||
| if (!lastInstant.isPresent) { // Return empty list if the table has no commit | ||
| List.empty | ||
| } else { | ||
| val latestCommit = fsView.getLastInstant.get().getTimestamp | ||
| val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, latestFiles.asJava).asScala | ||
| val fileSplits = fileGroup.map(kv => { | ||
| val baseFile = kv._1 | ||
| val logPaths = if (kv._2.isEmpty) Option.empty else Option(kv._2.asScala.toList) | ||
| val filePath = MergeOnReadSnapshotRelation.getFilePath(baseFile.getFileStatus.getPath) | ||
|
|
||
| val partitionedFile = PartitionedFile(InternalRow.empty, filePath, 0, baseFile.getFileLen) | ||
| HoodieMergeOnReadFileSplit(Option(partitionedFile), logPaths, latestCommit, | ||
| val latestCommit = lastInstant.get().getTimestamp | ||
| val baseAndLogsList = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, partitionPaths.asJava).asScala | ||
| val fileSplits = baseAndLogsList.map(kv => { | ||
| val baseFile = kv.getLeft | ||
| val logPaths = if (kv.getRight.isEmpty) Option.empty else Option(kv.getRight.asScala.toList) | ||
|
|
||
| val baseDataPath = if (baseFile.isPresent) { | ||
| Some(PartitionedFile( | ||
| InternalRow.empty, | ||
| MergeOnReadSnapshotRelation.getFilePath(baseFile.get.getFileStatus.getPath), | ||
| 0, baseFile.get.getFileLen) | ||
| ) | ||
| } else { | ||
| None | ||
| } | ||
| HoodieMergeOnReadFileSplit(baseDataPath, logPaths, latestCommit, | ||
| metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType) | ||
| }).toList | ||
| fileSplits | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
for log only file slice, is
fileSlice.getBaseFile() == null?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No, for log only file slice, the fileSlice.getBaseFile() is an empty Option.