1
0

[HUDI-509] Renaming code in sync with cWiki restructuring (#1212)

- Storage Type replaced with Table Type (remaining instances)
 - View types replaced with query types;
 - ReadOptimized view referred as Snapshot Query
 - TableFileSystemView sub interfaces renamed to BaseFileOnly and Slice Views
 - HoodieDataFile renamed to HoodieBaseFile
 - Hive Sync tool will register RO tables for MOR with a `_ro` suffix
 - Datasource/Deltastreamer options renamed accordingly
 - Support fallback to old config values as well, so migration is painless
 - Config for controlling _ro suffix addition
 - Renaming DataFile to BaseFile across DTOs, HoodieFileSlice and AbstractTableFileSystemView
This commit is contained in:
vinoth chandar
2020-01-16 23:58:47 -08:00
committed by GitHub
parent 8a3a50309b
commit c2c0f6b13d
92 changed files with 907 additions and 822 deletions

View File

@@ -22,14 +22,15 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.HoodieCommonTestHarness;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.SyncableFileSystemView;
import org.apache.hudi.common.table.TableFileSystemView;
import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.table.TableFileSystemView.SliceView;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
@@ -76,8 +77,8 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
private static String TEST_WRITE_TOKEN = "1-0-1";
protected SyncableFileSystemView fsView;
protected TableFileSystemView.ReadOptimizedView roView;
protected TableFileSystemView.RealtimeView rtView;
protected BaseFileOnlyView roView;
protected SliceView rtView;
@Before
public void init() throws IOException {
@@ -135,13 +136,13 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
refreshFsView();
List<HoodieDataFile> dataFiles = roView.getLatestDataFiles().collect(Collectors.toList());
List<HoodieBaseFile> dataFiles = roView.getLatestBaseFiles().collect(Collectors.toList());
assertTrue("No data file expected", dataFiles.isEmpty());
List<FileSlice> fileSliceList = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
assertEquals(1, fileSliceList.size());
FileSlice fileSlice = fileSliceList.get(0);
assertEquals("File-Id must be set correctly", fileId, fileSlice.getFileId());
assertFalse("Data file for base instant must be present", fileSlice.getDataFile().isPresent());
assertFalse("Data file for base instant must be present", fileSlice.getBaseFile().isPresent());
assertEquals("Base Instant for file-group set correctly", instantTime1, fileSlice.getBaseInstantTime());
List<HoodieLogFile> logFiles = fileSlice.getLogFiles().collect(Collectors.toList());
assertEquals("Correct number of log-files shows up in file-slice", 2, logFiles.size());
@@ -154,7 +155,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertEquals(1, fileSliceList.size());
fileSlice = fileSliceList.get(0);
assertEquals("File-Id must be set correctly", fileId, fileSlice.getFileId());
assertFalse("Data file for base instant must be present", fileSlice.getDataFile().isPresent());
assertFalse("Data file for base instant must be present", fileSlice.getBaseFile().isPresent());
assertEquals("Base Instant for file-group set correctly", instantTime1, fileSlice.getBaseInstantTime());
logFiles = fileSlice.getLogFiles().collect(Collectors.toList());
assertEquals("Correct number of log-files shows up in file-slice", 2, logFiles.size());
@@ -166,7 +167,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertEquals(1, fileSliceList.size());
fileSlice = fileSliceList.get(0);
assertEquals("File-Id must be set correctly", fileId, fileSlice.getFileId());
assertFalse("Data file for base instant must be present", fileSlice.getDataFile().isPresent());
assertFalse("Data file for base instant must be present", fileSlice.getBaseFile().isPresent());
assertEquals("Base Instant for file-group set correctly", instantTime1, fileSlice.getBaseInstantTime());
logFiles = fileSlice.getLogFiles().collect(Collectors.toList());
assertEquals("Correct number of log-files shows up in file-slice", 2, logFiles.size());
@@ -176,7 +177,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
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());
roView.getAllBaseFiles(partitionPath).count());
assertEquals("Total number of file-groups in view matches expected", 1,
fsView.getAllFileGroups(partitionPath).count());
}
@@ -293,7 +294,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
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());
assertFalse("Latest File Slice must not have data-file", slices.get(0).getBaseFile().isPresent());
assertEquals("Latest File Slice must not have any log-files", 0, slices.get(0).getLogFiles().count());
// Fake delta-ingestion after compaction-requested
@@ -313,7 +314,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
saveAsComplete(commitTimeline, deltaInstant5, Option.empty());
refreshFsView();
List<HoodieDataFile> dataFiles = roView.getAllDataFiles(partitionPath).collect(Collectors.toList());
List<HoodieBaseFile> dataFiles = roView.getAllBaseFiles(partitionPath).collect(Collectors.toList());
if (skipCreatingDataFile) {
assertTrue("No data file expected", dataFiles.isEmpty());
} else {
@@ -328,9 +329,9 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
FileSlice fileSlice = fileSliceList.get(0);
assertEquals(fileId, fileSlice.getFileId());
if (!skipCreatingDataFile) {
assertEquals("Data file must be present", dataFileName, fileSlice.getDataFile().get().getFileName());
assertEquals("Data file must be present", dataFileName, fileSlice.getBaseFile().get().getFileName());
} else {
assertFalse("No data-file expected as it was not created", fileSlice.getDataFile().isPresent());
assertFalse("No data-file expected as it was not created", fileSlice.getBaseFile().isPresent());
}
assertEquals("Base Instant of penultimate file-slice must be base instant", instantTime1,
fileSlice.getBaseInstantTime());
@@ -346,7 +347,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertEquals("Expect only one file-id", 1, fileSliceList.size());
fileSlice = fileSliceList.get(0);
assertEquals(fileId, fileSlice.getFileId());
assertFalse("No data-file expected in latest file-slice", fileSlice.getDataFile().isPresent());
assertFalse("No data-file expected in latest file-slice", fileSlice.getBaseFile().isPresent());
assertEquals("Compaction requested instant must be base instant", compactionRequestedTime,
fileSlice.getBaseInstantTime());
logFiles = fileSlice.getLogFiles().collect(Collectors.toList());
@@ -355,28 +356,28 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName());
/** Data Files API tests */
dataFiles = roView.getLatestDataFiles().collect(Collectors.toList());
dataFiles = roView.getLatestBaseFiles().collect(Collectors.toList());
if (skipCreatingDataFile) {
assertEquals("Expect no data file to be returned", 0, dataFiles.size());
} else {
assertEquals("Expect only one data-file to be sent", 1, dataFiles.size());
dataFiles.forEach(df -> assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1));
}
dataFiles = roView.getLatestDataFiles(partitionPath).collect(Collectors.toList());
dataFiles = roView.getLatestBaseFiles(partitionPath).collect(Collectors.toList());
if (skipCreatingDataFile) {
assertEquals("Expect no data file to be returned", 0, dataFiles.size());
} else {
assertEquals("Expect only one data-file to be sent", 1, dataFiles.size());
dataFiles.forEach(df -> assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1));
}
dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList());
dataFiles = roView.getLatestBaseFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList());
if (skipCreatingDataFile) {
assertEquals("Expect no data file to be returned", 0, dataFiles.size());
} else {
assertEquals("Expect only one data-file to be sent", 1, dataFiles.size());
dataFiles.forEach(df -> assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1));
}
dataFiles = roView.getLatestDataFilesInRange(allInstantTimes).collect(Collectors.toList());
dataFiles = roView.getLatestBaseFilesInRange(allInstantTimes).collect(Collectors.toList());
if (skipCreatingDataFile) {
assertEquals("Expect no data file to be returned", 0, dataFiles.size());
} else {
@@ -415,8 +416,8 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
List<FileSlice> allRawFileSlices = getAllRawFileSlices(partitionPath).collect(Collectors.toList());
dataFiles = allRawFileSlices.stream().flatMap(slice -> {
if (slice.getDataFile().isPresent()) {
return Stream.of(slice.getDataFile().get());
if (slice.getBaseFile().isPresent()) {
return Stream.of(slice.getBaseFile().get());
}
return Stream.empty();
}).collect(Collectors.toList());
@@ -424,7 +425,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
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());
Set<String> fileNames = dataFiles.stream().map(HoodieBaseFile::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) {
@@ -448,26 +449,26 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
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());
orphanFileSliceWithDataFile.getBaseFile().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());
inflightFileSliceWithDataFile.getBaseFile().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());
orphanFileSliceWithLogFile.getBaseFile().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());
inflightFileSliceWithLogFile.getBaseFile().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,
@@ -486,11 +487,11 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
commitTimeline.saveAsComplete(compactionInstant, Option.empty());
refreshFsView();
// populate the cache
roView.getAllDataFiles(partitionPath);
roView.getAllBaseFiles(partitionPath);
fileSliceList = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
LOG.info("FILESLICE LIST=" + fileSliceList);
dataFiles = fileSliceList.stream().map(FileSlice::getDataFile).filter(Option::isPresent).map(Option::get)
dataFiles = fileSliceList.stream().map(FileSlice::getBaseFile).filter(Option::isPresent).map(Option::get)
.collect(Collectors.toList());
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());
@@ -498,7 +499,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
fileSlice = fileSliceList.get(0);
assertEquals("Check file-Id is set correctly", fileId, fileSlice.getFileId());
assertEquals("Check data-filename is set correctly", compactDataFileName,
fileSlice.getDataFile().get().getFileName());
fileSlice.getBaseFile().get().getFileName());
assertEquals("Ensure base-instant is now compaction request instant", compactionRequestedTime,
fileSlice.getBaseInstantTime());
logFiles = fileSlice.getLogFiles().collect(Collectors.toList());
@@ -507,23 +508,23 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName());
/** Data Files API tests */
dataFiles = roView.getLatestDataFiles().collect(Collectors.toList());
dataFiles = roView.getLatestBaseFiles().collect(Collectors.toList());
assertEquals("Expect only one data-file to be sent", 1, dataFiles.size());
dataFiles.forEach(df -> assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime));
dataFiles = roView.getLatestDataFiles(partitionPath).collect(Collectors.toList());
dataFiles = roView.getLatestBaseFiles(partitionPath).collect(Collectors.toList());
assertEquals("Expect only one data-file to be sent", 1, dataFiles.size());
dataFiles.forEach(df -> assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime));
dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList());
dataFiles = roView.getLatestBaseFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList());
assertEquals("Expect only one data-file to be sent", 1, dataFiles.size());
dataFiles.forEach(df -> assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime));
dataFiles = roView.getLatestDataFilesInRange(allInstantTimes).collect(Collectors.toList());
dataFiles = roView.getLatestBaseFilesInRange(allInstantTimes).collect(Collectors.toList());
assertEquals("Expect only one data-file to be sent", 1, dataFiles.size());
dataFiles.forEach(df -> 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());
roView.getAllBaseFiles(partitionPath).count());
// file-groups includes inflight/invalid file-ids
assertEquals("Total number of file-groups in partitions matches expected", 5,
fsView.getAllFileGroups(partitionPath).count());
@@ -535,7 +536,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
new File(basePath + "/" + partitionPath).mkdirs();
String fileId = UUID.randomUUID().toString();
assertFalse("No commit, should not find any data file", roView.getLatestDataFiles(partitionPath)
assertFalse("No commit, should not find any data file", roView.getLatestBaseFiles(partitionPath)
.anyMatch(dfile -> dfile.getFileId().equals(fileId)));
// Only one commit, but is not safe
@@ -543,7 +544,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId);
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
refreshFsView();
assertFalse("No commit, should not find any data file", roView.getLatestDataFiles(partitionPath)
assertFalse("No commit, should not find any data file", roView.getLatestBaseFiles(partitionPath)
.anyMatch(dfile -> dfile.getFileId().equals(fileId)));
// Make this commit safe
@@ -551,7 +552,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
saveAsComplete(commitTimeline, instant1, Option.empty());
refreshFsView();
assertEquals("", fileName1, roView.getLatestDataFiles(partitionPath)
assertEquals("", fileName1, roView.getLatestBaseFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get().getFileName());
// Do another commit, but not safe
@@ -559,14 +560,14 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
String fileName2 = FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId);
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
refreshFsView();
assertEquals("", fileName1, roView.getLatestDataFiles(partitionPath)
assertEquals("", fileName1, roView.getLatestBaseFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get().getFileName());
// Make it safe
HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2);
saveAsComplete(commitTimeline, instant2, Option.empty());
refreshFsView();
assertEquals("", fileName2, roView.getLatestDataFiles(partitionPath)
assertEquals("", fileName2, roView.getLatestBaseFiles(partitionPath)
.filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get().getFileName());
}
@@ -648,11 +649,11 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertEquals(isLatestFileSliceOnly ? 1 : 2, fileSliceMap.get(fileId3).longValue());
assertEquals(1, fileSliceMap.get(fileId4).longValue());
List<HoodieDataFile> dataFileList =
roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime4).collect(Collectors.toList());
List<HoodieBaseFile> dataFileList =
roView.getLatestBaseFilesBeforeOrOn("2016/05/01", commitTime4).collect(Collectors.toList());
assertEquals(3, dataFileList.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFileList) {
for (HoodieBaseFile status : dataFileList) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)));
@@ -676,10 +677,10 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
.contains(FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0, TEST_WRITE_TOKEN)));
// Reset the max commit time
List<HoodieDataFile> dataFiles =
roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3).collect(Collectors.toList());
List<HoodieBaseFile> dataFiles =
roView.getLatestBaseFilesBeforeOrOn("2016/05/01", commitTime3).collect(Collectors.toList());
filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFiles) {
for (HoodieBaseFile status : dataFiles) {
filenames.add(status.getFileName());
}
if (!isLatestFileSliceOnly) {
@@ -739,7 +740,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
for (HoodieFileGroup fileGroup : fileGroups) {
String fileId = fileGroup.getFileGroupId().getFileId();
Set<String> filenames = Sets.newHashSet();
fileGroup.getAllDataFiles().forEach(dataFile -> {
fileGroup.getAllBaseFiles().forEach(dataFile -> {
assertEquals("All same fileId should be grouped", fileId, dataFile.getFileId());
filenames.add(dataFile.getFileName());
});
@@ -811,13 +812,13 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
refreshFsView();
// Populate view for partition
roView.getAllDataFiles("2016/05/01/");
roView.getAllBaseFiles("2016/05/01/");
List<HoodieDataFile> dataFiles =
roView.getLatestDataFilesInRange(Lists.newArrayList(commitTime2, commitTime3)).collect(Collectors.toList());
List<HoodieBaseFile> dataFiles =
roView.getLatestBaseFilesInRange(Lists.newArrayList(commitTime2, commitTime3)).collect(Collectors.toList());
assertEquals(isLatestFileSliceOnly ? 2 : 3, dataFiles.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFiles) {
for (HoodieBaseFile status : dataFiles) {
filenames.add(status.getFileName());
}
@@ -833,15 +834,15 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
for (FileSlice slice : slices) {
if (slice.getFileId().equals(fileId1)) {
assertEquals(slice.getBaseInstantTime(), commitTime3);
assertTrue(slice.getDataFile().isPresent());
assertTrue(slice.getBaseFile().isPresent());
assertEquals(slice.getLogFiles().count(), 0);
} else if (slice.getFileId().equals(fileId2)) {
assertEquals(slice.getBaseInstantTime(), commitTime3);
assertTrue(slice.getDataFile().isPresent());
assertTrue(slice.getBaseFile().isPresent());
assertEquals(slice.getLogFiles().count(), 1);
} else if (slice.getFileId().equals(fileId3)) {
assertEquals(slice.getBaseInstantTime(), commitTime4);
assertTrue(slice.getDataFile().isPresent());
assertTrue(slice.getBaseFile().isPresent());
assertEquals(slice.getLogFiles().count(), 0);
}
}
@@ -883,12 +884,12 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertEquals(7, statuses.length);
refreshFsView();
List<HoodieDataFile> dataFiles =
roView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2).collect(Collectors.toList());
List<HoodieBaseFile> dataFiles =
roView.getLatestBaseFilesBeforeOrOn(partitionPath, commitTime2).collect(Collectors.toList());
if (!isLatestFileSliceOnly) {
assertEquals(2, dataFiles.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : dataFiles) {
for (HoodieBaseFile status : dataFiles) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)));
@@ -952,7 +953,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertEquals(10, statuses.length);
refreshFsView();
fsView.getAllDataFiles(partitionPath);
fsView.getAllBaseFiles(partitionPath);
List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList());
assertEquals(3, fileGroups.size());
for (HoodieFileGroup fileGroup : fileGroups) {
@@ -980,10 +981,10 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
}
}
List<HoodieDataFile> statuses1 = roView.getLatestDataFiles().collect(Collectors.toList());
List<HoodieBaseFile> statuses1 = roView.getLatestBaseFiles().collect(Collectors.toList());
assertEquals(3, statuses1.size());
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : statuses1) {
for (HoodieBaseFile status : statuses1) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)));
@@ -1092,10 +1093,10 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
refreshFsView();
// Test Data Files
List<HoodieDataFile> dataFiles = roView.getAllDataFiles(partitionPath1).collect(Collectors.toList());
List<HoodieBaseFile> dataFiles = roView.getAllBaseFiles(partitionPath1).collect(Collectors.toList());
assertEquals("One data-file is expected as there is only one file-group", 1, dataFiles.size());
assertEquals("Expect only valid commit", "1", dataFiles.get(0).getCommitTime());
dataFiles = roView.getAllDataFiles(partitionPath2).collect(Collectors.toList());
dataFiles = roView.getAllBaseFiles(partitionPath2).collect(Collectors.toList());
assertEquals("One data-file is expected as there is only one file-group", 1, dataFiles.size());
assertEquals("Expect only valid commit", "1", dataFiles.get(0).getCommitTime());
@@ -1106,7 +1107,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertEquals("Expect file-slice to be merged", 1, fileSliceList.size());
FileSlice fileSlice = fileSliceList.get(0);
assertEquals(fileId, fileSlice.getFileId());
assertEquals("Data file must be present", dataFileName, fileSlice.getDataFile().get().getFileName());
assertEquals("Data file must be present", dataFileName, fileSlice.getBaseFile().get().getFileName());
assertEquals("Base Instant of penultimate file-slice must be base instant", instantTime1,
fileSlice.getBaseInstantTime());
List<HoodieLogFile> logFiles = fileSlice.getLogFiles().collect(Collectors.toList());
@@ -1120,7 +1121,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertEquals("Expect only one file-id", 1, fileSliceList.size());
fileSlice = fileSliceList.get(0);
assertEquals(fileId, fileSlice.getFileId());
assertFalse("No data-file expected in latest file-slice", fileSlice.getDataFile().isPresent());
assertFalse("No data-file expected in latest file-slice", fileSlice.getBaseFile().isPresent());
assertEquals("Compaction requested instant must be base instant", compactionRequestedTime,
fileSlice.getBaseInstantTime());
logFiles = fileSlice.getLogFiles().collect(Collectors.toList());

View File

@@ -29,7 +29,7 @@ import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieTableType;
@@ -502,7 +502,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
view.getLatestFileSlices(p).forEach(fs -> {
Assert.assertEquals(instantTime, fs.getBaseInstantTime());
Assert.assertEquals(p, fs.getPartitionPath());
Assert.assertFalse(fs.getDataFile().isPresent());
Assert.assertFalse(fs.getBaseFile().isPresent());
});
view.getLatestMergedFileSlicesBeforeOrOn(p, instantTime).forEach(fs -> {
Assert
@@ -625,7 +625,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
});
} else {
partitions.forEach(p -> {
view.getLatestDataFiles(p).forEach(f -> {
view.getLatestBaseFiles(p).forEach(f -> {
Assert.assertEquals(instant, f.getCommitTime());
});
});
@@ -676,10 +676,10 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
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(slice1.getBaseFile().isPresent(), slice2.getBaseFile().isPresent());
if (slice1.getBaseFile().isPresent()) {
HoodieBaseFile df1 = slice1.getBaseFile().get();
HoodieBaseFile df2 = slice2.getBaseFile().get();
Assert.assertEquals(df1.getCommitTime(), df2.getCommitTime());
Assert.assertEquals(df1.getFileId(), df2.getFileId());
Assert.assertEquals(df1.getFileName(), df2.getFileName());

View File

@@ -21,7 +21,7 @@ package org.apache.hudi.common.util;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTestUtils;
@@ -157,7 +157,7 @@ public class CompactionTestUtils {
instantId, fileId, Option.of(2));
FileSlice slice = new FileSlice(DEFAULT_PARTITION_PATHS[0], instantId, fileId);
if (createDataFile) {
slice.setDataFile(new TestHoodieDataFile(metaClient.getBasePath() + "/" + DEFAULT_PARTITION_PATHS[0] + "/"
slice.setBaseFile(new TestHoodieBaseFile(metaClient.getBasePath() + "/" + DEFAULT_PARTITION_PATHS[0] + "/"
+ FSUtils.makeDataFileName(instantId, TEST_WRITE_TOKEN, fileId)));
}
String logFilePath1 = HoodieTestUtils.getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
@@ -186,11 +186,11 @@ public class CompactionTestUtils {
/**
* The hoodie data file for testing.
*/
public static class TestHoodieDataFile extends HoodieDataFile {
public static class TestHoodieBaseFile extends HoodieBaseFile {
private final String path;
public TestHoodieDataFile(String path) {
public TestHoodieBaseFile(String path) {
super(path);
this.path = path;
}

View File

@@ -22,12 +22,12 @@ import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.HoodieCommonTestHarness;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.CompactionTestUtils.TestHoodieDataFile;
import org.apache.hudi.common.util.CompactionTestUtils.TestHoodieBaseFile;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.common.versioning.compaction.CompactionPlanMigrator;
@@ -97,7 +97,7 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
// 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_1_000.parquet"));
noLogFileSlice.setBaseFile(new TestHoodieBaseFile("/tmp/noLog_1_000.parquet"));
op = CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], noLogFileSlice, Option.of(metricsCaptureFn));
testFileSliceCompactionOpEquality(noLogFileSlice, op, DEFAULT_PARTITION_PATHS[0],
LATEST_COMPACTION_METADATA_VERSION);
@@ -113,7 +113,7 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
// 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_1_000.parquet"));
fileSlice.setBaseFile(new TestHoodieBaseFile("/tmp/noLog_1_000.parquet"));
fileSlice.addLogFile(
new HoodieLogFile(new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))));
fileSlice.addLogFile(
@@ -129,13 +129,13 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
Path fullPartitionPath = new Path(new Path(metaClient.getBasePath()), DEFAULT_PARTITION_PATHS[0]);
FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "empty1");
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1");
fileSlice.setDataFile(new TestHoodieDataFile(fullPartitionPath.toString() + "/data1_1_000.parquet"));
fileSlice.setBaseFile(new TestHoodieBaseFile(fullPartitionPath.toString() + "/data1_1_000.parquet"));
fileSlice.addLogFile(new HoodieLogFile(
new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN)))));
fileSlice.addLogFile(new HoodieLogFile(
new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN)))));
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noLog1");
noLogFileSlice.setDataFile(new TestHoodieDataFile(fullPartitionPath.toString() + "/noLog_1_000.parquet"));
noLogFileSlice.setBaseFile(new TestHoodieBaseFile(fullPartitionPath.toString() + "/noLog_1_000.parquet"));
FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1");
noDataFileSlice.addLogFile(new HoodieLogFile(
new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN)))));
@@ -241,8 +241,8 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
Assert.assertEquals("Partition path is correct", expPartitionPath, op.getPartitionPath());
Assert.assertEquals("Same base-instant", slice.getBaseInstantTime(), op.getBaseInstantTime());
Assert.assertEquals("Same file-id", slice.getFileId(), op.getFileId());
if (slice.getDataFile().isPresent()) {
HoodieDataFile df = slice.getDataFile().get();
if (slice.getBaseFile().isPresent()) {
HoodieBaseFile df = slice.getBaseFile().get();
Assert.assertEquals("Same data-file", version == COMPACTION_METADATA_VERSION_1 ? df.getPath() : df.getFileName(),
op.getDataFilePath());
}