Skip to content
Merged
Show file tree
Hide file tree
Changes from all 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 @@ -24,7 +24,14 @@
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.errors.RetriableException;
import org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy;
import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
import org.apache.storm.kafka.spout.internal.KafkaConsumerFactoryDefault;
import org.apache.storm.kafka.spout.internal.Timer;
import org.apache.storm.kafka.spout.internal.fetcher.KafkaRecordsFetcher;
import org.apache.storm.kafka.spout.internal.fetcher.KafkaRecordsFetchers;
import org.apache.storm.spout.SpoutOutputCollector;
import org.apache.storm.task.TopologyContext;
import org.apache.storm.topology.OutputFieldsDeclarer;
Expand All @@ -46,23 +53,19 @@
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;

import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.LATEST;
import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST;
import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_LATEST;
import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
import org.apache.storm.kafka.spout.internal.KafkaConsumerFactoryDefault;

import org.apache.kafka.common.errors.InterruptException;

public class KafkaSpout<K, V> extends BaseRichSpout {
private static final Logger LOG = LoggerFactory.getLogger(KafkaSpout.class);
private static final Comparator<KafkaSpoutMessageId> OFFSET_COMPARATOR = new OffsetComparator();

// Storm
protected SpoutOutputCollector collector;
private TopologyContext topologyContext;
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, it's used to create records fetcher.


// Kafka
private final KafkaSpoutConfig<K, V> kafkaSpoutConfig;
Expand All @@ -77,6 +80,7 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
private transient KafkaSpoutRetryService retryService; // Class that has the logic to handle tuple failure
private transient Timer commitTimer; // timer == null for auto commit mode
private transient boolean initialized; // Flag indicating that the spout is still undergoing initialization process.
private transient KafkaRecordsFetcher<K, V> recordsFetcher; // Class that encapsulates the logic of managing partitions and fetching records
// Initialization is only complete after the first call to KafkaSpoutConsumerRebalanceListener.onPartitionsAssigned()

private KafkaSpoutStreams kafkaSpoutStreams; // Object that wraps all the logic to declare output fields and emit tuples
Expand All @@ -102,9 +106,9 @@ public KafkaSpout(KafkaSpoutConfig<K, V> kafkaSpoutConfig) {
@Override
public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
initialized = false;

// Spout internals
this.collector = collector;
this.topologyContext = context;
maxRetries = kafkaSpoutConfig.getMaxTupleRetries();
numUncommittedOffsets = 0;

Expand Down Expand Up @@ -221,7 +225,11 @@ public void nextTuple() {
}

if (poll()) {
setWaitingToEmit(pollKafkaBroker());
try {
setWaitingToEmit(pollKafkaBroker());
} catch (RetriableException e) {
LOG.error("Failed to poll from kafka.", e);
}
}

if (waitingToEmit()) {
Expand Down Expand Up @@ -277,7 +285,7 @@ public void setWaitingToEmit(ConsumerRecords<K,V> consumerRecords) {
private ConsumerRecords<K, V> pollKafkaBroker() {
doSeekRetriableTopicPartitions();

final ConsumerRecords<K, V> consumerRecords = kafkaConsumer.poll(kafkaSpoutConfig.getPollTimeoutMs());
final ConsumerRecords<K, V> consumerRecords = recordsFetcher.fetchRecords(kafkaSpoutConfig.getPollTimeoutMs());
final int numPolledRecords = consumerRecords.count();
LOG.debug("Polled [{}] records from Kafka. [{}] uncommitted offsets across all topic partitions", numPolledRecords, numUncommittedOffsets);
return consumerRecords;
Expand Down Expand Up @@ -404,19 +412,8 @@ public void activate() {

private void subscribeKafkaConsumer() {
kafkaConsumer = kafkaConsumerFactory.createConsumer(kafkaSpoutConfig);

if (kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics) {
final List<String> topics = ((KafkaSpoutStreamsNamedTopics) kafkaSpoutStreams).getTopics();
kafkaConsumer.subscribe(topics, new KafkaSpoutConsumerRebalanceListener());
LOG.info("Kafka consumer subscribed topics {}", topics);
} else if (kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics) {
final Pattern pattern = ((KafkaSpoutStreamsWildcardTopics) kafkaSpoutStreams).getTopicWildcardPattern();
kafkaConsumer.subscribe(pattern, new KafkaSpoutConsumerRebalanceListener());
LOG.info("Kafka consumer subscribed topics matching wildcard pattern [{}]", pattern);
}
// Initial poll to get the consumer registration process going.
// KafkaSpoutConsumerRebalanceListener will be called following this poll, upon partition registration
kafkaConsumer.poll(0);
recordsFetcher = KafkaRecordsFetchers.create(kafkaSpoutConfig, kafkaConsumer, topologyContext,
new KafkaSpoutConsumerRebalanceListener());
}

@Override
Expand Down Expand Up @@ -612,60 +609,4 @@ public String toString() {
'}';
}
}

// =========== Timer ===========

private class Timer {
private final long delay;
private final long period;
private final TimeUnit timeUnit;
private final long periodNanos;
private long start;

/**
* Creates a class that mimics a single threaded timer that expires periodically. If a call to {@link
* #isExpiredResetOnTrue()} occurs later than {@code period} since the timer was initiated or reset, this method returns
* true. Each time the method returns true the counter is reset. The timer starts with the specified time delay.
*
* @param delay the initial delay before the timer starts
* @param period the period between calls {@link #isExpiredResetOnTrue()}
* @param timeUnit the time unit of delay and period
*/
public Timer(long delay, long period, TimeUnit timeUnit) {
this.delay = delay;
this.period = period;
this.timeUnit = timeUnit;

periodNanos = timeUnit.toNanos(period);
start = System.nanoTime() + timeUnit.toNanos(delay);
}

public long period() {
return period;
}

public long delay() {
return delay;
}

public TimeUnit getTimeUnit() {
return timeUnit;
}

/**
* Checks if a call to this method occurs later than {@code period} since the timer was initiated or reset. If that is the
* case the method returns true, otherwise it returns false. Each time this method returns true, the counter is reset
* (re-initiated) and a new cycle will start.
*
* @return true if the time elapsed since the last call returning true is greater than {@code period}. Returns false
* otherwise.
*/
public boolean isExpiredResetOnTrue() {
final boolean expired = System.nanoTime() - start > periodNanos;
if (expired) {
start = System.nanoTime();
}
return expired;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
public static final long DEFAULT_OFFSET_COMMIT_PERIOD_MS = 30_000; // 30s
public static final int DEFAULT_MAX_RETRIES = Integer.MAX_VALUE; // Retry forever
public static final int DEFAULT_MAX_UNCOMMITTED_OFFSETS = 10_000_000; // 10,000,000 records => 80MBs of memory footprint in the worst case
public static final long DEFAULT_PARTITION_REFRESH_PERIOD_MS = 2_000; // 2s

// Kafka property names
public interface Consumer {
Expand Down Expand Up @@ -76,6 +77,8 @@ public enum FirstPollOffsetStrategy {
private final long offsetCommitPeriodMs;
private final int maxRetries;
private final int maxUncommittedOffsets;
private final long partitionRefreshPeriodMs;
private final boolean manualPartitionAssignment;
private final FirstPollOffsetStrategy firstPollOffsetStrategy;
private final KafkaSpoutStreams kafkaSpoutStreams;
private final KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;
Expand All @@ -91,6 +94,8 @@ private KafkaSpoutConfig(Builder<K,V> builder) {
this.firstPollOffsetStrategy = builder.firstPollOffsetStrategy;
this.kafkaSpoutStreams = builder.kafkaSpoutStreams;
this.maxUncommittedOffsets = builder.maxUncommittedOffsets;
this.partitionRefreshPeriodMs = builder.partitionRefreshPeriodMs;
this.manualPartitionAssignment = builder.manualPartitionAssignment;
this.tuplesBuilder = builder.tuplesBuilder;
this.retryService = builder.retryService;
}
Expand All @@ -113,6 +118,8 @@ public static class Builder<K,V> {
private FirstPollOffsetStrategy firstPollOffsetStrategy = FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST;
private final KafkaSpoutStreams kafkaSpoutStreams;
private int maxUncommittedOffsets = DEFAULT_MAX_UNCOMMITTED_OFFSETS;
private long partitionRefreshPeriodMs = DEFAULT_PARTITION_REFRESH_PERIOD_MS;
private boolean manualPartitionAssignment = false;
private final KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;
private final KafkaSpoutRetryService retryService;

Expand Down Expand Up @@ -229,6 +236,25 @@ public Builder<K, V> setFirstPollOffsetStrategy(FirstPollOffsetStrategy firstPol
return this;
}

/**
* Sets partition refresh period in milliseconds in manual partition assignment model. Default is 2s.
* @param partitionRefreshPeriodMs time in milliseconds
*/
public Builder<K, V> setPartitionRefreshPeriodMs(long partitionRefreshPeriodMs) {
this.partitionRefreshPeriodMs = partitionRefreshPeriodMs;
return this;
}

/**
* Defines whether the consumer manages partition manually.
* If set to true, the consumer manage partition manually, otherwise it will rely on kafka to do partition assignment.
* @param manualPartitionAssignment True if using manual partition assignment.
*/
public Builder<K, V> setManualPartitionAssignment(boolean manualPartitionAssignment) {
this.manualPartitionAssignment = manualPartitionAssignment;
return this;
}

public KafkaSpoutConfig<K,V> build() {
return new KafkaSpoutConfig<>(this);
}
Expand Down Expand Up @@ -307,6 +333,14 @@ public KafkaSpoutRetryService getRetryService() {
return retryService;
}

public long getPartitionRefreshPeriodMs() {
return partitionRefreshPeriodMs;
}

public boolean isManualPartitionAssignment() {
return manualPartitionAssignment;
}

@Override
public String toString() {
return "KafkaSpoutConfig{" +
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
package org.apache.storm.kafka.spout;

import org.apache.kafka.common.TopicPartition;

import java.util.Comparator;

public class TopicPartitionComparator implements Comparator<TopicPartition> {
@Override
public int compare(TopicPartition o1, TopicPartition o2) {
if (!o1.topic().equals(o2.topic())) {
return o1.topic().compareTo(o2.topic());
} else {
return o1.partition() - o2.partition();
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.storm.kafka.spout.internal;

import java.util.concurrent.TimeUnit;

public class Timer {
private final long delay;
private final long period;
private final TimeUnit timeUnit;
private final long periodNanos;
private long start;

/**
* Creates a class that mimics a single threaded timer that expires periodically. If a call to {@link
* #isExpiredResetOnTrue()} occurs later than {@code period} since the timer was initiated or reset, this method returns
* true. Each time the method returns true the counter is reset. The timer starts with the specified time delay.
*
* @param delay the initial delay before the timer starts
* @param period the period between calls {@link #isExpiredResetOnTrue()}
* @param timeUnit the time unit of delay and period
*/
public Timer(long delay, long period, TimeUnit timeUnit) {
this.delay = delay;
this.period = period;
this.timeUnit = timeUnit;

periodNanos = timeUnit.toNanos(period);
start = System.nanoTime() + timeUnit.toNanos(delay);
}

public long period() {
return period;
}

public long delay() {
return delay;
}

public TimeUnit getTimeUnit() {
return timeUnit;
}

/**
* Checks if a call to this method occurs later than {@code period} since the timer was initiated or reset. If that is the
* case the method returns true, otherwise it returns false. Each time this method returns true, the counter is reset
* (re-initiated) and a new cycle will start.
*
* @return true if the time elapsed since the last call returning true is greater than {@code period}. Returns false
* otherwise.
*/
public boolean isExpiredResetOnTrue() {
final boolean expired = System.nanoTime() - start > periodNanos;
if (expired) {
start = System.nanoTime();
}
return expired;
}
}
Loading