1
0

[HUDI-4134] Fix Method naming consistency issues in FSUtils (#5655)

This commit is contained in:
Heap
2022-05-24 06:28:48 +08:00
committed by GitHub
parent 716e995a38
commit 47b764ec33
26 changed files with 130 additions and 130 deletions

View File

@@ -88,11 +88,11 @@ public class TestFileSystemViewCommand extends CLIFunctionalTestHarness {
// Write date files and log file
String testWriteToken = "1-0-1";
Files.createFile(Paths.get(fullPartitionPath, FSUtils
.makeDataFileName(commitTime1, testWriteToken, fileId1)));
.makeBaseFileName(commitTime1, testWriteToken, fileId1)));
Files.createFile(Paths.get(fullPartitionPath, FSUtils
.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0, testWriteToken)));
Files.createFile(Paths.get(fullPartitionPath, FSUtils
.makeDataFileName(commitTime2, testWriteToken, fileId1)));
.makeBaseFileName(commitTime2, testWriteToken, fileId1)));
Files.createFile(Paths.get(fullPartitionPath, FSUtils
.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0, testWriteToken)));

View File

@@ -180,7 +180,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
// base file to denote some log appends happened on a slice. writeToken will still fence concurrent
// writers.
// https://issues.apache.org/jira/browse/HUDI-1517
createMarkerFile(partitionPath, FSUtils.makeDataFileName(baseInstantTime, writeToken, fileId, hoodieTable.getBaseFileExtension()));
createMarkerFile(partitionPath, FSUtils.makeBaseFileName(baseInstantTime, writeToken, fileId, hoodieTable.getBaseFileExtension()));
this.writer = createLogWriter(fileSlice, baseInstantTime);
} catch (Exception e) {

View File

@@ -100,7 +100,7 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath),
hoodieTable.getPartitionMetafileFormat());
partitionMetadata.trySave(getPartitionId());
createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config,
writeSchemaWithMetaFields, this.taskContextSupplier);
} catch (IOException e) {

View File

@@ -174,7 +174,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
hoodieTable.getPartitionMetafileFormat());
partitionMetadata.trySave(getPartitionId());
String newFileName = FSUtils.makeDataFileName(instantTime, writeToken, fileId, hoodieTable.getBaseFileExtension());
String newFileName = FSUtils.makeBaseFileName(instantTime, writeToken, fileId, hoodieTable.getBaseFileExtension());
makeOldAndNewFilePaths(partitionPath, latestValidFilePath, newFileName);
LOG.info(String.format("Merging new data into oldPath %s, as newPath %s", oldFilePath.toString(),

View File

@@ -164,7 +164,7 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
throw new HoodieIOException("Failed to make dir " + path, e);
}
return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, writeToken, fileId,
return new Path(path.toString(), FSUtils.makeBaseFileName(instantTime, writeToken, fileId,
hoodieTable.getMetaClient().getTableConfig().getBaseFileFormat().getFileExtension()));
}

View File

@@ -137,6 +137,6 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler {
String baseInstant = FSUtils.getBaseCommitTimeFromLogPath(logPath);
String writeToken = FSUtils.getWriteTokenFromLogPath(logPath);
return FSUtils.makeDataFileName(baseInstant, writeToken, fileId, table.getBaseFileFormat().getFileExtension());
return FSUtils.makeBaseFileName(baseInstant, writeToken, fileId, table.getBaseFileFormat().getFileExtension());
}
}

View File

@@ -103,7 +103,7 @@ public class HoodieTestCommitGenerator {
}
public static String getBaseFilename(String instantTime, String fileId) {
return FSUtils.makeDataFileName(instantTime, BASE_FILE_WRITE_TOKEN, fileId);
return FSUtils.makeBaseFileName(instantTime, BASE_FILE_WRITE_TOKEN, fileId);
}
public static String getLogFilename(String instantTime, String fileId) {

View File

@@ -88,7 +88,7 @@ public class FlinkCreateHandle<T extends HoodieRecordPayload, I, K, O>
*/
private void deleteInvalidDataFile(long lastAttemptId) {
final String lastWriteToken = FSUtils.makeWriteToken(getPartitionId(), getStageId(), lastAttemptId);
final String lastDataFileName = FSUtils.makeDataFileName(instantTime,
final String lastDataFileName = FSUtils.makeBaseFileName(instantTime,
lastWriteToken, this.fileId, hoodieTable.getBaseFileExtension());
final Path path = makeNewFilePath(partitionPath, lastDataFileName);
try {
@@ -136,7 +136,7 @@ public class FlinkCreateHandle<T extends HoodieRecordPayload, I, K, O>
* Use the writeToken + "-" + rollNumber as the new writeToken of a mini-batch write.
*/
private Path newFilePathWithRollover(int rollNumber) {
final String dataFileName = FSUtils.makeDataFileName(instantTime, writeToken + "-" + rollNumber, fileId,
final String dataFileName = FSUtils.makeBaseFileName(instantTime, writeToken + "-" + rollNumber, fileId,
hoodieTable.getBaseFileExtension());
return makeNewFilePath(partitionPath, dataFileName);
}

View File

@@ -90,7 +90,7 @@ public class FlinkMergeAndReplaceHandle<T extends HoodieRecordPayload, I, K, O>
*/
private void deleteInvalidDataFile(long lastAttemptId) {
final String lastWriteToken = FSUtils.makeWriteToken(getPartitionId(), getStageId(), lastAttemptId);
final String lastDataFileName = FSUtils.makeDataFileName(instantTime,
final String lastDataFileName = FSUtils.makeBaseFileName(instantTime,
lastWriteToken, this.fileId, hoodieTable.getBaseFileExtension());
final Path path = makeNewFilePath(partitionPath, lastDataFileName);
try {
@@ -139,7 +139,7 @@ public class FlinkMergeAndReplaceHandle<T extends HoodieRecordPayload, I, K, O>
protected String newFileNameWithRollover(int rollNumber) {
// make the intermediate file as hidden
final String fileID = "." + this.fileId;
return FSUtils.makeDataFileName(instantTime, writeToken + "-" + rollNumber,
return FSUtils.makeBaseFileName(instantTime, writeToken + "-" + rollNumber,
fileID, hoodieTable.getBaseFileExtension());
}

View File

@@ -94,7 +94,7 @@ public class FlinkMergeHandle<T extends HoodieRecordPayload, I, K, O>
*/
private void deleteInvalidDataFile(long lastAttemptId) {
final String lastWriteToken = FSUtils.makeWriteToken(getPartitionId(), getStageId(), lastAttemptId);
final String lastDataFileName = FSUtils.makeDataFileName(instantTime,
final String lastDataFileName = FSUtils.makeBaseFileName(instantTime,
lastWriteToken, this.fileId, hoodieTable.getBaseFileExtension());
final Path path = makeNewFilePath(partitionPath, lastDataFileName);
if (path.equals(oldFilePath)) {
@@ -159,7 +159,7 @@ public class FlinkMergeHandle<T extends HoodieRecordPayload, I, K, O>
*/
protected String newFileNameWithRollover(int rollNumber) {
// make the intermediate file as hidden
return FSUtils.makeDataFileName(instantTime, writeToken + "-" + rollNumber,
return FSUtils.makeBaseFileName(instantTime, writeToken + "-" + rollNumber,
this.fileId, hoodieTable.getBaseFileExtension());
}

View File

@@ -97,7 +97,7 @@ public class HoodieRowDataCreateHandle implements Serializable {
FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath),
table.getPartitionMetafileFormat());
partitionMetadata.trySave(taskPartitionId);
createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension()));
createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension()));
this.fileWriter = createNewFileWriter(path, table, writeConfig, rowType);
} catch (IOException e) {
throw new HoodieInsertException("Failed to initialize file writer for path " + path, e);
@@ -180,7 +180,7 @@ public class HoodieRowDataCreateHandle implements Serializable {
throw new HoodieIOException("Failed to make dir " + path, e);
}
HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig();
return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, getWriteToken(), fileId,
return new Path(path.toString(), FSUtils.makeBaseFileName(instantTime, getWriteToken(), fileId,
tableConfig.getBaseFileFormat().getFileExtension()));
}

View File

@@ -103,7 +103,7 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase
}).collect(Collectors.toList()).get(0);
assertEquals(newPathWithWriteToken.getKey().toString(), Paths.get(this.basePath, partitionPath,
FSUtils.makeDataFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString());
FSUtils.makeBaseFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString());
}
private HoodieWriteConfig makeHoodieClientConfig() {

View File

@@ -96,7 +96,7 @@ public class HoodieRowCreateHandle implements Serializable {
FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath),
table.getPartitionMetafileFormat());
partitionMetadata.trySave(taskPartitionId);
createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension()));
createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension()));
this.fileWriter = createNewFileWriter(path, table, writeConfig, structType);
} catch (IOException e) {
throw new HoodieInsertException("Failed to initialize file writer for path " + path, e);
@@ -178,7 +178,7 @@ public class HoodieRowCreateHandle implements Serializable {
throw new HoodieIOException("Failed to make dir " + path, e);
}
HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig();
return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, getWriteToken(), fileId,
return new Path(path.toString(), FSUtils.makeBaseFileName(instantTime, getWriteToken(), fileId,
tableConfig.getBaseFileFormat().getFileExtension()));
}

View File

@@ -2496,7 +2496,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
Option<Path> markerFilePath = WriteMarkersFactory.get(
cfg.getMarkersType(), getHoodieTable(metaClient, cfg), instantTime)
.create(partitionPath,
FSUtils.makeDataFileName(instantTime, "1-0-1", UUID.randomUUID().toString()),
FSUtils.makeBaseFileName(instantTime, "1-0-1", UUID.randomUUID().toString()),
IOType.MERGE);
LOG.info("Created a dummy marker path=" + markerFilePath.get());

View File

@@ -127,7 +127,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
}).collect().get(0);
assertEquals(newPathWithWriteToken.getKey().toString(), Paths.get(this.basePath, partitionPath,
FSUtils.makeDataFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString());
FSUtils.makeBaseFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString());
}
private HoodieWriteConfig makeHoodieClientConfig() {

View File

@@ -159,12 +159,12 @@ public class FSUtils {
}
// TODO: this should be removed
public static String makeDataFileName(String instantTime, String writeToken, String fileId) {
public static String makeBaseFileName(String instantTime, String writeToken, String fileId) {
return String.format("%s_%s_%s%s", fileId, writeToken, instantTime,
HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension());
}
public static String makeDataFileName(String instantTime, String writeToken, String fileId, String fileExtension) {
public static String makeBaseFileName(String instantTime, String writeToken, String fileId, String fileExtension) {
return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, fileExtension);
}

View File

@@ -84,7 +84,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
public void testMakeDataFileName() {
String instantTime = HoodieActiveTimeline.formatDate(new Date());
String fileName = UUID.randomUUID().toString();
assertEquals(FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName), fileName + "_" + TEST_WRITE_TOKEN + "_" + instantTime + BASE_FILE_EXTENSION);
assertEquals(FSUtils.makeBaseFileName(instantTime, TEST_WRITE_TOKEN, fileName), fileName + "_" + TEST_WRITE_TOKEN + "_" + instantTime + BASE_FILE_EXTENSION);
}
@Test
@@ -159,7 +159,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
public void testGetCommitTime() {
String instantTime = HoodieActiveTimeline.formatDate(new Date());
String fileName = UUID.randomUUID().toString();
String fullFileName = FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName);
String fullFileName = FSUtils.makeBaseFileName(instantTime, TEST_WRITE_TOKEN, fileName);
assertEquals(instantTime, FSUtils.getCommitTime(fullFileName));
// test log file name
fullFileName = FSUtils.makeLogFileName(fileName, HOODIE_LOG.getFileExtension(), instantTime, 1, TEST_WRITE_TOKEN);
@@ -170,7 +170,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
public void testGetFileNameWithoutMeta() {
String instantTime = HoodieActiveTimeline.formatDate(new Date());
String fileName = UUID.randomUUID().toString();
String fullFileName = FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName);
String fullFileName = FSUtils.makeBaseFileName(instantTime, TEST_WRITE_TOKEN, fileName);
assertEquals(fileName, FSUtils.getFileId(fullFileName));
}
@@ -304,7 +304,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
final String LOG_EXTENTION = "." + LOG_STR;
// data file name
String dataFileName = FSUtils.makeDataFileName(instantTime, writeToken, fileId);
String dataFileName = FSUtils.makeBaseFileName(instantTime, writeToken, fileId);
assertEquals(instantTime, FSUtils.getCommitTime(dataFileName));
assertEquals(fileId, FSUtils.getFileId(dataFileName));

View File

@@ -45,7 +45,7 @@ public class TestHoodieWriteStat {
Path basePath = new Path(basePathString);
Path partitionPath = new Path(basePath, partitionPathString);
Path finalizeFilePath = new Path(partitionPath, FSUtils.makeDataFileName(instantTime, writeToken, fileName));
Path finalizeFilePath = new Path(partitionPath, FSUtils.makeBaseFileName(instantTime, writeToken, fileName));
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setPath(basePath, finalizeFilePath);
assertEquals(finalizeFilePath, new Path(basePath, writeStat.getPath()));

View File

@@ -378,7 +378,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
String dataFileName = null;
if (!skipCreatingDataFile) {
dataFileName = FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId);
dataFileName = FSUtils.makeBaseFileName(instantTime1, TEST_WRITE_TOKEN, fileId);
new File(basePath + "/" + partitionPath + "/" + dataFileName).createNewFile();
}
String fileName1 =
@@ -417,7 +417,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
checkExternalFile(srcFileStatus, fileSlice.getBaseFile().get().getBootstrapBaseFile(), testBootstrap);
}
String compactionRequestedTime = "4";
String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId);
String compactDataFileName = FSUtils.makeBaseFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId);
List<Pair<String, FileSlice>> partitionFileSlicesPairs = new ArrayList<>();
partitionFileSlicesPairs.add(Pair.of(partitionPath, fileSlices.get(0)));
HoodieCompactionPlan compactionPlan =
@@ -552,12 +552,12 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
final String orphanFileId2 = UUID.randomUUID().toString();
final String invalidInstantId = "INVALIDTIME";
String inflightDeltaInstantTime = "7";
String orphanDataFileName = FSUtils.makeDataFileName(invalidInstantId, TEST_WRITE_TOKEN, orphanFileId1);
String orphanDataFileName = FSUtils.makeBaseFileName(invalidInstantId, TEST_WRITE_TOKEN, orphanFileId1);
new File(basePath + "/" + partitionPath + "/" + orphanDataFileName).createNewFile();
String orphanLogFileName =
FSUtils.makeLogFileName(orphanFileId2, HoodieLogFile.DELTA_EXTENSION, invalidInstantId, 0, TEST_WRITE_TOKEN);
new File(basePath + "/" + partitionPath + "/" + orphanLogFileName).createNewFile();
String inflightDataFileName = FSUtils.makeDataFileName(inflightDeltaInstantTime, TEST_WRITE_TOKEN, inflightFileId1);
String inflightDataFileName = FSUtils.makeBaseFileName(inflightDeltaInstantTime, TEST_WRITE_TOKEN, inflightFileId1);
new File(basePath + "/" + partitionPath + "/" + inflightDataFileName).createNewFile();
String inflightLogFileName = FSUtils.makeLogFileName(inflightFileId2, HoodieLogFile.DELTA_EXTENSION,
inflightDeltaInstantTime, 0, TEST_WRITE_TOKEN);
@@ -712,7 +712,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
// Only one commit, but is not safe
String commitTime1 = "1";
String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId);
String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId);
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
refreshFsView();
assertFalse(roView.getLatestBaseFiles(partitionPath).anyMatch(dfile -> dfile.getFileId().equals(fileId)),
@@ -728,7 +728,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
// Do another commit, but not safe
String commitTime2 = "2";
String fileName2 = FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId);
String fileName2 = FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId);
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
refreshFsView();
assertEquals(fileName1, roView.getLatestBaseFiles(partitionPath)
@@ -762,22 +762,22 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
String fileId3 = UUID.randomUUID().toString();
String fileId4 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0, TEST_WRITE_TOKEN))
.createNewFile();
new File(fullPartitionPath
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1, TEST_WRITE_TOKEN))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath
+ FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0, TEST_WRITE_TOKEN))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath
+ FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0, TEST_WRITE_TOKEN))
.createNewFile();
@@ -827,9 +827,9 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
for (HoodieBaseFile status : dataFileList) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)));
filenames = new HashSet<>();
List<HoodieLogFile> logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4, true)
@@ -856,12 +856,12 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
}
if (!isLatestFileSliceOnly) {
assertEquals(3, dataFiles.size());
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)));
} else {
assertEquals(1, dataFiles.size());
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
}
logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3, true)
@@ -887,13 +887,13 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
@@ -918,22 +918,22 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
Set<String> expFileNames = new HashSet<>();
if (fileId.equals(fileId1)) {
if (!isLatestFileSliceOnly) {
expFileNames.add(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1));
expFileNames.add(FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1));
}
expFileNames.add(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1));
expFileNames.add(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1));
assertEquals(expFileNames, filenames);
} else if (fileId.equals(fileId2)) {
if (!isLatestFileSliceOnly) {
expFileNames.add(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2));
expFileNames.add(FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2));
expFileNames.add(FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2));
expFileNames.add(FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2));
}
expFileNames.add(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2));
expFileNames.add(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2));
assertEquals(expFileNames, filenames);
} else {
if (!isLatestFileSliceOnly) {
expFileNames.add(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3));
expFileNames.add(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3));
}
expFileNames.add(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3));
expFileNames.add(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3));
assertEquals(expFileNames, filenames);
}
}
@@ -956,21 +956,21 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0, TEST_WRITE_TOKEN))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath
+ FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0, TEST_WRITE_TOKEN))
.createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
@@ -993,10 +993,10 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId1)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
if (!isLatestFileSliceOnly) {
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)));
}
List<FileSlice> slices =
@@ -1037,13 +1037,13 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
@@ -1063,8 +1063,8 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
for (HoodieBaseFile status : dataFiles) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)));
} else {
assertEquals(0, dataFiles.size());
}
@@ -1088,30 +1088,30 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
String fileId2 = UUID.randomUUID().toString();
String fileId3 = UUID.randomUUID().toString();
new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1))
new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1))
.createNewFile();
new File(fullPartitionPath + "/"
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0, TEST_WRITE_TOKEN))
.createNewFile();
new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1))
new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1))
.createNewFile();
new File(fullPartitionPath + "/"
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0, TEST_WRITE_TOKEN))
.createNewFile();
new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2))
new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2))
.createNewFile();
new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2))
new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2))
.createNewFile();
new File(fullPartitionPath + "/"
+ FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0, TEST_WRITE_TOKEN))
.createNewFile();
new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))
new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))
.createNewFile();
new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3))
new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3))
.createNewFile();
new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3))
new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3))
.createNewFile();
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
@@ -1158,9 +1158,9 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
for (HoodieBaseFile status : statuses1) {
filenames.add(status.getFileName());
}
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)));
}
@Test
@@ -1181,15 +1181,15 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
String deltaInstantTime2 = "3";
String fileId = UUID.randomUUID().toString();
String dataFileName = FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId);
String dataFileName = FSUtils.makeBaseFileName(instantTime1, TEST_WRITE_TOKEN, fileId);
new File(fullPartitionPath1 + dataFileName).createNewFile();
String fileName1 =
FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 0, TEST_WRITE_TOKEN);
new File(fullPartitionPath1 + fileName1).createNewFile();
new File(fullPartitionPath2 + FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile();
new File(fullPartitionPath2 + FSUtils.makeBaseFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile();
new File(fullPartitionPath2 + fileName1).createNewFile();
new File(fullPartitionPath3 + FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile();
new File(fullPartitionPath3 + FSUtils.makeBaseFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile();
new File(fullPartitionPath3 + fileName1).createNewFile();
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
@@ -1228,7 +1228,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
partitionFileSlicesPairs.add(Pair.of(partitionPath3, fileSlices.get(0)));
String compactionRequestedTime = "2";
String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId);
String compactDataFileName = FSUtils.makeBaseFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId);
HoodieCompactionPlan compactionPlan =
CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs, Option.empty(), Option.empty());
@@ -1345,8 +1345,8 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
"No commit, should not find any data file");
// Only one commit
String commitTime1 = "1";
String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile();
new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile();
@@ -1362,8 +1362,8 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
// create commit2 - fileId1 is replaced. new file groups fileId3,fileId4 are created.
String fileId3 = UUID.randomUUID().toString();
String fileId4 = UUID.randomUUID().toString();
String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3);
String fileName4 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId4);
String fileName3 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId3);
String fileName4 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId4);
new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile();
new File(basePath + "/" + partitionPath1 + "/" + fileName4).createNewFile();
@@ -1440,10 +1440,10 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
// Only one commit
String commitTime1 = "1";
String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3);
String fileName4 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId4);
String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
String fileName3 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId3);
String fileName4 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId4);
new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile();
new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile();
new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile();
@@ -1500,9 +1500,9 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
"No commit, should not find any data file");
// Only one commit
String commitTime1 = "1";
String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3);
String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
String fileName3 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId3);
new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile();
new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile();
new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile();
@@ -1614,8 +1614,8 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
// first insert commit
String commitTime1 = "1";
String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
@@ -1636,7 +1636,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
// replace commit
String commitTime2 = "2";
String fileName3 = FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId3);
String fileName3 = FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId3);
new File(basePath + "/" + partitionPath + "/" + fileName3).createNewFile();
HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime2);
@@ -1658,7 +1658,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
// another insert commit
String commitTime3 = "3";
String fileName4 = FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId4);
String fileName4 = FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId4);
new File(basePath + "/" + partitionPath + "/" + fileName4).createNewFile();
HoodieInstant instant3 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime3);

View File

@@ -826,7 +826,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
File file = new File(basePath + "/" + p + "/"
+ (deltaCommit
? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant), TEST_WRITE_TOKEN)
: FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f)));
: FSUtils.makeBaseFileName(instant, TEST_WRITE_TOKEN, f)));
file.createNewFile();
HoodieWriteStat w = new HoodieWriteStat();
w.setFileId(f);

View File

@@ -80,7 +80,7 @@ public class FileCreateUtils {
}
public static String baseFileName(String instantTime, String fileId, String fileExtension) {
return FSUtils.makeDataFileName(instantTime, WRITE_TOKEN, fileId, fileExtension);
return FSUtils.makeBaseFileName(instantTime, WRITE_TOKEN, fileId, fileExtension);
}
public static String logFileName(String instantTime, String fileId, int version) {

View File

@@ -147,7 +147,7 @@ public class TestClusteringUtils extends HoodieCommonTestHarness {
private FileSlice generateFileSlice(String partitionPath, String fileId, String baseInstant) {
FileSlice fs = new FileSlice(new HoodieFileGroupId(partitionPath, fileId), baseInstant);
fs.setBaseFile(new HoodieBaseFile(FSUtils.makeDataFileName(baseInstant, "1-0-1", fileId)));
fs.setBaseFile(new HoodieBaseFile(FSUtils.makeBaseFileName(baseInstant, "1-0-1", fileId)));
return fs;
}

View File

@@ -101,7 +101,7 @@ public class InputFormatTestUtil {
throws IOException {
for (int i = 0; i < numberOfFiles; i++) {
Files.createFile(partitionPath.toPath()
.resolve(FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, fileId + i, baseFileExtension)));
.resolve(FSUtils.makeBaseFileName(commitNumber, TEST_WRITE_TOKEN, fileId + i, baseFileExtension)));
}
return partitionPath;
}
@@ -118,7 +118,7 @@ public class InputFormatTestUtil {
List<File> toUpdateList = dataFiles.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size()));
for (File file : toUpdateList) {
String fileId = FSUtils.getFileId(file.getName());
Files.createFile(directory.toPath().resolve(FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId,
Files.createFile(directory.toPath().resolve(FSUtils.makeBaseFileName(newCommit, TEST_WRITE_TOKEN, fileId,
baseFileExtension)));
}
}
@@ -270,7 +270,7 @@ public class InputFormatTestUtil {
String commitNumber) throws IOException {
AvroParquetWriter parquetWriter;
for (int i = 0; i < numberOfFiles; i++) {
String fileId = FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension());
String fileId = FSUtils.makeBaseFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension());
parquetWriter = new AvroParquetWriter(new Path(partitionPath.resolve(fileId).toString()), schema);
try {
for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber, fileId)) {
@@ -286,7 +286,7 @@ public class InputFormatTestUtil {
String commitNumber) throws Exception {
AvroParquetWriter parquetWriter;
for (int i = 0; i < numberOfFiles; i++) {
String fileId = FSUtils.makeDataFileName(commitNumber, "1", "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension());
String fileId = FSUtils.makeBaseFileName(commitNumber, "1", "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension());
parquetWriter = new AvroParquetWriter(new Path(partitionPath.resolve(fileId).toString()), schema);
try {
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, numberOfRecords);
@@ -318,7 +318,7 @@ public class InputFormatTestUtil {
File fileToUpdate = Objects.requireNonNull(directory.listFiles((dir, name) -> name.endsWith("parquet")))[0];
String fileId = FSUtils.getFileId(fileToUpdate.getName());
File dataFile = new File(directory,
FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId, HoodieFileFormat.PARQUET.getFileExtension()));
FSUtils.makeBaseFileName(newCommit, TEST_WRITE_TOKEN, fileId, HoodieFileFormat.PARQUET.getFileExtension()));
try (AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema)) {
for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit, fileId)) {
if (numberOfRecordsToUpdate > 0) {

View File

@@ -234,7 +234,7 @@ public class HiveTestUtil {
fileSystem.mkdirs(partPath);
List<HoodieWriteStat> writeStats = new ArrayList<>();
String fileId = UUID.randomUUID().toString();
Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeDataFileName(instantTime, "1-0-1", fileId));
Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeBaseFileName(instantTime, "1-0-1", fileId));
Schema schema = SchemaTestUtil.getSchemaFromResource(HiveTestUtil.class, schemaFileName);
generateParquetDataWithSchema(filePath, schema);
HoodieWriteStat writeStat = new HoodieWriteStat();
@@ -371,7 +371,7 @@ public class HiveTestUtil {
for (int i = 0; i < 5; i++) {
// Create 5 files
String fileId = UUID.randomUUID().toString();
Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeDataFileName(instantTime, "1-0-1", fileId));
Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeBaseFileName(instantTime, "1-0-1", fileId));
generateParquetData(filePath, isParquetSchemaSimple);
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId(fileId);

View File

@@ -211,7 +211,7 @@ public class TestCluster implements BeforeAllCallback, AfterAllCallback,
// Create 5 files
String fileId = UUID.randomUUID().toString();
Path filePath = new Path(partPath.toString() + "/" + FSUtils
.makeDataFileName(commitTime, "1-0-1", fileId));
.makeBaseFileName(commitTime, "1-0-1", fileId));
generateParquetData(filePath, isParquetSchemaSimple);
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId(fileId);

View File

@@ -99,27 +99,27 @@ public class TestHoodieSnapshotCopier extends FunctionalTestHarness {
HoodieTestDataGenerator.writePartitionMetadataDeprecated(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"},
basePath);
// Make commit1
File file11 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id11"));
File file11 = new File(basePath + "/2016/05/01/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, "id11"));
file11.createNewFile();
File file12 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id12"));
File file12 = new File(basePath + "/2016/05/02/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, "id12"));
file12.createNewFile();
File file13 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id13"));
File file13 = new File(basePath + "/2016/05/06/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, "id13"));
file13.createNewFile();
// Make commit2
File file21 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id21"));
File file21 = new File(basePath + "/2016/05/01/" + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, "id21"));
file21.createNewFile();
File file22 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id22"));
File file22 = new File(basePath + "/2016/05/02/" + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, "id22"));
file22.createNewFile();
File file23 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id23"));
File file23 = new File(basePath + "/2016/05/06/" + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, "id23"));
file23.createNewFile();
// Make commit3
File file31 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id31"));
File file31 = new File(basePath + "/2016/05/01/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, "id31"));
file31.createNewFile();
File file32 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id32"));
File file32 = new File(basePath + "/2016/05/02/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, "id32"));
file32.createNewFile();
File file33 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id33"));
File file33 = new File(basePath + "/2016/05/06/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, "id33"));
file33.createNewFile();
// Do a snapshot copy