1
0

[HUDI-596] Close KafkaConsumer every time (#1303)

This commit is contained in:
dengziming
2020-02-04 15:42:21 +08:00
committed by GitHub
parent d07ac588ac
commit 347e297ac1

View File

@@ -172,14 +172,15 @@ public class KafkaOffsetGen {
public OffsetRange[] getNextOffsetRanges(Option<String> lastCheckpointStr, long sourceLimit) { public OffsetRange[] getNextOffsetRanges(Option<String> lastCheckpointStr, long sourceLimit) {
// Obtain current metadata for the topic // Obtain current metadata for the topic
KafkaConsumer consumer = new KafkaConsumer(kafkaParams); Map<TopicPartition, Long> fromOffsets;
Map<TopicPartition, Long> toOffsets;
try (KafkaConsumer consumer = new KafkaConsumer(kafkaParams)) {
List<PartitionInfo> partitionInfoList; List<PartitionInfo> partitionInfoList;
partitionInfoList = consumer.partitionsFor(topicName); partitionInfoList = consumer.partitionsFor(topicName);
Set<TopicPartition> topicPartitions = partitionInfoList.stream() Set<TopicPartition> topicPartitions = partitionInfoList.stream()
.map(x -> new TopicPartition(x.topic(), x.partition())).collect(Collectors.toSet()); .map(x -> new TopicPartition(x.topic(), x.partition())).collect(Collectors.toSet());
// Determine the offset ranges to read from // Determine the offset ranges to read from
Map<TopicPartition, Long> fromOffsets;
if (lastCheckpointStr.isPresent()) { if (lastCheckpointStr.isPresent()) {
fromOffsets = checkupValidOffsets(consumer, lastCheckpointStr, topicPartitions); fromOffsets = checkupValidOffsets(consumer, lastCheckpointStr, topicPartitions);
} else { } else {
@@ -198,7 +199,8 @@ public class KafkaOffsetGen {
} }
// Obtain the latest offsets. // Obtain the latest offsets.
Map<TopicPartition, Long> toOffsets = consumer.endOffsets(topicPartitions); toOffsets = consumer.endOffsets(topicPartitions);
}
// Come up with final set of OffsetRanges to read (account for new partitions, limit number of events) // Come up with final set of OffsetRanges to read (account for new partitions, limit number of events)
long maxEventsToReadFromKafka = props.getLong(Config.MAX_EVENTS_FROM_KAFKA_SOURCE_PROP, long maxEventsToReadFromKafka = props.getLong(Config.MAX_EVENTS_FROM_KAFKA_SOURCE_PROP,