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

@@ -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());

View File

@@ -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());
}
/**

View File

@@ -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()));

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;

View File

@@ -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) {

View File

@@ -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;

View File

@@ -24,6 +24,7 @@ import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieFileGroup;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -65,7 +66,6 @@ import java.io.FileInputStream;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
@@ -74,7 +74,6 @@ import java.util.Optional;
import java.util.Set;
import java.util.TreeSet;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import scala.collection.Iterator;
import static org.junit.Assert.assertEquals;
@@ -415,8 +414,7 @@ public class TestHoodieClient implements Serializable {
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView view = table.getFileSystemView();
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
Stream<List<HoodieDataFile>> files = view.getEveryVersionInPartition(s);
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("002"));
return view.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 002 should not be cleaned", 3, dataFiles.size());
@@ -435,8 +433,7 @@ public class TestHoodieClient implements Serializable {
table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView view1 = table.getFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> {
Stream<List<HoodieDataFile>> files = view1.getEveryVersionInPartition(s);
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("002"));
return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 002 should be cleaned now", 0, dataFiles.size());
@@ -488,8 +485,7 @@ public class TestHoodieClient implements Serializable {
final TableFileSystemView view1 = table.getFileSystemView();
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
Stream<List<HoodieDataFile>> files = view1.getEveryVersionInPartition(s);
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("003"));
return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 003 should be present", 3, dataFiles.size());
@@ -508,8 +504,7 @@ public class TestHoodieClient implements Serializable {
final TableFileSystemView view2 = table.getFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> {
Stream<List<HoodieDataFile>> files = view2.getEveryVersionInPartition(s);
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("004"));
return view2.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 004 should be present", 3, dataFiles.size());
@@ -531,20 +526,17 @@ public class TestHoodieClient implements Serializable {
table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView view3 = table.getFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> {
Stream<List<HoodieDataFile>> files = view3.getEveryVersionInPartition(s);
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("002"));
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 002 be available", 3, dataFiles.size());
dataFiles = partitionPaths.stream().flatMap(s -> {
Stream<List<HoodieDataFile>> files = view3.getEveryVersionInPartition(s);
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("003"));
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 003 should be rolled back", 0, dataFiles.size());
dataFiles = partitionPaths.stream().flatMap(s -> {
Stream<List<HoodieDataFile>> files = view3.getEveryVersionInPartition(s);
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("004"));
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 004 should be rolled back", 0, dataFiles.size());
}
@@ -622,19 +614,21 @@ public class TestHoodieClient implements Serializable {
}
List<List<HoodieDataFile>> fileVersions = fsView.getEveryVersionInPartition(partitionPath).collect(Collectors.toList());
for (List<HoodieDataFile> entry : fileVersions) {
List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList());
for (HoodieFileGroup fileGroup : fileGroups) {
// No file has no more than max versions
String fileId = entry.iterator().next().getFileId();
String fileId = fileGroup.getId();
List<HoodieDataFile> dataFiles = fileGroup.getAllDataFiles().collect(Collectors.toList());
assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions",
entry.size() <= maxVersions);
dataFiles.size() <= maxVersions);
// Each file, has the latest N versions (i.e cleaning gets rid of older versions)
List<String> commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId));
for (int i = 0; i < entry.size(); i++) {
for (int i = 0; i < dataFiles.size(); i++) {
assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions,
Iterables.get(entry, i).getCommitTime(),
Iterables.get(dataFiles, i).getCommitTime(),
commitedVersions.get(commitedVersions.size() - 1 - i));
}
}
@@ -709,13 +703,13 @@ public class TestHoodieClient implements Serializable {
TableFileSystemView fsView = table1.getFileSystemView();
// Need to ensure the following
for (String partitionPath : dataGen.getPartitionPaths()) {
List<List<HoodieDataFile>> fileVersions = fsView.getEveryVersionInPartition(partitionPath).collect(Collectors.toList());
for (List<HoodieDataFile> entry : fileVersions) {
List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList());
for (HoodieFileGroup fileGroup : fileGroups) {
Set<String> commitTimes = new HashSet<>();
for(HoodieDataFile value:entry) {
fileGroup.getAllDataFiles().forEach(value -> {
System.out.println("Data File - " + value);
commitTimes.add(value.getCommitTime());
}
});
assertEquals("Only contain acceptable versions of file should be present",
acceptableCommits.stream().map(HoodieInstant::getTimestamp)
.collect(Collectors.toSet()), commitTimes);
@@ -968,7 +962,7 @@ public class TestHoodieClient implements Serializable {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
TableFileSystemView fileSystemView = table.getFileSystemView();
List<HoodieDataFile> files = fileSystemView.getLatestVersionInPartition(TEST_PARTITION_PATH, commitTime3).collect(
List<HoodieDataFile> files = fileSystemView.getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3).collect(
Collectors.toList());
int numTotalInsertsInCommit3 = 0;
for (HoodieDataFile file: files) {
@@ -1063,7 +1057,7 @@ public class TestHoodieClient implements Serializable {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
List<HoodieDataFile> files =
table.getFileSystemView().getLatestVersionInPartition(TEST_PARTITION_PATH, commitTime3)
table.getFileSystemView().getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3)
.collect(Collectors.toList());
assertEquals("Total of 2 valid data files", 2, files.size());

View File

@@ -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.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieIndexConfig;
@@ -126,13 +127,14 @@ public class TestMergeOnReadTable {
metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
assertFalse(commit.isPresent());
TableFileSystemView fsView = hoodieTable.getCompactedFileSystemView();
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
Stream<HoodieDataFile> dataFilesToRead = fsView.getLatestVersions(allFiles);
TableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCompletedCompactionCommitTimeline(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = fsView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent());
fsView = hoodieTable.getFileSystemView();
dataFilesToRead = fsView.getLatestVersions(allFiles);
fsView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
dataFilesToRead = fsView.getLatestDataFiles();
assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit",
dataFilesToRead.findAny().isPresent());
@@ -167,7 +169,8 @@ public class TestMergeOnReadTable {
compactor.compact(jsc, getConfig(), table);
allFiles = HoodieTestUtils.listAllDataFilesInPath(fs, cfg.getBasePath());
dataFilesToRead = fsView.getLatestVersions(allFiles);
fsView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
dataFilesToRead = fsView.getLatestDataFiles();
assertTrue(dataFilesToRead.findAny().isPresent());
// verify that there is a commit

View File

@@ -20,6 +20,7 @@ import com.uber.hoodie.HoodieReadClient;
import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -27,7 +28,7 @@ import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.model.HoodieTestUtils;
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.model.HoodieLogFile;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
@@ -43,7 +44,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SQLContext;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -53,6 +53,8 @@ import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@@ -164,11 +166,12 @@ public class TestHoodieCompactor {
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
for (String partitionPath : dataGen.getPartitionPaths()) {
Map<HoodieDataFile, List<HoodieLogFile>> groupedLogFiles =
table.getFileSystemView().groupLatestDataFileWithLogFiles(partitionPath);
for (List<HoodieLogFile> logFiles : groupedLogFiles.values()) {
List<FileSlice> groupedLogFiles =
table.getFileSystemView().getLatestFileSlices(partitionPath)
.collect(Collectors.toList());
for (FileSlice fileSlice : groupedLogFiles) {
assertEquals("There should be 1 log file written for every data file", 1,
logFiles.size());
fileSlice.getLogFiles().count());
}
}
@@ -189,12 +192,13 @@ public class TestHoodieCompactor {
HoodieTimeline.GREATER));
for (String partitionPath : dataGen.getPartitionPaths()) {
Map<HoodieDataFile, List<HoodieLogFile>> groupedLogFiles =
table.getFileSystemView().groupLatestDataFileWithLogFiles(partitionPath);
for (List<HoodieLogFile> logFiles : groupedLogFiles.values()) {
List<FileSlice> groupedLogFiles = table.getFileSystemView()
.getLatestFileSlices(partitionPath)
.collect(Collectors.toList());
for (FileSlice slice: groupedLogFiles) {
assertTrue(
"After compaction there should be no log files visiable on a Realtime view",
logFiles.isEmpty());
slice.getLogFiles().collect(Collectors.toList()).isEmpty());
}
assertTrue(result.getPartitionToCompactionWriteStats().containsKey(partitionPath));
}

View File

@@ -22,8 +22,7 @@ import static org.junit.Assert.assertTrue;
import com.beust.jcommander.internal.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.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.CompactionOperation;
@@ -32,7 +31,6 @@ import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.junit.Test;

View File

@@ -16,9 +16,9 @@
package com.uber.hoodie.io.strategy;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieLogFile;
import java.util.Optional;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
public class TestHoodieLogFile extends HoodieLogFile {