Skip to content
Merged
Show file tree
Hide file tree
Changes from 4 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 @@ -58,6 +58,7 @@
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.schema.SchemaSet;
import org.apache.hudi.utilities.sources.InputBatch;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen;
import org.apache.hudi.utilities.transform.Transformer;

import com.codahale.metrics.Timer;
Expand Down Expand Up @@ -473,7 +474,9 @@ private Pair<Option<String>, JavaRDD<WriteStatus>> writeToSink(JavaRDD<HoodieRec
boolean success = writeClient.commit(instantTime, writeStatusRDD, Option.of(checkpointCommitMetadata));
if (success) {
LOG.info("Commit " + instantTime + " successful!");

if (this.props.getBoolean(KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET,KafkaOffsetGen.Config.DEFAULT_ENABLE_KAFKA_COMMIT_OFFSET)) {
KafkaOffsetGen.commitOffsetToKafka(checkpointStr, this.props);
}
// Schedule compaction if needed
if (cfg.isAsyncCompactionEnabled()) {
scheduledCompactionInstant = writeClient.scheduleCompaction(Option.empty());
Expand Down Expand Up @@ -506,6 +509,8 @@ private Pair<Option<String>, JavaRDD<WriteStatus>> writeToSink(JavaRDD<HoodieRec
return Pair.of(scheduledCompactionInstant, writeStatusRDD);
}



/**
* Try to start a new commit.
* <p>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,13 @@
import org.apache.hudi.exception.HoodieNotSupportedException;

import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics;
import org.apache.kafka.clients.consumer.CommitFailedException;
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.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.streaming.kafka010.OffsetRange;
Expand Down Expand Up @@ -157,29 +161,23 @@ public static class Config {

private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic";
private static final String MAX_EVENTS_FROM_KAFKA_SOURCE_PROP = "hoodie.deltastreamer.kafka.source.maxEvents";
public static final String ENABLE_KAFKA_COMMIT_OFFSET = "hoodie.deltastreamer.source.enable.kafka.commit.offset";
// "auto.offset.reset" is kafka native config param. Do not change the config param name.
public static final String KAFKA_AUTO_OFFSET_RESET = "auto.offset.reset";
private static final KafkaResetOffsetStrategies DEFAULT_KAFKA_AUTO_OFFSET_RESET = KafkaResetOffsetStrategies.LATEST;
public static final long DEFAULT_MAX_EVENTS_FROM_KAFKA_SOURCE = 5000000;
public static long maxEventsFromKafkaSource = DEFAULT_MAX_EVENTS_FROM_KAFKA_SOURCE;
public static final Boolean DEFAULT_ENABLE_KAFKA_COMMIT_OFFSET = false;
}

private final HashMap<String, Object> kafkaParams;
private final Map<String, Object> kafkaParams;
private final TypedProperties props;
protected final String topicName;
private KafkaResetOffsetStrategies autoResetValue;

public KafkaOffsetGen(TypedProperties props) {
this.props = props;

kafkaParams = new HashMap<>();
props.keySet().stream().filter(prop -> {
// In order to prevent printing unnecessary warn logs, here filter out the hoodie
// configuration items before passing to kafkaParams
return !prop.toString().startsWith("hoodie.");
}).forEach(prop -> {
kafkaParams.put(prop.toString(), props.get(prop.toString()));
});
kafkaParams = KafkaOffsetGen.excludeHoodieConfigs(props);
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.KAFKA_TOPIC_NAME));
topicName = props.getString(Config.KAFKA_TOPIC_NAME);
String kafkaAutoResetOffsetsStr = props.getString(Config.KAFKA_AUTO_OFFSET_RESET, Config.DEFAULT_KAFKA_AUTO_OFFSET_RESET.name().toLowerCase());
Expand Down Expand Up @@ -297,7 +295,37 @@ public String getTopicName() {
return topicName;
}

public HashMap<String, Object> getKafkaParams() {
public Map<String, Object> getKafkaParams() {
return kafkaParams;
}

private static Map<String, Object> excludeHoodieConfigs(TypedProperties props) {
Map<String, Object> kafkaParams = new HashMap<>();
props.keySet().stream().filter(prop -> {
// In order to prevent printing unnecessary warn logs, here filter out the hoodie
// configuration items before passing to kafkaParams
return !prop.toString().startsWith("hoodie.");
}).forEach(prop -> {
kafkaParams.put(prop.toString(), props.get(prop.toString()));
});
return kafkaParams;
}

/**
* Commit offsets to Kafka only after hoodie commit is successful.
* @param checkpointStr checkpoint string containing offsets.
* @param props properties for Kafka consumer.
*/
public static void commitOffsetToKafka(String checkpointStr, TypedProperties props) {
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(ConsumerConfig.GROUP_ID_CONFIG));
Map<TopicPartition, Long> offsetMap = KafkaOffsetGen.CheckpointUtils.strToOffsets(checkpointStr);
Map<String, Object> kafkaParams = KafkaOffsetGen.excludeHoodieConfigs(props);
Map<TopicPartition, OffsetAndMetadata> offsetAndMetadataMap = new HashMap<>(offsetMap.size());
try (KafkaConsumer consumer = new KafkaConsumer(kafkaParams)) {
offsetMap.forEach((topicPartition, offset) -> offsetAndMetadataMap.put(topicPartition, new OffsetAndMetadata(offset)));
consumer.commitSync(offsetAndMetadataMap);
} catch (CommitFailedException | TimeoutException e) {
LOG.warn("Committing offsets to Kafka failed, this does not impact processing of records", e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,14 +22,19 @@
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;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config;
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;
Expand All @@ -40,9 +45,14 @@
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.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;

/**
Expand All @@ -58,7 +68,7 @@ public class TestKafkaSource extends UtilitiesTestBase {

@BeforeAll
public static void initClass() throws Exception {
UtilitiesTestBase.initClass();
UtilitiesTestBase.initClass(false);
}

@AfterAll
Expand All @@ -85,6 +95,7 @@ private TypedProperties createPropsForJsonSource(Long maxEventsToReadFromKafkaSo
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));
Expand Down Expand Up @@ -276,4 +287,64 @@ public void testJsonKafkaSourceWithConfigurableUpperCap() {
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");

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
KafkaOffsetGen.commitOffsetToKafka(fetch1.getCheckpointForNextBatch(), props);
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
KafkaOffsetGen.commitOffsetToKafka(fetch2.getCheckpointForNextBatch(), props);

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,() -> KafkaOffsetGen.commitOffsetToKafka("",props));
}
}