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

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