[HUDI-3361] Fixing missing begin checkpoint in HoodieIncremental pull (#4755)
This commit is contained in:
committed by
GitHub
parent
6a32cfe020
commit
60831d6906
@@ -20,12 +20,14 @@ package org.apache.hudi.utilities.sources;
|
||||
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
@@ -61,21 +63,31 @@ public class TestHoodieIncrSource extends HoodieClientTestHarness {
|
||||
|
||||
@Test
|
||||
public void testHoodieIncrSource() throws IOException {
|
||||
HoodieWriteConfig writeConfig = getConfigBuilder(basePath).build();
|
||||
HoodieWriteConfig writeConfig = getConfigBuilder(basePath)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2,3).retainCommits(1).build()).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build();
|
||||
|
||||
SparkRDDWriteClient writeClient = new SparkRDDWriteClient(context, writeConfig);
|
||||
Pair<String, List<HoodieRecord>> inserts = writeRecords(writeClient, true, null);
|
||||
Pair<String, List<HoodieRecord>> inserts2 = writeRecords(writeClient, true, null);
|
||||
Pair<String, List<HoodieRecord>> inserts3 = writeRecords(writeClient, true, null);
|
||||
Pair<String, List<HoodieRecord>> inserts = writeRecords(writeClient, true, null, "100");
|
||||
Pair<String, List<HoodieRecord>> inserts2 = writeRecords(writeClient, true, null, "200");
|
||||
Pair<String, List<HoodieRecord>> inserts3 = writeRecords(writeClient, true, null, "300");
|
||||
Pair<String, List<HoodieRecord>> inserts4 = writeRecords(writeClient, true, null, "400");
|
||||
Pair<String, List<HoodieRecord>> inserts5 = writeRecords(writeClient, true, null, "500");
|
||||
|
||||
|
||||
// read everything upto latest
|
||||
readAndAssert(IncrSourceHelper.MissingCheckpointStrategy.READ_UPTO_LATEST_COMMIT, 300, inserts3.getKey());
|
||||
readAndAssert(IncrSourceHelper.MissingCheckpointStrategy.READ_UPTO_LATEST_COMMIT, Option.empty(), 500, inserts5.getKey());
|
||||
|
||||
// even if the begin timestamp is archived (100), full table scan should kick in, but should filter for records having commit time > 100
|
||||
readAndAssert(IncrSourceHelper.MissingCheckpointStrategy.READ_UPTO_LATEST_COMMIT, Option.of("100"), 400, inserts5.getKey());
|
||||
|
||||
// even if the read upto latest is set, if begin timestamp is in active timeline, only incremental should kick in.
|
||||
readAndAssert(IncrSourceHelper.MissingCheckpointStrategy.READ_UPTO_LATEST_COMMIT, Option.of("400"), 100, inserts5.getKey());
|
||||
|
||||
// read just the latest
|
||||
readAndAssert(IncrSourceHelper.MissingCheckpointStrategy.READ_LATEST, 100, inserts3.getKey());
|
||||
readAndAssert(IncrSourceHelper.MissingCheckpointStrategy.READ_LATEST, Option.empty(), 100, inserts5.getKey());
|
||||
}
|
||||
|
||||
private void readAndAssert(IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy, int expectedCount, String expectedCheckpoint) {
|
||||
private void readAndAssert(IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy, Option<String> checkpointToPull, int expectedCount, String expectedCheckpoint) {
|
||||
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty("hoodie.deltastreamer.source.hoodieincr.path", basePath);
|
||||
@@ -84,14 +96,14 @@ public class TestHoodieIncrSource extends HoodieClientTestHarness {
|
||||
HoodieIncrSource incrSource = new HoodieIncrSource(typedProperties, jsc, sparkSession, new TestSchemaProvider(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
|
||||
// read everything until latest
|
||||
Pair<Option<Dataset<Row>>, String> batchCheckPoint = incrSource.fetchNextBatch(Option.empty(), 500);
|
||||
Pair<Option<Dataset<Row>>, String> batchCheckPoint = incrSource.fetchNextBatch(checkpointToPull, 500);
|
||||
Assertions.assertNotNull(batchCheckPoint.getValue());
|
||||
assertEquals(batchCheckPoint.getKey().get().count(), expectedCount);
|
||||
Assertions.assertEquals(batchCheckPoint.getRight(), expectedCheckpoint);
|
||||
}
|
||||
|
||||
public Pair<String, List<HoodieRecord>> writeRecords(SparkRDDWriteClient writeClient, boolean insert, List<HoodieRecord> insertRecords) throws IOException {
|
||||
String commit = writeClient.startCommit();
|
||||
public Pair<String, List<HoodieRecord>> writeRecords(SparkRDDWriteClient writeClient, boolean insert, List<HoodieRecord> insertRecords, String commit) throws IOException {
|
||||
writeClient.startCommitWithTime(commit);
|
||||
List<HoodieRecord> records = insert ? dataGen.generateInserts(commit, 100) : dataGen.generateUpdates(commit, insertRecords);
|
||||
JavaRDD<WriteStatus> result = writeClient.upsert(jsc.parallelize(records, 1), commit);
|
||||
List<WriteStatus> statuses = result.collect();
|
||||
|
||||
Reference in New Issue
Block a user