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