-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-1266] Add unit test for validating replacecommit rollback #2418
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,23 +18,26 @@ | |
|
|
||
| package org.apache.hudi.table.action.rollback; | ||
|
|
||
| import org.apache.hudi.client.HoodieWriteResult; | ||
| import org.apache.hudi.client.SparkRDDWriteClient; | ||
| import org.apache.hudi.client.WriteStatus; | ||
| import org.apache.hudi.common.model.FileSlice; | ||
| import org.apache.hudi.common.model.HoodieFileGroup; | ||
| import org.apache.hudi.common.model.HoodieRecord; | ||
| import org.apache.hudi.common.model.HoodieTableType; | ||
| import org.apache.hudi.common.table.timeline.HoodieTimeline; | ||
| import org.apache.hudi.common.table.view.SyncableFileSystemView; | ||
| import org.apache.hudi.common.testutils.HoodieTestDataGenerator; | ||
| import org.apache.hudi.common.util.Option; | ||
| import org.apache.hudi.config.HoodieWriteConfig; | ||
| import org.apache.hudi.table.HoodieTable; | ||
| import org.apache.hudi.testutils.Assertions; | ||
| import org.apache.hudi.testutils.HoodieClientTestBase; | ||
|
|
||
| import org.apache.spark.api.java.JavaRDD; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.List; | ||
| import java.util.Set; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH; | ||
|
|
@@ -96,4 +99,61 @@ protected void twoUpsertCommitDataWithTwoPartitions(List<FileSlice> firstPartiti | |
| assertEquals(1, secondPartitionCommit2FileSlices.size()); | ||
| } | ||
| } | ||
|
|
||
| protected void insertOverwriteCommitDataWithTwoPartitions(List<FileSlice> firstPartitionCommit2FileSlices, | ||
| List<FileSlice> secondPartitionCommit2FileSlices, | ||
| HoodieWriteConfig cfg, | ||
| boolean commitSecondInsertOverwrite) throws IOException { | ||
| //just generate two partitions | ||
| dataGen = new HoodieTestDataGenerator(new String[]{DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH}); | ||
| HoodieTestDataGenerator.writePartitionMetadata(fs, new String[]{DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH}, basePath); | ||
| SparkRDDWriteClient client = getHoodieWriteClient(cfg); | ||
| /** | ||
| * Write 1 (upsert) | ||
| */ | ||
| String newCommitTime = "001"; | ||
| List<HoodieRecord> records = dataGen.generateInsertsContainsAllPartitions(newCommitTime, 2); | ||
|
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. this code is the same as twoUpsertCommitDataWithTwoPartitions, can we reuse it ?
Member
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. We can only reuse two lines because some of the state generated (List) is needed for subsequent operations. So i'm not inclined to add another method for this. let me know if you have strong opinion.
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. ok,Make sense |
||
| JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1); | ||
| client.startCommitWithTime(newCommitTime); | ||
| JavaRDD<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime); | ||
| Assertions.assertNoWriteErrors(statuses.collect()); | ||
| client.commit(newCommitTime, statuses); | ||
|
|
||
| // get fileIds written | ||
| HoodieTable table = this.getHoodieTable(metaClient, cfg); | ||
| SyncableFileSystemView fsView = getFileSystemViewWithUnCommittedSlices(table.getMetaClient()); | ||
| List<HoodieFileGroup> firstPartitionCommit1FileGroups = fsView.getAllFileGroups(DEFAULT_FIRST_PARTITION_PATH).collect(Collectors.toList()); | ||
| assertEquals(1, firstPartitionCommit1FileGroups.size()); | ||
| Set<String> partition1Commit1FileIds = firstPartitionCommit1FileGroups.get(0).getAllFileSlices().map(FileSlice::getFileId).collect(Collectors.toSet()); | ||
| List<HoodieFileGroup> secondPartitionCommit1FileGroups = fsView.getAllFileGroups(DEFAULT_SECOND_PARTITION_PATH).collect(Collectors.toList()); | ||
| assertEquals(1, secondPartitionCommit1FileGroups.size()); | ||
| Set<String> partition2Commit1FileIds = secondPartitionCommit1FileGroups.get(0).getAllFileSlices().map(FileSlice::getFileId).collect(Collectors.toSet()); | ||
|
|
||
| /** | ||
| * Write 2 (one insert_overwrite) | ||
| */ | ||
| String commitActionType = HoodieTimeline.REPLACE_COMMIT_ACTION; | ||
| newCommitTime = "002"; | ||
| records = dataGen.generateInsertsContainsAllPartitions(newCommitTime, 2); | ||
| writeRecords = jsc.parallelize(records, 1); | ||
| client.startCommitWithTime(newCommitTime, commitActionType); | ||
| HoodieWriteResult result = client.insertOverwrite(writeRecords, newCommitTime); | ||
| statuses = result.getWriteStatuses(); | ||
| Assertions.assertNoWriteErrors(statuses.collect()); | ||
| if (commitSecondInsertOverwrite) { | ||
| client.commit(newCommitTime, statuses, Option.empty(), commitActionType, result.getPartitionToReplaceFileIds()); | ||
| } | ||
| metaClient.reloadActiveTimeline(); | ||
| // get new fileIds written as part of insert_overwrite | ||
| fsView = getFileSystemViewWithUnCommittedSlices(metaClient); | ||
| List<HoodieFileGroup> firstPartitionCommit2FileGroups = fsView.getAllFileGroups(DEFAULT_FIRST_PARTITION_PATH) | ||
| .filter(fg -> !partition1Commit1FileIds.contains(fg.getFileGroupId().getFileId())).collect(Collectors.toList()); | ||
| firstPartitionCommit2FileSlices.addAll(firstPartitionCommit2FileGroups.get(0).getAllFileSlices().collect(Collectors.toList())); | ||
| List<HoodieFileGroup> secondPartitionCommit2FileGroups = fsView.getAllFileGroups(DEFAULT_SECOND_PARTITION_PATH) | ||
| .filter(fg -> !partition2Commit1FileIds.contains(fg.getFileGroupId().getFileId())).collect(Collectors.toList()); | ||
| secondPartitionCommit2FileSlices.addAll(secondPartitionCommit2FileGroups.get(0).getAllFileSlices().collect(Collectors.toList())); | ||
|
|
||
| assertEquals(1, firstPartitionCommit2FileSlices.size()); | ||
| assertEquals(1, secondPartitionCommit2FileSlices.size()); | ||
| } | ||
| } | ||
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.
now have the insertOverwriteCommitDataWithTwoPartitions in HoodieClientRollbackTestBase.java . Is it necessary to add the unit test for TestMergeOnReadRollbackActionExecutor
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.
Yes, I want to add test for MOR too. But because of time constraints just added one for COW. I plan to add MOR test later on.