1
0

[HUDI-2559] Converting commit timestamp format to millisecs (#4024)

- Adds support for generating commit timestamps with millisecs granularity. 
- Older commit timestamps (in secs granularity) will be suffixed with 999 and parsed with millisecs format.
This commit is contained in:
Sivabalan Narayanan
2021-11-22 11:44:38 -05:00
committed by GitHub
parent 89452063b4
commit fc9ca6a07a
19 changed files with 132 additions and 66 deletions

View File

@@ -79,14 +79,14 @@ public class TestFSUtils extends HoodieCommonTestHarness {
@Test
public void testMakeDataFileName() {
String instantTime = HoodieActiveTimeline.formatInstantTime(new Date());
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);
}
@Test
public void testMaskFileName() {
String instantTime = HoodieActiveTimeline.formatInstantTime(new Date());
String instantTime = HoodieActiveTimeline.formatDate(new Date());
int taskPartitionId = 2;
assertEquals(FSUtils.maskWithoutFileId(instantTime, taskPartitionId), "*_" + taskPartitionId + "_" + instantTime + BASE_FILE_EXTENSION);
}
@@ -154,7 +154,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
@Test
public void testGetCommitTime() {
String instantTime = HoodieActiveTimeline.formatInstantTime(new Date());
String instantTime = HoodieActiveTimeline.formatDate(new Date());
String fileName = UUID.randomUUID().toString();
String fullFileName = FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName);
assertEquals(instantTime, FSUtils.getCommitTime(fullFileName));
@@ -165,7 +165,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
@Test
public void testGetFileNameWithoutMeta() {
String instantTime = HoodieActiveTimeline.formatInstantTime(new Date());
String instantTime = HoodieActiveTimeline.formatDate(new Date());
String fileName = UUID.randomUUID().toString();
String fullFileName = FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName);
assertEquals(fileName, FSUtils.getFileId(fullFileName));

View File

@@ -36,7 +36,7 @@ public class TestHoodieWriteStat {
@Test
public void testSetPaths() {
String instantTime = HoodieActiveTimeline.formatInstantTime(new Date());
String instantTime = HoodieActiveTimeline.formatDate(new Date());
String basePathString = "/data/tables/some-hoodie-table";
String partitionPathString = "2017/12/31";
String fileName = UUID.randomUUID().toString();

View File

@@ -444,7 +444,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
}
// All zero timestamp can be parsed
HoodieActiveTimeline.parseInstantTime("00000000000000");
HoodieActiveTimeline.parseDateFromInstantTime("00000000000000");
// Multiple thread test
final int numChecks = 100000;
@@ -455,9 +455,9 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
for (int idx = 0; idx < numThreads; ++idx) {
futures.add(executorService.submit(() -> {
Date date = new Date(System.currentTimeMillis() + (int)(Math.random() * numThreads) * milliSecondsInYear);
final String expectedFormat = HoodieActiveTimeline.formatInstantTime(date);
final String expectedFormat = HoodieActiveTimeline.formatDate(date);
for (int tidx = 0; tidx < numChecks; ++tidx) {
final String curFormat = HoodieActiveTimeline.formatInstantTime(date);
final String curFormat = HoodieActiveTimeline.formatDate(date);
if (!curFormat.equals(expectedFormat)) {
throw new HoodieException("Format error: expected=" + expectedFormat + ", curFormat=" + curFormat);
}
@@ -476,16 +476,37 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
@Test
public void testMetadataCompactionInstantDateParsing() throws ParseException {
// default second granularity instant ID
String secondGranularityInstant = "20210101120101";
Date defaultSecsGranularityDate = HoodieActiveTimeline.parseInstantTime(secondGranularityInstant);
String secondGranularityInstant = "20210101120101123";
Date defaultSecsGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(secondGranularityInstant);
// metadata table compaction/cleaning : ms granularity instant ID
String compactionInstant = secondGranularityInstant + "001";
Date msGranularityDate = HoodieActiveTimeline.parseInstantTime(compactionInstant);
assertEquals(0, msGranularityDate.getTime() - defaultSecsGranularityDate.getTime(), "Expected the ms part to be 0");
Date defaultMsGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(compactionInstant);
assertEquals(0, defaultMsGranularityDate.getTime() - defaultSecsGranularityDate.getTime(), "Expected the ms part to be 0");
assertTrue(HoodieTimeline.compareTimestamps(secondGranularityInstant, HoodieTimeline.LESSER_THAN, compactionInstant));
assertTrue(HoodieTimeline.compareTimestamps(compactionInstant, HoodieTimeline.GREATER_THAN, secondGranularityInstant));
}
@Test
public void testMillisGranularityInstantDateParsing() throws ParseException {
// Old second granularity instant ID
String secondGranularityInstant = "20210101120101";
Date defaultMsGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(secondGranularityInstant);
// New ms granularity instant ID
String specificMsGranularityInstant = secondGranularityInstant + "009";
Date msGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(specificMsGranularityInstant);
assertEquals(999, defaultMsGranularityDate.getTime() % 1000, "Expected the ms part to be 999");
assertEquals(9, msGranularityDate.getTime() % 1000, "Expected the ms part to be 9");
// Ensure that any date math which expects second granularity still works
String laterDateInstant = "20210101120111"; // + 10 seconds from original instant
assertEquals(
10,
HoodieActiveTimeline.parseDateFromInstantTime(laterDateInstant).getTime() / 1000
- HoodieActiveTimeline.parseDateFromInstantTime(secondGranularityInstant).getTime() / 1000,
"Expected the difference between later instant and previous instant to be 10 seconds"
);
}
/**
* Returns an exhaustive list of all possible HoodieInstant.
* @return list of HoodieInstant

View File

@@ -148,7 +148,7 @@ public class HoodieTestTable {
}
public static String makeNewCommitTime(Instant dateTime) {
return HoodieActiveTimeline.formatInstantTime(Date.from(dateTime));
return HoodieActiveTimeline.formatDate(Date.from(dateTime));
}
public static List<String> makeIncrementalCommitTimes(int num) {