[HUDI-2925] Fix duplicate cleaning of same files when unfinished clean operations are present using a config. (#4212)
Co-authored-by: sivabalan <n.siva.b@gmail.com>
This commit is contained in:
@@ -122,6 +122,7 @@ import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.awaitility.Awaitility.await;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
@@ -254,6 +255,73 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
SparkRDDWriteClient::upsertPreppedRecords, true);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Tests no more than 1 clean is scheduled/executed if HoodieCompactionConfig.allowMultipleCleanSchedule config is disabled.
|
||||
*/
|
||||
@Test
|
||||
public void testMultiClean() {
|
||||
HoodieWriteConfig writeConfig = getConfigBuilder()
|
||||
.withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder()
|
||||
.withEnableBackupForRemoteFileSystemView(false).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024)
|
||||
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1)
|
||||
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER)
|
||||
.allowMultipleCleans(false)
|
||||
.withAutoClean(false).retainCommits(1).retainFileVersions(1).build())
|
||||
.withEmbeddedTimelineServerEnabled(false).build();
|
||||
|
||||
int index = 0;
|
||||
String cleanInstantTime;
|
||||
final String partition = "2015/03/16";
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(context, writeConfig)) {
|
||||
// Three writes so we can initiate a clean
|
||||
for (; index < 3; ++index) {
|
||||
String newCommitTime = "00" + index;
|
||||
List<HoodieRecord> records = dataGen.generateInsertsForPartition(newCommitTime, 1, partition);
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
client.insert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
}
|
||||
}
|
||||
|
||||
// mimic failed/leftover clean by scheduling a clean but not performing it
|
||||
cleanInstantTime = "00" + index++;
|
||||
HoodieTable table = HoodieSparkTable.create(writeConfig, context);
|
||||
Option<HoodieCleanerPlan> cleanPlan = table.scheduleCleaning(context, cleanInstantTime, Option.empty());
|
||||
assertEquals(cleanPlan.get().getFilePathsToBeDeletedPerPartition().get(partition).size(), 1);
|
||||
assertEquals(metaClient.reloadActiveTimeline().getCleanerTimeline().filterInflightsAndRequested().countInstants(), 1);
|
||||
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(context, writeConfig)) {
|
||||
// Next commit. This is required so that there is an additional file version to clean.
|
||||
String newCommitTime = "00" + index++;
|
||||
List<HoodieRecord> records = dataGen.generateInsertsForPartition(newCommitTime, 1, partition);
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
client.insert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
|
||||
// Initiate another clean. The previous leftover clean will be attempted first, followed by another clean
|
||||
// due to the commit above.
|
||||
String newCleanInstantTime = "00" + index++;
|
||||
HoodieCleanMetadata cleanMetadata = client.clean(newCleanInstantTime);
|
||||
// subsequent clean should not be triggered since allowMultipleCleanSchedules is set to false
|
||||
assertNull(cleanMetadata);
|
||||
|
||||
// let the old clean complete
|
||||
table = HoodieSparkTable.create(writeConfig, context);
|
||||
cleanMetadata = table.clean(context, cleanInstantTime, false);
|
||||
assertNotNull(cleanMetadata);
|
||||
|
||||
// any new clean should go ahead
|
||||
cleanMetadata = client.clean(newCleanInstantTime);
|
||||
// subsequent clean should not be triggered since allowMultipleCleanSchedules is set to false
|
||||
assertNotNull(cleanMetadata);
|
||||
|
||||
// 1 file cleaned
|
||||
assertEquals(cleanMetadata.getPartitionMetadata().get(partition).getSuccessDeleteFiles().size(), 1);
|
||||
assertEquals(cleanMetadata.getPartitionMetadata().get(partition).getFailedDeleteFiles().size(), 0);
|
||||
assertEquals(cleanMetadata.getPartitionMetadata().get(partition).getDeletePathPatterns().size(), 1);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective.
|
||||
*
|
||||
|
||||
Reference in New Issue
Block a user