-
Notifications
You must be signed in to change notification settings - Fork 2.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[HUDI-4122] Fix NPE caused by adding kafka nodes #5632
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -22,17 +22,17 @@ | |
import org.apache.hudi.common.config.ConfigProperty; | ||
import org.apache.hudi.common.config.TypedProperties; | ||
import org.apache.hudi.common.util.Option; | ||
import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException; | ||
import org.apache.hudi.exception.HoodieException; | ||
import org.apache.hudi.exception.HoodieNotSupportedException; | ||
|
||
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; | ||
import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException; | ||
import org.apache.hudi.utilities.sources.AvroKafkaSource; | ||
|
||
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.OffsetAndTimestamp; | ||
import org.apache.kafka.clients.consumer.OffsetAndMetadata; | ||
import org.apache.kafka.clients.consumer.OffsetAndTimestamp; | ||
import org.apache.kafka.common.PartitionInfo; | ||
import org.apache.kafka.common.TopicPartition; | ||
import org.apache.kafka.common.errors.TimeoutException; | ||
|
@@ -48,6 +48,7 @@ | |
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Set; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.function.Function; | ||
import java.util.regex.Matcher; | ||
import java.util.regex.Pattern; | ||
|
@@ -169,9 +170,14 @@ public static class Config { | |
.withDocumentation("Kafka topic name."); | ||
|
||
public static final ConfigProperty<String> KAFKA_CHECKPOINT_TYPE = ConfigProperty | ||
.key("hoodie.deltastreamer.source.kafka.checkpoint.type") | ||
.defaultValue("string") | ||
.withDocumentation("Kafka chepoint type."); | ||
.key("hoodie.deltastreamer.source.kafka.checkpoint.type") | ||
.defaultValue("string") | ||
.withDocumentation("Kafka checkpoint type."); | ||
|
||
public static final ConfigProperty<Long> KAFKA_FETCH_PARTITION_TIME_OUT = ConfigProperty | ||
.key("hoodie.deltastreamer.source.kafka.fetch_partition.time.out") | ||
.defaultValue(300 * 1000L) | ||
.withDocumentation("Time out for fetching partitions. 5min by default"); | ||
|
||
public static final ConfigProperty<Boolean> ENABLE_KAFKA_COMMIT_OFFSET = ConfigProperty | ||
.key("hoodie.deltastreamer.source.kafka.enable.commit.offset") | ||
|
@@ -236,8 +242,7 @@ public OffsetRange[] getNextOffsetRanges(Option<String> lastCheckpointStr, long | |
if (!checkTopicExists(consumer)) { | ||
throw new HoodieException("Kafka topic:" + topicName + " does not exist"); | ||
} | ||
List<PartitionInfo> partitionInfoList; | ||
partitionInfoList = consumer.partitionsFor(topicName); | ||
List<PartitionInfo> partitionInfoList = fetchPartitionInfos(consumer, topicName); | ||
Set<TopicPartition> topicPartitions = partitionInfoList.stream() | ||
.map(x -> new TopicPartition(x.topic(), x.partition())).collect(Collectors.toSet()); | ||
|
||
|
@@ -287,6 +292,32 @@ public OffsetRange[] getNextOffsetRanges(Option<String> lastCheckpointStr, long | |
return CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets, numEvents); | ||
} | ||
|
||
/** | ||
* Fetch partition infos for given topic. | ||
* | ||
* @param consumer | ||
* @param topicName | ||
*/ | ||
private List<PartitionInfo> fetchPartitionInfos(KafkaConsumer consumer, String topicName) { | ||
long timeout = this.props.getLong(Config.KAFKA_FETCH_PARTITION_TIME_OUT.key(), Config.KAFKA_FETCH_PARTITION_TIME_OUT.defaultValue()); | ||
long start = System.currentTimeMillis(); | ||
|
||
List<PartitionInfo> partitionInfos; | ||
do { | ||
partitionInfos = consumer.partitionsFor(topicName); | ||
try { | ||
TimeUnit.SECONDS.sleep(10); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why here is a timeout no matter what? it should not be just a straight sleep. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @wangxianghu can you please elaborate? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes, I don't see a reason why we sleep here. May be, if first call fails, we can have a backup before retrying, but why sleep even if the call succeeds. Also, 10 secs is lot 1 or 2 secs would suffice. |
||
} catch (InterruptedException e) { | ||
LOG.error("Sleep failed while fetching partitions"); | ||
} | ||
} while (partitionInfos == null && (System.currentTimeMillis() <= (start + timeout))); | ||
|
||
if (partitionInfos == null) { | ||
throw new HoodieDeltaStreamerException(String.format("Can not find metadata for topic %s from kafka cluster", topicName)); | ||
} | ||
return partitionInfos; | ||
} | ||
|
||
/** | ||
* Fetch checkpoint offsets for each partition. | ||
* @param consumer instance of {@link KafkaConsumer} to fetch offsets from. | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
NPE is caused by
partitionInfoList
's being null