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

@@ -82,7 +82,7 @@ public class FileSlice implements Serializable {
}
public Optional<HoodieDataFile> getDataFile() {
return Optional.of(dataFile);
return Optional.ofNullable(dataFile);
}
@Override

View File

@@ -30,91 +30,69 @@ import java.util.stream.Stream;
/**
* Interface for viewing the table file system.
* Dependening on the Hoodie Table Type - The view of the filesystem changes.
* <p>
* ReadOptimizedView - Lets queries run only on organized columnar data files at the expense of latency
* WriteOptimizedView - Lets queries run on columnar data as well as delta files (sequential) at the expense of query execution time
*
* @since 0.3.0
*/
public interface TableFileSystemView {
/**
* Stream all the latest data files in the given partition
*
* @param partitionPath
* @return
* ReadOptimizedView - methods to provide a view of columnar data files only.
*/
Stream<HoodieDataFile> getLatestDataFiles(String partitionPath);
interface ReadOptimizedView {
/**
* Stream all the latest data files in the given partition
*/
Stream<HoodieDataFile> getLatestDataFiles(String partitionPath);
/**
* Stream all the latest data files, in the file system view
*/
Stream<HoodieDataFile> getLatestDataFiles();
/**
* Stream all the latest version data files in the given partition with precondition that
* commitTime(file) before maxCommitTime
*/
Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
String maxCommitTime);
/**
* Stream all the latest data files pass
*/
Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn);
/**
* Stream all the data file versions grouped by FileId for a given partition
*/
Stream<HoodieDataFile> getAllDataFiles(String partitionPath);
}
/**
* Stream all the latest data files, in the file system view
*
* @return
* RealtimeView - methods to access a combination of columnar data files + log files with real time data.
*/
Stream<HoodieDataFile> getLatestDataFiles();
interface RealtimeView {
/**
* Stream all the latest file slices in the given partition
*/
Stream<FileSlice> getLatestFileSlices(String partitionPath);
/**
* Stream all the latest version data files in the given partition
* with precondition that commitTime(file) before maxCommitTime
*
* @param partitionPath
* @param maxCommitTime
* @return
*/
Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
/**
* Stream all the latest file slices in the given partition with precondition that
* commitTime(file) before maxCommitTime
*/
Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath,
String maxCommitTime);
/**
* Stream all the latest data files pass
*
* @param commitsToReturn
* @return
*/
Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn);
/**
* Stream all the latest file slices, in the given range
*/
Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn);
/**
* Stream all the data file versions grouped by FileId for a given partition
*
* @param partitionPath
* @return
*/
Stream<HoodieDataFile> getAllDataFiles(String partitionPath);
/**
* Stream all the latest file slices in the given partition
*
* @param partitionPath
* @return
*/
Stream<FileSlice> getLatestFileSlices(String partitionPath);
/**
* Stream all the latest file slices in the given partition
* with precondition that commitTime(file) before maxCommitTime
*
* @param partitionPath
* @param maxCommitTime
* @return
*/
Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath,
String maxCommitTime);
/**
* Stream all the latest file slices, in the given range
*
* @param commitsToReturn
* @return
*/
Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn);
/**
* Stream all the file slices for a given partition, latest or not.
*
* @param partitionPath
* @return
*/
Stream<FileSlice> getAllFileSlices(String partitionPath);
/**
* Stream all the file slices for a given partition, latest or not.
*/
Stream<FileSlice> getAllFileSlices(String partitionPath);
}
/**
* Stream all the file groups for a given partition
@@ -123,12 +101,4 @@ public interface TableFileSystemView {
* @return
*/
Stream<HoodieFileGroup> getAllFileGroups(String partitionPath);
/**
* Get the file Status for the path specified
*
* @param path
* @return
*/
FileStatus getFileStatus(String path);
}

View File

@@ -16,8 +16,6 @@
package com.uber.hoodie.common.table.view;
import static java.util.stream.Collectors.toList;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieFileGroup;
@@ -44,7 +42,6 @@ import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.function.Predicate;
import java.util.stream.Collector;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -57,7 +54,8 @@ import java.util.stream.Stream;
* @see TableFileSystemView
* @since 0.3.0
*/
public class HoodieTableFileSystemView implements TableFileSystemView, Serializable {
public class HoodieTableFileSystemView implements TableFileSystemView, TableFileSystemView.ReadOptimizedView,
TableFileSystemView.RealtimeView, Serializable {
protected HoodieTableMetaClient metaClient;
protected transient FileSystem fs;
@@ -187,7 +185,7 @@ public class HoodieTableFileSystemView implements TableFileSystemView, Serializa
@Override
public Stream<HoodieDataFile> getLatestDataFiles() {
return fileGroupMap.values().stream()
return fileGroupMap.values().stream()
.map(fileGroup -> fileGroup.getLatestDataFile())
.filter(dataFileOpt -> dataFileOpt.isPresent())
.map(Optional::get);
@@ -271,13 +269,4 @@ public class HoodieTableFileSystemView implements TableFileSystemView, Serializa
"Failed to list data files in partition " + partitionPathStr, e);
}
}
@Override
public FileStatus getFileStatus(String path) {
try {
return fs.getFileStatus(new Path(path));
} catch (IOException e) {
throw new HoodieIOException("Could not get FileStatus on path " + path);
}
}
}

View File

@@ -18,8 +18,11 @@ package com.uber.hoodie.common.table.view;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieFileGroup;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
@@ -35,11 +38,15 @@ import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.Assert.*;
@@ -48,6 +55,8 @@ public class HoodieTableFileSystemViewTest {
private HoodieTableMetaClient metaClient;
private String basePath;
private TableFileSystemView fsView;
private TableFileSystemView.ReadOptimizedView roView;
private TableFileSystemView.RealtimeView rtView;
@Before
public void init() throws IOException {
@@ -57,6 +66,8 @@ public class HoodieTableFileSystemViewTest {
metaClient = HoodieTestUtils.init(basePath);
fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
roView = (TableFileSystemView.ReadOptimizedView) fsView;
rtView = (TableFileSystemView.RealtimeView) fsView;
}
private void refreshFsView(FileStatus[] statuses) {
@@ -69,6 +80,8 @@ public class HoodieTableFileSystemViewTest {
fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
}
roView = (TableFileSystemView.ReadOptimizedView) fsView;
rtView = (TableFileSystemView.RealtimeView) fsView;
}
@Test
@@ -78,7 +91,7 @@ public class HoodieTableFileSystemViewTest {
String fileId = UUID.randomUUID().toString();
assertFalse("No commit, should not find any data file",
fsView.getLatestDataFiles(partitionPath)
roView.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().isPresent());
// Only one commit, but is not safe
@@ -87,7 +100,7 @@ public class HoodieTableFileSystemViewTest {
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
refreshFsView(null);
assertFalse("No commit, should not find any data file",
fsView.getLatestDataFiles(partitionPath)
roView.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId))
.findFirst().isPresent());
@@ -97,7 +110,7 @@ public class HoodieTableFileSystemViewTest {
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
commitTimeline.saveAsComplete(instant1, Optional.empty());
refreshFsView(null);
assertEquals("", fileName1, fsView
assertEquals("", fileName1, roView
.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId))
.findFirst().get()
@@ -108,7 +121,7 @@ public class HoodieTableFileSystemViewTest {
String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId);
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
refreshFsView(null);
assertEquals("", fileName1, fsView
assertEquals("", fileName1, roView
.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId))
.findFirst().get()
@@ -119,7 +132,7 @@ public class HoodieTableFileSystemViewTest {
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2);
commitTimeline.saveAsComplete(instant2, Optional.empty());
refreshFsView(null);
assertEquals("", fileName2, fsView
assertEquals("", fileName2, roView
.getLatestDataFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId))
.findFirst().get()
@@ -138,21 +151,31 @@ public class HoodieTableFileSystemViewTest {
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
String fileId4 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
@@ -161,13 +184,24 @@ public class HoodieTableFileSystemViewTest {
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(statuses.length, 7);
assertEquals(11, statuses.length);
refreshFsView(null);
// Check files as of lastest commit.
List<FileSlice> allSlices = rtView.getAllFileSlices("2016/05/01").collect(Collectors.toList());
assertEquals(8, allSlices.size());
Map<String, Long> fileSliceMap = allSlices.stream().collect(Collectors.groupingBy(
slice -> slice.getFileId(), Collectors.counting()));
assertEquals(2, fileSliceMap.get(fileId1).longValue());
assertEquals(3, fileSliceMap.get(fileId2).longValue());
assertEquals(2, fileSliceMap.get(fileId3).longValue());
assertEquals(1, fileSliceMap.get(fileId4).longValue());
List<HoodieDataFile> dataFileList =
fsView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime4)
roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime4)
.collect(Collectors.toList());
assertEquals(dataFileList.size(), 3);
assertEquals(3, dataFileList.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFileList) {
filenames.add(status.getFileName());
@@ -176,18 +210,40 @@ public class HoodieTableFileSystemViewTest {
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
// Reset the max commit time
List<HoodieDataFile> statuses2 =
fsView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3)
.collect(Collectors.toList());
assertEquals(statuses2.size(), 3);
filenames = Sets.newHashSet();
for (HoodieDataFile status : statuses2) {
List<HoodieLogFile> logFilesList =
rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4)
.map(slice -> slice.getLogFiles())
.flatMap(logFileList -> logFileList)
.collect(Collectors.toList());
assertEquals(logFilesList.size(), 4);
for (HoodieLogFile logFile: logFilesList) {
filenames.add(logFile.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)));
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1)));
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)));
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)));
// Reset the max commit time
List<HoodieDataFile> dataFiles =
roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3)
.collect(Collectors.toList());
assertEquals(dataFiles.size(), 3);
filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFiles) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
logFilesList =
rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3)
.map(slice -> slice.getLogFiles())
.flatMap(logFileList -> logFileList).collect(Collectors.toList());
assertEquals(logFilesList.size(), 1);
assertTrue(logFilesList.get(0).getFileName().equals(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)));
}
@Test
@@ -225,12 +281,12 @@ public class HoodieTableFileSystemViewTest {
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(statuses.length, 7);
assertEquals(7, statuses.length);
refreshFsView(null);
List<HoodieFileGroup> fileGroups =
fsView.getAllFileGroups("2016/05/01").collect(Collectors.toList());
assertEquals(fileGroups.size(), 3);
assertEquals(3, fileGroups.size());
for (HoodieFileGroup fileGroup : fileGroups) {
String fileId = fileGroup.getId();
@@ -271,19 +327,28 @@ public class HoodieTableFileSystemViewTest {
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
@@ -291,19 +356,40 @@ public class HoodieTableFileSystemViewTest {
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(statuses.length, 7);
assertEquals(9, statuses.length);
refreshFsView(statuses);
List<HoodieDataFile> statuses1 = fsView
List<HoodieDataFile> dataFiles = roView
.getLatestDataFilesInRange(Lists.newArrayList(commitTime2, commitTime3))
.collect(Collectors.toList());
assertEquals(statuses1.size(), 2);
assertEquals(3, dataFiles.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : statuses1) {
for (HoodieDataFile status : dataFiles) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
List<FileSlice> slices = rtView
.getLatestFileSliceInRange(Lists.newArrayList(commitTime3, commitTime4))
.collect(Collectors.toList());
assertEquals(3, slices.size());
for (FileSlice slice: slices) {
if (slice.getFileId().equals(fileId1)) {
assertEquals(slice.getBaseCommitTime(), commitTime3);
assertTrue(slice.getDataFile().isPresent());
assertEquals(slice.getLogFiles().count(), 0);
} else if (slice.getFileId().equals(fileId2)) {
assertEquals(slice.getBaseCommitTime(), commitTime4);
assertFalse(slice.getDataFile().isPresent());
assertEquals(slice.getLogFiles().count(), 1);
} else if (slice.getFileId().equals(fileId3)) {
assertEquals(slice.getBaseCommitTime(), commitTime4);
assertTrue(slice.getDataFile().isPresent());
assertEquals(slice.getLogFiles().count(), 0);
}
}
}
@Test
@@ -342,20 +428,19 @@ public class HoodieTableFileSystemViewTest {
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(statuses.length, 7);
assertEquals(7, statuses.length);
refreshFsView(null);
List<HoodieDataFile> statuses1 =
fsView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2)
List<HoodieDataFile> dataFiles =
roView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2)
.collect(Collectors.toList());
assertEquals(statuses1.size(), 2);
assertEquals(2, dataFiles.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : statuses1) {
for (HoodieDataFile status : dataFiles) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, 1, fileId2)));
}
@Test
@@ -374,19 +459,28 @@ public class HoodieTableFileSystemViewTest {
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
@@ -394,12 +488,35 @@ public class HoodieTableFileSystemViewTest {
// Now we list the entire partition
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
assertEquals(statuses.length, 7);
assertEquals(10, statuses.length);
refreshFsView(statuses);
List<HoodieFileGroup> fileGroups = fsView
.getAllFileGroups(partitionPath)
.collect(Collectors.toList());
assertEquals(3, fileGroups.size());
for (HoodieFileGroup fileGroup: fileGroups) {
List<FileSlice> slices = fileGroup.getAllFileSlices().collect(Collectors.toList());
if (fileGroup.getId().equals(fileId1)) {
assertEquals(2, slices.size());
assertEquals(commitTime4, slices.get(0).getBaseCommitTime());
assertEquals(commitTime1, slices.get(1).getBaseCommitTime());
} else if (fileGroup.getId().equals(fileId2)) {
assertEquals(3, slices.size());
assertEquals(commitTime3, slices.get(0).getBaseCommitTime());
assertEquals(commitTime2, slices.get(1).getBaseCommitTime());
assertEquals(commitTime1, slices.get(2).getBaseCommitTime());
} else if (fileGroup.getId().equals(fileId3)) {
assertEquals(2, slices.size());
assertEquals(commitTime4, slices.get(0).getBaseCommitTime());
assertEquals(commitTime3, slices.get(1).getBaseCommitTime());
}
}
List<HoodieDataFile> statuses1 =
fsView.getLatestDataFiles().collect(Collectors.toList());
assertEquals(statuses1.size(), 3);
roView.getLatestDataFiles().collect(Collectors.toList());
assertEquals(3, statuses1.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : statuses1) {
filenames.add(status.getFileName());