1
0

[HUDI-1910] Commit Offset to Kafka after successful Hudi commit (#3092)

This commit is contained in:
Vinay Patil
2021-06-28 19:22:05 +05:30
committed by GitHub
parent 34fc8a8880
commit 039aeb6dce
8 changed files with 170 additions and 16 deletions

View File

@@ -22,6 +22,7 @@ import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics;
import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
@@ -30,6 +31,9 @@ import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
import org.apache.avro.generic.GenericRecord;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
@@ -40,9 +44,15 @@ import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import static org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.mockito.Mockito.mock;
/**
@@ -58,7 +68,7 @@ public class TestKafkaSource extends UtilitiesTestBase {
@BeforeAll
public static void initClass() throws Exception {
UtilitiesTestBase.initClass();
UtilitiesTestBase.initClass(false);
}
@AfterAll
@@ -85,6 +95,7 @@ public class TestKafkaSource extends UtilitiesTestBase {
props.setProperty("hoodie.deltastreamer.source.kafka.topic", TEST_TOPIC_NAME);
props.setProperty("bootstrap.servers", testUtils.brokerAddress());
props.setProperty(Config.KAFKA_AUTO_OFFSET_RESET, resetStrategy);
props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
props.setProperty("hoodie.deltastreamer.kafka.source.maxEvents",
maxEventsToReadFromKafkaSource != null ? String.valueOf(maxEventsToReadFromKafkaSource) :
String.valueOf(Config.maxEventsFromKafkaSource));
@@ -276,4 +287,64 @@ public class TestKafkaSource extends UtilitiesTestBase {
kafkaSource.fetchNewDataInAvroFormat(Option.of(fetch4.getCheckpointForNextBatch()), Long.MAX_VALUE);
assertEquals(Option.empty(), fetch6.getBatch());
}
@Test
public void testCommitOffsetToKafka() {
// topic setup.
testUtils.createTopic(TEST_TOPIC_NAME, 2);
List<TopicPartition> topicPartitions = new ArrayList<>();
TopicPartition topicPartition0 = new TopicPartition(TEST_TOPIC_NAME, 0);
topicPartitions.add(topicPartition0);
TopicPartition topicPartition1 = new TopicPartition(TEST_TOPIC_NAME, 1);
topicPartitions.add(topicPartition1);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
TypedProperties props = createPropsForJsonSource(null, "earliest");
props.put(ENABLE_KAFKA_COMMIT_OFFSET, "true");
Source jsonSource = new JsonKafkaSource(props, jsc, sparkSession, schemaProvider, metrics);
SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource);
// 1. Extract without any checkpoint => get all the data, respecting sourceLimit
assertEquals(Option.empty(), kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch());
testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000)));
InputBatch<JavaRDD<GenericRecord>> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 599);
// commit to kafka after first batch
kafkaSource.getSource().onCommit(fetch1.getCheckpointForNextBatch());
try (KafkaConsumer consumer = new KafkaConsumer(props)) {
consumer.assign(topicPartitions);
OffsetAndMetadata offsetAndMetadata = consumer.committed(topicPartition0);
assertNotNull(offsetAndMetadata);
assertEquals(300, offsetAndMetadata.offset());
offsetAndMetadata = consumer.committed(topicPartition1);
assertNotNull(offsetAndMetadata);
assertEquals(299, offsetAndMetadata.offset());
// end offsets will point to 500 for each partition because we consumed less messages from first batch
Map endOffsets = consumer.endOffsets(topicPartitions);
assertEquals(500L, endOffsets.get(topicPartition0));
assertEquals(500L, endOffsets.get(topicPartition1));
testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 500)));
InputBatch<Dataset<Row>> fetch2 =
kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
// commit to Kafka after second batch is processed completely
kafkaSource.getSource().onCommit(fetch2.getCheckpointForNextBatch());
offsetAndMetadata = consumer.committed(topicPartition0);
assertNotNull(offsetAndMetadata);
assertEquals(750, offsetAndMetadata.offset());
offsetAndMetadata = consumer.committed(topicPartition1);
assertNotNull(offsetAndMetadata);
assertEquals(750, offsetAndMetadata.offset());
endOffsets = consumer.endOffsets(topicPartitions);
assertEquals(750L, endOffsets.get(topicPartition0));
assertEquals(750L, endOffsets.get(topicPartition1));
}
// check failure case
props.remove(ConsumerConfig.GROUP_ID_CONFIG);
assertThrows(HoodieNotSupportedException.class,() -> kafkaSource.getSource().onCommit(""));
}
}