1
0

Refactoring HoodieTableFileSystemView using FileGroups/FileSlices

- Merged all filter* and get* methods
 - new constructor takes filestatus[]
 - All existing tests pass
 - FileGroup is all files that belong to a fileID within a partition
 - FileSlice is a generation of data and log files, starting at a base commit
This commit is contained in:
Vinoth Chandar
2017-06-19 00:34:37 -07:00
committed by prazanna
parent 23e7badd8a
commit c00f1a9ed9
42 changed files with 810 additions and 408 deletions

View File

@@ -85,15 +85,15 @@ public class HoodieInputFormat extends MapredParquetInputFormat
Map<HoodieTableMetaClient, List<FileStatus>> groupedFileStatus = groupFileStatus(fileStatuses);
LOG.info("Found a total of " + groupedFileStatus.size() + " groups");
List<FileStatus> returns = new ArrayList<>();
for(Map.Entry<HoodieTableMetaClient, List<FileStatus>> entry:groupedFileStatus.entrySet()) {
for(Map.Entry<HoodieTableMetaClient, List<FileStatus>> entry: groupedFileStatus.entrySet()) {
HoodieTableMetaClient metadata = entry.getKey();
if(metadata == null) {
if (metadata == null) {
// Add all the paths which are not hoodie specific
returns.addAll(entry.getValue());
continue;
}
FileStatus[] value = entry.getValue().toArray(new FileStatus[entry.getValue().size()]);
FileStatus[] statuses = entry.getValue().toArray(new FileStatus[entry.getValue().size()]);
if (LOG.isDebugEnabled()) {
LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metadata);
}
@@ -101,7 +101,7 @@ public class HoodieInputFormat extends MapredParquetInputFormat
String mode = HoodieHiveUtil.readMode(Job.getInstance(job), tableName);
// Get all commits, delta commits, compactions, as all of them produce a base parquet file today
HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
TableFileSystemView fsView = new HoodieTableFileSystemView(metadata, timeline);
TableFileSystemView fsView = new HoodieTableFileSystemView(metadata, timeline, statuses);
if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
// this is of the form commitTs_partition_sequenceNumber
@@ -112,8 +112,8 @@ public class HoodieInputFormat extends MapredParquetInputFormat
List<String> commitsToReturn =
timeline.findInstantsAfter(lastIncrementalTs, maxCommits).getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
List<HoodieDataFile> filteredFiles =
fsView.getLatestVersionInRange(value, commitsToReturn)
List<HoodieDataFile> filteredFiles = fsView
.getLatestDataFilesInRange(commitsToReturn)
.collect(Collectors.toList());
for (HoodieDataFile filteredFile : filteredFiles) {
LOG.info("Processing incremental hoodie file - " + filteredFile.getPath());
@@ -124,7 +124,7 @@ public class HoodieInputFormat extends MapredParquetInputFormat
"Total paths to process after hoodie incremental filter " + filteredFiles.size());
} else {
// filter files on the latest commit found
List<HoodieDataFile> filteredFiles = fsView.getLatestVersions(value).collect(Collectors.toList());
List<HoodieDataFile> filteredFiles = fsView.getLatestDataFiles().collect(Collectors.toList());
LOG.info("Total paths to process after hoodie filter " + filteredFiles.size());
for (HoodieDataFile filteredFile : filteredFiles) {
if (LOG.isDebugEnabled()) {
@@ -146,15 +146,15 @@ public class HoodieInputFormat extends MapredParquetInputFormat
* 3. Generation of splits looks at FileStatus size to create splits, which skips this file
*
* @param fsView
* @param fileStatus
* @param dataFile
* @return
*/
private HoodieDataFile checkFileStatus(TableFileSystemView fsView, HoodieDataFile fileStatus) {
if(fileStatus.getFileSize() == 0) {
LOG.info("Refreshing file status " + fileStatus.getPath());
return new HoodieDataFile(fsView.getFileStatus(fileStatus.getPath()));
private HoodieDataFile checkFileStatus(TableFileSystemView fsView, HoodieDataFile dataFile) {
if(dataFile.getFileSize() == 0) {
LOG.info("Refreshing file status " + dataFile.getPath());
return new HoodieDataFile(fsView.getFileStatus(dataFile.getPath()));
}
return fileStatus;
return dataFile;
}
private Map<HoodieTableMetaClient, List<FileStatus>> groupFileStatus(FileStatus[] fileStatuses)
@@ -232,8 +232,9 @@ public class HoodieInputFormat extends MapredParquetInputFormat
* @param tableName
* @return
*/
private FilterPredicate constructHoodiePredicate(JobConf job, String tableName,
InputSplit split) throws IOException {
private FilterPredicate constructHoodiePredicate(JobConf job,
String tableName,
InputSplit split) throws IOException {
FilterPredicate commitTimePushdown = constructCommitTimePushdownPredicate(job, tableName);
LOG.info("Commit time predicate - " + commitTimePushdown.toString());
FilterPredicate existingPushdown = constructHQLPushdownPredicate(job, split);

View File

@@ -130,13 +130,14 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
new HoodieTableMetaClient(fs, baseDir.toString());
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline()
.filterCompletedInstants());
List<HoodieDataFile>
latestFiles = fsView.getLatestVersions(fs.listStatus(folder)).collect(
Collectors.toList());
.filterCompletedInstants(),
fs.listStatus(folder));
List<HoodieDataFile> latestFiles = fsView
.getLatestDataFiles()
.collect(Collectors.toList());
// populate the cache
if (!hoodiePathCache.containsKey(folder.toString())) {
hoodiePathCache.put(folder.toString(), new HashSet<Path>());
hoodiePathCache.put(folder.toString(), new HashSet<>());
}
LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() +
", caching " + latestFiles.size() + " files under "+ folder);

View File

@@ -21,13 +21,14 @@ package com.uber.hoodie.hadoop.realtime;
import com.google.common.base.Preconditions;
import com.google.common.collect.Sets;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.hadoop.HoodieInputFormat;
import com.uber.hoodie.hadoop.UseFileSplitsFromInputFormat;
@@ -106,16 +107,16 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
String relPartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath);
try {
Map<HoodieDataFile, List<HoodieLogFile>> dataLogFileGrouping = fsView.groupLatestDataFileWithLogFiles(relPartitionPath);
Stream<FileSlice> latestFileSlices = fsView.getLatestFileSlices(relPartitionPath);
// subgroup splits again by file id & match with log files.
Map<String, List<FileSplit>> groupedInputSplits = partitionsToParquetSplits.get(partitionPath).stream()
.collect(Collectors.groupingBy(split -> FSUtils.getFileId(split.getPath().getName())));
dataLogFileGrouping.forEach((dataFile, logFiles) -> {
List<FileSplit> dataFileSplits = groupedInputSplits.get(dataFile.getFileId());
latestFileSlices.forEach(fileSlice -> {
List<FileSplit> dataFileSplits = groupedInputSplits.get(fileSlice.getFileId());
dataFileSplits.forEach(split -> {
try {
List<String> logFilePaths = logFiles.stream()
List<String> logFilePaths = fileSlice.getLogFiles()
.map(logFile -> logFile.getPath().toString())
.collect(Collectors.toList());
// Get the maxCommit from the last delta or compaction or commit - when bootstrapped from COW table
@@ -132,8 +133,8 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
}
});
});
} catch (IOException e) {
throw new HoodieIOException("Error obtaining data file/log file grouping: " + partitionPath, e);
} catch (Exception e) {
throw new HoodieException("Error obtaining data file/log file grouping: " + partitionPath, e);
}
});
LOG.info("Returning a total splits of " + rtSplits.size());