-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-5569] Maintain commit timeline even in case of long standing inflights #8783
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -90,6 +90,7 @@ | |
| import java.util.HashSet; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Set; | ||
| import java.util.UUID; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.CountDownLatch; | ||
|
|
@@ -913,19 +914,21 @@ public void testNoArchivalWithInflightCompactionInMiddle(boolean enableMetadata) | |
| if (i != 7) { | ||
| assertEquals(originalCommits, commitsAfterArchival); | ||
| } else { | ||
| // on 7th commit, archival will kick in. but will archive only one commit since 2nd compaction commit is inflight. | ||
| assertEquals(originalCommits.size() - commitsAfterArchival.size(), 1); | ||
| // on 7th commit, archival will kick in, but cannot archive any commit, | ||
| // since 1st deltacommit is the greatest completed commit before an oldest inflight commit. | ||
| assertEquals(originalCommits.size() - commitsAfterArchival.size(), 0); | ||
| } | ||
| } else { | ||
| if (i != 7) { | ||
| assertEquals(originalCommits, commitsAfterArchival); | ||
| } else { | ||
| // on 7th commit, archival will kick in. but will archive only one commit since 2nd compaction commit is inflight. | ||
| assertEquals(originalCommits.size() - commitsAfterArchival.size(), 1); | ||
| // on 7th commit, archival will kick in, but cannot archive any commit, | ||
| // since 1st deltacommit is the greatest completed commit before an oldest inflight commit. | ||
| assertEquals(originalCommits.size() - commitsAfterArchival.size(), 0); | ||
| for (int j = 1; j <= 7; j++) { | ||
| if (j == 1) { | ||
| // first commit should be archived | ||
| assertFalse(commitsAfterArchival.contains(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j))); | ||
| // first commit should not be archived | ||
| assertTrue(commitsAfterArchival.contains(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j))); | ||
| } else if (j == 2) { | ||
| // 2nd compaction should not be archived | ||
| assertFalse(commitsAfterArchival.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "0000000" + j))); | ||
|
|
@@ -1418,6 +1421,115 @@ public void testArchivalWithMaxDeltaCommitsGuaranteeForCompaction(boolean enable | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * Test archival functionality when there are inflights files. | ||
| * Archive should hold on to the greatest completed commit that is less than the oldes inflight commit. | ||
| * @throws Exception | ||
| */ | ||
| @Test | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can we add some java docs on what we are testing here.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added. |
||
| public void testGetCommitInstantsToArchiveDuringInflightCommits() throws Exception { | ||
| HoodieWriteConfig cfg = initTestTableAndGetWriteConfig(false, 3, 4, 2); | ||
|
|
||
| Set<String> expectedInstants = new HashSet<>(); | ||
| // Create 3 completed commits. | ||
| for (int i = 0; i < 3; i++) { | ||
| String instantTime = "100" + i; | ||
| HoodieTestDataGenerator.createCommitFile(basePath, instantTime, wrapperFs.getConf()); | ||
| expectedInstants.add(instantTime); | ||
| } | ||
| // Create an inflight file. | ||
| String replaceInstant = "1003"; | ||
| HoodieTestDataGenerator.createReplaceCommitRequestedFile(basePath, replaceInstant, wrapperFs.getConf()); | ||
| expectedInstants.add(replaceInstant); | ||
| // Create 3 more instants | ||
| for (int i = 4; i < 7; i++) { | ||
| String instantTime = "100" + i; | ||
| HoodieTestDataGenerator.createCommitFile(basePath, instantTime, wrapperFs.getConf()); | ||
| expectedInstants.add(instantTime); | ||
| } | ||
| // Create another inflight commit | ||
| HoodieTestDataGenerator.createRequestedCommitFile(basePath, "1007", wrapperFs.getConf()); | ||
| HoodieTestDataGenerator.createPendingCommitFile(basePath, "1007", wrapperFs.getConf()); | ||
| expectedInstants.add("1007"); | ||
| // Create 6 more instants | ||
| for (int i = 0; i < 6; i++) { | ||
| String instantTime = "101" + i; | ||
| HoodieTestDataGenerator.createCommitFile(basePath, instantTime, wrapperFs.getConf()); | ||
| expectedInstants.add(instantTime); | ||
| } | ||
| HoodieTimeline timeline = metaClient.reloadActiveTimeline().getWriteTimeline(); | ||
|
|
||
| // Check the count of instants. | ||
| assertEquals(expectedInstants.size(), timeline.countInstants(), "Loaded 14 commits and the count should match"); | ||
|
|
||
| // Run archival | ||
| HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); | ||
| HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); | ||
| boolean result = archiver.archiveIfRequired(context); | ||
| expectedInstants.remove("1000"); | ||
| expectedInstants.remove("1001"); | ||
| assertTrue(result); | ||
| timeline = metaClient.reloadActiveTimeline().getWriteTimeline(); | ||
|
|
||
| // Check the count of instants after archive it should have 2 less instants | ||
| // because 103 replacecommit's inflight will block archival. | ||
| assertEquals(12, timeline.countInstants(), "After archival only first 2 commits should be archived"); | ||
| assertEquals(expectedInstants.size(), timeline.countInstants(), "After archival only first 2 commits should be archived"); | ||
|
|
||
| HoodieTimeline finalTimeline = timeline; | ||
| assertEquals(12, expectedInstants.stream().filter(instant -> finalTimeline.containsInstant(instant)).count()); | ||
| assertEquals("1002", timeline.getInstantsAsStream().findFirst().get().getTimestamp()); | ||
|
|
||
| // Delete replacecommit requested instant. | ||
| Path replaceCommitRequestedPath = new Path( | ||
| basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" | ||
| + HoodieTimeline.makeRequestedReplaceFileName(replaceInstant)); | ||
| metaClient.getFs().delete(replaceCommitRequestedPath); | ||
| metaClient.reloadActiveTimeline(); | ||
|
|
||
| // Run archival | ||
| assertTrue(archiver.archiveIfRequired(context)); | ||
| timeline = metaClient.reloadActiveTimeline().getWriteTimeline(); | ||
| expectedInstants.removeAll(Arrays.asList("1002", "1003", "1004", "1005")); | ||
|
|
||
| // Check the count of instants after archive it should have 3 more less instants | ||
| // This time 1007 inflight commit will block archival. | ||
| assertEquals(8, timeline.countInstants(), "After archival only first 2 commits should be archived"); | ||
| assertEquals(expectedInstants.size(), timeline.countInstants(), "After archival only first 2 commits should be archived"); | ||
| HoodieTimeline refreshedTimeline = timeline; | ||
| assertEquals(8, expectedInstants.stream().filter(instant -> refreshedTimeline.containsInstant(instant)).count()); | ||
| assertEquals("1006", timeline.getInstantsAsStream().findFirst().get().getTimestamp()); | ||
| } | ||
|
|
||
| /** | ||
| * If replacecommit inflight is the oldest commit in the timeline or for that matter any inflight commit is present | ||
| * then the archival is blocked from there. This method test this scenario. | ||
| */ | ||
| @Test | ||
| public void testWithOldestReplaceCommit() throws Exception { | ||
| HoodieWriteConfig cfg = initTestTableAndGetWriteConfig(false, 2, 3, 2); | ||
|
|
||
| HoodieTestDataGenerator.createReplaceCommitRequestedFile(basePath, "1001", wrapperFs.getConf()); | ||
| HoodieTestDataGenerator.createReplaceCommitInflightFile(basePath, "1001", wrapperFs.getConf()); | ||
| // Create 8 completed commits. | ||
| for (int i = 2; i < 10; i++) { | ||
| String instantTime = "100" + i; | ||
| HoodieTestDataGenerator.createCommitFile(basePath, instantTime, wrapperFs.getConf()); | ||
| } | ||
|
|
||
| HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); | ||
| HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); | ||
|
|
||
| HoodieTimeline timeline = metaClient.reloadActiveTimeline(); | ||
| assertEquals(9, timeline.countInstants(), "Loaded 9 commits and the count should match"); | ||
| boolean result = archiver.archiveIfRequired(context); | ||
| assertTrue(result); | ||
| timeline = metaClient.reloadActiveTimeline(); | ||
| assertEquals(9, timeline.countInstants(), | ||
| "Since we have a pending replacecommit at 1001, we should never archive any commit after 1001"); | ||
| assertEquals("1001", timeline.getInstantsAsStream().findFirst().get().getTimestamp()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testArchivalAndCompactionInMetadataTable() throws Exception { | ||
| init(HoodieTableType.COPY_ON_WRITE); | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this simplifies a lot :)
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, it can fix #7738, although a little obscure to understand.