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:
@@ -26,6 +26,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
@@ -167,7 +168,6 @@ public class HoodieReadClient implements Serializable {
|
||||
public Dataset<Row> read(String... paths) {
|
||||
assertSqlContext();
|
||||
List<String> filteredPaths = new ArrayList<>();
|
||||
TableFileSystemView fileSystemView = hoodieTable.getFileSystemView();
|
||||
|
||||
try {
|
||||
for (String path : paths) {
|
||||
@@ -177,7 +177,9 @@ public class HoodieReadClient implements Serializable {
|
||||
+ hoodieTable.getMetaClient().getBasePath());
|
||||
}
|
||||
|
||||
List<HoodieDataFile> latestFiles = fileSystemView.getLatestVersions(fs.globStatus(new Path(path))).collect(
|
||||
TableFileSystemView fileSystemView = new HoodieTableFileSystemView(hoodieTable.getMetaClient(),
|
||||
hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path)));
|
||||
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(
|
||||
Collectors.toList());
|
||||
for (HoodieDataFile file : latestFiles) {
|
||||
filteredPaths.add(file.getPath());
|
||||
|
||||
@@ -30,7 +30,6 @@ import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
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.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
@@ -482,7 +481,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
logger.info("Collecting latest files in partition path " + partitionPath);
|
||||
TableFileSystemView view = table.getFileSystemView();
|
||||
List<String> latestFiles =
|
||||
view.getLatestVersionInPartition(partitionPath, commitTime)
|
||||
view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime)
|
||||
.map(HoodieDataFile::getFileName).collect(Collectors.toList());
|
||||
return new Tuple2<>(partitionPath, latestFiles);
|
||||
}).collectAsMap();
|
||||
@@ -801,26 +800,16 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* Deduplicate Hoodie records, using the given deduplication funciton.
|
||||
*/
|
||||
private JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records, int parallelism) {
|
||||
return records.mapToPair(new PairFunction<HoodieRecord<T>, HoodieKey, HoodieRecord<T>>() {
|
||||
@Override
|
||||
public Tuple2<HoodieKey, HoodieRecord<T>> call(HoodieRecord<T> record) {
|
||||
return new Tuple2<>(record.getKey(), record);
|
||||
}
|
||||
}).reduceByKey(new Function2<HoodieRecord<T>, HoodieRecord<T>, HoodieRecord<T>>() {
|
||||
@Override
|
||||
public HoodieRecord<T> call(HoodieRecord<T> rec1, HoodieRecord<T> rec2) {
|
||||
@SuppressWarnings("unchecked")
|
||||
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
|
||||
// we cannot allow the user to change the key or partitionPath, since that will affect everything
|
||||
// so pick it from one of the records.
|
||||
return new HoodieRecord<T>(rec1.getKey(), reducedData);
|
||||
}
|
||||
}, parallelism).map(new Function<Tuple2<HoodieKey, HoodieRecord<T>>, HoodieRecord<T>>() {
|
||||
@Override
|
||||
public HoodieRecord<T> call(Tuple2<HoodieKey, HoodieRecord<T>> recordTuple) {
|
||||
return recordTuple._2();
|
||||
}
|
||||
});
|
||||
return records
|
||||
.mapToPair(record -> new Tuple2<>(record.getKey(), record))
|
||||
.reduceByKey((rec1, rec2) -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
|
||||
// we cannot allow the user to change the key or partitionPath, since that will affect everything
|
||||
// so pick it from one of the records.
|
||||
return new HoodieRecord<T>(rec1.getKey(), reducedData);
|
||||
}, parallelism)
|
||||
.map(recordTuple -> recordTuple._2());
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -184,7 +184,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
List<Tuple2<String, String>> list = new ArrayList<>();
|
||||
if (latestCommitTime.isPresent()) {
|
||||
List<HoodieDataFile> filteredFiles =
|
||||
hoodieTable.getFileSystemView().getLatestVersionInPartition(partitionPath,
|
||||
hoodieTable.getFileSystemView().getLatestDataFilesBeforeOrOn(partitionPath,
|
||||
latestCommitTime.get().getTimestamp()).collect(Collectors.toList());
|
||||
for (HoodieDataFile file : filteredFiles) {
|
||||
list.add(new Tuple2<>(partitionPath, file.getFileName()));
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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");
|
||||
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -40,7 +40,6 @@ import java.util.Optional;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroParquetReader;
|
||||
@@ -306,7 +305,6 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
* @return
|
||||
*/
|
||||
private List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
HoodieTimeline commitTimeline = getCompletedCommitTimeline();
|
||||
@@ -314,7 +312,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
if (!commitTimeline.empty()) { // if we have some commits
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
List<HoodieDataFile> allFiles = getFileSystemView()
|
||||
.getLatestVersionInPartition(partitionPath, latestCommitTime.getTimestamp())
|
||||
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
for (HoodieDataFile file : allFiles) {
|
||||
|
||||
@@ -41,6 +41,8 @@ import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
Reference in New Issue
Block a user