Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1327,7 +1327,7 @@ public void testSmallInsertHandlingForInserts(boolean mergeAllowDuplicateInserts
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
List<WriteStatus> statuses = client.insert(insertRecordsRDD1, commitTime1).collect();
assertNoWriteErrors(statuses);
assertPartitionMetadata(new String[] {testPartitionPath}, fs);
assertPartitionMetadata(basePath, new String[] {testPartitionPath}, fs);
assertEquals(1, statuses.size(), "Just 1 file needs to be added.");
String file1 = statuses.get(0).getFileId();
assertEquals(100,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
import org.apache.hudi.avro.model.HoodieSliceInfo;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.transaction.lock.InProcessLockProvider;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.HoodieCleanStat;
import org.apache.hudi.common.bootstrap.TestBootstrapIndex;
import org.apache.hudi.common.config.HoodieMetadataConfig;
Expand Down Expand Up @@ -73,7 +73,6 @@
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieCleanConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieLockConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.index.HoodieIndex;
Expand All @@ -86,8 +85,6 @@

import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
Expand All @@ -101,10 +98,8 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -132,21 +127,24 @@
public class TestCleaner extends HoodieClientTestBase {

private static final int BIG_BATCH_INSERT_SIZE = 500;
private static final Logger LOG = LogManager.getLogger(TestCleaner.class);
private static final int PARALLELISM = 10;

/**
* Helper method to do first batch of insert for clean by versions/commits tests.
*
* @param cfg Hoodie Write Config
* @param context Spark engine context
* @param metaClient Hoodie table meta client
* @param client Hoodie Client
* @param recordGenFunction Function to generate records for insertion
* @param insertFn Insertion API for testing
* @throws Exception in case of error
*/
private Pair<String, JavaRDD<WriteStatus>> insertFirstBigBatchForClientCleanerTest(HoodieWriteConfig cfg, SparkRDDWriteClient client,
public static Pair<String, JavaRDD<WriteStatus>> insertFirstBigBatchForClientCleanerTest(
HoodieSparkEngineContext context,
HoodieTableMetaClient metaClient,
SparkRDDWriteClient client,
Function2<List<HoodieRecord>, String, Integer> recordGenFunction,
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
HoodieCleaningPolicy cleaningPolicy) throws Exception {
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> insertFn) throws Exception {

/*
* do a big insert (this is basically same as insert part of upsert, just adding it here so we can catch breakages
Expand All @@ -155,7 +153,7 @@ private Pair<String, JavaRDD<WriteStatus>> insertFirstBigBatchForClientCleanerTe
String newCommitTime = client.startCommit();

List<HoodieRecord> records = recordGenFunction.apply(newCommitTime, BIG_BATCH_INSERT_SIZE);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 5);
JavaRDD<HoodieRecord> writeRecords = context.getJavaSparkContext().parallelize(records, PARALLELISM);

JavaRDD<WriteStatus> statuses = insertFn.apply(client, writeRecords, newCommitTime);
// Verify there are no errors
Expand All @@ -174,8 +172,8 @@ private Pair<String, JavaRDD<WriteStatus>> insertFirstBigBatchForClientCleanerTe
assertTrue(table.getCompletedCleanTimeline().empty());

if (client.getConfig().shouldAutoCommit()) {
HoodieIndex index = SparkHoodieIndexFactory.createIndex(cfg);
List<HoodieRecord> taggedRecords = tagLocation(index, jsc.parallelize(records, 1), table).collect();
HoodieIndex index = SparkHoodieIndexFactory.createIndex(client.getConfig());
List<HoodieRecord> taggedRecords = tagLocation(index, context, context.getJavaSparkContext().parallelize(records, PARALLELISM), table).collect();
checkTaggedRecords(taggedRecords, newCommitTime);
}
return Pair.of(newCommitTime, statuses);
Expand All @@ -184,16 +182,17 @@ private Pair<String, JavaRDD<WriteStatus>> insertFirstBigBatchForClientCleanerTe
/**
* Helper method to do first batch of insert for clean by versions/commits tests.
*
* @param cfg Hoodie Write Config
* @param context Spark engine context
* @param client Hoodie Client
* @param recordGenFunction Function to generate records for insertion
* @param insertFn Insertion API for testing
* @throws Exception in case of error
*/
private Pair<String, JavaRDD<WriteStatus>> insertFirstFailedBigBatchForClientCleanerTest(HoodieWriteConfig cfg, SparkRDDWriteClient client,
Function2<List<HoodieRecord>, String, Integer> recordGenFunction,
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
HoodieCleaningPolicy cleaningPolicy) throws Exception {
public static Pair<String, JavaRDD<WriteStatus>> insertFirstFailedBigBatchForClientCleanerTest(
HoodieSparkEngineContext context,
SparkRDDWriteClient client,
Function2<List<HoodieRecord>, String, Integer> recordGenFunction,
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> insertFn) throws Exception {

/*
* do a big insert (this is basically same as insert part of upsert, just adding it here so we can catch breakages
Expand All @@ -202,7 +201,7 @@ private Pair<String, JavaRDD<WriteStatus>> insertFirstFailedBigBatchForClientCle
String newCommitTime = client.startCommit();

List<HoodieRecord> records = recordGenFunction.apply(newCommitTime, BIG_BATCH_INSERT_SIZE);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 5);
JavaRDD<HoodieRecord> writeRecords = context.getJavaSparkContext().parallelize(records, 5);

JavaRDD<WriteStatus> statuses = insertFn.apply(client, writeRecords, newCommitTime);
// Verify there are no errors
Expand Down Expand Up @@ -359,8 +358,7 @@ private void testInsertAndCleanByVersions(
final Function2<List<HoodieRecord>, String, Integer> recordUpsertGenWrappedFunction =
generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateUniqueUpdates);

insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn,
HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS);
insertFirstBigBatchForClientCleanerTest(context, metaClient, client, recordInsertGenWrappedFunction, insertFn);

Map<HoodieFileGroupId, FileSlice> compactionFileIdToLatestFileSlice = new HashMap<>();
metaClient = HoodieTableMetaClient.reload(metaClient);
Expand Down Expand Up @@ -458,15 +456,6 @@ private void testInsertAndCleanByVersions(
}
}

/**
* Test Clean-By-Commits using insert/upsert API.
*/
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testInsertAndCleanByCommits(boolean isAsync) throws Exception {
testInsertAndCleanByCommits(SparkRDDWriteClient::insert, SparkRDDWriteClient::upsert, false, isAsync);
}

/**
* Test Clean-By-Commits using insert/upsert API.
*/
Expand All @@ -475,117 +464,6 @@ public void testFailedInsertAndCleanByCommits() throws Exception {
testFailedInsertAndCleanByCommits(SparkRDDWriteClient::insert, false);
}

/**
* Test Clean-By-Commits using prepped version of insert/upsert API.
*/
@Test
public void testInsertPreppedAndCleanByCommits() throws Exception {
testInsertAndCleanByCommits(SparkRDDWriteClient::insertPreppedRecords, SparkRDDWriteClient::upsertPreppedRecords,
true, false);
}

/**
* Test Clean-By-Commits using prepped versions of bulk-insert/upsert API.
*/
@Test
public void testBulkInsertPreppedAndCleanByCommits() throws Exception {
testInsertAndCleanByCommits(
(client, recordRDD, instantTime) -> client.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty()),
SparkRDDWriteClient::upsertPreppedRecords, true, false);
}

/**
* Test Clean-By-Commits using bulk-insert/upsert API.
*/
@Test
public void testBulkInsertAndCleanByCommits() throws Exception {
testInsertAndCleanByCommits(SparkRDDWriteClient::bulkInsert, SparkRDDWriteClient::upsert, false, false);
}

/**
* Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective.
*
* @param insertFn Insert API to be tested
* @param upsertFn Upsert API to be tested
* @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during
* record generation to also tag the regards (de-dupe is implicit as we use unique record-gen APIs)
* @throws Exception in case of errors
*/
private void testInsertAndCleanByCommits(
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> upsertFn, boolean isPreppedAPI, boolean isAsync)
throws Exception {
int maxCommits = 3; // keep upto 3 commits from the past
HoodieWriteConfig cfg = getConfigBuilder()
.withCleanConfig(HoodieCleanConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).withAsyncClean(isAsync).retainCommits(maxCommits).build())
.withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build())
.build();
SparkRDDWriteClient client = getHoodieWriteClient(cfg);

final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =
generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts);

final Function2<List<HoodieRecord>, String, Integer> recordUpsertGenWrappedFunction =
generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateUniqueUpdates);

insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn,
HoodieCleaningPolicy.KEEP_LATEST_COMMITS);

// Keep doing some writes and clean inline. Make sure we have expected number of files remaining.
for (int i = 0; i < 8; i++) {
String newCommitTime = makeNewCommitTime();
try {
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = recordUpsertGenWrappedFunction.apply(newCommitTime, 100);

List<WriteStatus> statuses = upsertFn.apply(client, jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);

metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table1 = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimeline activeTimeline = table1.getCompletedCommitsTimeline();
HoodieInstant lastInstant = activeTimeline.lastInstant().get();
if (cfg.isAsyncClean()) {
activeTimeline = activeTimeline.findInstantsBefore(lastInstant.getTimestamp());
}
// NOTE: See CleanPlanner#getFilesToCleanKeepingLatestCommits. We explicitly keep one commit before earliest
// commit
Option<HoodieInstant> earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits);
Set<HoodieInstant> acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet());
if (earliestRetainedCommit.isPresent()) {
acceptableCommits
.removeAll(activeTimeline.findInstantsInRange("000", earliestRetainedCommit.get().getTimestamp())
.getInstants().collect(Collectors.toSet()));
acceptableCommits.add(earliestRetainedCommit.get());
}

TableFileSystemView fsView = table1.getFileSystemView();
// Need to ensure the following
for (String partitionPath : dataGen.getPartitionPaths()) {
List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList());
for (HoodieFileGroup fileGroup : fileGroups) {
Set<String> commitTimes = new HashSet<>();
fileGroup.getAllBaseFiles().forEach(value -> {
LOG.debug("Data File - " + value);
commitTimes.add(value.getCommitTime());
});
if (cfg.isAsyncClean()) {
commitTimes.remove(lastInstant.getTimestamp());
}
assertEquals(acceptableCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()), commitTimes,
"Only contain acceptable versions of file should be present");
}
}
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
}

/**
* Test Helper for Cleaning failed commits by commits logic from HoodieWriteClient API perspective.
*
Expand All @@ -612,22 +490,18 @@ private void testFailedInsertAndCleanByCommits(
final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =
generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts);

Pair<String, JavaRDD<WriteStatus>> result = insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn,
HoodieCleaningPolicy.KEEP_LATEST_COMMITS);
Pair<String, JavaRDD<WriteStatus>> result = insertFirstBigBatchForClientCleanerTest(context, metaClient, client, recordInsertGenWrappedFunction, insertFn);
client.commit(result.getLeft(), result.getRight());

HoodieTable table = HoodieSparkTable.create(client.getConfig(), context, metaClient);
assertTrue(table.getCompletedCleanTimeline().empty());

insertFirstFailedBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn,
HoodieCleaningPolicy.KEEP_LATEST_COMMITS);
insertFirstFailedBigBatchForClientCleanerTest(context, client, recordInsertGenWrappedFunction, insertFn);

insertFirstFailedBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn,
HoodieCleaningPolicy.KEEP_LATEST_COMMITS);
insertFirstFailedBigBatchForClientCleanerTest(context, client, recordInsertGenWrappedFunction, insertFn);

Pair<String, JavaRDD<WriteStatus>> ret =
insertFirstFailedBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn,
HoodieCleaningPolicy.KEEP_LATEST_COMMITS);
insertFirstFailedBigBatchForClientCleanerTest(context, client, recordInsertGenWrappedFunction, insertFn);
// Await till enough time passes such that the last failed commits heartbeats are expired
await().atMost(10, TimeUnit.SECONDS).until(() -> client.getHeartbeatClient()
.isHeartbeatExpired(ret.getLeft()));
Expand Down Expand Up @@ -1352,24 +1226,20 @@ private void testInsertAndCleanFailedWritesByVersions(
final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =
generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts);

Pair<String, JavaRDD<WriteStatus>> result = insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn,
HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS);
Pair<String, JavaRDD<WriteStatus>> result = insertFirstBigBatchForClientCleanerTest(context, metaClient, client, recordInsertGenWrappedFunction, insertFn);

client.commit(result.getLeft(), result.getRight());

HoodieTable table = HoodieSparkTable.create(client.getConfig(), context, metaClient);

assertTrue(table.getCompletedCleanTimeline().empty());

insertFirstFailedBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn,
HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS);
insertFirstFailedBigBatchForClientCleanerTest(context, client, recordInsertGenWrappedFunction, insertFn);

insertFirstFailedBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn,
HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS);
insertFirstFailedBigBatchForClientCleanerTest(context, client, recordInsertGenWrappedFunction, insertFn);

Pair<String, JavaRDD<WriteStatus>> ret =
insertFirstFailedBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn,
HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS);
insertFirstFailedBigBatchForClientCleanerTest(context, client, recordInsertGenWrappedFunction, insertFn);

// Await till enough time passes such that the last failed commits heartbeats are expired
await().atMost(10, TimeUnit.SECONDS).until(() -> client.getHeartbeatClient()
Expand Down
Loading