Timeline Service with Incremental View Syncing support
This commit is contained in:
committed by
vinoth chandar
parent
446f99aa0f
commit
64fec64097
@@ -89,9 +89,14 @@ public class HoodieTestUtils {
|
||||
|
||||
public static HoodieTableMetaClient init(String basePath)
|
||||
throws IOException {
|
||||
return initTableType(getDefaultHadoopConf(), basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
return init(basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
}
|
||||
|
||||
|
||||
public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType)
|
||||
throws IOException {
|
||||
return initTableType(getDefaultHadoopConf(), basePath, tableType);
|
||||
}
|
||||
|
||||
public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath)
|
||||
throws IOException {
|
||||
return initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
|
||||
@@ -29,13 +29,13 @@ public class MockHoodieTimeline extends HoodieActiveTimeline {
|
||||
|
||||
public MockHoodieTimeline(Stream<String> completed, Stream<String> inflights) throws IOException {
|
||||
super();
|
||||
this.instants = Stream.concat(completed.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)),
|
||||
this.setInstants(Stream.concat(completed.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)),
|
||||
inflights.map(s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s)))
|
||||
.sorted(Comparator.comparing(new Function<HoodieInstant, String>() {
|
||||
@Override
|
||||
public String apply(HoodieInstant hoodieInstant) {
|
||||
return hoodieInstant.getFileName();
|
||||
}
|
||||
})).collect(Collectors.toList());
|
||||
})).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -23,14 +23,17 @@ import static org.junit.Assert.assertTrue;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
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.HoodieFileGroupId;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
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.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
@@ -43,6 +46,7 @@ import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
@@ -61,35 +65,38 @@ import org.junit.rules.TemporaryFolder;
|
||||
@SuppressWarnings("ResultOfMethodCallIgnored")
|
||||
public class HoodieTableFileSystemViewTest {
|
||||
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private String basePath;
|
||||
private HoodieTableFileSystemView fsView;
|
||||
private TableFileSystemView.ReadOptimizedView roView;
|
||||
private TableFileSystemView.RealtimeView rtView;
|
||||
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
protected String basePath;
|
||||
protected SyncableFileSystemView fsView;
|
||||
protected TableFileSystemView.ReadOptimizedView roView;
|
||||
protected TableFileSystemView.RealtimeView rtView;
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder tmpFolder = new TemporaryFolder();
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath());;
|
||||
basePath = metaClient.getBasePath();
|
||||
fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
|
||||
roView = (TableFileSystemView.ReadOptimizedView) fsView;
|
||||
rtView = (TableFileSystemView.RealtimeView) fsView;
|
||||
initializeMetaClient();
|
||||
refreshFsView();
|
||||
}
|
||||
|
||||
private void refreshFsView(FileStatus[] statuses) {
|
||||
protected void initializeMetaClient() throws IOException {
|
||||
metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
|
||||
basePath = metaClient.getBasePath();
|
||||
}
|
||||
|
||||
protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) throws IOException {
|
||||
return new HoodieTableFileSystemView(metaClient, timeline);
|
||||
}
|
||||
|
||||
protected void refreshFsView() throws IOException {
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
if (statuses != null) {
|
||||
fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants(),
|
||||
statuses);
|
||||
} else {
|
||||
fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants());
|
||||
if (null != fsView) {
|
||||
fsView.close();
|
||||
fsView = null;
|
||||
}
|
||||
fsView = getFileSystemView(
|
||||
metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants());
|
||||
roView = (TableFileSystemView.ReadOptimizedView) fsView;
|
||||
rtView = (TableFileSystemView.RealtimeView) fsView;
|
||||
}
|
||||
@@ -100,6 +107,11 @@ public class HoodieTableFileSystemViewTest {
|
||||
*/
|
||||
@Test
|
||||
public void testViewForFileSlicesWithNoBaseFile() throws Exception {
|
||||
testViewForFileSlicesWithNoBaseFile(1, 0);
|
||||
}
|
||||
|
||||
protected void testViewForFileSlicesWithNoBaseFile(int expNumTotalFileSlices, int expNumTotalDataFiles)
|
||||
throws Exception {
|
||||
String partitionPath = "2016/05/01";
|
||||
new File(basePath + "/" + partitionPath).mkdirs();
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
@@ -120,7 +132,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
commitTimeline.saveAsComplete(deltaInstant2, Optional.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant3, Optional.empty());
|
||||
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
|
||||
List<HoodieDataFile> dataFiles = roView.getLatestDataFiles().collect(Collectors.toList());
|
||||
assertTrue("No data file expected", dataFiles.isEmpty());
|
||||
@@ -159,26 +171,33 @@ public class HoodieTableFileSystemViewTest {
|
||||
assertEquals("Correct number of log-files shows up in file-slice", 2, logFiles.size());
|
||||
assertEquals("Log File Order check", fileName2, logFiles.get(0).getFileName());
|
||||
assertEquals("Log File Order check", fileName1, logFiles.get(1).getFileName());
|
||||
|
||||
assertEquals("Total number of file-slices in view matches expected", expNumTotalFileSlices,
|
||||
rtView.getAllFileSlices(partitionPath).count());
|
||||
assertEquals("Total number of data-files in view matches expected", expNumTotalDataFiles,
|
||||
roView.getAllDataFiles(partitionPath).count());
|
||||
assertEquals("Total number of file-groups in view matches expected", 1,
|
||||
fsView.getAllFileGroups(partitionPath).count());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testViewForFileSlicesWithNoBaseFileAndRequestedCompaction() throws Exception {
|
||||
testViewForFileSlicesWithAsyncCompaction(true, false);
|
||||
testViewForFileSlicesWithAsyncCompaction(true, false, 2, 1, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testViewForFileSlicesWithBaseFileAndRequestedCompaction() throws Exception {
|
||||
testViewForFileSlicesWithAsyncCompaction(false, false);
|
||||
testViewForFileSlicesWithAsyncCompaction(false, false, 2, 2, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testViewForFileSlicesWithNoBaseFileAndInflightCompaction() throws Exception {
|
||||
testViewForFileSlicesWithAsyncCompaction(true, true);
|
||||
testViewForFileSlicesWithAsyncCompaction(true, true, 2, 1, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testViewForFileSlicesWithBaseFileAndInflightCompaction() throws Exception {
|
||||
testViewForFileSlicesWithAsyncCompaction(false, true);
|
||||
testViewForFileSlicesWithAsyncCompaction(false, true, 2, 2, true);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -210,10 +229,14 @@ public class HoodieTableFileSystemViewTest {
|
||||
* simulate inserts going directly to log files
|
||||
* @param isCompactionInFlight if set, compaction was inflight (running) when view was tested first time,
|
||||
* otherwise compaction was in requested state
|
||||
* @param expTotalFileSlices Total number of file-slices across file-groups in the partition path
|
||||
* @param expTotalDataFiles Total number of data-files across file-groups in the partition path
|
||||
* @param includeInvalidAndInflight Whether view includes inflight and invalid file-groups.
|
||||
* @throws Exception
|
||||
*/
|
||||
private void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFile,
|
||||
boolean isCompactionInFlight) throws Exception {
|
||||
protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFile,
|
||||
boolean isCompactionInFlight, int expTotalFileSlices, int expTotalDataFiles,
|
||||
boolean includeInvalidAndInflight) throws Exception {
|
||||
String partitionPath = "2016/05/01";
|
||||
new File(basePath + "/" + partitionPath).mkdirs();
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
@@ -241,7 +264,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
commitTimeline.saveAsComplete(deltaInstant2, Optional.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant3, Optional.empty());
|
||||
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
List<FileSlice> fileSlices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
String compactionRequestedTime = "4";
|
||||
String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, 1, fileId);
|
||||
@@ -262,6 +285,15 @@ public class HoodieTableFileSystemViewTest {
|
||||
commitTimeline.saveToCompactionRequested(compactionInstant, AvroUtils.serializeCompactionPlan(compactionPlan));
|
||||
}
|
||||
|
||||
// View immediately after scheduling compaction
|
||||
refreshFsView();
|
||||
List<FileSlice> slices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
assertEquals("Expected latest file-slices", 1, slices.size());
|
||||
assertEquals("Base-Instant must be compaction Instant", compactionRequestedTime,
|
||||
slices.get(0).getBaseInstantTime());
|
||||
assertFalse("Latest File Slice must not have data-file", slices.get(0).getDataFile().isPresent());
|
||||
assertTrue("Latest File Slice must not have any log-files", slices.get(0).getLogFiles().count() == 0);
|
||||
|
||||
// Fake delta-ingestion after compaction-requested
|
||||
String deltaInstantTime4 = "5";
|
||||
String deltaInstantTime5 = "6";
|
||||
@@ -275,7 +307,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
|
||||
commitTimeline.saveAsComplete(deltaInstant4, Optional.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant5, Optional.empty());
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
|
||||
List<HoodieDataFile> dataFiles = roView.getAllDataFiles(partitionPath).collect(Collectors.toList());
|
||||
if (skipCreatingDataFile) {
|
||||
@@ -381,7 +413,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
// Mark instant as inflight
|
||||
commitTimeline.saveToInflight(new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION,
|
||||
inflightDeltaInstantTime), Optional.empty());
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
|
||||
List<FileSlice> allRawFileSlices = getAllRawFileSlices(partitionPath).collect(Collectors.toList());
|
||||
dataFiles = allRawFileSlices.stream().flatMap(slice -> {
|
||||
@@ -390,58 +422,60 @@ public class HoodieTableFileSystemViewTest {
|
||||
}
|
||||
return Stream.empty();
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("Inflight/Orphan data-file is also expected", 2
|
||||
+ (isCompactionInFlight ? 1 : 0) + (skipCreatingDataFile ? 0 : 1), dataFiles.size());
|
||||
Set<String> fileNames = dataFiles.stream().map(HoodieDataFile::getFileName).collect(Collectors.toSet());
|
||||
assertTrue("Expect orphan data-file to be present", fileNames.contains(orphanDataFileName));
|
||||
assertTrue("Expect inflight data-file to be present", fileNames.contains(inflightDataFileName));
|
||||
if (!skipCreatingDataFile) {
|
||||
assertTrue("Expect old committed data-file", fileNames.contains(dataFileName));
|
||||
|
||||
if (includeInvalidAndInflight) {
|
||||
assertEquals("Inflight/Orphan data-file is also expected", 2
|
||||
+ (isCompactionInFlight ? 1 : 0) + (skipCreatingDataFile ? 0 : 1), dataFiles.size());
|
||||
Set<String> fileNames = dataFiles.stream().map(HoodieDataFile::getFileName).collect(Collectors.toSet());
|
||||
assertTrue("Expect orphan data-file to be present", fileNames.contains(orphanDataFileName));
|
||||
assertTrue("Expect inflight data-file to be present", fileNames.contains(inflightDataFileName));
|
||||
if (!skipCreatingDataFile) {
|
||||
assertTrue("Expect old committed data-file", fileNames.contains(dataFileName));
|
||||
}
|
||||
|
||||
if (isCompactionInFlight) {
|
||||
assertTrue("Expect inflight compacted data file to be present", fileNames.contains(compactDataFileName));
|
||||
}
|
||||
|
||||
fileSliceList = getLatestRawFileSlices(partitionPath).collect(Collectors.toList());
|
||||
assertEquals("Expect both inflight and orphan file-slice to be included",
|
||||
includeInvalidAndInflight ? 5 : 1, fileSliceList.size());
|
||||
Map<String, FileSlice> fileSliceMap =
|
||||
fileSliceList.stream().collect(Collectors.toMap(FileSlice::getFileId, r -> r));
|
||||
FileSlice orphanFileSliceWithDataFile = fileSliceMap.get(orphanFileId1);
|
||||
FileSlice orphanFileSliceWithLogFile = fileSliceMap.get(orphanFileId2);
|
||||
FileSlice inflightFileSliceWithDataFile = fileSliceMap.get(inflightFileId1);
|
||||
FileSlice inflightFileSliceWithLogFile = fileSliceMap.get(inflightFileId2);
|
||||
|
||||
assertEquals("Orphan File Slice with data-file check base-commit", invalidInstantId,
|
||||
orphanFileSliceWithDataFile.getBaseInstantTime());
|
||||
assertEquals("Orphan File Slice with data-file check data-file", orphanDataFileName,
|
||||
orphanFileSliceWithDataFile.getDataFile().get().getFileName());
|
||||
assertEquals("Orphan File Slice with data-file check data-file", 0,
|
||||
orphanFileSliceWithDataFile.getLogFiles().count());
|
||||
assertEquals("Inflight File Slice with data-file check base-commit", inflightDeltaInstantTime,
|
||||
inflightFileSliceWithDataFile.getBaseInstantTime());
|
||||
assertEquals("Inflight File Slice with data-file check data-file", inflightDataFileName,
|
||||
inflightFileSliceWithDataFile.getDataFile().get().getFileName());
|
||||
assertEquals("Inflight File Slice with data-file check data-file", 0,
|
||||
inflightFileSliceWithDataFile.getLogFiles().count());
|
||||
assertEquals("Orphan File Slice with log-file check base-commit", invalidInstantId,
|
||||
orphanFileSliceWithLogFile.getBaseInstantTime());
|
||||
assertFalse("Orphan File Slice with log-file check data-file",
|
||||
orphanFileSliceWithLogFile.getDataFile().isPresent());
|
||||
logFiles = orphanFileSliceWithLogFile.getLogFiles().collect(Collectors.toList());
|
||||
assertEquals("Orphan File Slice with log-file check data-file", 1, logFiles.size());
|
||||
assertEquals("Orphan File Slice with log-file check data-file", orphanLogFileName,
|
||||
logFiles.get(0).getFileName());
|
||||
assertEquals("Inflight File Slice with log-file check base-commit", inflightDeltaInstantTime,
|
||||
inflightFileSliceWithLogFile.getBaseInstantTime());
|
||||
assertFalse("Inflight File Slice with log-file check data-file",
|
||||
inflightFileSliceWithLogFile.getDataFile().isPresent());
|
||||
logFiles = inflightFileSliceWithLogFile.getLogFiles().collect(Collectors.toList());
|
||||
assertEquals("Inflight File Slice with log-file check data-file", 1, logFiles.size());
|
||||
assertEquals("Inflight File Slice with log-file check data-file", inflightLogFileName,
|
||||
logFiles.get(0).getFileName());
|
||||
}
|
||||
|
||||
if (isCompactionInFlight) {
|
||||
assertTrue("Expect inflight compacted data file to be present", fileNames.contains(compactDataFileName));
|
||||
}
|
||||
|
||||
fileSliceList = getLatestRawFileSlices(partitionPath).collect(Collectors.toList());
|
||||
assertEquals("Expect both inflight and orphan file-slice to be included",
|
||||
5, fileSliceList.size());
|
||||
Map<String, FileSlice> fileSliceMap =
|
||||
fileSliceList.stream().collect(Collectors.toMap(FileSlice::getFileId, r -> r));
|
||||
FileSlice orphanFileSliceWithDataFile = fileSliceMap.get(orphanFileId1);
|
||||
FileSlice orphanFileSliceWithLogFile = fileSliceMap.get(orphanFileId2);
|
||||
FileSlice inflightFileSliceWithDataFile = fileSliceMap.get(inflightFileId1);
|
||||
FileSlice inflightFileSliceWithLogFile = fileSliceMap.get(inflightFileId2);
|
||||
|
||||
assertEquals("Orphan File Slice with data-file check base-commit", invalidInstantId,
|
||||
orphanFileSliceWithDataFile.getBaseInstantTime());
|
||||
assertEquals("Orphan File Slice with data-file check data-file", orphanDataFileName,
|
||||
orphanFileSliceWithDataFile.getDataFile().get().getFileName());
|
||||
assertEquals("Orphan File Slice with data-file check data-file", 0,
|
||||
orphanFileSliceWithDataFile.getLogFiles().count());
|
||||
assertEquals("Inflight File Slice with data-file check base-commit", inflightDeltaInstantTime,
|
||||
inflightFileSliceWithDataFile.getBaseInstantTime());
|
||||
assertEquals("Inflight File Slice with data-file check data-file", inflightDataFileName,
|
||||
inflightFileSliceWithDataFile.getDataFile().get().getFileName());
|
||||
assertEquals("Inflight File Slice with data-file check data-file", 0,
|
||||
inflightFileSliceWithDataFile.getLogFiles().count());
|
||||
assertEquals("Orphan File Slice with log-file check base-commit", invalidInstantId,
|
||||
orphanFileSliceWithLogFile.getBaseInstantTime());
|
||||
assertFalse("Orphan File Slice with log-file check data-file",
|
||||
orphanFileSliceWithLogFile.getDataFile().isPresent());
|
||||
logFiles = orphanFileSliceWithLogFile.getLogFiles().collect(Collectors.toList());
|
||||
assertEquals("Orphan File Slice with log-file check data-file", 1, logFiles.size());
|
||||
assertEquals("Orphan File Slice with log-file check data-file", orphanLogFileName,
|
||||
logFiles.get(0).getFileName());
|
||||
assertEquals("Inflight File Slice with log-file check base-commit", inflightDeltaInstantTime,
|
||||
inflightFileSliceWithLogFile.getBaseInstantTime());
|
||||
assertFalse("Inflight File Slice with log-file check data-file",
|
||||
inflightFileSliceWithLogFile.getDataFile().isPresent());
|
||||
logFiles = inflightFileSliceWithLogFile.getLogFiles().collect(Collectors.toList());
|
||||
assertEquals("Inflight File Slice with log-file check data-file", 1, logFiles.size());
|
||||
assertEquals("Inflight File Slice with log-file check data-file", inflightLogFileName,
|
||||
logFiles.get(0).getFileName());
|
||||
|
||||
// Now simulate Compaction completing - Check the view
|
||||
if (!isCompactionInFlight) {
|
||||
// For inflight compaction, we already create a data-file to test concurrent inflight case.
|
||||
@@ -455,14 +489,14 @@ public class HoodieTableFileSystemViewTest {
|
||||
}
|
||||
compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime);
|
||||
commitTimeline.saveAsComplete(compactionInstant, Optional.empty());
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
// populate the cache
|
||||
roView.getAllDataFiles(partitionPath);
|
||||
|
||||
fileSliceList = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
System.out.println("FILESLICE LIST=" + fileSliceList);
|
||||
dataFiles = fileSliceList.stream().map(FileSlice::getDataFile)
|
||||
.filter(Optional::isPresent).map(Optional::get).collect(Collectors.toList());
|
||||
System.out.println("fileSliceList : " + fileSliceList);
|
||||
assertEquals("Expect only one data-files in latest view as there is only one file-group", 1, dataFiles.size());
|
||||
assertEquals("Data Filename must match", compactDataFileName, dataFiles.get(0).getFileName());
|
||||
assertEquals("Only one latest file-slice in the partition", 1, fileSliceList.size());
|
||||
@@ -502,6 +536,14 @@ public class HoodieTableFileSystemViewTest {
|
||||
assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(),
|
||||
compactionRequestedTime);
|
||||
});
|
||||
|
||||
assertEquals("Total number of file-slices in partitions matches expected", expTotalFileSlices,
|
||||
rtView.getAllFileSlices(partitionPath).count());
|
||||
assertEquals("Total number of data-files in partitions matches expected", expTotalDataFiles,
|
||||
roView.getAllDataFiles(partitionPath).count());
|
||||
// file-groups includes inflight/invalid file-ids
|
||||
assertEquals("Total number of file-groups in partitions matches expected",
|
||||
5, fsView.getAllFileGroups(partitionPath).count());
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -518,7 +560,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
String commitTime1 = "1";
|
||||
String fileName1 = FSUtils.makeDataFileName(commitTime1, 1, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
assertFalse("No commit, should not find any data file",
|
||||
roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst()
|
||||
.isPresent());
|
||||
@@ -527,7 +569,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
|
||||
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
|
||||
commitTimeline.saveAsComplete(instant1, Optional.empty());
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
assertEquals("", fileName1,
|
||||
roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get()
|
||||
.getFileName());
|
||||
@@ -536,7 +578,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
String commitTime2 = "2";
|
||||
String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
assertEquals("", fileName1,
|
||||
roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get()
|
||||
.getFileName());
|
||||
@@ -544,7 +586,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
// Make it safe
|
||||
HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2);
|
||||
commitTimeline.saveAsComplete(instant2, Optional.empty());
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
assertEquals("", fileName2,
|
||||
roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get()
|
||||
.getFileName());
|
||||
@@ -552,6 +594,10 @@ public class HoodieTableFileSystemViewTest {
|
||||
|
||||
@Test
|
||||
public void testStreamLatestVersionInPartition() throws IOException {
|
||||
testStreamLatestVersionInPartition(false);
|
||||
}
|
||||
|
||||
public void testStreamLatestVersionInPartition(boolean isLatestFileSliceOnly) throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
@@ -588,16 +634,16 @@ public class HoodieTableFileSystemViewTest {
|
||||
// Now we list the entire partition
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(11, statuses.length);
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
|
||||
// Check files as of lastest commit.
|
||||
List<FileSlice> allSlices = rtView.getAllFileSlices("2016/05/01").collect(Collectors.toList());
|
||||
assertEquals(8, allSlices.size());
|
||||
assertEquals(isLatestFileSliceOnly ? 4 : 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(isLatestFileSliceOnly ? 1 : 2, fileSliceMap.get(fileId1).longValue());
|
||||
assertEquals(isLatestFileSliceOnly ? 1 : 3, fileSliceMap.get(fileId2).longValue());
|
||||
assertEquals(isLatestFileSliceOnly ? 1 : 2, fileSliceMap.get(fileId3).longValue());
|
||||
assertEquals(1, fileSliceMap.get(fileId4).longValue());
|
||||
|
||||
List<HoodieDataFile> dataFileList = roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime4)
|
||||
@@ -627,24 +673,35 @@ public class HoodieTableFileSystemViewTest {
|
||||
// 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)));
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertEquals(3, dataFiles.size());
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
} else {
|
||||
assertEquals(1, dataFiles.size());
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
}
|
||||
|
||||
logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3).map(slice -> slice.getLogFiles())
|
||||
.flatMap(logFileList -> logFileList).collect(Collectors.toList());
|
||||
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
|
||||
public void testStreamEveryVersionInPartition() throws IOException {
|
||||
testStreamEveryVersionInPartition(false);
|
||||
}
|
||||
|
||||
protected void testStreamEveryVersionInPartition(boolean isLatestFileSliceOnly) throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
@@ -673,7 +730,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(7, statuses.length);
|
||||
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups("2016/05/01").collect(Collectors.toList());
|
||||
assertEquals(3, fileGroups.size());
|
||||
|
||||
@@ -684,21 +741,36 @@ public class HoodieTableFileSystemViewTest {
|
||||
assertEquals("All same fileId should be grouped", fileId, dataFile.getFileId());
|
||||
filenames.add(dataFile.getFileName());
|
||||
});
|
||||
Set<String> expFileNames = new HashSet<>();
|
||||
if (fileId.equals(fileId1)) {
|
||||
assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId1),
|
||||
FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
|
||||
if (!isLatestFileSliceOnly) {
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime1, 1, fileId1));
|
||||
}
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime4, 1, fileId1));
|
||||
assertEquals(expFileNames, filenames);
|
||||
} else if (fileId.equals(fileId2)) {
|
||||
assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId2),
|
||||
FSUtils.makeDataFileName(commitTime2, 1, fileId2), FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
if (!isLatestFileSliceOnly) {
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime1, 1, fileId2));
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime2, 1, fileId2));
|
||||
}
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime3, 1, fileId2));
|
||||
assertEquals(expFileNames, filenames);
|
||||
} else {
|
||||
assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime3, 1, fileId3),
|
||||
FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
|
||||
if (!isLatestFileSliceOnly) {
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime3, 1, fileId3));
|
||||
}
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime4, 1, fileId3));
|
||||
assertEquals(expFileNames, filenames);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void streamLatestVersionInRange() throws IOException {
|
||||
public void testStreamLatestVersionInRange() throws IOException {
|
||||
testStreamLatestVersionInRange(false);
|
||||
}
|
||||
|
||||
protected void testStreamLatestVersionInRange(boolean isLatestFileSliceOnly) throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
@@ -718,7 +790,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
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))
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))
|
||||
.createNewFile();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
@@ -733,16 +805,23 @@ public class HoodieTableFileSystemViewTest {
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(9, statuses.length);
|
||||
|
||||
refreshFsView(statuses);
|
||||
refreshFsView();
|
||||
// Populate view for partition
|
||||
roView.getAllDataFiles("2016/05/01/");
|
||||
|
||||
List<HoodieDataFile> dataFiles = roView.getLatestDataFilesInRange(Lists.newArrayList(commitTime2, commitTime3))
|
||||
.collect(Collectors.toList());
|
||||
assertEquals(3, dataFiles.size());
|
||||
assertEquals(isLatestFileSliceOnly ? 2 : 3, dataFiles.size());
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile status : dataFiles) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
}
|
||||
|
||||
List<FileSlice> slices = rtView.getLatestFileSliceInRange(Lists.newArrayList(commitTime3, commitTime4))
|
||||
.collect(Collectors.toList());
|
||||
@@ -753,8 +832,8 @@ public class HoodieTableFileSystemViewTest {
|
||||
assertTrue(slice.getDataFile().isPresent());
|
||||
assertEquals(slice.getLogFiles().count(), 0);
|
||||
} else if (slice.getFileId().equals(fileId2)) {
|
||||
assertEquals(slice.getBaseInstantTime(), commitTime4);
|
||||
assertFalse(slice.getDataFile().isPresent());
|
||||
assertEquals(slice.getBaseInstantTime(), commitTime3);
|
||||
assertTrue(slice.getDataFile().isPresent());
|
||||
assertEquals(slice.getLogFiles().count(), 1);
|
||||
} else if (slice.getFileId().equals(fileId3)) {
|
||||
assertEquals(slice.getBaseInstantTime(), commitTime4);
|
||||
@@ -765,7 +844,11 @@ public class HoodieTableFileSystemViewTest {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void streamLatestVersionsBefore() throws IOException {
|
||||
public void testStreamLatestVersionsBefore() throws IOException {
|
||||
testStreamLatestVersionsBefore(false);
|
||||
}
|
||||
|
||||
protected void testStreamLatestVersionsBefore(boolean isLatestFileSliceOnly) throws IOException {
|
||||
// Put some files in the partition
|
||||
String partitionPath = "2016/05/01/";
|
||||
String fullPartitionPath = basePath + "/" + partitionPath;
|
||||
@@ -795,22 +878,30 @@ public class HoodieTableFileSystemViewTest {
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(7, statuses.length);
|
||||
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
List<HoodieDataFile> dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2)
|
||||
.collect(Collectors.toList());
|
||||
assertEquals(2, dataFiles.size());
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile status : dataFiles) {
|
||||
filenames.add(status.getFileName());
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertEquals(2, dataFiles.size());
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile status : dataFiles) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, 1, fileId2)));
|
||||
} else {
|
||||
assertEquals(0, dataFiles.size());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, 1, fileId2)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void streamLatestVersions() throws IOException {
|
||||
public void testStreamLatestVersions() throws IOException {
|
||||
testStreamLatestVersions(false);
|
||||
}
|
||||
|
||||
protected void testStreamLatestVersions(boolean isLatestFileSliceOnly) throws IOException {
|
||||
// Put some files in the partition
|
||||
String partitionPath = "2016/05/01/";
|
||||
String partitionPath = "2016/05/01";
|
||||
String fullPartitionPath = basePath + "/" + partitionPath;
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
String commitTime1 = "1";
|
||||
@@ -821,21 +912,28 @@ public class HoodieTableFileSystemViewTest {
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
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, 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))
|
||||
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, fileId2)).createNewFile();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).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();
|
||||
@@ -846,26 +944,32 @@ public class HoodieTableFileSystemViewTest {
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(10, statuses.length);
|
||||
|
||||
refreshFsView(statuses);
|
||||
|
||||
refreshFsView();
|
||||
fsView.getAllDataFiles(partitionPath);
|
||||
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());
|
||||
String fileId = fileGroup.getFileGroupId().getFileId();
|
||||
if (fileId.equals(fileId1)) {
|
||||
assertEquals(2, slices.size());
|
||||
assertEquals(isLatestFileSliceOnly ? 1 : 2, slices.size());
|
||||
assertEquals(commitTime4, slices.get(0).getBaseInstantTime());
|
||||
assertEquals(commitTime1, slices.get(1).getBaseInstantTime());
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertEquals(commitTime1, slices.get(1).getBaseInstantTime());
|
||||
}
|
||||
} else if (fileId.equals(fileId2)) {
|
||||
assertEquals(3, slices.size());
|
||||
assertEquals(isLatestFileSliceOnly ? 1 : 3, slices.size());
|
||||
assertEquals(commitTime3, slices.get(0).getBaseInstantTime());
|
||||
assertEquals(commitTime2, slices.get(1).getBaseInstantTime());
|
||||
assertEquals(commitTime1, slices.get(2).getBaseInstantTime());
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertEquals(commitTime2, slices.get(1).getBaseInstantTime());
|
||||
assertEquals(commitTime1, slices.get(2).getBaseInstantTime());
|
||||
}
|
||||
} else if (fileId.equals(fileId3)) {
|
||||
assertEquals(2, slices.size());
|
||||
assertEquals(isLatestFileSliceOnly ? 1 : 2, slices.size());
|
||||
assertEquals(commitTime4, slices.get(0).getBaseInstantTime());
|
||||
assertEquals(commitTime3, slices.get(1).getBaseInstantTime());
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertEquals(commitTime3, slices.get(1).getBaseInstantTime());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -925,9 +1029,11 @@ public class HoodieTableFileSystemViewTest {
|
||||
new Path(fullPartitionPath1), new Path(fullPartitionPath2), new Path(fullPartitionPath3)
|
||||
});
|
||||
assertEquals(6, statuses.length);
|
||||
refreshFsView(statuses);
|
||||
refreshFsView();
|
||||
Arrays.asList(partitionPath1, partitionPath2, partitionPath3).forEach(p -> fsView.getAllFileGroups(p).count());
|
||||
|
||||
List<HoodieFileGroup> groups = fsView.getAllFileGroups().collect(Collectors.toList());
|
||||
List<HoodieFileGroup> groups = Stream.of(partitionPath1, partitionPath2, partitionPath3)
|
||||
.flatMap(p -> fsView.getAllFileGroups(p)).collect(Collectors.toList());
|
||||
Assert.assertEquals("Expected number of file-groups", 3, groups.size());
|
||||
Assert.assertEquals("Partitions must be different for file-groups", 3,
|
||||
groups.stream().map(HoodieFileGroup::getPartitionPath).collect(Collectors.toSet()).size());
|
||||
@@ -979,7 +1085,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
|
||||
commitTimeline.saveAsComplete(deltaInstant4, Optional.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant5, Optional.empty());
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
|
||||
// Test Data Files
|
||||
List<HoodieDataFile> dataFiles = roView.getAllDataFiles(partitionPath1).collect(Collectors.toList());
|
||||
@@ -1019,18 +1125,16 @@ public class HoodieTableFileSystemViewTest {
|
||||
assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName());
|
||||
});
|
||||
|
||||
Assert.assertEquals(3, fsView.getFgIdToPendingCompaction().size());
|
||||
Set<String> partitionsInCompaction =
|
||||
fsView.getFgIdToPendingCompaction().keySet().stream().map(HoodieFileGroupId::getPartitionPath)
|
||||
.collect(Collectors.toSet());
|
||||
Assert.assertEquals(3, fsView.getPendingCompactionOperations().count());
|
||||
Set<String> partitionsInCompaction = fsView.getPendingCompactionOperations().map(Pair::getValue)
|
||||
.map(CompactionOperation::getPartitionPath).collect(Collectors.toSet());
|
||||
Assert.assertEquals(3, partitionsInCompaction.size());
|
||||
Assert.assertTrue(partitionsInCompaction.contains(partitionPath1));
|
||||
Assert.assertTrue(partitionsInCompaction.contains(partitionPath2));
|
||||
Assert.assertTrue(partitionsInCompaction.contains(partitionPath3));
|
||||
|
||||
Set<String> fileIdsInCompaction =
|
||||
fsView.getFgIdToPendingCompaction().keySet().stream().map(HoodieFileGroupId::getFileId)
|
||||
.collect(Collectors.toSet());
|
||||
Set<String> fileIdsInCompaction = fsView.getPendingCompactionOperations().map(Pair::getValue)
|
||||
.map(CompactionOperation::getFileId).collect(Collectors.toSet());
|
||||
Assert.assertEquals(1, fileIdsInCompaction.size());
|
||||
Assert.assertTrue(fileIdsInCompaction.contains(fileId));
|
||||
}
|
||||
|
||||
@@ -0,0 +1,783 @@
|
||||
/*
|
||||
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.table.view;
|
||||
|
||||
import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.avro.model.HoodieRestoreMetadata;
|
||||
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
|
||||
import com.uber.hoodie.common.HoodieCleanStat;
|
||||
import com.uber.hoodie.common.HoodieRollbackStat;
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
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.HoodieFileGroupId;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
|
||||
import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.common.util.CompactionUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
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.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class IncrementalFSViewSyncTest {
|
||||
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
protected String basePath;
|
||||
|
||||
private final List<String> partitions = Arrays.asList("2018/01/01", "2018/01/02",
|
||||
"2019/03/01");
|
||||
private final List<String> fileIdsPerPartition =
|
||||
IntStream.range(0, 10).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList());
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder tmpFolder = new TemporaryFolder();
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
initializeMetaClient();
|
||||
refreshFsView();
|
||||
}
|
||||
|
||||
protected void initializeMetaClient() throws IOException {
|
||||
metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
|
||||
basePath = metaClient.getBasePath();
|
||||
partitions.forEach(p -> new File(basePath + "/" + p).mkdirs());
|
||||
}
|
||||
|
||||
protected void refreshFsView() throws IOException {
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
}
|
||||
|
||||
protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient) throws IOException {
|
||||
return getNewFileSystemView(metaClient, metaClient.getActiveTimeline().filterCompletedAndCompactionInstants());
|
||||
}
|
||||
|
||||
protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline timeline)
|
||||
throws IOException {
|
||||
return new HoodieTableFileSystemView(metaClient, timeline, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyPartitionsAndTimeline() throws IOException {
|
||||
SyncableFileSystemView view = getNewFileSystemView(metaClient);
|
||||
Assert.assertFalse(view.getLastInstant().isPresent());
|
||||
partitions.forEach(p -> Assert.assertEquals(0, view.getLatestFileSlices(p).count()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAsyncCompaction() throws IOException {
|
||||
SyncableFileSystemView view = getNewFileSystemView(metaClient);
|
||||
view.sync();
|
||||
|
||||
// Run 3 ingestion on MOR table (3 delta commits)
|
||||
Map<String, List<String>> instantsToFiles =
|
||||
testMultipleWriteSteps(view, Arrays.asList("11", "12", "13"), true, "11");
|
||||
|
||||
// Schedule Compaction
|
||||
scheduleCompaction(view, "14");
|
||||
|
||||
// Restore pending compaction
|
||||
unscheduleCompaction(view, "14", "13", "11");
|
||||
|
||||
// Add one more delta instant
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("15"), true, "11"));
|
||||
|
||||
// Schedule Compaction again
|
||||
scheduleCompaction(view, "16");
|
||||
|
||||
// Run Compaction - This will be the second file-slice
|
||||
testMultipleWriteSteps(view, Arrays.asList("16"), false, "16", 2);
|
||||
|
||||
// Run 2 more ingest
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("17", "18"), true, "16", 2));
|
||||
|
||||
// Schedule Compaction again
|
||||
scheduleCompaction(view, "19");
|
||||
|
||||
// Run one more ingestion after pending compaction. THis will be 3rd slice
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("20"), true, "19", 3));
|
||||
|
||||
// Clean first slice
|
||||
testCleans(view, Arrays.asList("21"),
|
||||
new ImmutableMap.Builder<String, List<String>>().put("11", Arrays.asList("12", "13", "15")).build(),
|
||||
instantsToFiles,
|
||||
Arrays.asList("11"));
|
||||
|
||||
// Add one more ingestion instant. This should be 2nd slice now
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("22"), true, "19", 2));
|
||||
|
||||
// Restore last ingestion
|
||||
testRestore(view, Arrays.asList("23"), true, new HashMap<>(), Arrays.asList("22"), "24", false);
|
||||
|
||||
// Run one more ingestion. THis is still 2nd slice
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("24"), true, "19", 2));
|
||||
|
||||
// Finish Compaction
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("19"), false, "19", 2,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24"))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIngestion() throws IOException {
|
||||
SyncableFileSystemView view = getNewFileSystemView(metaClient);
|
||||
|
||||
// Add an empty ingestion
|
||||
String firstEmptyInstantTs = "11";
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs),
|
||||
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
|
||||
view.sync();
|
||||
Assert.assertTrue(view.getLastInstant().isPresent());
|
||||
Assert.assertEquals("11", view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
Assert.assertEquals(HoodieTimeline.COMMIT_ACTION, view.getLastInstant().get().getAction());
|
||||
partitions.forEach(p -> Assert.assertEquals(0, view.getLatestFileSlices(p).count()));
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
for (String partition : partitions) {
|
||||
newView.getAllFileGroups(partition).count();
|
||||
}
|
||||
|
||||
areViewsConsistent(view, newView, 0L);
|
||||
|
||||
// Add 3 non-empty ingestions to COW table
|
||||
Map<String, List<String>> instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("12", "13", "14"));
|
||||
|
||||
// restore instants in reverse order till we rollback all
|
||||
testRestore(view, Arrays.asList("15", "16", "17"), false, instantsToFiles,
|
||||
Arrays.asList("14", "13", "12"), "17", true);
|
||||
|
||||
// Add 5 non-empty ingestions back-to-back
|
||||
instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("18", "19", "20"));
|
||||
|
||||
// Clean instants.
|
||||
testCleans(view, Arrays.asList("21", "22"), instantsToFiles, Arrays.asList("18", "19"));
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests FS View incremental syncing behavior when multiple instants gets committed
|
||||
*/
|
||||
@Test
|
||||
public void testMultipleTransitions() throws IOException {
|
||||
|
||||
SyncableFileSystemView view1 = getNewFileSystemView(metaClient);
|
||||
view1.sync();
|
||||
Map<String, List<String>> instantsToFiles = null;
|
||||
|
||||
/**
|
||||
* Case where incremental syncing is catching up on more than one ingestion at a time
|
||||
*/
|
||||
// Run 1 ingestion on MOR table (1 delta commits). View1 is now sync up to this point
|
||||
instantsToFiles =
|
||||
testMultipleWriteSteps(view1, Arrays.asList("11"), true, "11");
|
||||
|
||||
SyncableFileSystemView view2 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
|
||||
// Run 2 more ingestion on MOR table. View1 is not yet synced but View2 is
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("12", "13"), true, "11"));
|
||||
|
||||
// Now Sync view1 and add 1 more ingestion. Check if view1 is able to catchup correctly
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view1, Arrays.asList("14"), true, "11"));
|
||||
|
||||
view2.sync();
|
||||
SyncableFileSystemView view3 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
partitions.stream().forEach(p -> view3.getLatestFileSlices(p).count());
|
||||
view3.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size());
|
||||
|
||||
/**
|
||||
* Case where a compaction is scheduled and then unscheduled
|
||||
*/
|
||||
scheduleCompaction(view2, "15");
|
||||
unscheduleCompaction(view2, "15", "14", "11");
|
||||
view1.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size());
|
||||
SyncableFileSystemView view4 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
partitions.stream().forEach(p -> view4.getLatestFileSlices(p).count());
|
||||
view4.sync();
|
||||
|
||||
/**
|
||||
* Case where a compaction is scheduled, 2 ingestion happens and then a compaction happens
|
||||
*/
|
||||
scheduleCompaction(view2, "16");
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("17", "18"), true, "16", 2));
|
||||
// Compaction
|
||||
testMultipleWriteSteps(view2, Arrays.asList("16"), false, "16", 2,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "18")));
|
||||
view1.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size() * 2);
|
||||
SyncableFileSystemView view5 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
partitions.stream().forEach(p -> view5.getLatestFileSlices(p).count());
|
||||
view5.sync();
|
||||
|
||||
/**
|
||||
* Case where a clean happened and then rounds of ingestion and compaction happened
|
||||
*/
|
||||
testCleans(view2, Arrays.asList("19"),
|
||||
new ImmutableMap.Builder<String, List<String>>().put("11", Arrays.asList("12", "13", "14")).build(),
|
||||
instantsToFiles,
|
||||
Arrays.asList("11"));
|
||||
scheduleCompaction(view2, "20");
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("21", "22"), true, "20", 2));
|
||||
// Compaction
|
||||
testMultipleWriteSteps(view2, Arrays.asList("20"), false, "20", 2,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "22")));
|
||||
//Run one more round of ingestion
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("23", "24"), true, "20", 2));
|
||||
view1.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size() * 2);
|
||||
SyncableFileSystemView view6 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
partitions.stream().forEach(p -> view5.getLatestFileSlices(p).count());
|
||||
view6.sync();
|
||||
|
||||
/**
|
||||
* Case where multiple restores and ingestions happened
|
||||
*/
|
||||
testRestore(view2, Arrays.asList("25"), true, new HashMap<>(), Arrays.asList("24"), "29", true);
|
||||
testRestore(view2, Arrays.asList("26"), true, new HashMap<>(), Arrays.asList("23"), "29", false);
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("27"), true, "20", 2));
|
||||
scheduleCompaction(view2, "28");
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("29"), true, "28", 3));
|
||||
// Compaction
|
||||
testMultipleWriteSteps(view2, Arrays.asList("28"), false, "28", 3,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "29")));
|
||||
|
||||
Arrays.asList(view1, view2, view3, view4, view5, view6).stream().forEach(v -> {
|
||||
v.sync();
|
||||
areViewsConsistent(v, view1, partitions.size() * fileIdsPerPartition.size() * 3);
|
||||
});
|
||||
}
|
||||
|
||||
/*
|
||||
********************************************************************************************************
|
||||
* HELPER METHODS
|
||||
*********************************************************************************************************
|
||||
*/
|
||||
|
||||
|
||||
/**
|
||||
* Helper to run one or more rounds of cleaning, incrementally syncing the view and then validate
|
||||
*/
|
||||
private void testCleans(SyncableFileSystemView view, List<String> newCleanerInstants,
|
||||
Map<String, List<String>> instantsToFiles, List<String> cleanedInstants) {
|
||||
Map<String, List<String>> deltaInstantMap = cleanedInstants.stream().map(e -> Pair.of(e, new ArrayList()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
testCleans(view, newCleanerInstants, deltaInstantMap, instantsToFiles, cleanedInstants);
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulates one of more cleaning, incrementally sync the view and validate the view
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param newCleanerInstants Cleaner Instants
|
||||
* @param deltaInstantMap File-Slice Base Instants to Delta Instants
|
||||
* @param instantsToFiles List of files associated with each instant
|
||||
* @param cleanedInstants List of cleaned instants
|
||||
*/
|
||||
private void testCleans(SyncableFileSystemView view, List<String> newCleanerInstants,
|
||||
Map<String, List<String>> deltaInstantMap,
|
||||
Map<String, List<String>> instantsToFiles, List<String> cleanedInstants) {
|
||||
Assert.assertEquals(newCleanerInstants.size(), cleanedInstants.size());
|
||||
long initialFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).findAny().getAsLong();
|
||||
long exp = initialFileSlices;
|
||||
System.out.println("Initial File Slices :" + exp);
|
||||
for (int idx = 0; idx < newCleanerInstants.size(); idx++) {
|
||||
String instant = cleanedInstants.get(idx);
|
||||
try {
|
||||
List<String> filesToDelete = new ArrayList<>(instantsToFiles.get(instant));
|
||||
deltaInstantMap.get(instant).stream().forEach(n -> filesToDelete.addAll(instantsToFiles.get(n)));
|
||||
|
||||
performClean(view, instant, filesToDelete, newCleanerInstants.get(idx));
|
||||
|
||||
exp -= fileIdsPerPartition.size();
|
||||
final long expTotalFileSlicesPerPartition = exp;
|
||||
view.sync();
|
||||
Assert.assertTrue(view.getLastInstant().isPresent());
|
||||
Assert.assertEquals(newCleanerInstants.get(idx), view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
Assert.assertEquals(HoodieTimeline.CLEAN_ACTION, view.getLastInstant().get().getAction());
|
||||
partitions.forEach(p -> {
|
||||
System.out.println("PARTTITION : " + p);
|
||||
System.out.println("\tFileSlices :" + view.getAllFileSlices(p).collect(Collectors.toList()));
|
||||
});
|
||||
|
||||
partitions.forEach(p -> Assert.assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
|
||||
partitions.forEach(p -> Assert.assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
for (String partition : partitions) {
|
||||
newView.getAllFileGroups(partition).count();
|
||||
}
|
||||
areViewsConsistent(view, newView, expTotalFileSlicesPerPartition * partitions.size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulates one of more restores, incrementally sync the view and validate the view
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param newRestoreInstants Restore Instants
|
||||
* @param isDeltaCommit is Delta Commit ?
|
||||
* @param instantsToFiles List of files associated with each instant
|
||||
* @param rolledBackInstants List of rolled-back instants
|
||||
* @param emptyRestoreInstant Restore instant at which dataset becomes empty
|
||||
*/
|
||||
private void testRestore(SyncableFileSystemView view, List<String> newRestoreInstants, boolean isDeltaCommit,
|
||||
Map<String, List<String>> instantsToFiles, List<String> rolledBackInstants,
|
||||
String emptyRestoreInstant, boolean isRestore)
|
||||
throws IOException {
|
||||
Assert.assertEquals(newRestoreInstants.size(), rolledBackInstants.size());
|
||||
long initialFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).findAny().getAsLong();
|
||||
IntStream.range(0, newRestoreInstants.size()).forEach(idx -> {
|
||||
String instant = rolledBackInstants.get(idx);
|
||||
try {
|
||||
performRestore(view, instant, instantsToFiles.get(instant), newRestoreInstants.get(idx), isRestore);
|
||||
final long expTotalFileSlicesPerPartition = isDeltaCommit ? initialFileSlices :
|
||||
initialFileSlices - ((idx + 1) * fileIdsPerPartition.size());
|
||||
view.sync();
|
||||
Assert.assertTrue(view.getLastInstant().isPresent());
|
||||
System.out.println("Last Instant is :" + view.getLastInstant().get());
|
||||
if (isRestore) {
|
||||
Assert.assertEquals(newRestoreInstants.get(idx), view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(isRestore ? HoodieTimeline.RESTORE_ACTION : HoodieTimeline.ROLLBACK_ACTION,
|
||||
view.getLastInstant().get().getAction());
|
||||
}
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
|
||||
if (HoodieTimeline.compareTimestamps(newRestoreInstants.get(idx), emptyRestoreInstant,
|
||||
HoodieTimeline.GREATER_OR_EQUAL)) {
|
||||
partitions.forEach(p -> Assert.assertEquals(0, view.getLatestFileSlices(p).count()));
|
||||
} else {
|
||||
partitions.forEach(p -> Assert.assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
|
||||
}
|
||||
partitions.forEach(p -> Assert.assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
for (String partition : partitions) {
|
||||
newView.getAllFileGroups(partition).count();
|
||||
}
|
||||
areViewsConsistent(view, newView, expTotalFileSlicesPerPartition * partitions.size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulate a Cleaner operation cleaning up an instant
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instant Instant to be cleaner
|
||||
* @param files List of files to be deleted
|
||||
* @param cleanInstant Cleaner Instant
|
||||
*/
|
||||
private void performClean(SyncableFileSystemView view, String instant, List<String> files, String cleanInstant)
|
||||
throws IOException {
|
||||
Map<String, List<String>> partititonToFiles = deleteFiles(files);
|
||||
List<HoodieCleanStat> cleanStats = partititonToFiles.entrySet().stream().map(e -> {
|
||||
return new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_COMMITS,
|
||||
e.getKey(), e.getValue(), e.getValue(), new ArrayList<>(),
|
||||
Integer.toString(Integer.parseInt(instant) + 1));
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
HoodieCleanMetadata cleanMetadata = AvroUtils.convertCleanMetadata(cleanInstant, Optional.empty(), cleanStats);
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, cleanInstant),
|
||||
AvroUtils.serializeCleanMetadata(cleanMetadata));
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulate Restore of an instant in timeline and fsview
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instant Instant to be rolled-back
|
||||
* @param files List of files to be deleted as part of rollback
|
||||
* @param rollbackInstant Restore Instant
|
||||
*/
|
||||
private void performRestore(SyncableFileSystemView view, String instant, List<String> files, String rollbackInstant,
|
||||
boolean isRestore) throws IOException {
|
||||
Map<String, List<String>> partititonToFiles = deleteFiles(files);
|
||||
List<HoodieRollbackStat> rollbackStats = partititonToFiles.entrySet().stream().map(e -> {
|
||||
return new HoodieRollbackStat(e.getKey(), e.getValue(), new ArrayList<>(), new HashMap<>());
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
List<String> rollbacks = new ArrayList<>();
|
||||
rollbacks.add(instant);
|
||||
|
||||
HoodieRollbackMetadata rollbackMetadata = AvroUtils
|
||||
.convertRollbackMetadata(rollbackInstant, Optional.empty(), rollbacks, rollbackStats);
|
||||
if (isRestore) {
|
||||
HoodieRestoreMetadata metadata = new HoodieRestoreMetadata();
|
||||
|
||||
List<HoodieRollbackMetadata> rollbackM = new ArrayList<>();
|
||||
rollbackM.add(rollbackMetadata);
|
||||
metadata.setHoodieRestoreMetadata(
|
||||
new ImmutableMap.Builder().put(rollbackInstant, rollbackM).build());
|
||||
List<String> rollbackInstants = new ArrayList<>();
|
||||
rollbackInstants.add(rollbackInstant);
|
||||
metadata.setInstantsToRollback(rollbackInstants);
|
||||
metadata.setStartRestoreTime(rollbackInstant);
|
||||
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, rollbackInstant),
|
||||
AvroUtils.serializeRestoreMetadata(metadata));
|
||||
} else {
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant),
|
||||
AvroUtils.serializeRollbackMetadata(rollbackMetadata));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to delete a list of files and group the deleted files by partitions
|
||||
*
|
||||
* @param files List of files to be deleted
|
||||
*/
|
||||
private Map<String, List<String>> deleteFiles(List<String> files) {
|
||||
|
||||
if (null == files) {
|
||||
return new HashMap<>();
|
||||
}
|
||||
|
||||
Map<String, List<String>> partititonToFiles = new HashMap<>();
|
||||
partitions.forEach(p -> partititonToFiles.put(p, new ArrayList<>()));
|
||||
|
||||
for (String f : files) {
|
||||
String fullPath = String.format("%s/%s", metaClient.getBasePath(), f);
|
||||
new File(fullPath).delete();
|
||||
String partition = partitions.stream().filter(p -> f.startsWith(p)).findAny().get();
|
||||
partititonToFiles.get(partition).add(fullPath);
|
||||
}
|
||||
return partititonToFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedule a pending compaction and validate
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instantTime COmpaction Instant Time
|
||||
*/
|
||||
private void scheduleCompaction(SyncableFileSystemView view, String instantTime) throws IOException {
|
||||
List<Pair<String, FileSlice>> slices = partitions.stream().flatMap(p -> view.getLatestFileSlices(p)
|
||||
.map(s -> Pair.of(p, s))).collect(Collectors.toList());
|
||||
|
||||
long initialExpTotalFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).sum();
|
||||
|
||||
HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(slices, Optional.empty(), Optional.empty());
|
||||
HoodieInstant compactionInstant =
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime);
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant,
|
||||
AvroUtils.serializeCompactionPlan(plan));
|
||||
|
||||
view.sync();
|
||||
partitions.stream().forEach(p -> {
|
||||
view.getLatestFileSlices(p).forEach(fs -> {
|
||||
Assert.assertEquals(instantTime, fs.getBaseInstantTime());
|
||||
Assert.assertEquals(p, fs.getPartitionPath());
|
||||
Assert.assertFalse(fs.getDataFile().isPresent());
|
||||
});
|
||||
view.getLatestMergedFileSlicesBeforeOrOn(p, instantTime).forEach(fs -> {
|
||||
Assert.assertTrue(HoodieTimeline.compareTimestamps(instantTime, fs.getBaseInstantTime(),
|
||||
HoodieTimeline.GREATER));
|
||||
Assert.assertEquals(p, fs.getPartitionPath());
|
||||
});
|
||||
});
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
partitions.forEach(p -> newView.getLatestFileSlices(p).count());
|
||||
areViewsConsistent(view, newView, initialExpTotalFileSlices + partitions.size() * fileIdsPerPartition.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Unschedule a compaction instant and validate incremental fs view
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param compactionInstantTime Compaction Instant to be removed
|
||||
* @param newLastInstant New Last instant
|
||||
* @param newBaseInstant New Base instant of last file-slice
|
||||
*/
|
||||
private void unscheduleCompaction(SyncableFileSystemView view, String compactionInstantTime, String newLastInstant,
|
||||
String newBaseInstant) throws IOException {
|
||||
HoodieInstant instant = new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstantTime);
|
||||
boolean deleted = metaClient.getFs().delete(new Path(metaClient.getMetaPath(), instant.getFileName()), false);
|
||||
Preconditions.checkArgument(deleted, "Unable to delete compaction instant.");
|
||||
|
||||
view.sync();
|
||||
Assert.assertEquals(newLastInstant, view.getLastInstant().get().getTimestamp());
|
||||
partitions.stream().forEach(p -> {
|
||||
view.getLatestFileSlices(p).forEach(fs -> {
|
||||
Assert.assertEquals(newBaseInstant, fs.getBaseInstantTime());
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instants Ingestion/Commit INstants
|
||||
* @param deltaCommit Delta COmmit ?
|
||||
* @param baseInstantForDeltaCommit Base Instant to be used in case of delta-commit
|
||||
* @return List of new file created
|
||||
*/
|
||||
private Map<String, List<String>> testMultipleWriteSteps(SyncableFileSystemView view, List<String> instants,
|
||||
boolean deltaCommit, String baseInstantForDeltaCommit) throws IOException {
|
||||
return testMultipleWriteSteps(view, instants, deltaCommit, baseInstantForDeltaCommit, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instants Ingestion/Commit INstants
|
||||
* @param deltaCommit Delta COmmit ?
|
||||
* @param baseInstantForDeltaCommit Base Instant to be used in case of delta-commit
|
||||
* @param begin initial file-slice offset
|
||||
* @return List of new file created
|
||||
*/
|
||||
private Map<String, List<String>> testMultipleWriteSteps(SyncableFileSystemView view, List<String> instants,
|
||||
boolean deltaCommit, String baseInstantForDeltaCommit, int begin)
|
||||
throws IOException {
|
||||
return testMultipleWriteSteps(view, instants, deltaCommit, baseInstantForDeltaCommit, begin,
|
||||
instants.stream().map(i -> new HoodieInstant(State.COMPLETED,
|
||||
deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, i))
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instants Ingestion/Commit INstants
|
||||
* @return List of new file created
|
||||
*/
|
||||
private Map<String, List<String>> testMultipleWriteSteps(SyncableFileSystemView view, List<String> instants)
|
||||
throws IOException {
|
||||
return testMultipleWriteSteps(view, instants, false, null, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instants Ingestion/Commit INstants
|
||||
* @param deltaCommit Delta COmmit ?
|
||||
* @param baseInstantForDeltaCommit Base Instant to be used in case of delta-commit
|
||||
* @param begin initial file-slice offset
|
||||
* @param lastInstants List of Last Instants at each time we ingest/compact
|
||||
* @return List of new file created
|
||||
*/
|
||||
private Map<String, List<String>> testMultipleWriteSteps(SyncableFileSystemView view, List<String> instants,
|
||||
boolean deltaCommit, String baseInstantForDeltaCommit, int begin, List<HoodieInstant> lastInstants)
|
||||
throws IOException {
|
||||
Map<String, List<String>> instantToFiles = new HashMap<>();
|
||||
|
||||
int multiple = begin;
|
||||
for (int idx = 0; idx < instants.size(); idx++) {
|
||||
String instant = instants.get(idx);
|
||||
System.out.println("Adding instant=" + instant);
|
||||
HoodieInstant lastInstant = lastInstants.get(idx);
|
||||
// Add a non-empty ingestion to COW table
|
||||
List<String> filePaths = addInstant(metaClient, instant, deltaCommit,
|
||||
deltaCommit ? baseInstantForDeltaCommit : instant);
|
||||
view.sync();
|
||||
Assert.assertTrue(view.getLastInstant().isPresent());
|
||||
Assert.assertEquals(lastInstant.getTimestamp(), view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
Assert.assertEquals("Expected Last=" + lastInstant + ", Found Instants="
|
||||
+ view.getTimeline().getInstants().collect(Collectors.toList()),
|
||||
lastInstant.getAction(), view.getLastInstant().get().getAction());
|
||||
partitions.forEach(p -> Assert.assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
|
||||
final long expTotalFileSlicesPerPartition = fileIdsPerPartition.size() * multiple;
|
||||
partitions.forEach(p -> Assert.assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
|
||||
if (deltaCommit) {
|
||||
partitions.forEach(p -> {
|
||||
view.getLatestFileSlices(p).forEach(f -> {
|
||||
Assert.assertEquals(baseInstantForDeltaCommit, f.getBaseInstantTime());
|
||||
});
|
||||
});
|
||||
} else {
|
||||
partitions.forEach(p -> {
|
||||
view.getLatestDataFiles(p).forEach(f -> {
|
||||
Assert.assertEquals(instant, f.getCommitTime());
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
for (String partition : partitions) {
|
||||
newView.getAllFileGroups(partition).count();
|
||||
}
|
||||
areViewsConsistent(view, newView, fileIdsPerPartition.size() * partitions.size() * multiple);
|
||||
instantToFiles.put(instant, filePaths);
|
||||
if (!deltaCommit) {
|
||||
multiple++;
|
||||
}
|
||||
}
|
||||
return instantToFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check for equality of views
|
||||
*
|
||||
* @param view1 View1
|
||||
* @param view2 View2
|
||||
*/
|
||||
private void areViewsConsistent(SyncableFileSystemView view1, SyncableFileSystemView view2,
|
||||
long expectedTotalFileSlices) {
|
||||
//Timeline check
|
||||
HoodieTimeline timeline1 = view1.getTimeline();
|
||||
HoodieTimeline timeline2 = view2.getTimeline();
|
||||
Assert.assertEquals(view1.getLastInstant(), view2.getLastInstant());
|
||||
Iterators.elementsEqual(timeline1.getInstants().iterator(), timeline2.getInstants().iterator());
|
||||
|
||||
//View Checks
|
||||
Map<HoodieFileGroupId, HoodieFileGroup> fileGroupsMap1 =
|
||||
partitions.stream().flatMap(p -> view1.getAllFileGroups(p))
|
||||
.collect(Collectors.toMap(fg -> fg.getFileGroupId(), fg -> fg));
|
||||
Map<HoodieFileGroupId, HoodieFileGroup> fileGroupsMap2 =
|
||||
partitions.stream().flatMap(p -> view2.getAllFileGroups(p))
|
||||
.collect(Collectors.toMap(fg -> fg.getFileGroupId(), fg -> fg));
|
||||
Assert.assertEquals(fileGroupsMap1.keySet(), fileGroupsMap2.keySet());
|
||||
long gotSlicesCount =
|
||||
fileGroupsMap1.keySet().stream().map(k -> Pair.of(fileGroupsMap1.get(k), fileGroupsMap2.get(k)))
|
||||
.mapToLong(e -> {
|
||||
HoodieFileGroup fg1 = e.getKey();
|
||||
HoodieFileGroup fg2 = e.getValue();
|
||||
Assert.assertEquals(fg1.getFileGroupId(), fg2.getFileGroupId());
|
||||
List<FileSlice> slices1 = fg1.getAllRawFileSlices().collect(Collectors.toList());
|
||||
List<FileSlice> slices2 = fg2.getAllRawFileSlices().collect(Collectors.toList());
|
||||
Assert.assertEquals(slices1.size(), slices2.size());
|
||||
IntStream.range(0, slices1.size()).mapToObj(idx -> Pair.of(slices1.get(idx), slices2.get(idx)))
|
||||
.forEach(e2 -> {
|
||||
FileSlice slice1 = e2.getKey();
|
||||
FileSlice slice2 = e2.getValue();
|
||||
Assert.assertEquals(slice1.getBaseInstantTime(), slice2.getBaseInstantTime());
|
||||
Assert.assertEquals(slice1.getFileId(), slice2.getFileId());
|
||||
Assert.assertEquals(slice1.getDataFile().isPresent(), slice2.getDataFile().isPresent());
|
||||
if (slice1.getDataFile().isPresent()) {
|
||||
HoodieDataFile df1 = slice1.getDataFile().get();
|
||||
HoodieDataFile df2 = slice2.getDataFile().get();
|
||||
Assert.assertEquals(df1.getCommitTime(), df2.getCommitTime());
|
||||
Assert.assertEquals(df1.getFileId(), df2.getFileId());
|
||||
Assert.assertEquals(df1.getFileName(), df2.getFileName());
|
||||
Assert.assertEquals(Path.getPathWithoutSchemeAndAuthority(new Path(df1.getPath())),
|
||||
Path.getPathWithoutSchemeAndAuthority(new Path(df2.getPath())));
|
||||
}
|
||||
List<Path> logPaths1 = slice1.getLogFiles()
|
||||
.map(lf -> Path.getPathWithoutSchemeAndAuthority(lf.getPath())).collect(Collectors.toList());
|
||||
List<Path> logPaths2 = slice2.getLogFiles()
|
||||
.map(lf -> Path.getPathWithoutSchemeAndAuthority(lf.getPath())).collect(Collectors.toList());
|
||||
Assert.assertEquals(logPaths1, logPaths2);
|
||||
});
|
||||
return slices1.size();
|
||||
}).sum();
|
||||
Assert.assertEquals(expectedTotalFileSlices, gotSlicesCount);
|
||||
|
||||
// Pending Compaction Operations Check
|
||||
Set<Pair<String, CompactionOperation>> ops1 = view1.getPendingCompactionOperations().collect(Collectors.toSet());
|
||||
Set<Pair<String, CompactionOperation>> ops2 = view2.getPendingCompactionOperations().collect(Collectors.toSet());
|
||||
Assert.assertEquals(ops1, ops2);
|
||||
}
|
||||
|
||||
private List<String> addInstant(HoodieTableMetaClient metaClient, String instant, boolean deltaCommit,
|
||||
String baseInstant)
|
||||
throws IOException {
|
||||
List<Pair<String, HoodieWriteStat>> writeStats = partitions.stream().flatMap(p -> {
|
||||
return fileIdsPerPartition.stream().map(f -> {
|
||||
try {
|
||||
File file = new File(basePath + "/" + p + "/"
|
||||
+ (deltaCommit ? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant)) :
|
||||
FSUtils.makeDataFileName(instant, 0, f)));
|
||||
file.createNewFile();
|
||||
HoodieWriteStat w = new HoodieWriteStat();
|
||||
w.setFileId(f);
|
||||
w.setPath(String.format("%s/%s", p, file.getName()));
|
||||
return Pair.of(p, w);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
writeStats.forEach(e -> metadata.addWriteStat(e.getKey(), e.getValue()));
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION,
|
||||
instant),
|
||||
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
// Delete pending compaction if present
|
||||
metaClient.getFs().delete(new Path(metaClient.getMetaPath(),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instant).getFileName()));
|
||||
return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,33 @@
|
||||
/*
|
||||
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.table.view;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import java.io.IOException;
|
||||
|
||||
public class RocksDBBasedIncrementalFSViewSyncTest extends IncrementalFSViewSyncTest {
|
||||
|
||||
@Override
|
||||
protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline timeline)
|
||||
throws IOException {
|
||||
return new RocksDbBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withRocksDBPath(tmpFolder.newFolder().getAbsolutePath())
|
||||
.withIncrementalTimelineSync(true).build());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.table.view;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import java.io.IOException;
|
||||
|
||||
public class RocksDbBasedFileSystemViewTest extends HoodieTableFileSystemViewTest {
|
||||
|
||||
protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) throws IOException {
|
||||
return new RocksDbBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withRocksDBPath(tmpFolder.newFolder().getAbsolutePath())
|
||||
.build());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,29 @@
|
||||
/*
|
||||
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.table.view;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
|
||||
public class SpillableMapBasedFileSystemViewTest extends HoodieTableFileSystemViewTest {
|
||||
|
||||
protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) {
|
||||
return new SpillableMapBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.SPILLABLE_DISK)
|
||||
.withMaxMemoryForView(0L).build()); // pure disk base View
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.table.view;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
|
||||
public class SpillableMapBasedIncrementalFSViewSyncTest extends IncrementalFSViewSyncTest {
|
||||
|
||||
@Override
|
||||
protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline timeline) {
|
||||
return new SpillableMapBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withMaxMemoryForView(0L).withIncrementalTimelineSync(true).build());
|
||||
}
|
||||
}
|
||||
@@ -182,7 +182,7 @@ public class CompactionTestUtils {
|
||||
private final String path;
|
||||
|
||||
public TestHoodieDataFile(String path) {
|
||||
super(null);
|
||||
super("/tmp/ce481ee7-9e53-4a2e-9992-f9e295fa79c0_11_20180918020003.parquet");
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
|
||||
@@ -77,7 +77,7 @@ public class TestCompactionUtils {
|
||||
|
||||
// File Slice with data-file but no log files
|
||||
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noLog1");
|
||||
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet"));
|
||||
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
op = CompactionUtils.buildFromFileSlice(
|
||||
DEFAULT_PARTITION_PATHS[0], noLogFileSlice, Optional.of(metricsCaptureFn));
|
||||
testFileSliceCompactionOpEquality(noLogFileSlice, op, DEFAULT_PARTITION_PATHS[0]);
|
||||
@@ -94,7 +94,7 @@ public class TestCompactionUtils {
|
||||
|
||||
//File Slice with data-file and log files present
|
||||
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet"));
|
||||
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
@@ -110,13 +110,13 @@ public class TestCompactionUtils {
|
||||
private Pair<List<Pair<String, FileSlice>>, HoodieCompactionPlan> buildCompactionPlan() {
|
||||
FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "empty1");
|
||||
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet"));
|
||||
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2))));
|
||||
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noLog1");
|
||||
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet"));
|
||||
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
|
||||
|
||||
@@ -0,0 +1,183 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestRocksDBManager {
|
||||
|
||||
static {
|
||||
RocksDBDAO.log.setLevel(Level.INFO);
|
||||
}
|
||||
|
||||
private static RocksDBDAO dbManager;
|
||||
|
||||
@AfterClass
|
||||
public static void drop() throws IOException {
|
||||
if (dbManager != null) {
|
||||
dbManager.close();
|
||||
dbManager = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRocksDBManager() throws Exception {
|
||||
String prefix1 = "prefix1_";
|
||||
String prefix2 = "prefix2_";
|
||||
String prefix3 = "prefix3_";
|
||||
String prefix4 = "prefix4_";
|
||||
List<String> prefixes = Arrays.asList(prefix1, prefix2, prefix3, prefix4);
|
||||
String family1 = "family1";
|
||||
String family2 = "family2";
|
||||
List<String> colFamilies = Arrays.asList(family1, family2);
|
||||
|
||||
List<Payload> payloads = IntStream.range(0, 100).mapToObj(index -> {
|
||||
String prefix = prefixes.get(index % 4);
|
||||
String key = prefix + UUID.randomUUID().toString();
|
||||
String family = colFamilies.get(index % 2);
|
||||
String val = "VALUE_" + UUID.randomUUID().toString();
|
||||
return new Payload(prefix, key, val, family);
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
dbManager = new RocksDBDAO("/dummy/path",
|
||||
FileSystemViewStorageConfig.newBuilder().build().newBuilder().build());
|
||||
colFamilies.stream().forEach(family -> dbManager.dropColumnFamily(family));
|
||||
colFamilies.stream().forEach(family -> dbManager.addColumnFamily(family));
|
||||
|
||||
Map<String, Map<String, Integer>> countsMap = new HashMap<>();
|
||||
payloads.stream().forEach(payload -> {
|
||||
dbManager.put(payload.getFamily(), payload.getKey(), payload);
|
||||
|
||||
if (!countsMap.containsKey(payload.family)) {
|
||||
countsMap.put(payload.family, new HashMap<>());
|
||||
}
|
||||
Map<String, Integer> c = countsMap.get(payload.family);
|
||||
if (!c.containsKey(payload.prefix)) {
|
||||
c.put(payload.prefix, 0);
|
||||
}
|
||||
int currCount = c.get(payload.prefix);
|
||||
c.put(payload.prefix, currCount + 1);
|
||||
});
|
||||
|
||||
colFamilies.stream().forEach(family -> {
|
||||
prefixes.stream().forEach(prefix -> {
|
||||
List<Pair<String, Payload>> gotPayloads =
|
||||
dbManager.<Payload>prefixSearch(family, prefix).collect(Collectors.toList());
|
||||
Integer expCount = countsMap.get(family).get(prefix);
|
||||
Assert.assertEquals("Size check for prefix (" + prefix + ") and family (" + family + ")",
|
||||
expCount == null ? 0L : expCount.longValue(), gotPayloads.size());
|
||||
gotPayloads.stream().forEach(p -> {
|
||||
Assert.assertEquals(p.getRight().getFamily(), family);
|
||||
Assert.assertTrue(p.getRight().getKey().startsWith(prefix));
|
||||
});
|
||||
});
|
||||
});
|
||||
|
||||
payloads.stream().forEach(payload -> {
|
||||
Payload p = dbManager.get(payload.getFamily(), payload.getKey());
|
||||
Assert.assertEquals("Retrieved correct payload for key :" + payload.getKey(), payload, p);
|
||||
|
||||
// Now, delete the key
|
||||
dbManager.delete(payload.getFamily(), payload.getKey());
|
||||
|
||||
// Now retrieve
|
||||
Payload p2 = dbManager.get(payload.getFamily(), payload.getKey());
|
||||
Assert.assertNull("Retrieved correct payload for key :" + p.getKey(), p2);
|
||||
});
|
||||
|
||||
// Now do a prefix search
|
||||
colFamilies.stream().forEach(family -> {
|
||||
prefixes.stream().forEach(prefix -> {
|
||||
List<Pair<String, Payload>> gotPayloads =
|
||||
dbManager.<Payload>prefixSearch(family, prefix).collect(Collectors.toList());
|
||||
Assert.assertEquals("Size check for prefix (" + prefix + ") and family (" + family + ")",
|
||||
0, gotPayloads.size());
|
||||
});
|
||||
});
|
||||
|
||||
String rocksDBBasePath = dbManager.getRocksDBBasePath();
|
||||
dbManager.close();
|
||||
Assert.assertFalse(new File(rocksDBBasePath).exists());
|
||||
}
|
||||
|
||||
public static class Payload implements Serializable {
|
||||
|
||||
private final String prefix;
|
||||
private final String key;
|
||||
private final String val;
|
||||
private final String family;
|
||||
|
||||
public Payload(String prefix, String key, String val, String family) {
|
||||
this.prefix = prefix;
|
||||
this.key = key;
|
||||
this.val = val;
|
||||
this.family = family;
|
||||
}
|
||||
|
||||
public String getPrefix() {
|
||||
return prefix;
|
||||
}
|
||||
|
||||
public String getKey() {
|
||||
return key;
|
||||
}
|
||||
|
||||
public String getVal() {
|
||||
return val;
|
||||
}
|
||||
|
||||
public String getFamily() {
|
||||
return family;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
Payload payload = (Payload) o;
|
||||
return Objects.equals(prefix, payload.prefix)
|
||||
&& Objects.equals(key, payload.key)
|
||||
&& Objects.equals(val, payload.val)
|
||||
&& Objects.equals(family, payload.family);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(prefix, key, val, family);
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user