1
0

[HUDI-1944] Support Hudi to read from committed offset (#3175)

* [HUDI-1944] Support Hudi to read from committed offset

* [HUDI-1944] Adding group option to KafkaResetOffsetStrategies

* [HUDI-1944] Update Exception msg
This commit is contained in:
Vinay Patil
2021-06-30 14:11:28 +05:30
committed by GitHub
parent 1cbf43b6e7
commit 94f0f40fec
2 changed files with 51 additions and 3 deletions

View File

@@ -25,6 +25,7 @@ import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase.Helpers;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.spark.streaming.kafka010.KafkaTestUtils;
@@ -33,6 +34,8 @@ import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.UUID;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -66,6 +69,7 @@ public class TestKafkaOffsetGen {
props.setProperty("bootstrap.servers", testUtils.brokerAddress());
props.setProperty("key.deserializer", StringDeserializer.class.getName());
props.setProperty("value.deserializer", StringDeserializer.class.getName());
props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
return props;
}
@@ -127,6 +131,30 @@ public class TestKafkaOffsetGen {
assertEquals(249, nextOffsetRanges[1].untilOffset());
}
@Test
public void testGetNextOffsetRangesFromGroup() {
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
testUtils.createTopic(TEST_TOPIC_NAME, 2);
testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000)));
KafkaOffsetGen kafkaOffsetGen = new KafkaOffsetGen(getConsumerConfigs("group"));
String lastCheckpointString = TEST_TOPIC_NAME + ",0:250,1:249";
kafkaOffsetGen.commitOffsetToKafka(lastCheckpointString);
// don't pass lastCheckpointString as we want to read from group committed offset
OffsetRange[] nextOffsetRanges = kafkaOffsetGen.getNextOffsetRanges(Option.empty(), 300, metrics);
assertEquals(250, nextOffsetRanges[0].fromOffset());
assertEquals(400, nextOffsetRanges[0].untilOffset());
assertEquals(249, nextOffsetRanges[1].fromOffset());
assertEquals(399, nextOffsetRanges[1].untilOffset());
// committed offsets are not present for the consumer group
kafkaOffsetGen = new KafkaOffsetGen(getConsumerConfigs("group"));
nextOffsetRanges = kafkaOffsetGen.getNextOffsetRanges(Option.empty(), 300, metrics);
assertEquals(500, nextOffsetRanges[0].fromOffset());
assertEquals(500, nextOffsetRanges[0].untilOffset());
assertEquals(500, nextOffsetRanges[1].fromOffset());
assertEquals(500, nextOffsetRanges[1].untilOffset());
}
@Test
public void testCheckTopicExists() {
TypedProperties props = getConsumerConfigs("latest");