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

@@ -22,10 +22,10 @@ import com.uber.hoodie.common.model.HoodieDeltaWriteStat;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -86,8 +86,9 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
partitionPath = record.getPartitionPath();
// HACK(vc) This also assumes a base file. It will break, if appending without one.
String latestValidFilePath =
fileSystemView.getLatestDataFilesForFileId(record.getPartitionPath(), fileId)
.findFirst().get().getFileName();
fileSystemView.getLatestDataFiles(record.getPartitionPath())
.filter(dataFile -> dataFile.getFileId().equals(fileId))
.findFirst().get().getFileName();
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
writeStatus.getStat().setPrevCommit(baseCommitTime);
writeStatus.setFileId(fileId);

View File

@@ -18,11 +18,12 @@ package com.uber.hoodie.io;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieFileGroup;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -78,17 +79,17 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
throws IOException {
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
.getCleanerFileVersionsRetained() + " file versions. ");
List<List<HoodieDataFile>> fileVersions =
fileSystemView.getEveryVersionInPartition(partitionPath)
List<HoodieFileGroup> fileGroups =
fileSystemView.getAllFileGroups(partitionPath)
.collect(Collectors.toList());
List<String> deletePaths = new ArrayList<>();
// Collect all the datafiles savepointed by all the savepoints
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
for (List<HoodieDataFile> versionsForFileId : fileVersions) {
for (HoodieFileGroup fileGroup : fileGroups) {
int keepVersions = config.getCleanerFileVersionsRetained();
Iterator<HoodieDataFile> commitItr = versionsForFileId.iterator();
Iterator<HoodieDataFile> commitItr = fileGroup.getAllDataFiles().iterator();
while (commitItr.hasNext() && keepVersions > 0) {
// Skip this most recent version
HoodieDataFile next = commitItr.next();
@@ -150,10 +151,11 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
// determine if we have enough commits, to start cleaning.
if (commitTimeline.countInstants() > commitsRetained) {
HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get();
List<List<HoodieDataFile>> fileVersions =
fileSystemView.getEveryVersionInPartition(partitionPath)
List<HoodieFileGroup> fileGroups =
fileSystemView.getAllFileGroups(partitionPath)
.collect(Collectors.toList());
for (List<HoodieDataFile> fileList : fileVersions) {
for (HoodieFileGroup fileGroup : fileGroups) {
List<HoodieDataFile> fileList = fileGroup.getAllDataFiles().collect(Collectors.toList());
String lastVersion = FSUtils.getCommitTime(fileList.get(0).getFileName());
String lastVersionBeforeEarliestCommitToRetain =
getLatestVersionBeforeCommit(fileList, earliestCommitToRetain);

View File

@@ -78,8 +78,10 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
// If the first record, we need to extract some info out
if (oldFilePath == null) {
String latestValidFilePath = fileSystemView
.getLatestDataFilesForFileId(record.getPartitionPath(), fileId).findFirst()
.get().getFileName();
.getLatestDataFiles(record.getPartitionPath())
.filter(dataFile -> dataFile.getFileId().equals(fileId))
.findFirst()
.get().getFileName();
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,

View File

@@ -17,7 +17,7 @@
package com.uber.hoodie.io.compact;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;

View File

@@ -35,6 +35,7 @@ import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
import com.uber.hoodie.table.HoodieTable;
import java.util.Collection;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
@@ -84,9 +85,9 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
jsc.parallelize(partitionPaths, partitionPaths.size())
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> hoodieTable
.getFileSystemView()
.groupLatestDataFileWithLogFiles(partitionPath).entrySet()
.stream()
.map(s -> new CompactionOperation(s.getKey(), partitionPath, s.getValue(), config))
.getLatestFileSlices(partitionPath)
.map(s -> new CompactionOperation(s.getDataFile().get(),
partitionPath, s.getLogFiles().collect(Collectors.toList()), config))
.collect(toList()).iterator()).collect();
log.info("Total of " + operations.size() + " compactions are retrieved");

View File

@@ -19,7 +19,7 @@ package com.uber.hoodie.io.compact.strategy;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.CompactionOperation;

View File

@@ -17,7 +17,7 @@
package com.uber.hoodie.io.compact.strategy;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.CompactionOperation;
import java.io.Serializable;

View File

@@ -17,7 +17,7 @@
package com.uber.hoodie.io.compact.strategy;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.CompactionOperation;
import java.util.Comparator;

View File

@@ -18,7 +18,7 @@ package com.uber.hoodie.io.compact.strategy;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.CompactionOperation;
import java.util.List;