1
0

Introduce ReadOptimizedView & RealtimeView out of TableFileSystemView

- Usage now marks code as clearly using either RO or RT views, for future evolution
  - Tests on all of FileGroups and FileSlices
This commit is contained in:
Vinoth Chandar
2017-06-19 17:16:45 -07:00
committed by prazanna
parent c00f1a9ed9
commit 754ab88a2d
16 changed files with 264 additions and 166 deletions

View File

@@ -177,7 +177,7 @@ public class HoodieReadClient implements Serializable {
+ hoodieTable.getMetaClient().getBasePath());
}
TableFileSystemView fileSystemView = new HoodieTableFileSystemView(hoodieTable.getMetaClient(),
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(hoodieTable.getMetaClient(),
hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path)));
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(
Collectors.toList());

View File

@@ -479,7 +479,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
.mapToPair((PairFunction<String, String, List<String>>) partitionPath -> {
// Scan all partitions files with this commit time
logger.info("Collecting latest files in partition path " + partitionPath);
TableFileSystemView view = table.getFileSystemView();
TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView();
List<String> latestFiles =
view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime)
.map(HoodieDataFile::getFileName).collect(Collectors.toList());

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().getLatestDataFilesBeforeOrOn(partitionPath,
hoodieTable.getROFileSystemView().getLatestDataFilesBeforeOrOn(partitionPath,
latestCommitTime.get().getTimestamp()).collect(Collectors.toList());
for (HoodieDataFile file : filteredFiles) {
list.add(new Tuple2<>(partitionPath, file.getFileName()));

View File

@@ -40,7 +40,7 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
protected final FileSystem fs;
protected final HoodieTable<T> hoodieTable;
protected HoodieTimeline hoodieTimeline;
protected TableFileSystemView fileSystemView;
protected TableFileSystemView.ReadOptimizedView fileSystemView;
protected final Schema schema;
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
@@ -50,7 +50,7 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
this.fs = FSUtils.getFs();
this.hoodieTable = hoodieTable;
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
this.fileSystemView = hoodieTable.getFileSystemView();
this.fileSystemView = hoodieTable.getROFileSystemView();
this.schema =
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
}

View File

@@ -84,7 +84,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
List<CompactionOperation> operations =
jsc.parallelize(partitionPaths, partitionPaths.size())
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> hoodieTable
.getFileSystemView()
.getRTFileSystemView()
.getLatestFileSlices(partitionPath)
.map(s -> new CompactionOperation(s.getDataFile().get(),
partitionPath, s.getLogFiles().collect(Collectors.toList()), config))

View File

@@ -311,7 +311,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()
List<HoodieDataFile> allFiles = getROFileSystemView()
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp())
.collect(Collectors.toList());

View File

@@ -108,6 +108,24 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
}
/**
* Get the read optimized view of the file system for this table
*
* @return
*/
public TableFileSystemView.ReadOptimizedView getROFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
}
/**
* Get the real time view of the file system for this table
*
* @return
*/
public TableFileSystemView.RealtimeView getRTFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
}
/**
* Get the view of the file system for this table
*

View File

@@ -44,6 +44,7 @@ import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.util.Collection;
import java.util.Map;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils;
@@ -412,7 +413,7 @@ public class TestHoodieClient implements Serializable {
List<String> partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView view = table.getFileSystemView();
final TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView();
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
return view.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
}).collect(Collectors.toList());
@@ -431,7 +432,7 @@ public class TestHoodieClient implements Serializable {
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView view1 = table.getFileSystemView();
final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> {
return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
}).collect(Collectors.toList());
@@ -482,7 +483,7 @@ public class TestHoodieClient implements Serializable {
List<String> partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView view1 = table.getFileSystemView();
final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView();
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003"));
@@ -501,7 +502,7 @@ public class TestHoodieClient implements Serializable {
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView view2 = table.getFileSystemView();
final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> {
return view2.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"));
@@ -524,7 +525,7 @@ public class TestHoodieClient implements Serializable {
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView view3 = table.getFileSystemView();
final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> {
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
}).collect(Collectors.toList());
@@ -961,7 +962,7 @@ public class TestHoodieClient implements Serializable {
assertEquals("2 files needs to be committed.", 2, statuses.size());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
TableFileSystemView fileSystemView = table.getFileSystemView();
TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView();
List<HoodieDataFile> files = fileSystemView.getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3).collect(
Collectors.toList());
int numTotalInsertsInCommit3 = 0;
@@ -1057,7 +1058,7 @@ public class TestHoodieClient implements Serializable {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
List<HoodieDataFile> files =
table.getFileSystemView().getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3)
table.getROFileSystemView().getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3)
.collect(Collectors.toList());
assertEquals("Total of 2 valid data files", 2, files.size());

View File

@@ -128,13 +128,13 @@ public class TestMergeOnReadTable {
assertFalse(commit.isPresent());
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCompletedCompactionCommitTimeline(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = fsView.getLatestDataFiles();
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent());
fsView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
dataFilesToRead = fsView.getLatestDataFiles();
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
dataFilesToRead = roView.getLatestDataFiles();
assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit",
dataFilesToRead.findAny().isPresent());
@@ -169,8 +169,8 @@ public class TestMergeOnReadTable {
compactor.compact(jsc, getConfig(), table);
allFiles = HoodieTestUtils.listAllDataFilesInPath(fs, cfg.getBasePath());
fsView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
dataFilesToRead = fsView.getLatestDataFiles();
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
dataFilesToRead = roView.getLatestDataFiles();
assertTrue(dataFilesToRead.findAny().isPresent());
// verify that there is a commit

View File

@@ -167,7 +167,7 @@ public class TestHoodieCompactor {
table = HoodieTable.getHoodieTable(metaClient, config);
for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles =
table.getFileSystemView().getLatestFileSlices(partitionPath)
table.getRTFileSystemView().getLatestFileSlices(partitionPath)
.collect(Collectors.toList());
for (FileSlice fileSlice : groupedLogFiles) {
assertEquals("There should be 1 log file written for every data file", 1,
@@ -192,7 +192,7 @@ public class TestHoodieCompactor {
HoodieTimeline.GREATER));
for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles = table.getFileSystemView()
List<FileSlice> groupedLogFiles = table.getRTFileSystemView()
.getLatestFileSlices(partitionPath)
.collect(Collectors.toList());
for (FileSlice slice: groupedLogFiles) {