diff --git a/docs/storm-kafka-client.md b/docs/storm-kafka-client.md index c8e038f1905..90aca55acbb 100644 --- a/docs/storm-kafka-client.md +++ b/docs/storm-kafka-client.md @@ -1,90 +1,257 @@ -#Storm Kafka Spout with New Kafka Consumer API +#Storm Apache Kafka integration using the kafka-client jar +This includes the new Apache Kafka copnsumer API. -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). +##Compatibility -The Apache Storm Spout allows clients to consume data from Kafka starting at offsets as defined by the offset strategy specified in `FirstPollOffsetStrategy`. -In case of failure, the Kafka Spout will re-start consuming messages from the offset that matches the chosen `FirstPollOffsetStrategy`. +Apache Kafka versions 0.10 onwards -The Kafka Spout implementation allows you to specify the stream (`KafkaSpoutStream`) associated with each topic or topic wildcard. `KafkaSpoutStream` represents the stream and output fields. For named topics use `KafkaSpoutStreamsNamedTopics`, and for topic wildcards use `KafkaSpoutStreamsWildcardTopics`. +##Writing to Kafka as part of your topology +You can create an instance of org.apache.storm.kafka.bolt.KafkaBolt and attach it as a component to your topology or if you +are using trident you can use org.apache.storm.kafka.trident.TridentState, org.apache.storm.kafka.trident.TridentStateFactory and +org.apache.storm.kafka.trident.TridentKafkaUpdater. -The `KafkaSpoutTuplesBuilder` wraps all the logic that builds `Tuple`s from `ConsumerRecord`s. The logic is provided by the user through implementing the appropriate number of `KafkaSpoutTupleBuilder` instances. For named topics use `KafkaSpoutTuplesBuilderNamedTopics`, and for topic wildcard use `KafkaSpoutTuplesBuilderWildcardTopics`. +You need to provide implementations for the following 2 interfaces -Multiple topics and topic wildcards can use the same `KafkaSpoutTupleBuilder` implementation, as long as the logic to build `Tuple`s from `ConsumerRecord`s is identical. +###TupleToKafkaMapper and TridentTupleToKafkaMapper +These interfaces have 2 methods defined: +```java + K getKeyFromTuple(Tuple/TridentTuple tuple); + V getMessageFromTuple(Tuple/TridentTuple tuple); +``` + +As the name suggests, these methods are called to map a tuple to a Kafka key and a Kafka message. If you just want one field +as key and one field as value, then you can use the provided FieldNameBasedTupleToKafkaMapper.java +implementation. In the KafkaBolt, the implementation always looks for a field with field name "key" and "message" if you +use the default constructor to construct FieldNameBasedTupleToKafkaMapper for backward compatibility +reasons. Alternatively you could also specify a different key and message field by using the non default constructor. +In the TridentKafkaState you must specify what is the field name for key and message as there is no default constructor. +These should be specified while constructing an instance of FieldNameBasedTupleToKafkaMapper. + +###KafkaTopicSelector and trident KafkaTopicSelector +This interface has only one method +```java +public interface KafkaTopicSelector { + String getTopics(Tuple/TridentTuple tuple); +} +``` +The implementation of this interface should return the topic to which the tuple's key/message mapping needs to be published +You can return a null and the message will be ignored. If you have one static topic name then you can use +DefaultTopicSelector.java and set the name of the topic in the constructor. +`FieldNameTopicSelector` and `FieldIndexTopicSelector` can be used to select the topic should to publish a tuple to. +A user just needs to specify the field name or field index for the topic name in the tuple itself. +When the topic is name not found , the `Field*TopicSelector` will write messages into default topic . +Please make sure the default topic has been created . + +### Specifying Kafka producer properties +You can provide all the producer properties in your Storm topology by calling `KafkaBolt.withProducerProperties()` and `TridentKafkaStateFactory.withProducerProperties()`. Please see http://kafka.apache.org/documentation.html#newproducerconfigs +Section "Important configuration properties for the producer" for more details. +These are also defined in `org.apache.kafka.clients.producer.ProducerConfig` + +###Using wildcard kafka topic match +You can do a wildcard topic match by adding the following config +``` + Config config = new Config(); + config.put("kafka.topic.wildcard.match",true); + +``` + +After this you can specify a wildcard topic for matching e.g. clickstream.*.log. This will match all streams matching clickstream.my.log, clickstream.cart.log etc + + +###Putting it all together + +For the bolt : +```java + TopologyBuilder builder = new TopologyBuilder(); + + Fields fields = new Fields("key", "message"); + FixedBatchSpout spout = new FixedBatchSpout(fields, 4, + new Values("storm", "1"), + new Values("trident", "1"), + new Values("needs", "1"), + new Values("javadoc", "1") + ); + spout.setCycle(true); + builder.setSpout("spout", spout, 5); + //set producer properties. + Properties props = new Properties(); + props.put("bootstrap.servers", "localhost:9092"); + props.put("acks", "1"); + props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + + KafkaBolt bolt = new KafkaBolt() + .withProducerProperties(props) + .withTopicSelector(new DefaultTopicSelector("test")) + .withTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper()); + builder.setBolt("forwardToKafka", bolt, 8).shuffleGrouping("spout"); + + Config conf = new Config(); + + StormSubmitter.submitTopology("kafkaboltTest", conf, builder.createTopology()); +``` + +For Trident: + +```java + Fields fields = new Fields("word", "count"); + FixedBatchSpout spout = new FixedBatchSpout(fields, 4, + new Values("storm", "1"), + new Values("trident", "1"), + new Values("needs", "1"), + new Values("javadoc", "1") + ); + spout.setCycle(true); + + TridentTopology topology = new TridentTopology(); + Stream stream = topology.newStream("spout1", spout); + + //set producer properties. + Properties props = new Properties(); + props.put("bootstrap.servers", "localhost:9092"); + props.put("acks", "1"); + props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + + TridentKafkaStateFactory stateFactory = new TridentKafkaStateFactory() + .withProducerProperties(props) + .withKafkaTopicSelector(new DefaultTopicSelector("test")) + .withTridentTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper("word", "count")); + stream.partitionPersist(stateFactory, fields, new TridentKafkaUpdater(), new Fields()); + + Config conf = new Config(); + StormSubmitter.submitTopology("kafkaTridentTest", conf, topology.build()); +``` + +## Reading From kafka (Spouts) + +### Configuration + +The spout implementations are configured by use of the `KafkaSpoutConfig` class. This class uses a Builder pattern and can be started either by calling one of +the Builders constructors or by calling the static method builder in the KafkaSpoutConfig class. + +The Constructor or static method to create the builder require a few key values (that can be changed later on) but are the minimum config needed to start +a spout. + +`bootstrapServers` is the same as the Kafka Consumer Property "bootstrap.servers". +`topics` The topics the spout will consume can either be a `Collection` of specific topic names (1 or more) or a regular expression `Pattern`, which specifies +that any topics that match that regular expression will be consumed. -# Usage Examples +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer. This is to help guarantee type safety through the use +of Java generics. The defaults are `StringDeserializer`s and can be overwritten by calling `setKeyDeserializer` and/or `setValueDeserializer`. +If these are set to null the code will fall back to what is set in the kafka properties, but it is preferable to be explicit here, again to maintain +type safety with the generics. -### Create a Kafka Spout +There are a few key configs to pay attention to. -The code snippet bellow is extracted from the example in the module [test] (https://github.com/apache/storm/tree/master/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test). The code that is common for named topics and topic wildcards is in the first box. The specific implementations are in the appropriate section. +`setFirstPollOffsetStrategy` allows you to set where to start consuming data from. This is used both in case of failure recovery and starting the spout +for the first time. Allowed values include -These snippets serve as a reference and do not compile. If you would like to reuse this code in your implementation, please obtain it from the test module, where it is complete. + * `EARLIEST` means that the kafka spout polls records starting in the first offset of the partition, regardless of previous commits + * `LATEST` means that the kafka spout polls records with offsets greater than the last offset in the partition, regardless of previous commits + * `UNCOMMITTED_EARLIEST` (DEFAULT) means that the kafka spout polls records from the last committed offset, if any. If no offset has been committed, it behaves as `EARLIEST`. + * `UNCOMMITTED_LATEST` means that the kafka spout polls records from the last committed offset, if any. If no offset has been committed, it behaves as `LATEST`. +`setRecordTranslator` allows you to modify how the spout converts a Kafka Consumer Record into a Tuple, and which stream that tuple will be published into. +By default the "topic", "partition", "offset", "key", and "value" will be emitted to the "default" stream. If you want to output entries to different +streams based on the topic, storm provides `ByTopicRecordTranslator`. See below for more examples on how to use these. + +`setProp` can be used to set kafka properties that do not have a convenience method. + +`setGroupId` lets you set the id of the kafka consumer group property "group.id' + +`setSSLKeystore` and `setSSLTruststore` allow you to configure SSL authentication. + +### Usage Examples + +The API is written with java 8 lambda expressions in mind. It works with java7 and below though. + +#### Create a Simple Insecure Spout +The following will consume all events published to "topic" and send them to MyBolt with the fields "topic", "partition", "offset", "key", "value". ```java -KafkaSpout kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig); - -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService) - .setOffsetCommitPeriodMs(10_000) - .setFirstPollOffsetStrategy(EARLIEST) - .setMaxUncommittedOffsets(250) - .build(); - -Map kafkaConsumerProps= new HashMap<>(); -kafkaConsumerProps.put(KafkaSpoutConfig.Consumer.BOOTSTRAP_SERVERS,"127.0.0.1:9092"); -kafkaConsumerProps.put(KafkaSpoutConfig.Consumer.GROUP_ID,"kafkaSpoutTestGroup"); -kafkaConsumerProps.put(KafkaSpoutConfig.Consumer.KEY_DESERIALIZER,"org.apache.kafka.common.serialization.StringDeserializer"); -kafkaConsumerProps.put(KafkaSpoutConfig.Consumer.VALUE_DESERIALIZER,"org.apache.kafka.common.serialization.StringDeserializer"); - -KafkaSpoutRetryService retryService = new KafkaSpoutRetryExponentialBackoff(KafkaSpoutRetryExponentialBackoff.TimeInterval.microSeconds(500), - KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(2), Integer.MAX_VALUE, KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(10)); + +final TopologyBuilder tp = new TopologyBuilder(); +tp.setSpout("kafka_spout", new KafkaSpout<>(KafkaSpoutConfig.builder("127.0.0.1:" + port, "topic").build()), 1); +tp.setBolt("bolt", new myBolt()).shuffleGrouping("kafka_spout"); +... + ``` -### Named Topics +#### Wildcard Topics +Wildcard topics will consume from all topics that exist in the specified brokers list and match the pattern. So in the following example +"topic", "topic_foo" and "topic_bar" will all match the pattern "topic.*", but "not_my_topic" would not match. ```java -KafkaSpoutStreams kafkaSpoutStreams = new KafkaSpoutStreamsNamedTopics.Builder(outputFields, STREAMS[0], new String[]{TOPICS[0], TOPICS[1]}) - .addStream(outputFields, STREAMS[0], new String[]{TOPICS[2]}) // contents of topic test2 sent to test_stream - .addStream(outputFields1, STREAMS[2], new String[]{TOPICS[2]}) // contents of topic test2 sent to test2_stream - .build(); - -KafkaSpoutTuplesBuilder tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>( - new TopicsTest0Test1TupleBuilder(TOPICS[0], TOPICS[1]), - new TopicTest2TupleBuilder(TOPICS[2])) - .build(); - -String[] STREAMS = new String[]{"test_stream", "test1_stream", "test2_stream"}; -String[] TOPICS = new String[]{"test", "test1", "test2"}; - -Fields outputFields = new Fields("topic", "partition", "offset", "key", "value"); -Fields outputFields1 = new Fields("topic", "partition", "offset"); + +final TopologyBuilder tp = new TopologyBuilder(); +tp.setSpout("kafka_spout", new KafkaSpout<>(KafkaSpoutConfig.builder("127.0.0.1:" + port, Pattern.compile("topic.*")).build()), 1); +tp.setBolt("bolt", new myBolt()).shuffleGrouping("kafka_spout"); +... + ``` -### Topic Wildcards +#### Multiple Streams +This uses java 8 lambda expressions. ```java -KafkaSpoutStreams kafkaSpoutStreams = new KafkaSpoutStreamsWildcardTopics( - new KafkaSpoutStream(outputFields, STREAM, Pattern.compile(TOPIC_WILDCARD_PATTERN))); -KafkaSpoutTuplesBuilder tuplesBuilder = new TopicsTest0Test1TupleBuilder<>(TOPIC_WILDCARD_PATTERN); +final TopologyBuilder tp = new TopologyBuilder(); + +//By default all topics not covered by another rule, but consumed by the spout will be emitted to "STREAM_1" as "topic", "key", and "value" +ByTopicRecordTranslator byTopic = new ByTopicRecordTranslator<>( + (r) -> new Values(r.topic(), r.key(), r.value()), + new Fields("topic", "key", "value"), "STREAM_1"); +//For topic_2 all events will be emitted to "STREAM_2" as just "key" and "value" +byTopic.forTopic("topic_2", (r) -> new Values(r.key(), r.value()), new Fields("key", "value"), "STREAM_2"); + +tp.setSpout("kafka_spout", new KafkaSpout<>(KafkaSpoutConfig.builder("127.0.0.1:" + port, "topic_1", "topic_2", "topic_3").build()), 1); +tp.setBolt("bolt", new myBolt()).shuffleGrouping("kafka_spout", "STREAM_1"); +tp.setBolt("another", new myOtherBolt()).shuffleGrouping("kafka_spout", "STREAM_2"); +... -String STREAM = "test_wildcard_stream"; -String TOPIC_WILDCARD_PATTERN = "test[1|2]"; +``` + +#### Trident + +```java +final TridentTopology tridentTopology = new TridentTopology(); +final Stream spoutStream = tridentTopology.newStream("kafkaSpout", + new KafkaTridentSpoutOpaque<>(KafkaSpoutConfig.builder("127.0.0.1:" + port, Pattern.compile("topic.*")).build())) + .parallelismHint(1) +... -Fields outputFields = new Fields("topic", "partition", "offset", "key", "value"); ``` -### Create a simple Toplogy using the Kafka Spout: +Trident does not support multiple streams and will ignore any streams set for output. If however the Fields are not identical for each +output topic it will throw an exception and not continue. + +### Custom RecordTranslators (ADVANCED) + +In most cases the built in SimpleRecordTranslator and ByTopicRecordTranslator should cover your use case. If you do run into a situation where you need a custom one +then this documentation will describe how to do this properly, and some of the less than obvious classes involved. +The point of apply is to take a ConsumerRecord and turn it into a `List` that can be emitted. What is not obvious is how to tell the spout to emit it to a +specific stream. To do this you will need to return an instance of `org.apache.storm.kafka.spout.KafkaTuple`. This provides a method `routedTo` that will say which +specific stream the tuple should go to. +For Example: ```java -TopologyBuilder tp = new TopologyBuilder(); -tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams())), 1); -tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAMS[0]); -tp.setBolt("kafka_bolt_1", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAMS[2]); -tp.createTopology(); +return new KafkaTuple(1, 2, 3, 4).routedTo("bar"); ``` -# Build And Run Bundled Examples -To be able to run the examples you must first build the java code in the package `storm-kafka-client`, -and then generate an uber jar with all the dependencies. +Will cause the tuple to be emitted on the "bar" stream. + +Be careful when writing custom record translators because just like in a storm spout it needs to be self consistent. The `streams` method should return +a full set of streams that this translator will ever try to emit on. Additionally `getFieldsFor` should return a valid Fields object for each of those +streams. If you are doing this for Trident a value must be in the List returned by apply for every field in the Fields object for that stream, +otherwise trident can throw exceptions. + + +### Manual Partition Control (ADVANCED) + +By default Kafka will automatically assign partitions to the current set of spouts. It handles lots of things, but in some cases you may want to manually assign the partitions. +This can cause less churn in the assignments when spouts go down and come back up, but it can result in a lot of issues if not done right. This can all be handled by subclassing +Subscription and we have a few implementations that you can look at for examples on how to do this. ManualPartitionNamedSubscription and ManualPartitionPatternSubscription. Again +please be careful when using these or implementing your own. ## Use the Maven Shade Plugin to Build the Uber Jar @@ -112,7 +279,7 @@ Add the following to `REPO_HOME/storm/external/storm-kafka-client/pom.xml` ``` -create the uber jar by running the commmand: +create the uber jar by running the command: `mvn package -f REPO_HOME/storm/external/storm-kafka-client/pom.xml` @@ -122,11 +289,11 @@ This will create the uber jar file with the name and location matching the follo ### Run Storm Topology -Copy the file `REPO_HOME/storm/external/storm-kafka-client/target/storm-kafka-client-1.0.x.jar` to `STORM_HOME/extlib` +Copy the file `REPO_HOME/storm/external/storm-kafka-client/target/storm-kafka-client-*.jar` to `STORM_HOME/extlib` Using the Kafka command line tools create three topics [test, test1, test2] and use the Kafka console producer to populate the topics with some data -Execute the command `STORM_HOME/bin/storm jar REPO_HOME/storm/external/storm/target/storm-kafka-client-1.0.x.jar org.apache.storm.kafka.spout.test.KafkaSpoutTopologyMain` +Execute the command `STORM_HOME/bin/storm jar REPO_HOME/storm/external/storm/target/storm-kafka-client-*.jar org.apache.storm.kafka.spout.test.KafkaSpoutTopologyMain` With the debug level logs enabled it is possible to see the messages of each topic being redirected to the appropriate Bolt as defined by the streams defined and choice of shuffle grouping. @@ -181,8 +348,3 @@ Currently the Kafka spout has has the following default values, which have shown * offset.commit.period.ms = 30000 (30s) * max.uncommitted.offsets = 10000000
- -There will be a blog post coming soon analyzing the trade-offs of this tuning parameters, and comparing the performance of the Kafka Spouts using the Kafka client API introduced in 0.9 (new implementation) and in prior versions (prior implementation) - -#Future Work - Implement comprehensive metrics. Trident spout is coming soon. diff --git a/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java b/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java index db265491a31..1ae69c825c2 100644 --- a/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java +++ b/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java @@ -18,87 +18,58 @@ package org.apache.storm.kafka.trident; +import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; + import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.storm.Config; import org.apache.storm.StormSubmitter; import org.apache.storm.generated.AlreadyAliveException; import org.apache.storm.generated.AuthorizationException; import org.apache.storm.generated.InvalidTopologyException; +import org.apache.storm.kafka.spout.Func; import org.apache.storm.kafka.spout.KafkaSpoutConfig; import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff; +import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff.TimeInterval; import org.apache.storm.kafka.spout.KafkaSpoutRetryService; -import org.apache.storm.kafka.spout.KafkaSpoutStreams; -import org.apache.storm.kafka.spout.KafkaSpoutStreamsNamedTopics; -import org.apache.storm.kafka.spout.KafkaSpoutTupleBuilder; -import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilder; -import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilderNamedTopics; -import org.apache.storm.kafka.spout.trident.KafkaTridentSpoutManager; import org.apache.storm.kafka.spout.trident.KafkaTridentSpoutOpaque; import org.apache.storm.tuple.Fields; import org.apache.storm.tuple.Values; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; - public class TridentKafkaClientWordCountNamedTopics { private static final String TOPIC_1 = "test-trident"; private static final String TOPIC_2 = "test-trident-1"; private static final String KAFKA_LOCAL_BROKER = "localhost:9092"; private KafkaTridentSpoutOpaque newKafkaTridentSpoutOpaque() { - return new KafkaTridentSpoutOpaque<>(new KafkaTridentSpoutManager<>( - newKafkaSpoutConfig( - newKafkaSpoutStreams()))); + return new KafkaTridentSpoutOpaque<>(newKafkaSpoutConfig()); } - private KafkaSpoutConfig newKafkaSpoutConfig(KafkaSpoutStreams kafkaSpoutStreams) { - return new KafkaSpoutConfig.Builder<>(newKafkaConsumerProps(), - kafkaSpoutStreams, newTuplesBuilder(), newRetryService()) + private static Func, List> JUST_VALUE_FUNC = new Func, List>() { + @Override + public List apply(ConsumerRecord record) { + return new Values(record.value()); + } + }; + + protected KafkaSpoutConfig newKafkaSpoutConfig() { + return KafkaSpoutConfig.builder(KAFKA_LOCAL_BROKER, TOPIC_1, TOPIC_2) + .setGroupId("kafkaSpoutTestGroup") + .setMaxPartitionFectchBytes(200) + .setRecordTranslator(JUST_VALUE_FUNC, new Fields("str")) + .setRetry(newRetryService()) .setOffsetCommitPeriodMs(10_000) .setFirstPollOffsetStrategy(EARLIEST) .setMaxUncommittedOffsets(250) .build(); } - protected Map newKafkaConsumerProps() { - Map props = new HashMap<>(); - props.put(KafkaSpoutConfig.Consumer.BOOTSTRAP_SERVERS, "127.0.0.1:9092"); - props.put(KafkaSpoutConfig.Consumer.GROUP_ID, "kafkaSpoutTestGroup"); - props.put(KafkaSpoutConfig.Consumer.KEY_DESERIALIZER, "org.apache.kafka.common.serialization.StringDeserializer"); - props.put(KafkaSpoutConfig.Consumer.VALUE_DESERIALIZER, "org.apache.kafka.common.serialization.StringDeserializer"); - props.put("max.partition.fetch.bytes", 200); - return props; - } - - protected KafkaSpoutTuplesBuilder newTuplesBuilder() { - return new KafkaSpoutTuplesBuilderNamedTopics.Builder<>( - new TopicsTupleBuilder(TOPIC_1, TOPIC_2)) - .build(); - } - protected KafkaSpoutRetryService newRetryService() { - return new KafkaSpoutRetryExponentialBackoff(new KafkaSpoutRetryExponentialBackoff.TimeInterval(500L, TimeUnit.MICROSECONDS), - KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(2), - Integer.MAX_VALUE, KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(10)); - } - - protected KafkaSpoutStreams newKafkaSpoutStreams() { - return new KafkaSpoutStreamsNamedTopics.Builder(new Fields("str"), new String[]{"test-trident","test-trident-1"}).build(); - } - - protected static class TopicsTupleBuilder extends KafkaSpoutTupleBuilder { - public TopicsTupleBuilder(String... topics) { - super(topics); - } - @Override - public List buildTuple(ConsumerRecord consumerRecord) { - return new Values(consumerRecord.value()); - } + return new KafkaSpoutRetryExponentialBackoff(new TimeInterval(500L, TimeUnit.MICROSECONDS), + TimeInterval.milliSeconds(2), Integer.MAX_VALUE, TimeInterval.seconds(10)); } public static void main(String[] args) throws Exception { diff --git a/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountWildcardTopics.java b/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountWildcardTopics.java index 02471c5ac08..94a976564f1 100644 --- a/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountWildcardTopics.java +++ b/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountWildcardTopics.java @@ -18,26 +18,37 @@ package org.apache.storm.kafka.trident; -import org.apache.storm.kafka.spout.KafkaSpoutStream; -import org.apache.storm.kafka.spout.KafkaSpoutStreams; -import org.apache.storm.kafka.spout.KafkaSpoutStreamsWildcardTopics; -import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilder; -import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilderWildcardTopics; -import org.apache.storm.tuple.Fields; +import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; +import java.util.List; import java.util.regex.Pattern; -public class TridentKafkaClientWordCountWildcardTopics extends TridentKafkaClientWordCountNamedTopics { - private static final String TOPIC_WILDCARD_PATTERN = "test-trident(-1)?"; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.storm.kafka.spout.Func; +import org.apache.storm.kafka.spout.KafkaSpoutConfig; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; - protected KafkaSpoutTuplesBuilder newTuplesBuilder() { - return new KafkaSpoutTuplesBuilderWildcardTopics<>(new TopicsTupleBuilder(TOPIC_WILDCARD_PATTERN)); - } +public class TridentKafkaClientWordCountWildcardTopics extends TridentKafkaClientWordCountNamedTopics { + private static final Pattern TOPIC_WILDCARD_PATTERN = Pattern.compile("test-trident(-1)?"); - protected KafkaSpoutStreams newKafkaSpoutStreams() { - final Fields outputFields = new Fields("str"); - final KafkaSpoutStream kafkaSpoutStream = new KafkaSpoutStream(outputFields, Pattern.compile(TOPIC_WILDCARD_PATTERN)); - return new KafkaSpoutStreamsWildcardTopics(kafkaSpoutStream); + private static Func, List> JUST_VALUE_FUNC = new Func, List>() { + @Override + public List apply(ConsumerRecord record) { + return new Values(record.value()); + } + }; + + protected KafkaSpoutConfig newKafkaSpoutConfig() { + return KafkaSpoutConfig.builder("127.0.0.1:9092", TOPIC_WILDCARD_PATTERN) + .setGroupId("kafkaSpoutTestGroup") + .setMaxPartitionFectchBytes(200) + .setRecordTranslator(JUST_VALUE_FUNC, new Fields("str")) + .setRetry(newRetryService()) + .setOffsetCommitPeriodMs(10_000) + .setFirstPollOffsetStrategy(EARLIEST) + .setMaxUncommittedOffsets(250) + .build(); } public static void main(String[] args) throws Exception { diff --git a/external/storm-kafka-client/README.md b/external/storm-kafka-client/README.md index 5784b8add52..465e466e362 100644 --- a/external/storm-kafka-client/README.md +++ b/external/storm-kafka-client/README.md @@ -1,192 +1,5 @@ -#Storm Kafka Spout with New Kafka Consumer API +#Storm Apache Kafka integration using the kafka-client jar (This includes the new Apache Kafka consumer API) -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). +Spouts and Bolts that write to and read from Kafka through the kafka-client library. -The Apache Storm Spout allows clients to consume data from Kafka starting at offsets as defined by the offset strategy specified in `FirstPollOffsetStrategy`. -In case of failure, the Kafka Spout will re-start consuming messages from the offset that matches the chosen `FirstPollOffsetStrategy`. - -The Kafka Spout implementation allows you to specify the stream (`KafkaSpoutStream`) associated with each topic or topic wildcard. `KafkaSpoutStream` represents the stream and output fields. For named topics use `KafkaSpoutStreamsNamedTopics`, and for topic wildcards use `KafkaSpoutStreamsWildcardTopics`. - -The `KafkaSpoutTuplesBuilder` wraps all the logic that builds `Tuple`s from `ConsumerRecord`s. The logic is provided by the user through implementing the appropriate number of `KafkaSpoutTupleBuilder` instances. For named topics use `KafkaSpoutTuplesBuilderNamedTopics`, and for topic wildcard use `KafkaSpoutTuplesBuilderWildcardTopics`. - -Multiple topics and topic wildcards can use the same `KafkaSpoutTupleBuilder` implementation, as long as the logic to build `Tuple`s from `ConsumerRecord`s is identical. - - -# Usage Examples - -### Create a Kafka Spout - -The code snippet bellow is extracted from the example in the module [test] (https://github.com/apache/storm/tree/master/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test). The code that is common for named topics and topic wildcards is in the first box. The specific implementations are in the appropriate section. - -These snippets serve as a reference and do not compile. If you would like to reuse this code in your implementation, please obtain it from the test module, where it is complete. - -```java -KafkaSpout kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig); - -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService) - .setOffsetCommitPeriodMs(10_000) - .setFirstPollOffsetStrategy(EARLIEST) - .setMaxUncommittedOffsets(250) - .build(); - -Map kafkaConsumerProps= new HashMap<>(); -kafkaConsumerProps.put(KafkaSpoutConfig.Consumer.BOOTSTRAP_SERVERS,"127.0.0.1:9092"); -kafkaConsumerProps.put(KafkaSpoutConfig.Consumer.GROUP_ID,"kafkaSpoutTestGroup"); -kafkaConsumerProps.put(KafkaSpoutConfig.Consumer.KEY_DESERIALIZER,"org.apache.kafka.common.serialization.StringDeserializer"); -kafkaConsumerProps.put(KafkaSpoutConfig.Consumer.VALUE_DESERIALIZER,"org.apache.kafka.common.serialization.StringDeserializer"); - -KafkaSpoutRetryService retryService = new KafkaSpoutRetryExponentialBackoff(KafkaSpoutRetryExponentialBackoff.TimeInterval.microSeconds(500), - KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(2), Integer.MAX_VALUE, KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(10)); -``` - -### Named Topics -```java -KafkaSpoutStreams kafkaSpoutStreams = new KafkaSpoutStreamsNamedTopics.Builder(outputFields, STREAMS[0], new String[]{TOPICS[0], TOPICS[1]}) - .addStream(outputFields, STREAMS[0], new String[]{TOPICS[2]}) // contents of topic test2 sent to test_stream - .addStream(outputFields1, STREAMS[2], new String[]{TOPICS[2]}) // contents of topic test2 sent to test2_stream - .build(); - -KafkaSpoutTuplesBuilder tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>( - new TopicsTest0Test1TupleBuilder(TOPICS[0], TOPICS[1]), - new TopicTest2TupleBuilder(TOPICS[2])) - .build(); - -String[] STREAMS = new String[]{"test_stream", "test1_stream", "test2_stream"}; -String[] TOPICS = new String[]{"test", "test1", "test2"}; - -Fields outputFields = new Fields("topic", "partition", "offset", "key", "value"); -Fields outputFields1 = new Fields("topic", "partition", "offset"); -``` - -### Topic Wildcards -```java -KafkaSpoutStreams kafkaSpoutStreams = new KafkaSpoutStreamsWildcardTopics( - new KafkaSpoutStream(outputFields, STREAM, Pattern.compile(TOPIC_WILDCARD_PATTERN))); - -KafkaSpoutTuplesBuilder tuplesBuilder = new TopicsTest0Test1TupleBuilder<>(TOPIC_WILDCARD_PATTERN); - -String STREAM = "test_wildcard_stream"; -String TOPIC_WILDCARD_PATTERN = "test[1|2]"; - -Fields outputFields = new Fields("topic", "partition", "offset", "key", "value"); -``` - -### Create a simple Toplogy using the Kafka Spout: - - -```java -TopologyBuilder tp = new TopologyBuilder(); -tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams())), 1); -tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAMS[0]); -tp.setBolt("kafka_bolt_1", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAMS[2]); -tp.createTopology(); -``` - -# Build And Run Bundled Examples -To be able to run the examples you must first build the java code in the package `storm-kafka-client`, -and then generate an uber jar with all the dependencies. - -## Use the Maven Shade Plugin to Build the Uber Jar - -Add the following to `REPO_HOME/storm/external/storm-kafka-client/pom.xml` -```xml - - org.apache.maven.plugins - maven-shade-plugin - 2.4.1 - - - package - - shade - - - - - org.apache.storm.kafka.spout.test.KafkaSpoutTopologyMain - - - - - - -``` - -create the uber jar by running the commmand: - -`mvn package -f REPO_HOME/storm/external/storm-kafka-client/pom.xml` - -This will create the uber jar file with the name and location matching the following pattern: - -`REPO_HOME/storm/external/storm-kafka-client/target/storm-kafka-client-1.0.x.jar` - -### Run Storm Topology - -Copy the file `REPO_HOME/storm/external/storm-kafka-client/target/storm-kafka-client-1.0.x.jar` to `STORM_HOME/extlib` - -Using the Kafka command line tools create three topics [test, test1, test2] and use the Kafka console producer to populate the topics with some data - -Execute the command `STORM_HOME/bin/storm jar REPO_HOME/storm/external/storm/target/storm-kafka-client-1.0.x.jar org.apache.storm.kafka.spout.test.KafkaSpoutTopologyMain` - -With the debug level logs enabled it is possible to see the messages of each topic being redirected to the appropriate Bolt as defined -by the streams defined and choice of shuffle grouping. - -## Using storm-kafka-client with different versions of kafka - -Storm-kafka-client's Kafka dependency is defined as `provided` scope in maven, meaning it will not be pulled in -as a transitive dependency. This allows you to use a version of Kafka dependency compatible with your kafka cluster. - -When building a project with storm-kafka-client, you must explicitly add the Kafka clients dependency. For example, to -use Kafka-clients 0.10.0.0, you would use the following dependency in your `pom.xml`: - -```xml - - org.apache.kafka - kafka-clients - 0.10.0.0 - -``` - -You can also override the kafka clients version while building from maven, with parameter `storm.kafka.client.version` -e.g. `mvn clean install -Dstorm.kafka.client.version=0.10.0.0` - -When selecting a kafka client version, you should ensure - - 1. kafka api is compatible. storm-kafka-client module only supports **0.10 or newer** kafka client API. For older versions, - you can use storm-kafka module (https://github.com/apache/storm/tree/master/external/storm-kafka). - 2. The kafka client selected by you should be wire compatible with the broker. e.g. 0.9.x client will not work with - 0.8.x broker. - - -#Kafka Spout Performance Tuning - -The Kafka spout provides two internal parameters to control its performance. The parameters can be set using the [KafkaSpoutConfig] (https://github.com/apache/storm/blob/1.0.x-branch/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java) methods [setOffsetCommitPeriodMs] (https://github.com/apache/storm/blob/1.0.x-branch/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java#L189-L193) and [setMaxUncommittedOffsets] (https://github.com/apache/storm/blob/1.0.x-branch/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java#L211-L217). - -* "offset.commit.period.ms" controls how often the spout commits to Kafka -* "max.uncommitted.offsets" controls how many offsets can be pending commit before another poll can take place -
- -The [Kafka consumer config] (http://kafka.apache.org/documentation.html#consumerconfigs) parameters may also have an impact on the performance of the spout. The following Kafka parameters are likely the most influential in the spout performance: - -* “fetch.min.bytes” -* “fetch.max.wait.ms” -* [Kafka Consumer] (http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html) instance poll timeout, which is specified for each Kafka spout using the [KafkaSpoutConfig] (https://github.com/apache/storm/blob/1.0.x-branch/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java) method [setPollTimeoutMs] (https://github.com/apache/storm/blob/1.0.x-branch/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java#L180-L184) -
- -Depending on the structure of your Kafka cluster, distribution of the data, and availability of data to poll, these parameters will have to be configured appropriately. Please refer to the Kafka documentation on Kafka parameter tuning. - -###Default values - -Currently the Kafka spout has has the following default values, which have shown to give good performance in the test environment as described in this [blog post] (https://hortonworks.com/blog/microbenchmarking-storm-1-0-performance/) - -* poll.timeout.ms = 200 -* offset.commit.period.ms = 30000 (30s) -* max.uncommitted.offsets = 10000000 -
- -There will be a blog post coming soon analyzing the trade-offs of this tuning parameters, and comparing the performance of the Kafka Spouts using the Kafka client API introduced in 0.9 (new implementation) and in prior versions (prior implementation) - -#Future Work - Implement comprehensive metrics. Trident spout is coming soon. - -## Committer Sponsors - * Sriharsha Chintalapani ([sriharsha@apache.org](mailto:sriharsha@apache.org)) +Please see [here](../../docs/storm-kafka-client.md) for details on how to use it. diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java new file mode 100644 index 00000000000..84d3334621c --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java @@ -0,0 +1,223 @@ +/** + * 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.bolt; + +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseRichBolt; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.utils.TupleUtils; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.clients.producer.Callback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.storm.kafka.bolt.mapper.FieldNameBasedTupleToKafkaMapper; +import org.apache.storm.kafka.bolt.mapper.TupleToKafkaMapper; +import org.apache.storm.kafka.bolt.selector.DefaultTopicSelector; +import org.apache.storm.kafka.bolt.selector.KafkaTopicSelector; +import java.util.concurrent.Future; +import java.util.concurrent.ExecutionException; +import java.util.Map; +import java.util.Properties; + + +/** + * Bolt implementation that can send Tuple data to Kafka. + *

+ * Most configuration for this bolt should be through the various + * setter methods in the bolt. + * For backwards compatibility it supports the producer + * configuration and topic to be placed in the storm config under + *

+ * 'kafka.broker.properties' and 'topic' + *

+ * respectively. + */ +public class KafkaBolt extends BaseRichBolt { + private static final long serialVersionUID = -5205886631877033478L; + + private static final Logger LOG = LoggerFactory.getLogger(KafkaBolt.class); + + public static final String TOPIC = "topic"; + + private KafkaProducer producer; + private OutputCollector collector; + private TupleToKafkaMapper mapper; + private KafkaTopicSelector topicSelector; + private Properties boltSpecifiedProperties = new Properties(); + /** + * {@see KafkaBolt#setFireAndForget(boolean)} for more details on this. + */ + private boolean fireAndForget = false; + /** + * {@see KafkaBolt#setAsync(boolean)} for more details on this. + */ + private boolean async = true; + + public KafkaBolt() {} + + public KafkaBolt withTupleToKafkaMapper(TupleToKafkaMapper mapper) { + this.mapper = mapper; + return this; + } + + /** + * Set the messages to be published to a single topic + * @param topic the topic to publish to + * @return this + */ + public KafkaBolt withTopicSelector(String topic) { + return withTopicSelector(new DefaultTopicSelector(topic)); + } + + public KafkaBolt withTopicSelector(KafkaTopicSelector selector) { + this.topicSelector = selector; + return this; + } + + public KafkaBolt withProducerProperties(Properties producerProperties) { + this.boltSpecifiedProperties = producerProperties; + return this; + } + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + LOG.info("Preparing bolt with configuration {}", this); + //for backward compatibility. + if (mapper == null) { + LOG.info("Mapper not specified. Setting default mapper to {}", FieldNameBasedTupleToKafkaMapper.class.getSimpleName()); + this.mapper = new FieldNameBasedTupleToKafkaMapper(); + } + + //for backward compatibility. + if (topicSelector == null) { + if (stormConf.containsKey(TOPIC)) { + LOG.info("TopicSelector not specified. Using [{}] for topic [{}] specified in bolt configuration,", + DefaultTopicSelector.class.getSimpleName(), stormConf.get(TOPIC)); + this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC)); + } else { + throw new IllegalStateException("topic should be specified in bolt's configuration"); + } + } + + producer = mkProducer(boltSpecifiedProperties); + this.collector = collector; + } + + /** + * Intended to be overridden for tests. Make the producer with the given props + */ + protected KafkaProducer mkProducer(Properties props) { + return new KafkaProducer<>(props); + } + + @Override + public void execute(final Tuple input) { + if (TupleUtils.isTick(input)) { + collector.ack(input); + return; // Do not try to send ticks to Kafka + } + K key = null; + V message = null; + String topic = null; + try { + key = mapper.getKeyFromTuple(input); + message = mapper.getMessageFromTuple(input); + topic = topicSelector.getTopic(input); + if (topic != null ) { + Callback callback = null; + + if (!fireAndForget && async) { + callback = new Callback() { + @Override + public void onCompletion(RecordMetadata ignored, Exception e) { + synchronized (collector) { + if (e != null) { + collector.reportError(e); + collector.fail(input); + } else { + collector.ack(input); + } + } + } + }; + } + Future result = producer.send(new ProducerRecord(topic, key, message), callback); + if (!async) { + try { + result.get(); + collector.ack(input); + } catch (ExecutionException err) { + collector.reportError(err); + collector.fail(input); + } + } else if (fireAndForget) { + collector.ack(input); + } + } else { + LOG.warn("skipping key = " + key + ", topic selector returned null."); + collector.ack(input); + } + } catch (Exception ex) { + collector.reportError(ex); + collector.fail(input); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + + } + + @Override + public void cleanup() { + producer.close(); + } + + /** + * If set to true the bolt will assume that sending a message to kafka will succeed and will ack + * the tuple as soon as it has handed the message off to the producer API + * if false (the default) the message will be acked after it was successfully sent to kafka or + * failed if it was not successfully sent. + * @param fireAndForget + */ + public void setFireAndForget(boolean fireAndForget) { + this.fireAndForget = fireAndForget; + } + + /** + * If set to true(the default) the bolt will not wait for the message + * to be fully sent to Kafka before getting another tuple to send. + * @param async true to have multiple tuples in flight to kafka, else false. + */ + public void setAsync(boolean async) { + this.async = async; + } + + @Override + public String toString() { + return "KafkaBolt: {mapper: " + mapper + + " topicSelector: " + topicSelector + + " fireAndForget: " + fireAndForget + + " async: " + async + + " proerties: " + boltSpecifiedProperties; + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/FieldNameBasedTupleToKafkaMapper.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/FieldNameBasedTupleToKafkaMapper.java new file mode 100644 index 00000000000..f7638aa2d49 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/FieldNameBasedTupleToKafkaMapper.java @@ -0,0 +1,48 @@ +/** + * 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.bolt.mapper; + +import org.apache.storm.tuple.Tuple; + +public class FieldNameBasedTupleToKafkaMapper implements TupleToKafkaMapper { + private static final long serialVersionUID = -8794262989021702349L; + public static final String BOLT_KEY = "key"; + public static final String BOLT_MESSAGE = "message"; + public String boltKeyField; + public String boltMessageField; + + public FieldNameBasedTupleToKafkaMapper() { + this(BOLT_KEY, BOLT_MESSAGE); + } + + public FieldNameBasedTupleToKafkaMapper(String boltKeyField, String boltMessageField) { + this.boltKeyField = boltKeyField; + this.boltMessageField = boltMessageField; + } + + @Override + public K getKeyFromTuple(Tuple tuple) { + //for backward compatibility, we return null when key is not present. + return tuple.contains(boltKeyField) ? (K) tuple.getValueByField(boltKeyField) : null; + } + + @Override + public V getMessageFromTuple(Tuple tuple) { + return (V) tuple.getValueByField(boltMessageField); + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/TupleToKafkaMapper.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/TupleToKafkaMapper.java new file mode 100644 index 00000000000..9f11fc9c292 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/TupleToKafkaMapper.java @@ -0,0 +1,32 @@ +/** + * 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.bolt.mapper; + +import org.apache.storm.tuple.Tuple; + +import java.io.Serializable; + +/** + * Interface defining a mapping from storm tuple to kafka key and message. + * @param type of key. + * @param type of value. + */ +public interface TupleToKafkaMapper extends Serializable { + K getKeyFromTuple(Tuple tuple); + V getMessageFromTuple(Tuple tuple); +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/DefaultTopicSelector.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/DefaultTopicSelector.java new file mode 100644 index 00000000000..3d00fc1be4d --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/DefaultTopicSelector.java @@ -0,0 +1,34 @@ +/** + * 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.bolt.selector; + +import org.apache.storm.tuple.Tuple; + +public class DefaultTopicSelector implements KafkaTopicSelector { + private static final long serialVersionUID = 4601118062437851265L; + private final String topicName; + + public DefaultTopicSelector(final String topicName) { + this.topicName = topicName; + } + + @Override + public String getTopic(Tuple tuple) { + return topicName; + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java new file mode 100644 index 00000000000..ffe0b35ab9a --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java @@ -0,0 +1,52 @@ +/** + * 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.bolt.selector; + +import org.apache.storm.tuple.Tuple; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Uses field with a given index to select the topic name from a tuple . + */ +public class FieldIndexTopicSelector implements KafkaTopicSelector { + private static final long serialVersionUID = -3830575380208166367L; + + private static final Logger LOG = LoggerFactory.getLogger(FieldIndexTopicSelector.class); + + private final int fieldIndex; + private final String defaultTopicName; + + public FieldIndexTopicSelector(int fieldIndex, String defaultTopicName) { + this.fieldIndex = fieldIndex; + if (fieldIndex < 0) { + throw new IllegalArgumentException("fieldIndex cannot be negative"); + } + this.defaultTopicName = defaultTopicName; + } + + @Override + public String getTopic(Tuple tuple) { + if (fieldIndex < tuple.size()) { + return tuple.getString(fieldIndex); + } else { + LOG.warn("Field index {} is out of bounds. Using default topic {}", fieldIndex, defaultTopicName); + return defaultTopicName; + } + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java new file mode 100644 index 00000000000..e90b26ff307 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java @@ -0,0 +1,49 @@ +/** + * 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.bolt.selector; + +import org.apache.storm.tuple.Tuple; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Uses field name to select topic name from tuple . + */ +public class FieldNameTopicSelector implements KafkaTopicSelector { + private static final long serialVersionUID = -3903708904533396833L; + private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class); + + private final String fieldName; + private final String defaultTopicName; + + + public FieldNameTopicSelector(String fieldName, String defaultTopicName) { + this.fieldName = fieldName; + this.defaultTopicName = defaultTopicName; + } + + @Override + public String getTopic(Tuple tuple) { + if (tuple.contains(fieldName)) { + return tuple.getStringByField(fieldName); + } else { + LOG.warn("Field {} Not Found. Returning default topic {}", fieldName, defaultTopicName); + return defaultTopicName; + } + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/KafkaTopicSelector.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/KafkaTopicSelector.java new file mode 100644 index 00000000000..cb7fb44a047 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/KafkaTopicSelector.java @@ -0,0 +1,26 @@ +/** + * 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.bolt.selector; + +import org.apache.storm.tuple.Tuple; + +import java.io.Serializable; + +public interface KafkaTopicSelector extends Serializable { + String getTopic(Tuple tuple); +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ByTopicRecordTranslator.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ByTopicRecordTranslator.java new file mode 100644 index 00000000000..8ad527d6df1 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ByTopicRecordTranslator.java @@ -0,0 +1,149 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.storm.tuple.Fields; + +/** + * Based off of a given Kafka topic a ConsumerRecord came from it will be translated to a Storm tuple + * and emitted to a given stream + * @param the key of the incoming Records + * @param the value of the incoming Records + */ +public class ByTopicRecordTranslator implements RecordTranslator { + private static final long serialVersionUID = -121699733778988688L; + private final RecordTranslator defaultTranslator; + private final Map> topicToTranslator = new HashMap<>(); + private final Map streamToFields = new HashMap<>(); + + /** + * Create a simple record translator that will use func to extract the fields of the tuple, + * named by fields, and emit them to stream. This will handle all topics not explicitly set + * elsewhere. + * @param func extracts and turns them into a list of objects to be emitted + * @param fields the names of the fields extracted + * @param stream the stream to emit these fields on. + */ + public ByTopicRecordTranslator(Func, List> func, Fields fields, String stream) { + this(new SimpleRecordTranslator<>(func, fields, stream)); + } + + /** + * Create a simple record translator that will use func to extract the fields of the tuple, + * named by fields, and emit them to the default stream. This will handle all topics not explicitly set + * elsewhere. + * @param func extracts and turns them into a list of objects to be emitted + * @param fields the names of the fields extracted + */ + public ByTopicRecordTranslator(Func, List> func, Fields fields) { + this(new SimpleRecordTranslator<>(func, fields)); + } + + /** + * @param defaultTranslator a translator that will be used for all topics not explicitly set + * elsewhere. + */ + public ByTopicRecordTranslator(RecordTranslator defaultTranslator) { + this.defaultTranslator = defaultTranslator; + //This shouldn't throw on a Check, because nothing is configured yet + cacheNCheckFields(defaultTranslator); + } + + /** + * Configure a translator for a given topic with tuples to be emitted to the default stream. + * @param topic the topic this should be used for + * @param func extracts and turns them into a list of objects to be emitted + * @param fields the names of the fields extracted + * @return this to be able to chain configuration + * @throws IllegalStateException if the topic is already registered to another translator + * @throws IllegalArgumentException if the Fields for the stream this emits to do not match any already configured Fields for the same stream + */ + public ByTopicRecordTranslator forTopic(String topic, Func, List> func, Fields fields) { + return forTopic(topic, new SimpleRecordTranslator<>(func, fields)); + } + + /** + * Configure a translator for a given topic. + * @param topic the topic this should be used for + * @param func extracts and turns them into a list of objects to be emitted + * @param fields the names of the fields extracted + * @param stream the stream to emit the tuples to. + * @return this to be able to chain configuration + * @throws IllegalStateException if the topic is already registered to another translator + * @throws IllegalArgumentException if the Fields for the stream this emits to do not match any already configured Fields for the same stream + */ + public ByTopicRecordTranslator forTopic(String topic, Func, List> func, Fields fields, String stream) { + return forTopic(topic, new SimpleRecordTranslator<>(func, fields, stream)); + } + + /** + * Configure a translator for a given kafka topic + * @param topic the topic this translator should handle + * @param translator the translator itself + * @return this to be able to chain configuration + * @throws IllegalStateException if the topic is already registered to another translator + * @throws IllegalArgumentException if the Fields for the stream this emits to do not match any already configured Fields for the same stream + */ + public ByTopicRecordTranslator forTopic(String topic, RecordTranslator translator) { + if (topicToTranslator.containsKey(topic)) { + throw new IllegalStateException("Topic " + topic + " is already registered"); + } + cacheNCheckFields(translator); + topicToTranslator.put(topic, translator); + return this; + } + + private void cacheNCheckFields(RecordTranslator translator) { + for (String stream : translator.streams()) { + Fields fromTrans = translator.getFieldsFor(stream); + Fields cached = streamToFields.get(stream); + if (cached != null && !fromTrans.equals(cached)) { + throw new IllegalArgumentException("Stream " + stream + " currently has Fields of " + cached + " which is not the same as those being added in " + fromTrans); + } + + if (cached == null) { + streamToFields.put(stream, fromTrans); + } + } + } + + @Override + public List apply(ConsumerRecord record) { + RecordTranslator trans = topicToTranslator.get(record.topic()); + if (trans == null) { + trans = defaultTranslator; + } + return trans.apply(record); + } + + @Override + public Fields getFieldsFor(String stream) { + return streamToFields.get(stream); + } + + @Override + public List streams() { + return new ArrayList<>(streamToFields.keySet()); + } +} diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/TopicKeyValueTupleBuilder.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/DefaultRecordTranslator.java similarity index 57% rename from external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/TopicKeyValueTupleBuilder.java rename to external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/DefaultRecordTranslator.java index 4f20b58a2dc..4b0262bd7a2 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/TopicKeyValueTupleBuilder.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/DefaultRecordTranslator.java @@ -15,26 +15,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.storm.kafka.spout.builders; +package org.apache.storm.kafka.spout; + +import java.util.List; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.storm.kafka.spout.KafkaSpoutTupleBuilder; +import org.apache.storm.tuple.Fields; import org.apache.storm.tuple.Values; -import java.util.List; +public class DefaultRecordTranslator implements RecordTranslator { + private static final long serialVersionUID = -5782462870112305750L; + public static final Fields FIELDS = new Fields("topic", "partition", "offset", "key", "value"); + @Override + public List apply(ConsumerRecord record) { + return new Values(record.topic(), + record.partition(), + record.offset(), + record.key(), + record.value()); + } -public class TopicKeyValueTupleBuilder extends KafkaSpoutTupleBuilder { - /** - * @param topics list of topics that use this implementation to build tuples - */ - public TopicKeyValueTupleBuilder(String... topics) { - super(topics); + @Override + public Fields getFieldsFor(String stream) { + return FIELDS; } @Override - public List buildTuple(ConsumerRecord consumerRecord) { - return new Values(consumerRecord.topic(), - consumerRecord.key(), - consumerRecord.value()); + public List streams() { + return DEFAULT_STREAM; } } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetcher.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Func.java similarity index 78% rename from external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetcher.java rename to external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Func.java index 47a61c137c5..a631d96081e 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetcher.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Func.java @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.storm.kafka.spout; -package org.apache.storm.kafka.spout.internal.fetcher; - -import org.apache.kafka.clients.consumer.ConsumerRecords; - -public interface KafkaRecordsFetcher { - ConsumerRecords fetchRecords(long fetchTimeoutMs); +/** + * A simple interface to allow compatibility with non java 8 + * code bases + */ +public interface Func { + R apply(V record); } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java index c92b6eef077..e0a345188ad 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java @@ -18,6 +18,25 @@ package org.apache.storm.kafka.spout; +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 java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.TimeUnit; + import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -30,8 +49,6 @@ 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; @@ -39,33 +56,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.NavigableSet; -import java.util.Set; -import java.util.TreeSet; -import java.util.concurrent.TimeUnit; - -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; - public class KafkaSpout extends BaseRichSpout { + private static final long serialVersionUID = 4151921085047987154L; private static final Logger LOG = LoggerFactory.getLogger(KafkaSpout.class); private static final Comparator OFFSET_COMPARATOR = new OffsetComparator(); // Storm protected SpoutOutputCollector collector; - private TopologyContext topologyContext; // Kafka private final KafkaSpoutConfig kafkaSpoutConfig; @@ -80,16 +77,14 @@ public class KafkaSpout 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 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 - private transient KafkaSpoutTuplesBuilder tuplesBuilder; // Object that contains the logic to build tuples for each ConsumerRecord - transient Map acked; // Tuples that were successfully acked. These tuples will be committed periodically when the commit timer expires, after consumer rebalance, or on close/deactivate private transient Set emitted; // Tuples that have been emitted but that are "on the wire", i.e. pending being acked or failed private transient Iterator> waitingToEmit; // Records that have been polled and are queued to be emitted in the nextTuple() call. One record is emitted per nextTuple() private transient long numUncommittedOffsets; // Number of offsets that have been polled and emitted but not yet been committed + private transient TopologyContext context; + private transient Timer refreshSubscriptionTimer; // Used to say when a subscription should be refreshed public KafkaSpout(KafkaSpoutConfig kafkaSpoutConfig) { @@ -99,16 +94,16 @@ public KafkaSpout(KafkaSpoutConfig kafkaSpoutConfig) { //This constructor is here for testing KafkaSpout(KafkaSpoutConfig kafkaSpoutConfig, KafkaConsumerFactory kafkaConsumerFactory) { this.kafkaSpoutConfig = kafkaSpoutConfig; // Pass in configuration - this.kafkaSpoutStreams = kafkaSpoutConfig.getKafkaSpoutStreams(); this.kafkaConsumerFactory = kafkaConsumerFactory; } @Override public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { initialized = false; + this.context = context; + // Spout internals this.collector = collector; - this.topologyContext = context; maxRetries = kafkaSpoutConfig.getMaxTupleRetries(); numUncommittedOffsets = 0; @@ -119,12 +114,10 @@ public void open(Map conf, TopologyContext context, SpoutOutputCollector collect // Retries management retryService = kafkaSpoutConfig.getRetryService(); - // Tuples builder delegate - tuplesBuilder = kafkaSpoutConfig.getTuplesBuilder(); - if (!consumerAutoCommitMode) { // If it is auto commit, no need to commit offsets manually commitTimer = new Timer(500, kafkaSpoutConfig.getOffsetsCommitPeriodMs(), TimeUnit.MILLISECONDS); } + refreshSubscriptionTimer = new Timer(500, kafkaSpoutConfig.getPartitionRefreshPeriodMs(), TimeUnit.MILLISECONDS); acked = new HashMap<>(); emitted = new HashSet<>(); @@ -187,10 +180,10 @@ private long doSeek(TopicPartition tp, OffsetAndMetadata committedOffset) { long fetchOffset; if (committedOffset != null) { // offset was committed for this TopicPartition if (firstPollOffsetStrategy.equals(EARLIEST)) { - kafkaConsumer.seekToBeginning(toArrayList(tp)); + kafkaConsumer.seekToBeginning(Collections.singleton(tp)); fetchOffset = kafkaConsumer.position(tp); } else if (firstPollOffsetStrategy.equals(LATEST)) { - kafkaConsumer.seekToEnd(toArrayList(tp)); + kafkaConsumer.seekToEnd(Collections.singleton(tp)); fetchOffset = kafkaConsumer.position(tp); } else { // By default polling starts at the last committed offset. +1 to point fetch to the first uncommitted offset. @@ -199,9 +192,9 @@ private long doSeek(TopicPartition tp, OffsetAndMetadata committedOffset) { } } else { // no commits have ever been done, so start at the beginning or end depending on the strategy if (firstPollOffsetStrategy.equals(EARLIEST) || firstPollOffsetStrategy.equals(UNCOMMITTED_EARLIEST)) { - kafkaConsumer.seekToBeginning(toArrayList(tp)); + kafkaConsumer.seekToBeginning(Collections.singleton(tp)); } else if (firstPollOffsetStrategy.equals(LATEST) || firstPollOffsetStrategy.equals(UNCOMMITTED_LATEST)) { - kafkaConsumer.seekToEnd(toArrayList(tp)); + kafkaConsumer.seekToEnd(Collections.singleton(tp)); } fetchOffset = kafkaConsumer.position(tp); } @@ -209,10 +202,6 @@ private long doSeek(TopicPartition tp, OffsetAndMetadata committedOffset) { } } - private Collection toArrayList(final TopicPartition tp) { - return new ArrayList(1){{add(tp);}}; - } - private void setAcked(TopicPartition tp, long fetchOffset) { // If this partition was previously assigned to this spout, leave the acked offsets as they were to resume where it left off if (!consumerAutoCommitMode && !acked.containsKey(tp)) { @@ -290,8 +279,10 @@ public void setWaitingToEmit(ConsumerRecords consumerRecords) { // ======== poll ========= private ConsumerRecords pollKafkaBroker() { doSeekRetriableTopicPartitions(); - - final ConsumerRecords consumerRecords = recordsFetcher.fetchRecords(kafkaSpoutConfig.getPollTimeoutMs()); + if (refreshSubscriptionTimer.isExpiredResetOnTrue()) { + kafkaSpoutConfig.getSubscription().refreshAssignment(); + } + final ConsumerRecords consumerRecords = kafkaConsumer.poll(kafkaSpoutConfig.getPollTimeoutMs()); final int numPolledRecords = consumerRecords.count(); LOG.debug("Polled [{}] records from Kafka. [{}] uncommitted offsets across all topic partitions", numPolledRecords, numUncommittedOffsets); return consumerRecords; @@ -317,7 +308,6 @@ private void emit() { } } - //Emits one tuple per record //@return true if tuple was emitted private boolean emitTupleIfNotEmitted(ConsumerRecord record) { @@ -331,8 +321,12 @@ private boolean emitTupleIfNotEmitted(ConsumerRecord record) { } else { boolean isScheduled = retryService.isScheduled(msgId); if (!isScheduled || retryService.isReady(msgId)) { // not scheduled <=> never failed (i.e. never emitted) or ready to be retried - final List tuple = tuplesBuilder.buildTuple(record); - kafkaSpoutStreams.emit(collector, tuple, msgId); + final List tuple = kafkaSpoutConfig.getTranslator().apply(record); + if (tuple instanceof KafkaTuple) { + collector.emit(((KafkaTuple)tuple).getStream(), tuple, msgId); + } else { + collector.emit(tuple, msgId); + } emitted.add(msgId); numUncommittedOffsets++; if (isScheduled) { // Was scheduled for retry, now being re-emitted. Remove from schedule. @@ -418,8 +412,8 @@ public void activate() { private void subscribeKafkaConsumer() { kafkaConsumer = kafkaConsumerFactory.createConsumer(kafkaSpoutConfig); - recordsFetcher = KafkaRecordsFetchers.create(kafkaSpoutConfig, kafkaConsumer, topologyContext, - new KafkaSpoutConsumerRebalanceListener()); + + kafkaSpoutConfig.getSubscription().subscribe(kafkaConsumer, new KafkaSpoutConsumerRebalanceListener(), context); } @Override @@ -453,7 +447,10 @@ private void shutdown() { @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { - kafkaSpoutStreams.declareOutputFields(declarer); + RecordTranslator translator = kafkaSpoutConfig.getTranslator(); + for (String stream: translator.streams()) { + declarer.declareStream(stream, translator.getFieldsFor(stream)); + } } @Override @@ -472,11 +469,7 @@ public Map getComponentConfiguration () { } String configKeyPrefix = "config."; - if (kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics) { - configuration.put(configKeyPrefix + "topics", getNamedTopics()); - } else if (kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics) { - configuration.put(configKeyPrefix + "topics", getWildCardTopics()); - } + configuration.put(configKeyPrefix + "topics", getTopicsString()); configuration.put(configKeyPrefix + "groupid", kafkaSpoutConfig.getConsumerGroupId()); configuration.put(configKeyPrefix + "bootstrap.servers", kafkaSpoutConfig.getKafkaProps().get("bootstrap.servers")); @@ -484,16 +477,8 @@ public Map getComponentConfiguration () { return configuration; } - private String getNamedTopics() { - StringBuilder topics = new StringBuilder(); - for (String topic: kafkaSpoutConfig.getSubscribedTopics()) { - topics.append(topic).append(","); - } - return topics.toString(); - } - - private String getWildCardTopics() { - return kafkaSpoutConfig.getTopicWildcardPattern().toString(); + private String getTopicsString() { + return kafkaSpoutConfig.getSubscription().getTopicsString(); } // ======= Offsets Commit Management ========== @@ -597,7 +582,7 @@ public boolean isEmpty() { return ackedMsgs.isEmpty(); } - public boolean contains(ConsumerRecord record) { + public boolean contains(ConsumerRecord record) { return contains(new KafkaSpoutMessageId(record)); } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java index 5daf13fcee3..db07fdac9f2 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java @@ -18,35 +18,40 @@ package org.apache.storm.kafka.spout; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff.TimeInterval; - import java.io.Serializable; -import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.regex.Pattern; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff.TimeInterval; +import org.apache.storm.tuple.Fields; + /** * KafkaSpoutConfig defines the required configuration to connect a consumer to a consumer group, as well as the subscribing topics */ public class KafkaSpoutConfig implements Serializable { + private static final long serialVersionUID = 141902646130682494L; public static final long DEFAULT_POLL_TIMEOUT_MS = 200; // 200ms 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 { - String GROUP_ID = "group.id"; - String BOOTSTRAP_SERVERS = "bootstrap.servers"; - String ENABLE_AUTO_COMMIT = "enable.auto.commit"; - String AUTO_COMMIT_INTERVAL_MS = "auto.commit.interval.ms"; - String KEY_DESERIALIZER = "key.deserializer"; - String VALUE_DESERIALIZER = "value.deserializer"; - } + public static final KafkaSpoutRetryService DEFAULT_RETRY_SERVICE = + new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2), + DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)); + /** + * Retry in a tight loop (keep unit tests fasts) do not use in production. + */ + public static final KafkaSpoutRetryService UNIT_TEST_RETRY_SERVICE = + new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(0), + DEFAULT_MAX_RETRIES, TimeInterval.milliSeconds(0)); /** * The offset used by the Kafka spout in the first poll to Kafka broker. The choice of this parameter will @@ -61,129 +66,254 @@ public interface Consumer { * If no offset has been committed, it behaves as LATEST. * * */ - public enum FirstPollOffsetStrategy { + public static enum FirstPollOffsetStrategy { EARLIEST, LATEST, UNCOMMITTED_EARLIEST, UNCOMMITTED_LATEST } - - // Kafka consumer configuration - private final Map kafkaProps; - private final Deserializer keyDeserializer; - private final Deserializer valueDeserializer; - private final long pollTimeoutMs; - - // Kafka spout configuration - 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 tuplesBuilder; - private final KafkaSpoutRetryService retryService; - - private KafkaSpoutConfig(Builder builder) { - this.kafkaProps = setDefaultsAndGetKafkaProps(builder.kafkaProps); - this.keyDeserializer = builder.keyDeserializer; - this.valueDeserializer = builder.valueDeserializer; - this.pollTimeoutMs = builder.pollTimeoutMs; - this.offsetCommitPeriodMs = builder.offsetCommitPeriodMs; - this.maxRetries = builder.maxRetries; - 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; + + public static Builder builder(String bootstrapServers, String ... topics) { + return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics); } - - private Map setDefaultsAndGetKafkaProps(Map kafkaProps) { + + public static Builder builder(String bootstrapServers, Collection topics) { + return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics); + } + + public static Builder builder(String bootstrapServers, Pattern topics) { + return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics); + } + + private static Map setDefaultsAndGetKafkaProps(Map kafkaProps) { // set defaults for properties not specified - if (!kafkaProps.containsKey(Consumer.ENABLE_AUTO_COMMIT)) { - kafkaProps.put(Consumer.ENABLE_AUTO_COMMIT, "false"); + if (!kafkaProps.containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) { + kafkaProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); } return kafkaProps; } - + public static class Builder { private final Map kafkaProps; - private SerializableDeserializer keyDeserializer; - private SerializableDeserializer valueDeserializer; + private Subscription subscription; + private final SerializableDeserializer keyDes; + private final Class> keyDesClazz; + private final SerializableDeserializer valueDes; + private final Class> valueDesClazz; + private RecordTranslator translator; private long pollTimeoutMs = DEFAULT_POLL_TIMEOUT_MS; private long offsetCommitPeriodMs = DEFAULT_OFFSET_COMMIT_PERIOD_MS; private int maxRetries = DEFAULT_MAX_RETRIES; private FirstPollOffsetStrategy firstPollOffsetStrategy = FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST; - private final KafkaSpoutStreams kafkaSpoutStreams; private int maxUncommittedOffsets = DEFAULT_MAX_UNCOMMITTED_OFFSETS; + private KafkaSpoutRetryService retryService = DEFAULT_RETRY_SERVICE; private long partitionRefreshPeriodMs = DEFAULT_PARTITION_REFRESH_PERIOD_MS; - private boolean manualPartitionAssignment = false; - private final KafkaSpoutTuplesBuilder tuplesBuilder; - private final KafkaSpoutRetryService retryService; + + public Builder(String bootstrapServers, SerializableDeserializer keyDes, SerializableDeserializer valDes, String ... topics) { + this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics)); + } + + public Builder(String bootstrapServers, SerializableDeserializer keyDes, SerializableDeserializer valDes, Collection topics) { + this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics)); + } + + public Builder(String bootstrapServers, SerializableDeserializer keyDes, SerializableDeserializer valDes, Pattern topics) { + this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics)); + } + + public Builder(String bootstrapServers, SerializableDeserializer keyDes, SerializableDeserializer valDes, Subscription subscription) { + this(bootstrapServers, keyDes, null, valDes, null, subscription); + } + + public Builder(String bootstrapServers, Class> keyDes, Class> valDes, String ... topics) { + this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics)); + } + + public Builder(String bootstrapServers, Class> keyDes, Class> valDes, Collection topics) { + this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics)); + } + + public Builder(String bootstrapServers, Class> keyDes, Class> valDes, Pattern topics) { + this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics)); + } + + public Builder(String bootstrapServers, Class> keyDes, Class> valDes, Subscription subscription) { + this(bootstrapServers, null, keyDes, null, valDes, subscription); + } + + private Builder(String bootstrapServers, SerializableDeserializer keyDes, Class> keyDesClazz, + SerializableDeserializer valDes, Class> valDesClazz, Subscription subscription) { + kafkaProps = new HashMap<>(); + if (bootstrapServers == null || bootstrapServers.isEmpty()) { + throw new IllegalArgumentException("bootstrap servers cannot be null"); + } + kafkaProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + this.keyDes = keyDes; + this.keyDesClazz = keyDesClazz; + this.valueDes = valDes; + this.valueDesClazz = valDesClazz; + this.subscription = subscription; + this.translator = new DefaultRecordTranslator(); + } + + private Builder(Builder builder, SerializableDeserializer keyDes, Class> keyDesClazz, + SerializableDeserializer valueDes, Class> valueDesClazz) { + this.kafkaProps = new HashMap<>(builder.kafkaProps); + this.subscription = builder.subscription; + this.pollTimeoutMs = builder.pollTimeoutMs; + this.offsetCommitPeriodMs = builder.offsetCommitPeriodMs; + this.maxRetries = builder.maxRetries; + this.firstPollOffsetStrategy = builder.firstPollOffsetStrategy; + this.maxUncommittedOffsets = builder.maxUncommittedOffsets; + //this could result in a lot of class case exceptions at runtime, + // but because some translators will work no matter what the generics + // are I thought it best not to force someone to reset the translator + // when they change the key/value types. + this.translator = (RecordTranslator) builder.translator; + this.retryService = builder.retryService; + this.keyDes = keyDes; + this.keyDesClazz = keyDesClazz; + this.valueDes = valueDes; + this.valueDesClazz = valueDesClazz; + } /** - * Please refer to javadoc in {@link #Builder(Map, KafkaSpoutStreams, KafkaSpoutTuplesBuilder, KafkaSpoutRetryService)}.

- * This constructor uses by the default the following implementation for {@link KafkaSpoutRetryService}:

- * {@code new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2), - * DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)))} + * Specifying this key deserializer overrides the property key.deserializer. If you have + * set a custom RecordTranslator before calling this it may result in class cast + * exceptions at runtime. */ - public Builder(Map kafkaProps, KafkaSpoutStreams kafkaSpoutStreams, - KafkaSpoutTuplesBuilder tuplesBuilder) { - this(kafkaProps, kafkaSpoutStreams, tuplesBuilder, - new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2), - DEFAULT_MAX_RETRIES, TimeInterval.seconds(10))); + public Builder setKey(SerializableDeserializer keyDeserializer) { + return new Builder<>(this, keyDeserializer, null, valueDes, valueDesClazz); } - - /*** - * KafkaSpoutConfig defines the required configuration to connect a consumer to a consumer group, as well as the subscribing topics - * The optional configuration can be specified using the set methods of this builder - * @param kafkaProps properties defining consumer connection to Kafka broker as specified in @see KafkaConsumer - * @param kafkaSpoutStreams streams to where the tuples are emitted for each tuple. Multiple topics can emit in the same stream. - * @param tuplesBuilder logic to build tuples from {@link ConsumerRecord}s. - * @param retryService logic that manages the retrial of failed tuples + + /** + * Specify a class that can be instantiated to create a key.deserializer + * This is the same as setting key.deserializer, but overrides it. If you have + * set a custom RecordTranslator before calling this it may result in class cast + * exceptions at runtime. */ - public Builder(Map kafkaProps, KafkaSpoutStreams kafkaSpoutStreams, - KafkaSpoutTuplesBuilder tuplesBuilder, KafkaSpoutRetryService retryService) { - if (kafkaProps == null || kafkaProps.isEmpty()) { - throw new IllegalArgumentException("Properties defining consumer connection to Kafka broker are required: " + kafkaProps); - } - - if (kafkaSpoutStreams == null) { - throw new IllegalArgumentException("Must specify stream associated with each topic. Multiple topics can emit to the same stream"); - } - - if (tuplesBuilder == null) { - throw new IllegalArgumentException("Must specify at last one tuple builder per topic declared in KafkaSpoutStreams"); - } - - if (retryService == null) { - throw new IllegalArgumentException("Must specify at implementation of retry service"); - } - - this.kafkaProps = kafkaProps; - this.kafkaSpoutStreams = kafkaSpoutStreams; - this.tuplesBuilder = tuplesBuilder; - this.retryService = retryService; + public Builder setKey(Class> clazz) { + return new Builder<>(this, null, clazz, valueDes, valueDesClazz); } /** - * Specifying this key deserializer overrides the property key.deserializer + * Specifying this value deserializer overrides the property value.deserializer. If you have + * set a custom RecordTranslator before calling this it may result in class cast + * exceptions at runtime. */ - public Builder setKeyDeserializer(SerializableDeserializer keyDeserializer) { - this.keyDeserializer = keyDeserializer; + public Builder setValue(SerializableDeserializer valueDeserializer) { + return new Builder<>(this, keyDes, keyDesClazz, valueDeserializer, null); + } + + /** + * Specify a class that can be instantiated to create a value.deserializer + * This is the same as setting value.deserializer, but overrides it. If you have + * set a custom RecordTranslator before calling this it may result in class cast + * exceptions at runtime. + */ + public Builder setValue(Class> clazz) { + return new Builder<>(this, keyDes, keyDesClazz, null, clazz); + } + + /** + * Set a Kafka property config + */ + public Builder setProp(String key, Object value) { + kafkaProps.put(key, value); return this; } - + + /** + * Set multiple Kafka property configs + */ + public Builder setProp(Map props) { + kafkaProps.putAll(props); + return this; + } + /** - * Specifying this value deserializer overrides the property value.deserializer + * Set multiple Kafka property configs */ - public Builder setValueDeserializer(SerializableDeserializer valueDeserializer) { - this.valueDeserializer = valueDeserializer; + public Builder setProp(Properties props) { + for (String name: props.stringPropertyNames()) { + kafkaProps.put(name, props.get(name)); + } return this; } + + /** + * Set the group.id for the consumers + */ + public Builder setGroupId(String id) { + return setProp("group.id", id); + } + + /** + * reset the bootstrap servers for the Consumer + */ + public Builder setBootstrapServers(String servers) { + return setProp(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, servers); + } + + /** + * The minimum amount of data the broker should return for a fetch request. + */ + public Builder setFetchMinBytes(int bytes) { + return setProp(ConsumerConfig.FETCH_MIN_BYTES_CONFIG, bytes); + } + + /** + * The maximum amount of data per-partition the broker will return. + */ + public Builder setMaxPartitionFectchBytes(int bytes) { + return setProp(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, bytes); + } + + /** + * The maximum number of records a poll will return. + * Will only work with Kafka 0.10.0 and above. + */ + public Builder setMaxPollRecords(int records) { + //to avoid issues with 0.9 versions that technically still work + // with this we do not use ConsumerConfig.MAX_POLL_RECORDS_CONFIG + return setProp("max.poll.records", records); + } + + //Security Related Configs + + /** + * Configure the SSL Keystore for mutual authentication + */ + public Builder setSSLKeystore(String location, String password) { + return setProp("ssl.keystore.location", location) + .setProp("ssl.keystore.password", password); + } + + /** + * Configure the SSL Keystore for mutual authentication + */ + public Builder setSSLKeystore(String location, String password, String keyPassword) { + return setProp("ssl.key.password", keyPassword) + .setSSLKeystore(location, password); + } + + /** + * Configure the SSL Truststore to authenticate with the brokers + */ + public Builder setSSLTruststore(String location, String password) { + return setSecurityProtocol("SSL") + .setProp("ssl.truststore.location", location) + .setProp("ssl.truststore.password", password); + } + + /** + * Protocol used to communicate with brokers. + * Valid values are: PLAINTEXT, SSL, SASL_PLAINTEXT, SASL_SSL. + */ + public Builder setSecurityProtocol(String protocol) { + return setProp("security.protocol", protocol); + } + //Spout Settings /** * Specifies the time, in milliseconds, spent waiting in poll if data is not available. Default is 2s * @param pollTimeoutMs time in ms @@ -235,41 +365,131 @@ public Builder setFirstPollOffsetStrategy(FirstPollOffsetStrategy firstPol this.firstPollOffsetStrategy = firstPollOffsetStrategy; return this; } - + /** - * Sets partition refresh period in milliseconds in manual partition assignment model. Default is 2s. - * @param partitionRefreshPeriodMs time in milliseconds + * Sets the retry service for the spout to use. + * @param retryService the new retry service + * @return the builder (this). */ - public Builder setPartitionRefreshPeriodMs(long partitionRefreshPeriodMs) { - this.partitionRefreshPeriodMs = partitionRefreshPeriodMs; + public Builder setRetry(KafkaSpoutRetryService retryService) { + if (retryService == null) { + throw new NullPointerException("retryService cannot be null"); + } + this.retryService = retryService; return this; } + public Builder setRecordTranslator(RecordTranslator translator) { + this.translator = translator; + return this; + } + + /** + * Configure a translator with tuples to be emitted on the default stream. + * @param func extracts and turns a Kafka ConsumerRecord into a list of objects to be emitted + * @param fields the names of the fields extracted + * @return this to be able to chain configuration + */ + public Builder setRecordTranslator(Func, List> func, Fields fields) { + return setRecordTranslator(new SimpleRecordTranslator<>(func, fields)); + } + /** - * 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. + * Configure a translator with tuples to be emitted to a given stream. + * @param func extracts and turns a Kafka ConsumerRecord into a list of objects to be emitted + * @param fields the names of the fields extracted + * @param stream the stream to emit the tuples on + * @return this to be able to chain configuration */ - public Builder setManualPartitionAssignment(boolean manualPartitionAssignment) { - this.manualPartitionAssignment = manualPartitionAssignment; + public Builder setRecordTranslator(Func, List> func, Fields fields, String stream) { + return setRecordTranslator(new SimpleRecordTranslator<>(func, fields, stream)); + } + + /** + * Sets partition refresh period in milliseconds. This is how often kafka will be polled + * to check for new topics and/or new partitions. + * This is mostly for Subscription implementations that manually assign partitions. NamedSubscription and + * PatternSubscription rely on kafka to handle this instead. + * @param partitionRefreshPeriodMs time in milliseconds + * @return the builder (this) + */ + public Builder setPartitionRefreshPeriodMs(long partitionRefreshPeriodMs) { + this.partitionRefreshPeriodMs = partitionRefreshPeriodMs; return this; } - + public KafkaSpoutConfig build() { return new KafkaSpoutConfig<>(this); } } + // Kafka consumer configuration + private final Map kafkaProps; + private final Subscription subscription; + private final SerializableDeserializer keyDes; + private final Class> keyDesClazz; + private final SerializableDeserializer valueDes; + private final Class> valueDesClazz; + private final long pollTimeoutMs; + + // Kafka spout configuration + private final RecordTranslator translator; + private final long offsetCommitPeriodMs; + private final int maxRetries; + private final int maxUncommittedOffsets; + private final FirstPollOffsetStrategy firstPollOffsetStrategy; + private final KafkaSpoutRetryService retryService; + private final long partitionRefreshPeriodMs; + + private KafkaSpoutConfig(Builder builder) { + this.kafkaProps = setDefaultsAndGetKafkaProps(builder.kafkaProps); + this.subscription = builder.subscription; + this.translator = builder.translator; + this.pollTimeoutMs = builder.pollTimeoutMs; + this.offsetCommitPeriodMs = builder.offsetCommitPeriodMs; + this.maxRetries = builder.maxRetries; + this.firstPollOffsetStrategy = builder.firstPollOffsetStrategy; + this.maxUncommittedOffsets = builder.maxUncommittedOffsets; + this.retryService = builder.retryService; + this.keyDes = builder.keyDes; + this.keyDesClazz = builder.keyDesClazz; + this.valueDes = builder.valueDes; + this.valueDesClazz = builder.valueDesClazz; + this.partitionRefreshPeriodMs = builder.partitionRefreshPeriodMs; + } + public Map getKafkaProps() { return kafkaProps; } public Deserializer getKeyDeserializer() { - return keyDeserializer; + if (keyDesClazz != null) { + try { + return keyDesClazz.newInstance(); + } catch (InstantiationException | IllegalAccessException e) { + throw new RuntimeException("Could not instantiate key deserializer " + keyDesClazz); + } + } + return keyDes; } public Deserializer getValueDeserializer() { - return valueDeserializer; + if (valueDesClazz != null) { + try { + return valueDesClazz.newInstance(); + } catch (InstantiationException | IllegalAccessException e) { + throw new RuntimeException("Could not instantiate value deserializer " + valueDesClazz); + } + } + return valueDes; + } + + public Subscription getSubscription() { + return subscription; + } + + public RecordTranslator getTranslator() { + return translator; } public long getPollTimeoutMs() { @@ -281,32 +501,12 @@ public long getOffsetsCommitPeriodMs() { } public boolean isConsumerAutoCommitMode() { - return kafkaProps.get(Consumer.ENABLE_AUTO_COMMIT) == null // default is true - || Boolean.valueOf((String)kafkaProps.get(Consumer.ENABLE_AUTO_COMMIT)); + return kafkaProps.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG) == null // default is true + || Boolean.valueOf((String)kafkaProps.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)); } public String getConsumerGroupId() { - return (String) kafkaProps.get(Consumer.GROUP_ID); - } - - /** - * @return list of topics subscribed and emitting tuples to a stream as configured by {@link KafkaSpoutStream}, - * or null if this stream is associated with a wildcard pattern topic - */ - public List getSubscribedTopics() { - return kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics ? - new ArrayList<>(((KafkaSpoutStreamsNamedTopics) kafkaSpoutStreams).getTopics()) : - null; - } - - /** - * @return the wildcard pattern topic associated with this {@link KafkaSpoutStream}, or null - * if this stream is associated with a specific named topic - */ - public Pattern getTopicWildcardPattern() { - return kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics ? - ((KafkaSpoutStreamsWildcardTopics)kafkaSpoutStreams).getTopicWildcardPattern() : - null; + return (String) kafkaProps.get(ConsumerConfig.GROUP_ID_CONFIG); } public int getMaxTupleRetries() { @@ -317,46 +517,32 @@ public FirstPollOffsetStrategy getFirstPollOffsetStrategy() { return firstPollOffsetStrategy; } - public KafkaSpoutStreams getKafkaSpoutStreams() { - return kafkaSpoutStreams; - } - public int getMaxUncommittedOffsets() { return maxUncommittedOffsets; } - public KafkaSpoutTuplesBuilder getTuplesBuilder() { - return tuplesBuilder; - } - public KafkaSpoutRetryService getRetryService() { return retryService; } - + public long getPartitionRefreshPeriodMs() { return partitionRefreshPeriodMs; } - public boolean isManualPartitionAssignment() { - return manualPartitionAssignment; - } - @Override public String toString() { return "KafkaSpoutConfig{" + "kafkaProps=" + kafkaProps + - ", keyDeserializer=" + keyDeserializer + - ", valueDeserializer=" + valueDeserializer + + ", key=" + getKeyDeserializer() + + ", value=" + getValueDeserializer() + ", pollTimeoutMs=" + pollTimeoutMs + ", offsetCommitPeriodMs=" + offsetCommitPeriodMs + ", maxRetries=" + maxRetries + ", maxUncommittedOffsets=" + maxUncommittedOffsets + ", firstPollOffsetStrategy=" + firstPollOffsetStrategy + - ", kafkaSpoutStreams=" + kafkaSpoutStreams + - ", tuplesBuilder=" + tuplesBuilder + + ", subscription=" + subscription + + ", translator=" + translator + ", retryService=" + retryService + - ", topics=" + getSubscribedTopics() + - ", topicWildcardPattern=" + getTopicWildcardPattern() + '}'; } } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutMessageId.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutMessageId.java index 71f83276059..3cfad9d4c39 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutMessageId.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutMessageId.java @@ -26,7 +26,7 @@ public class KafkaSpoutMessageId { private transient long offset; private transient int numFails = 0; - public KafkaSpoutMessageId(ConsumerRecord consumerRecord) { + public KafkaSpoutMessageId(ConsumerRecord consumerRecord) { this(new TopicPartition(consumerRecord.topic(), consumerRecord.partition()), consumerRecord.offset()); } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStream.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStream.java deleted file mode 100644 index 0f444b40721..00000000000 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStream.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * 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; - -import org.apache.storm.spout.SpoutOutputCollector; -import org.apache.storm.topology.OutputFieldsDeclarer; -import org.apache.storm.tuple.Fields; -import org.apache.storm.utils.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.Serializable; -import java.util.List; -import java.util.regex.Pattern; - -/** - * Represents the stream and output fields used by a topic - */ -public class KafkaSpoutStream implements Serializable { - private static final Logger LOG = LoggerFactory.getLogger(KafkaSpoutStream.class); - - private final Fields outputFields; - private final String streamId; - private final String topic; - private Pattern topicWildcardPattern; - - /** Represents the specified outputFields and topic with the default stream */ - public KafkaSpoutStream(Fields outputFields, String topic) { - this(outputFields, Utils.DEFAULT_STREAM_ID, topic); - } - - /** Represents the specified outputFields and topic with the specified stream */ - public KafkaSpoutStream(Fields outputFields, String streamId, String topic) { - if (outputFields == null || streamId == null || topic == null) { - throw new IllegalArgumentException(String.format("Constructor parameters cannot be null. " + - "[outputFields=%s, streamId=%s, topic=%s]", outputFields, streamId, topic)); - } - this.outputFields = outputFields; - this.streamId = streamId; - this.topic = topic; - this.topicWildcardPattern = null; - } - - /** Represents the specified outputFields and topic wild card with the default stream */ - public KafkaSpoutStream(Fields outputFields, Pattern topicWildcardPattern) { - this(outputFields, Utils.DEFAULT_STREAM_ID, topicWildcardPattern); - } - - /** Represents the specified outputFields and topic wild card with the specified stream */ - public KafkaSpoutStream(Fields outputFields, String streamId, Pattern topicWildcardPattern) { - - if (outputFields == null || streamId == null || topicWildcardPattern == null) { - throw new IllegalArgumentException(String.format("Constructor parameters cannot be null. " + - "[outputFields=%s, streamId=%s, topicWildcardPattern=%s]", outputFields, streamId, topicWildcardPattern)); - } - this.outputFields = outputFields; - this.streamId = streamId; - this.topic = null; - this.topicWildcardPattern = topicWildcardPattern; - } - - public void emit(SpoutOutputCollector collector, List tuple, KafkaSpoutMessageId messageId) { - collector.emit(streamId, tuple, messageId); - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - LOG.info("Declared [streamId = {}], [outputFields = {}] for [topic = {}]", streamId, outputFields, topic); - declarer.declareStream(streamId, outputFields); - } - - - public Fields getOutputFields() { - return outputFields; - } - - public String getStreamId() { - return streamId; - } - - /** - * @return the topic associated with this {@link KafkaSpoutStream}, or null - * if this stream is associated with a wildcard pattern topic - */ - public String getTopic() { - return topic; - } - - /** - * @return the wildcard pattern topic associated with this {@link KafkaSpoutStream}, or null - * if this stream is associated with a specific named topic - */ - public Pattern getTopicWildcardPattern() { - return topicWildcardPattern; - } - - @Override - public String toString() { - return "KafkaSpoutStream{" + - "outputFields=" + outputFields + - ", streamId='" + streamId + '\'' + - ", topic='" + topic + '\'' + - ", topicWildcardPattern=" + topicWildcardPattern + - '}'; - } -} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopics.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopics.java deleted file mode 100644 index bc2426a5f31..00000000000 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopics.java +++ /dev/null @@ -1,165 +0,0 @@ -/* - * 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; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.storm.spout.SpoutOutputCollector; -import org.apache.storm.topology.OutputFieldsDeclarer; -import org.apache.storm.topology.OutputFieldsGetter; -import org.apache.storm.tuple.Fields; -import org.apache.storm.utils.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Represents the {@link KafkaSpoutStream} associated with each topic, and provides a public API to - * declare output streams and emmit tuples, on the appropriate stream, for all the topics specified. - */ -public class KafkaSpoutStreamsNamedTopics implements KafkaSpoutStreams { - private static final Logger LOG = LoggerFactory.getLogger(KafkaSpoutStreamsNamedTopics.class); - - private final Map topicToStream; - - private KafkaSpoutStreamsNamedTopics(Builder builder) { - this.topicToStream = builder.topicToStream; - LOG.debug("Built {}", this); - } - - /** - * @param topic the topic for which to get output fields - * @return the declared output fields - */ - public Fields getOutputFields(String topic) { - if (topicToStream.containsKey(topic)) { - final Fields outputFields = topicToStream.get(topic).getOutputFields(); - LOG.trace("Topic [{}] has output fields [{}]", topic, outputFields); - return outputFields; - } - throw new IllegalStateException(this.getClass().getName() + " not configured for topic: " + topic); - } - - @Override - public Fields getOutputFields() { - final Set allFields = new LinkedHashSet<>(); - for (KafkaSpoutStream kafkaSpoutStream : topicToStream.values()) { - allFields.addAll(kafkaSpoutStream.getOutputFields().toList()); - } - return new Fields(new ArrayList<>(allFields)); - } - - /** - * @param topic the topic to for which to get the stream id - * @return the id of the stream to where the tuples are emitted - */ - public KafkaSpoutStream getStream(String topic) { - if (topicToStream.containsKey(topic)) { - return topicToStream.get(topic); - } - throw new IllegalStateException(this.getClass().getName() + " not configured for topic: " + topic); - } - - /** - * @return list of topics subscribed and emitting tuples to a stream as configured by {@link KafkaSpoutStream} - */ - public List getTopics() { - return new ArrayList<>(topicToStream.keySet()); - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - for (KafkaSpoutStream stream : topicToStream.values()) { - if (!((OutputFieldsGetter)declarer).getFieldsDeclaration().containsKey(stream.getStreamId())) { - stream.declareOutputFields(declarer); - } - } - } - - public void emit(SpoutOutputCollector collector, List tuple, KafkaSpoutMessageId messageId) { - getStream(messageId.topic()).emit(collector, tuple, messageId); - } - - @Override - public String toString() { - return "KafkaSpoutStreamsNamedTopics{" + - "topicToStream=" + topicToStream + - '}'; - } - - public static class Builder { - private final Map topicToStream = new HashMap<>();; - - /** - * Creates a {@link KafkaSpoutStream} with the given output Fields for each topic specified. - * All topics will have the default stream id and the same output fields. - */ - public Builder(Fields outputFields, String... topics) { - addStream(outputFields, topics); - } - - /** - * Creates a {@link KafkaSpoutStream} with this particular stream for each topic specified. - * All the topics will have the specified stream id and the same output fields. - */ - public Builder (Fields outputFields, String streamId, String... topics) { - addStream(outputFields, streamId, topics); - } - - /** - * Adds this stream to the state representing the streams associated with each topic - */ - public Builder(KafkaSpoutStream stream) { - addStream(stream); - } - - /** - * Adds this stream to the state representing the streams associated with each topic - */ - public Builder addStream(KafkaSpoutStream stream) { - topicToStream.put(stream.getTopic(), stream); - return this; - } - - /** - * Please refer to javadoc in {@link #Builder(Fields, String...)} - */ - public Builder addStream(Fields outputFields, String... topics) { - addStream(outputFields, Utils.DEFAULT_STREAM_ID, topics); - return this; - } - - /** - * Please refer to javadoc in {@link #Builder(Fields, String, String...)} - */ - public Builder addStream(Fields outputFields, String streamId, String... topics) { - for (String topic : topics) { - topicToStream.put(topic, new KafkaSpoutStream(outputFields, streamId, topic)); - } - return this; - } - - public KafkaSpoutStreamsNamedTopics build() { - return new KafkaSpoutStreamsNamedTopics(this); - } - } -} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsWildcardTopics.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsWildcardTopics.java deleted file mode 100644 index 64132b37139..00000000000 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsWildcardTopics.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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; - -import org.apache.storm.spout.SpoutOutputCollector; -import org.apache.storm.topology.OutputFieldsDeclarer; -import org.apache.storm.tuple.Fields; - -import java.util.List; -import java.util.regex.Pattern; - -public class KafkaSpoutStreamsWildcardTopics implements KafkaSpoutStreams { - private KafkaSpoutStream kafkaSpoutStream; - - public KafkaSpoutStreamsWildcardTopics(KafkaSpoutStream kafkaSpoutStream) { - this.kafkaSpoutStream = kafkaSpoutStream; - if (kafkaSpoutStream.getTopicWildcardPattern() == null) { - throw new IllegalStateException("KafkaSpoutStream must be configured for wildcard topic"); - } - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - kafkaSpoutStream.declareOutputFields(declarer); - } - - @Override - public void emit(SpoutOutputCollector collector, List tuple, KafkaSpoutMessageId messageId) { - kafkaSpoutStream.emit(collector, tuple, messageId); - } - - @Override - public Fields getOutputFields() { - return kafkaSpoutStream.getOutputFields(); - } - - public KafkaSpoutStream getStream() { - return kafkaSpoutStream; - } - - public Pattern getTopicWildcardPattern() { - return kafkaSpoutStream.getTopicWildcardPattern(); - } - - @Override - public String toString() { - return "KafkaSpoutStreamsWildcardTopics{" + - "kafkaSpoutStream=" + kafkaSpoutStream + - '}'; - } -} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTupleBuilder.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTupleBuilder.java deleted file mode 100644 index 3bb71a859c5..00000000000 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTupleBuilder.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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; - -import org.apache.kafka.clients.consumer.ConsumerRecord; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -/** - * Implementations of {@link KafkaSpoutTupleBuilder} contain the logic to build tuples from {@link ConsumerRecord}s. - * Users must subclass this abstract class to provide their implementation. See also {@link KafkaSpoutTuplesBuilder} - */ -public abstract class KafkaSpoutTupleBuilder implements Serializable { - private List topics; - - /** - * @param topics list of topics that use this implementation to build tuples - */ - public KafkaSpoutTupleBuilder(String... topics) { - if (topics == null || topics.length == 0) { - throw new IllegalArgumentException("Must specify at least one topic. It cannot be null or empty"); - } - this.topics = Arrays.asList(topics); - } - - /** - * @return list of topics that use this implementation to build tuples - */ - public List getTopics() { - return Collections.unmodifiableList(topics); - } - - /** - * Builds a list of tuples using the ConsumerRecord specified as parameter - * @param consumerRecord whose contents are used to build tuples - * @return list of tuples - */ - public abstract List buildTuple(ConsumerRecord consumerRecord); -} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderNamedTopics.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderNamedTopics.java deleted file mode 100644 index 80fe5436b34..00000000000 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderNamedTopics.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class KafkaSpoutTuplesBuilderNamedTopics implements KafkaSpoutTuplesBuilder { - private static final Logger LOG = LoggerFactory.getLogger(KafkaSpoutTuplesBuilderNamedTopics.class); - - private Map> topicToTupleBuilders; - - private KafkaSpoutTuplesBuilderNamedTopics(Builder builder) { - this.topicToTupleBuilders = builder.topicToTupleBuilders; - LOG.debug("Instantiated {}", this); - } - - public static class Builder { - private List> tupleBuilders; - private Map> topicToTupleBuilders; - - @SafeVarargs - public Builder(KafkaSpoutTupleBuilder... tupleBuilders) { - if (tupleBuilders == null || tupleBuilders.length == 0) { - throw new IllegalArgumentException("Must specify at last one tuple builder per topic declared in KafkaSpoutStreams"); - } - - this.tupleBuilders = Arrays.asList(tupleBuilders); - topicToTupleBuilders = new HashMap<>(); - } - - public KafkaSpoutTuplesBuilderNamedTopics build() { - for (KafkaSpoutTupleBuilder tupleBuilder : tupleBuilders) { - for (String topic : tupleBuilder.getTopics()) { - if (!topicToTupleBuilders.containsKey(topic)) { - topicToTupleBuilders.put(topic, tupleBuilder); - } - } - } - return new KafkaSpoutTuplesBuilderNamedTopics<>(this); - } - } - - public ListbuildTuple(ConsumerRecord consumerRecord) { - final String topic = consumerRecord.topic(); - return topicToTupleBuilders.get(topic).buildTuple(consumerRecord); - } - - @Override - public String toString() { - return "KafkaSpoutTuplesBuilderNamedTopics {" + - "topicToTupleBuilders=" + topicToTupleBuilders + - '}'; - } - -} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilder.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTuple.java similarity index 57% rename from external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilder.java rename to external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTuple.java index 2ba0a79cdd4..f5953aded26 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilder.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTuple.java @@ -15,18 +15,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.storm.kafka.spout; -import org.apache.kafka.clients.consumer.ConsumerRecord; - -import java.io.Serializable; -import java.util.List; +import org.apache.storm.tuple.Values; /** - * {@link KafkaSpoutTuplesBuilder} wraps all the logic that builds tuples from {@link ConsumerRecord}s. - * The logic is provided by the user by implementing the appropriate number of {@link KafkaSpoutTupleBuilder} instances + * A list of Values in a tuple that can be routed + * to a given stream. {@see org.apache.storm.kafka.spout.RecordTranslator#apply} */ -public interface KafkaSpoutTuplesBuilder extends Serializable { - List buildTuple(ConsumerRecord consumerRecord); +public class KafkaTuple extends Values { + private static final long serialVersionUID = 4803794470450587992L; + private String stream = null; + + public KafkaTuple() { + super(); + } + + public KafkaTuple(Object... vals) { + super(vals); + } + + public KafkaTuple routedTo(String stream) { + assert(this.stream == null); + this.stream = stream; + return this; + } + + public String getStream() { + return stream; + } } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionNamedSubscription.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionNamedSubscription.java new file mode 100644 index 00000000000..df3e8008ca4 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionNamedSubscription.java @@ -0,0 +1,78 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.storm.task.TopologyContext; + +public class ManualPartitionNamedSubscription extends NamedSubscription { + private static final long serialVersionUID = 5633018073527583826L; + private final ManualPartitioner partitioner; + private Set currentAssignment = null; + private KafkaConsumer consumer = null; + private ConsumerRebalanceListener listener = null; + private TopologyContext context = null; + + public ManualPartitionNamedSubscription(ManualPartitioner parter, Collection topics) { + super(topics); + this.partitioner = parter; + } + + public ManualPartitionNamedSubscription(ManualPartitioner parter, String ... topics) { + this(parter, Arrays.asList(topics)); + } + + @Override + public void subscribe(KafkaConsumer consumer, ConsumerRebalanceListener listener, TopologyContext context) { + this.consumer = consumer; + this.listener = listener; + this.context = context; + refreshAssignment(); + } + + @Override + public void refreshAssignment() { + List allPartitions = new ArrayList<>(); + for (String topic : topics) { + for (PartitionInfo partitionInfo: consumer.partitionsFor(topic)) { + allPartitions.add(new TopicPartition(partitionInfo.topic(), partitionInfo.partition())); + } + } + Collections.sort(allPartitions, TopicPartitionComparator.INSTANCE); + Set newAssignment = new HashSet<>(partitioner.partition(allPartitions, context)); + if (!newAssignment.equals(currentAssignment)) { + if (currentAssignment != null) { + listener.onPartitionsRevoked(currentAssignment); + listener.onPartitionsAssigned(newAssignment); + } + currentAssignment = newAssignment; + consumer.assign(currentAssignment); + } + } +} \ No newline at end of file diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionPatternSubscription.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionPatternSubscription.java new file mode 100644 index 00000000000..cf4dfcbe382 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionPatternSubscription.java @@ -0,0 +1,76 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.storm.task.TopologyContext; + +public class ManualPartitionPatternSubscription extends PatternSubscription { + private static final long serialVersionUID = 5633018073527583826L; + private final ManualPartitioner parter; + private Set currentAssignment = null; + private KafkaConsumer consumer = null; + private ConsumerRebalanceListener listener = null; + private TopologyContext context = null; + + public ManualPartitionPatternSubscription(ManualPartitioner parter, Pattern pattern) { + super(pattern); + this.parter = parter; + } + + @Override + public void subscribe(KafkaConsumer consumer, ConsumerRebalanceListener listener, TopologyContext context) { + this.consumer = consumer; + this.listener = listener; + this.context = context; + refreshAssignment(); + } + + @Override + public void refreshAssignment() { + List allPartitions = new ArrayList<>(); + for(Map.Entry> entry: consumer.listTopics().entrySet()) { + if (pattern.matcher(entry.getKey()).matches()) { + for (PartitionInfo partitionInfo: entry.getValue()) { + allPartitions.add(new TopicPartition(partitionInfo.topic(), partitionInfo.partition())); + } + } + } + Collections.sort(allPartitions, TopicPartitionComparator.INSTANCE); + Set newAssignment = new HashSet<>(parter.partition(allPartitions, context)); + if (!newAssignment.equals(currentAssignment)) { + if (currentAssignment != null) { + listener.onPartitionsRevoked(currentAssignment); + listener.onPartitionsAssigned(newAssignment); + } + currentAssignment = newAssignment; + consumer.assign(currentAssignment); + } + } +} \ No newline at end of file diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReader.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitioner.java similarity index 50% rename from external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReader.java rename to external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitioner.java index e3480acc4fc..f4034ad739f 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReader.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitioner.java @@ -15,14 +15,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.storm.kafka.spout; -package org.apache.storm.kafka.spout.internal.partition; +import java.util.List; -import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.TopicPartition; +import org.apache.storm.task.TopologyContext; -import java.util.List; - -public interface KafkaPartitionReader { - List readPartitions(KafkaConsumer consumer); +/** + * A function used to assign partitions to this spout. + * WARNING if this is not done correctly you can really mess things up, like not reading data in some partitions. + * The complete TopologyContext is passed in, but it is suggested that you use the index of the spout and the total + * number of spouts to avoid missing partitions or double assigning partitions. + */ +public interface ManualPartitioner { + /** + * Get the partitions for this assignment + * @param allPartitions all of the partitions that the set of spouts want to subscribe to, in a strict ordering + * @param context the context of the topology + * @return the subset of the partitions that this spout should use. + */ + public List partition(List allPartitions, TopologyContext context); } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedSubscription.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedSubscription.java new file mode 100644 index 00000000000..6c5dcfb07af --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedSubscription.java @@ -0,0 +1,61 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; + +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.storm.task.TopologyContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Subscribe to all topics that follow a given list of values + */ +public class NamedSubscription extends Subscription { + private static final Logger LOG = LoggerFactory.getLogger(NamedSubscription.class); + private static final long serialVersionUID = 3438543305215813839L; + protected final Collection topics; + + public NamedSubscription(Collection topics) { + this.topics = Collections.unmodifiableCollection(new ArrayList<>(topics)); + } + + public NamedSubscription(String ... topics) { + this(Arrays.asList(topics)); + } + + @Override + public void subscribe(KafkaConsumer consumer, ConsumerRebalanceListener listener, TopologyContext unused) { + consumer.subscribe(topics, listener); + LOG.info("Kafka consumer subscribed topics {}", topics); + + // Initial poll to get the consumer registration process going. + // KafkaSpoutConsumerRebalanceListener will be called following this poll, upon partition registration + consumer.poll(0); + } + + @Override + public String getTopicsString() { + return String.valueOf(topics); + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/PatternSubscription.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/PatternSubscription.java new file mode 100644 index 00000000000..9a8de0f6b8f --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/PatternSubscription.java @@ -0,0 +1,54 @@ +/* + * 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; + +import java.util.regex.Pattern; + +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.storm.task.TopologyContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Subscribe to all topics that match a given pattern + */ +public class PatternSubscription extends Subscription { + private static final Logger LOG = LoggerFactory.getLogger(PatternSubscription.class); + private static final long serialVersionUID = 3438543305215813839L; + protected final Pattern pattern; + + public PatternSubscription(Pattern pattern) { + this.pattern = pattern; + } + + @Override + public void subscribe(KafkaConsumer consumer, ConsumerRebalanceListener listener, TopologyContext unused) { + consumer.subscribe(pattern, listener); + 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 + consumer.poll(0); + } + + @Override + public String getTopicsString() { + return pattern.pattern(); + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RecordTranslator.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RecordTranslator.java new file mode 100644 index 00000000000..2e72c9907d3 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RecordTranslator.java @@ -0,0 +1,53 @@ +/* + * 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; + +import java.io.Serializable; +import java.util.Collections; +import java.util.List; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.storm.tuple.Fields; + +/** + * Translate a {@link org.apache.kafka.clients.consumer.ConsumerRecord} to a tuple. + */ +public interface RecordTranslator extends Serializable, Func, List> { + public static final List DEFAULT_STREAM = Collections.singletonList("default"); + + /** + * Translate the ConsumerRecord into a list of objects that can be emitted + * @param record the record to translate + * @return the objects in the tuple. Return a {@link KafkaTuple} + * if you want to route the tuple to a non-default stream + */ + List apply(ConsumerRecord record); + + /** + * Get the fields associated with a stream. The streams passed in are + * returned by the {@link RecordTranslator.streams} method. + * @param stream the stream the fields are for + * @return the fields for that stream. + */ + Fields getFieldsFor(String stream); + + /** + * @return the list of streams that this will handle. + */ + List streams(); +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RoundRobinManualPartitioner.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RoundRobinManualPartitioner.java new file mode 100644 index 00000000000..e23e2dc1a42 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RoundRobinManualPartitioner.java @@ -0,0 +1,50 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.apache.kafka.common.TopicPartition; +import org.apache.storm.task.TopologyContext; + +/** + * Assign partitions in a round robin fashion for all spouts, + * not just the ones that are alive. Because the parallelism of + * the spouts does not typically change while running this makes + * the assignments more stable in the face of crashing spouts. + * + * Round Robin means that first spout of N spouts will get the first + * partition, and the N+1th partition... The second spout will get the second partition and + * N+2th partition etc. + */ +public class RoundRobinManualPartitioner implements ManualPartitioner { + + @Override + public List partition(List allPartitions, TopologyContext context) { + int thisTaskIndex = context.getThisTaskIndex(); + int totalTaskCount = context.getComponentTasks(context.getThisComponentId()).size(); + Set myPartitions = new HashSet<>(allPartitions.size()/totalTaskCount+1); + for (int i = thisTaskIndex; i < allPartitions.size(); i += totalTaskCount) { + myPartitions.add(allPartitions.get(i)); + } + return new ArrayList<>(myPartitions); + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/SimpleRecordTranslator.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/SimpleRecordTranslator.java new file mode 100644 index 00000000000..46c28496bf3 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/SimpleRecordTranslator.java @@ -0,0 +1,58 @@ +/* + * 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; + +import java.util.Arrays; +import java.util.List; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.storm.tuple.Fields; + +public class SimpleRecordTranslator implements RecordTranslator { + private static final long serialVersionUID = 4678369144122009596L; + private final Fields fields; + private final Func, List> func; + private final String stream; + + public SimpleRecordTranslator(Func, List> func, Fields fields) { + this(func, fields, "default"); + } + + public SimpleRecordTranslator(Func, List> func, Fields fields, String stream) { + this.func = func; + this.fields = fields; + this.stream = stream; + } + + @Override + public List apply(ConsumerRecord record) { + KafkaTuple ret = new KafkaTuple(); + ret.addAll(func.apply(record)); + return ret.routedTo(stream); + } + + @Override + public Fields getFieldsFor(String stream) { + return fields; + } + + @Override + public List streams() { + return Arrays.asList(stream); + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Subscription.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Subscription.java new file mode 100644 index 00000000000..db2a7bbdf68 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Subscription.java @@ -0,0 +1,53 @@ +/* + * 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; + +import java.io.Serializable; + +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.storm.task.TopologyContext; + +/** + * A subscription to kafka. + */ +public abstract class Subscription implements Serializable { + private static final long serialVersionUID = -216136367240198716L; + + /** + * Subscribe the KafkaConsumer to the proper topics + * @param consumer the Consumer to get. + * @param listener the rebalance listener to include in the subscription + */ + public abstract void subscribe(KafkaConsumer consumer, ConsumerRebalanceListener listener, TopologyContext context); + + /** + * @return a string representing the subscribed topics. + */ + public abstract String getTopicsString(); + + /** + * NOOP is the default behavior, which means that Kafka will internally handle partition assignment. + * If you wish to do manual partition management, you must provide an implementation of this method + * that will check with kafka for any changes and call the ConsumerRebalanceListener from subscribe + * to inform the rest of the system of those changes. + */ + public void refreshAssignment() { + //NOOP + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java index b90800109b9..dafb97c8269 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java @@ -15,14 +15,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.storm.kafka.spout; -import org.apache.kafka.common.TopicPartition; - import java.util.Comparator; +import org.apache.kafka.common.TopicPartition; + +/** + * Singleton comparator of TopicPartitions. Topics have precedence over partitions. + * Topics are compared through String.compare and partitions are compared + * numerically. + * + * Use INSTANCE for all sorting. + */ public class TopicPartitionComparator implements Comparator { + public static final TopicPartitionComparator INSTANCE = new TopicPartitionComparator(); + + /** + * Private to make it a singleton + */ + private TopicPartitionComparator() { + //Empty + } + @Override public int compare(TopicPartition o1, TopicPartition o2) { if (!o1.topic().equals(o2.topic())) { diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java index 45ae3309976..d51104df82f 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java @@ -15,7 +15,6 @@ * 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; @@ -72,4 +71,4 @@ public boolean isExpiredResetOnTrue() { } return expired; } -} +} \ No newline at end of file diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/AutomaticKafkaRecordsFetcher.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/AutomaticKafkaRecordsFetcher.java deleted file mode 100644 index 8ba70986f22..00000000000 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/AutomaticKafkaRecordsFetcher.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.fetcher; - -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.storm.kafka.spout.KafkaSpoutStreams; -import org.apache.storm.kafka.spout.KafkaSpoutStreamsNamedTopics; -import org.apache.storm.kafka.spout.KafkaSpoutStreamsWildcardTopics; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; -import java.util.regex.Pattern; - -public class AutomaticKafkaRecordsFetcher implements KafkaRecordsFetcher { - private static final Logger LOG = LoggerFactory.getLogger(AutomaticKafkaRecordsFetcher.class); - - private final KafkaConsumer kafkaConsumer; - private final ConsumerRebalanceListener consumerRebalanceListener; - - public AutomaticKafkaRecordsFetcher(KafkaConsumer kafkaConsumer, - ConsumerRebalanceListener consumerRebalanceListener, - KafkaSpoutStreams kafkaSpoutStreams) { - this.kafkaConsumer = kafkaConsumer; - this.consumerRebalanceListener = consumerRebalanceListener; - - subscribe(kafkaSpoutStreams); - } - - private void subscribe(KafkaSpoutStreams kafkaSpoutStreams) { - if (kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics) { - final List topics = ((KafkaSpoutStreamsNamedTopics) kafkaSpoutStreams).getTopics(); - kafkaConsumer.subscribe(topics, consumerRebalanceListener); - LOG.info("Kafka consumer subscribed topics {}", topics); - } else if (kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics) { - final Pattern pattern = ((KafkaSpoutStreamsWildcardTopics) kafkaSpoutStreams).getTopicWildcardPattern(); - kafkaConsumer.subscribe(pattern, consumerRebalanceListener); - 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); - } - - @Override - public ConsumerRecords fetchRecords(long fetchTimeoutMs) { - return kafkaConsumer.poll(fetchTimeoutMs); - } -} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetchers.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetchers.java deleted file mode 100644 index e3b7cd64305..00000000000 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetchers.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.fetcher; - -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.storm.kafka.spout.KafkaSpoutConfig; -import org.apache.storm.kafka.spout.internal.Timer; -import org.apache.storm.kafka.spout.internal.partition.KafkaPartitionReader; -import org.apache.storm.kafka.spout.internal.partition.KafkaPartitionReaders; -import org.apache.storm.task.TopologyContext; - -import java.util.concurrent.TimeUnit; - -public final class KafkaRecordsFetchers { - public static KafkaRecordsFetcher create(KafkaSpoutConfig kafkaSpoutConfig, - KafkaConsumer consumer, - TopologyContext context, - ConsumerRebalanceListener rebalanceListener) { - if (kafkaSpoutConfig.isManualPartitionAssignment()) { - int thisTaskIndex = context.getThisTaskIndex(); - int totalTaskCount = context.getComponentTasks(context.getThisComponentId()).size(); - KafkaPartitionReader partitionReader = KafkaPartitionReaders.create( - kafkaSpoutConfig.getKafkaSpoutStreams()); - Timer partitionRefreshTimer = new Timer(500, - kafkaSpoutConfig.getPartitionRefreshPeriodMs(), TimeUnit.MILLISECONDS); - - ManualKafkaRecordsFetcher.PartitionAssignmentChangeListener partitionAssignmentChangeListener = - ManualKafkaRecordsFetcher.listenerOf(rebalanceListener); - - return new ManualKafkaRecordsFetcher<>(consumer, thisTaskIndex, totalTaskCount, partitionReader, - partitionRefreshTimer, partitionAssignmentChangeListener); - } else { - return new AutomaticKafkaRecordsFetcher<>(consumer, rebalanceListener, - kafkaSpoutConfig.getKafkaSpoutStreams()); - } - } -} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/ManualKafkaRecordsFetcher.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/ManualKafkaRecordsFetcher.java deleted file mode 100644 index 124afc4abef..00000000000 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/ManualKafkaRecordsFetcher.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * 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.fetcher; - -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.TopicPartition; -import org.apache.storm.kafka.spout.internal.Timer; -import org.apache.storm.kafka.spout.internal.partition.KafkaPartitionReader; -import org.apache.storm.kafka.spout.TopicPartitionComparator; - -import java.util.Collections; -import java.util.Comparator; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - - -public class ManualKafkaRecordsFetcher implements KafkaRecordsFetcher { - private static final Comparator KAFKA_TOPIC_PARTITION_COMPARATOR = new TopicPartitionComparator(); - - private final KafkaConsumer consumer; - private final int thisTaskIndex; - private final int totalTaskCount; - private final KafkaPartitionReader partitionReader; - private final Timer partitionRefreshTimer; - private final PartitionAssignmentChangeListener partitionAssignmentChangeListener; - private Set myPartitions = Collections.emptySet(); - - public ManualKafkaRecordsFetcher(KafkaConsumer consumer, - int thisTaskIndex, - int totalTaskCount, - KafkaPartitionReader partitionReader, - Timer partitionRefreshTimer, - PartitionAssignmentChangeListener partitionAssignmentChangeListener) { - this.consumer = consumer; - this.thisTaskIndex = thisTaskIndex; - this.totalTaskCount = totalTaskCount; - this.partitionReader = partitionReader; - this.partitionRefreshTimer = partitionRefreshTimer; - this.partitionAssignmentChangeListener = partitionAssignmentChangeListener; - - doRefreshMyPartitions(); - } - - private void refreshMyPartitionsIfNeeded() { - if (!partitionRefreshTimer.isExpiredResetOnTrue()) { - return; - } - - doRefreshMyPartitions(); - } - - private void doRefreshMyPartitions() { - List topicPartitions = partitionReader.readPartitions(consumer); - Collections.sort(topicPartitions, KAFKA_TOPIC_PARTITION_COMPARATOR); - - Set curPartitions = new HashSet<>(topicPartitions.size()/totalTaskCount+1); - for (int i=thisTaskIndex; i fetchRecords(long fetchTimeoutMs) { - refreshMyPartitionsIfNeeded(); - - return consumer.poll(fetchTimeoutMs); - } - - public interface PartitionAssignmentChangeListener { - void onPartitionAssignmentChange(Set oldPartitions, Set newPartitions); - } - - public static PartitionAssignmentChangeListener listenerOf(final ConsumerRebalanceListener consumerRebalanceListener) { - return new PartitionAssignmentChangeListener() { - @Override - public void onPartitionAssignmentChange(Set oldPartitions, Set newPartitions) { - consumerRebalanceListener.onPartitionsRevoked(oldPartitions); - consumerRebalanceListener.onPartitionsAssigned(newPartitions); - } - }; - } -} - diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReaders.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReaders.java deleted file mode 100644 index 4e51c1e52ec..00000000000 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReaders.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.partition; - -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; -import org.apache.storm.kafka.spout.KafkaSpoutStreams; -import org.apache.storm.kafka.spout.KafkaSpoutStreamsNamedTopics; -import org.apache.storm.kafka.spout.KafkaSpoutStreamsWildcardTopics; - -import java.util.HashSet; - -public final class KafkaPartitionReaders { - public static KafkaPartitionReader create(KafkaSpoutStreams kafkaSpoutStreams) { - if (kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics) { - return new NamedTopicPartitionReader(new HashSet<>( - KafkaSpoutStreamsNamedTopics.class.cast(kafkaSpoutStreams).getTopics())); - } else if (kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics) { - return new WildcardTopicPartitionReader( - KafkaSpoutStreamsWildcardTopics.class.cast(kafkaSpoutStreams).getTopicWildcardPattern()); - } else { - throw new IllegalArgumentException("Unrecognized kafka spout stream: " + kafkaSpoutStreams.getClass()); - } - } - - public static TopicPartition toTopicPartition(PartitionInfo partitionInfo) { - return new TopicPartition(partitionInfo.topic(), partitionInfo.partition()); - } -} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/NamedTopicPartitionReader.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/NamedTopicPartitionReader.java deleted file mode 100644 index 41db16914e4..00000000000 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/NamedTopicPartitionReader.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.partition; - -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; - -import java.util.ArrayList; -import java.util.List; -import java.util.Set; - -public class NamedTopicPartitionReader implements KafkaPartitionReader { - private final Set topics; - - public NamedTopicPartitionReader(Set topics) { - this.topics = topics; - } - - @Override - public List readPartitions(KafkaConsumer consumer) { - List topicPartitions = new ArrayList<>(); - for (String topic : topics) { - for (PartitionInfo partitionInfo: consumer.partitionsFor(topic)) { - topicPartitions.add(KafkaPartitionReaders.toTopicPartition(partitionInfo)); - } - } - - return topicPartitions; - } -} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/WildcardTopicPartitionReader.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/WildcardTopicPartitionReader.java deleted file mode 100644 index fcac1c1fb83..00000000000 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/WildcardTopicPartitionReader.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.partition; - -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.regex.Pattern; - -public class WildcardTopicPartitionReader implements KafkaPartitionReader { - private final Pattern wildcardTopicPattern; - - public WildcardTopicPartitionReader(Pattern wildcardTopicPattern) { - this.wildcardTopicPattern = wildcardTopicPattern; - } - - @Override - public List readPartitions(KafkaConsumer consumer) { - List topicPartitions = new ArrayList<>(); - - for(Map.Entry> entry: consumer.listTopics().entrySet()) { - if (wildcardTopicPattern.matcher(entry.getKey()).matches()) { - for (PartitionInfo partitionInfo: entry.getValue()) { - topicPartitions.add(KafkaPartitionReaders.toTopicPartition(partitionInfo)); - } - } - } - - return topicPartitions; - } -} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java index 0218f0709c6..19b4f01fdac 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java @@ -18,33 +18,38 @@ package org.apache.storm.kafka.spout.trident; +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 java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; + import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.storm.kafka.spout.KafkaSpoutConfig; -import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilder; +import org.apache.storm.kafka.spout.RecordTranslator; +import org.apache.storm.kafka.spout.internal.Timer; +import org.apache.storm.task.TopologyContext; import org.apache.storm.trident.operation.TridentCollector; import org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout; import org.apache.storm.trident.topology.TransactionAttempt; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -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; - public class KafkaTridentSpoutEmitter implements IOpaquePartitionedTridentSpout.Emitter, KafkaTridentSpoutTopicPartition, KafkaTridentSpoutBatchMetadata>, Serializable { + private static final long serialVersionUID = -7343927794834130435L; + private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutEmitter.class); // Kafka @@ -53,18 +58,20 @@ public class KafkaTridentSpoutEmitter implements IOpaquePartitionedTridentS // Bookkeeping private final KafkaTridentSpoutManager kafkaManager; // Declare some KafkaTridentSpoutManager references for convenience - private final KafkaSpoutTuplesBuilder tuplesBuilder; private final long pollTimeoutMs; private final KafkaSpoutConfig.FirstPollOffsetStrategy firstPollOffsetStrategy; + private final RecordTranslator translator; + private final Timer refreshSubscriptionTimer; - public KafkaTridentSpoutEmitter(KafkaTridentSpoutManager kafkaManager) { + public KafkaTridentSpoutEmitter(KafkaTridentSpoutManager kafkaManager, TopologyContext context) { this.kafkaManager = kafkaManager; - this.kafkaManager.subscribeKafkaConsumer(); + this.kafkaManager.subscribeKafkaConsumer(context); + refreshSubscriptionTimer = new Timer(500, kafkaManager.getKafkaSpoutConfig().getPartitionRefreshPeriodMs(), TimeUnit.MILLISECONDS); //must subscribeKafkaConsumer before this line kafkaConsumer = kafkaManager.getKafkaConsumer(); + translator = kafkaManager.getKafkaSpoutConfig().getTranslator(); - tuplesBuilder = kafkaManager.getTuplesBuilder(); final KafkaSpoutConfig kafkaSpoutConfig = kafkaManager.getKafkaSpoutConfig(); pollTimeoutMs = kafkaSpoutConfig.getPollTimeoutMs(); firstPollOffsetStrategy = kafkaSpoutConfig.getFirstPollOffsetStrategy(); @@ -88,6 +95,9 @@ public KafkaTridentSpoutBatchMetadata emitPartitionBatch(TransactionAttemp seek(topicPartition, lastBatch); // poll + if (refreshSubscriptionTimer.isExpiredResetOnTrue()) { + kafkaManager.getKafkaSpoutConfig().getSubscription().refreshAssignment(); + } final ConsumerRecords records = kafkaConsumer.poll(pollTimeoutMs); LOG.debug("Polled [{}] records from Kafka.", records.count()); @@ -106,7 +116,7 @@ public KafkaTridentSpoutBatchMetadata emitPartitionBatch(TransactionAttemp private void emitTuples(TridentCollector collector, ConsumerRecords records) { for (ConsumerRecord record : records) { - final List tuple = tuplesBuilder.buildTuple(record); + final List tuple = translator.apply(record); collector.emit(tuple); LOG.debug("Emitted tuple [{}] for record: [{}]", tuple, record); } @@ -131,18 +141,18 @@ private long seek(TopicPartition tp, KafkaTridentSpoutBatchMetadata lastBa final OffsetAndMetadata committedOffset = kafkaConsumer.committed(tp); if (committedOffset != null) { // offset was committed for this TopicPartition if (firstPollOffsetStrategy.equals(EARLIEST)) { - kafkaConsumer.seekToBeginning(toArrayList(tp)); + kafkaConsumer.seekToBeginning(Collections.singleton(tp)); } else if (firstPollOffsetStrategy.equals(LATEST)) { - kafkaConsumer.seekToEnd(toArrayList(tp)); + kafkaConsumer.seekToEnd(Collections.singleton(tp)); } else { // By default polling starts at the last committed offset. +1 to point fetch to the first uncommitted offset. kafkaConsumer.seek(tp, committedOffset.offset() + 1); } } else { // no commits have ever been done, so start at the beginning or end depending on the strategy if (firstPollOffsetStrategy.equals(EARLIEST) || firstPollOffsetStrategy.equals(UNCOMMITTED_EARLIEST)) { - kafkaConsumer.seekToBeginning(toArrayList(tp)); + kafkaConsumer.seekToBeginning(Collections.singleton(tp)); } else if (firstPollOffsetStrategy.equals(LATEST) || firstPollOffsetStrategy.equals(UNCOMMITTED_LATEST)) { - kafkaConsumer.seekToEnd(toArrayList(tp)); + kafkaConsumer.seekToEnd(Collections.singleton(tp)); } } } @@ -151,10 +161,6 @@ private long seek(TopicPartition tp, KafkaTridentSpoutBatchMetadata lastBa return fetchOffset; } - private Collection toArrayList(final TopicPartition tp) { - return new ArrayList(1){{add(tp);}}; - } - // returns paused topic partitions private Collection pauseTopicPartitions(TopicPartition excludedTp) { final Set pausedTopicPartitions = new HashSet<>(kafkaConsumer.assignment()); diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java index d2c170076c3..4b60f33d0a0 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java @@ -18,23 +18,22 @@ package org.apache.storm.kafka.spout.trident; +import java.io.Serializable; +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.TimeUnit; + import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.TopicPartition; import org.apache.storm.kafka.spout.KafkaSpoutConfig; -import org.apache.storm.kafka.spout.KafkaSpoutStreams; -import org.apache.storm.kafka.spout.KafkaSpoutStreamsNamedTopics; -import org.apache.storm.kafka.spout.KafkaSpoutStreamsWildcardTopics; -import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilder; +import org.apache.storm.kafka.spout.RecordTranslator; +import org.apache.storm.kafka.spout.internal.Timer; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.tuple.Fields; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Serializable; -import java.util.Collection; -import java.util.List; -import java.util.Set; -import java.util.regex.Pattern; - public class KafkaTridentSpoutManager implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutManager.class); @@ -44,29 +43,30 @@ public class KafkaTridentSpoutManager implements Serializable { // Bookkeeping private final KafkaSpoutConfig kafkaSpoutConfig; // Declare some KafkaSpoutConfig references for convenience - private KafkaSpoutStreams kafkaSpoutStreams; // Object that wraps all the logic to declare output fields and emit tuples - private KafkaSpoutTuplesBuilder tuplesBuilder; // Object that contains the logic to build tuples for each ConsumerRecord + private final Fields fields; public KafkaTridentSpoutManager(KafkaSpoutConfig kafkaSpoutConfig) { this.kafkaSpoutConfig = kafkaSpoutConfig; - kafkaSpoutStreams = kafkaSpoutConfig.getKafkaSpoutStreams(); - tuplesBuilder = kafkaSpoutConfig.getTuplesBuilder(); + RecordTranslator translator = kafkaSpoutConfig.getTranslator(); + Fields fields = null; + for (String stream: translator.streams()) { + if (fields == null) { + fields = translator.getFieldsFor(stream); + } else { + if (!fields.equals(translator.getFieldsFor(stream))) { + throw new IllegalArgumentException("Trident Spouts do not support multiple output Fields"); + } + } + } + this.fields = fields; LOG.debug("Created {}", this); } - void subscribeKafkaConsumer() { + void subscribeKafkaConsumer(TopologyContext context) { kafkaConsumer = new KafkaConsumer<>(kafkaSpoutConfig.getKafkaProps(), kafkaSpoutConfig.getKeyDeserializer(), kafkaSpoutConfig.getValueDeserializer()); - if (kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics) { - final List subTopics = kafkaSpoutConfig.getSubscribedTopics(); - kafkaConsumer.subscribe(subTopics, new KafkaSpoutConsumerRebalanceListener()); - LOG.info("Kafka consumer subscribed topics {}", subTopics); - } else if (kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics) { - final Pattern pattern = kafkaSpoutConfig.getTopicWildcardPattern(); - kafkaConsumer.subscribe(pattern, new KafkaSpoutConsumerRebalanceListener()); - LOG.info("Kafka consumer subscribed topics matching wildcard pattern [{}]", pattern); - } + kafkaSpoutConfig.getSubscription().subscribe(kafkaConsumer, new KafkaSpoutConsumerRebalanceListener(), context); // Initial poll to get the consumer registration process going. // KafkaSpoutConsumerRebalanceListener will be called following this poll, upon partition registration @@ -93,16 +93,12 @@ KafkaConsumer getKafkaConsumer() { return kafkaConsumer; } - KafkaSpoutTuplesBuilder getTuplesBuilder() { - return tuplesBuilder; - } - Set getTopicPartitions() { return KafkaTridentSpoutTopicPartitionRegistry.INSTANCE.getTopicPartitions(); } - - KafkaSpoutStreams getKafkaSpoutStreams() { - return kafkaSpoutStreams; + + Fields getFields() { + return fields; } KafkaSpoutConfig getKafkaSpoutConfig() { diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java index b77ad111561..5c5856c215f 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java @@ -19,6 +19,7 @@ package org.apache.storm.kafka.spout.trident; import org.apache.kafka.common.TopicPartition; +import org.apache.storm.kafka.spout.KafkaSpoutConfig; import org.apache.storm.task.TopologyContext; import org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout; import org.apache.storm.tuple.Fields; @@ -29,12 +30,19 @@ import java.util.Map; public class KafkaTridentSpoutOpaque implements IOpaquePartitionedTridentSpout, KafkaTridentSpoutTopicPartition, KafkaTridentSpoutBatchMetadata> { + private static final long serialVersionUID = -8003272486566259640L; + private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutOpaque.class); private KafkaTridentSpoutManager kafkaManager; private KafkaTridentSpoutEmitter kafkaTridentSpoutEmitter; private KafkaTridentSpoutOpaqueCoordinator coordinator; + + public KafkaTridentSpoutOpaque(KafkaSpoutConfig conf) { + this(new KafkaTridentSpoutManager<>(conf)); + } + public KafkaTridentSpoutOpaque(KafkaTridentSpoutManager kafkaManager) { this.kafkaManager = kafkaManager; LOG.debug("Created {}", this); @@ -44,7 +52,7 @@ public KafkaTridentSpoutOpaque(KafkaTridentSpoutManager kafkaManager) { public Emitter, KafkaTridentSpoutTopicPartition, KafkaTridentSpoutBatchMetadata> getEmitter(Map conf, TopologyContext context) { // Instance is created on first call rather than in constructor to avoid NotSerializableException caused by KafkaConsumer if (kafkaTridentSpoutEmitter == null) { - kafkaTridentSpoutEmitter = new KafkaTridentSpoutEmitter<>(kafkaManager); + kafkaTridentSpoutEmitter = new KafkaTridentSpoutEmitter<>(kafkaManager, context); } return kafkaTridentSpoutEmitter; } @@ -64,7 +72,7 @@ public Map getComponentConfiguration() { @Override public Fields getOutputFields() { - final Fields outputFields = kafkaManager.getKafkaSpoutStreams().getOutputFields(); + final Fields outputFields = kafkaManager.getFields(); LOG.debug("OutputFields = {}", outputFields); return outputFields; } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java new file mode 100644 index 00000000000..f24fed5976a --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java @@ -0,0 +1,115 @@ +/** + * 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.trident; + +import org.apache.storm.task.OutputCollector; +import org.apache.storm.topology.FailedException; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.storm.kafka.trident.mapper.TridentTupleToKafkaMapper; +import org.apache.storm.kafka.trident.selector.KafkaTopicSelector; +import org.apache.storm.trident.operation.TridentCollector; +import org.apache.storm.trident.state.State; +import org.apache.storm.trident.tuple.TridentTuple; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +public class TridentKafkaState implements State { + private static final Logger LOG = LoggerFactory.getLogger(TridentKafkaState.class); + + private KafkaProducer producer; + private OutputCollector collector; + + private TridentTupleToKafkaMapper mapper; + private KafkaTopicSelector topicSelector; + + public TridentKafkaState withTridentTupleToKafkaMapper(TridentTupleToKafkaMapper mapper) { + this.mapper = mapper; + return this; + } + + public TridentKafkaState withKafkaTopicSelector(KafkaTopicSelector selector) { + this.topicSelector = selector; + return this; + } + + @Override + public void beginCommit(Long txid) { + LOG.debug("beginCommit is Noop."); + } + + @Override + public void commit(Long txid) { + LOG.debug("commit is Noop."); + } + + public void prepare(Properties options) { + Objects.requireNonNull(mapper, "mapper can not be null"); + Objects.requireNonNull(topicSelector, "topicSelector can not be null"); + producer = new KafkaProducer(options); + } + + public void updateState(List tuples, TridentCollector collector) { + String topic = null; + try { + List> futures = new ArrayList<>(tuples.size()); + for (TridentTuple tuple : tuples) { + topic = topicSelector.getTopic(tuple); + + if(topic != null) { + Future result = producer.send(new ProducerRecord(topic, + mapper.getKeyFromTuple(tuple), mapper.getMessageFromTuple(tuple))); + futures.add(result); + } else { + LOG.warn("skipping key = " + mapper.getKeyFromTuple(tuple) + ", topic selector returned null."); + } + } + + List exceptions = new ArrayList<>(futures.size()); + for (Future future : futures) { + try { + future.get(); + } catch (ExecutionException e) { + exceptions.add(e); + } + } + + if(exceptions.size() > 0){ + String errorMsg = "Could not retrieve result for messages " + tuples + " from topic = " + topic + + " because of the following exceptions: \n"; + for (ExecutionException exception : exceptions) { + errorMsg = errorMsg + exception.getMessage() + "\n"; + } + LOG.error(errorMsg); + throw new FailedException(errorMsg); + } + } catch (Exception ex) { + String errorMsg = "Could not send messages " + tuples + " to topic = " + topic; + LOG.warn(errorMsg, ex); + throw new FailedException(errorMsg, ex); + } + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaStateFactory.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaStateFactory.java new file mode 100644 index 00000000000..f564510775d --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaStateFactory.java @@ -0,0 +1,63 @@ +/** + * 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.trident; + +import org.apache.storm.task.IMetricsContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.storm.kafka.trident.mapper.TridentTupleToKafkaMapper; +import org.apache.storm.kafka.trident.selector.KafkaTopicSelector; +import org.apache.storm.trident.state.State; +import org.apache.storm.trident.state.StateFactory; + +import java.util.Map; +import java.util.Properties; + +public class TridentKafkaStateFactory implements StateFactory { + + private static final Logger LOG = LoggerFactory.getLogger(TridentKafkaStateFactory.class); + + private TridentTupleToKafkaMapper mapper; + private KafkaTopicSelector topicSelector; + private Properties producerProperties = new Properties(); + + public TridentKafkaStateFactory withTridentTupleToKafkaMapper(TridentTupleToKafkaMapper mapper) { + this.mapper = mapper; + return this; + } + + public TridentKafkaStateFactory withKafkaTopicSelector(KafkaTopicSelector selector) { + this.topicSelector = selector; + return this; + } + + public TridentKafkaStateFactory withProducerProperties(Properties props) { + this.producerProperties = props; + return this; + } + + @Override + public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) { + LOG.info("makeState(partitonIndex={}, numpartitions={}", partitionIndex, numPartitions); + TridentKafkaState state = new TridentKafkaState() + .withKafkaTopicSelector(this.topicSelector) + .withTridentTupleToKafkaMapper(this.mapper); + state.prepare(producerProperties); + return state; + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/FieldNameBasedTupleToKafkaMapper.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/FieldNameBasedTupleToKafkaMapper.java new file mode 100644 index 00000000000..2d049715d71 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/FieldNameBasedTupleToKafkaMapper.java @@ -0,0 +1,41 @@ +/** + * 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.trident.mapper; + +import org.apache.storm.trident.tuple.TridentTuple; + +public class FieldNameBasedTupleToKafkaMapper implements TridentTupleToKafkaMapper { + + public final String keyFieldName; + public final String msgFieldName; + + public FieldNameBasedTupleToKafkaMapper(String keyFieldName, String msgFieldName) { + this.keyFieldName = keyFieldName; + this.msgFieldName = msgFieldName; + } + + @Override + public K getKeyFromTuple(TridentTuple tuple) { + return (K) tuple.getValueByField(keyFieldName); + } + + @Override + public V getMessageFromTuple(TridentTuple tuple) { + return (V) tuple.getValueByField(msgFieldName); + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/TridentTupleToKafkaMapper.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/TridentTupleToKafkaMapper.java new file mode 100644 index 00000000000..28c6c89465e --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/TridentTupleToKafkaMapper.java @@ -0,0 +1,28 @@ +/** + * 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.trident.mapper; + +import org.apache.storm.tuple.Tuple; +import org.apache.storm.trident.tuple.TridentTuple; + +import java.io.Serializable; + +public interface TridentTupleToKafkaMapper extends Serializable { + K getKeyFromTuple(TridentTuple tuple); + V getMessageFromTuple(TridentTuple tuple); +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/DefaultTopicSelector.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/DefaultTopicSelector.java new file mode 100644 index 00000000000..607c996e567 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/DefaultTopicSelector.java @@ -0,0 +1,34 @@ +/** + * 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.trident.selector; + +import org.apache.storm.trident.tuple.TridentTuple; + +public class DefaultTopicSelector implements KafkaTopicSelector { + private static final long serialVersionUID = -1172454882072591493L; + private final String topicName; + + public DefaultTopicSelector(final String topicName) { + this.topicName = topicName; + } + + @Override + public String getTopic(TridentTuple tuple) { + return topicName; + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/KafkaTopicSelector.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/KafkaTopicSelector.java new file mode 100644 index 00000000000..012a6c73b5c --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/KafkaTopicSelector.java @@ -0,0 +1,26 @@ +/** + * 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.trident.selector; + +import org.apache.storm.trident.tuple.TridentTuple; + +import java.io.Serializable; + +public interface KafkaTopicSelector extends Serializable { + String getTopic(TridentTuple tuple); +} diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/bolt/KafkaBoltTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/bolt/KafkaBoltTest.java new file mode 100644 index 00000000000..8c8a94562ed --- /dev/null +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/bolt/KafkaBoltTest.java @@ -0,0 +1,91 @@ +/** + * 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.bolt; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.argThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.storm.Testing; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.testing.MkTupleParam; +import org.apache.storm.tuple.Tuple; +import org.junit.Test; +import org.mockito.ArgumentMatcher; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class KafkaBoltTest { + private static final Logger LOG = LoggerFactory.getLogger(KafkaBoltTest.class); + + @SuppressWarnings({ "unchecked", "serial" }) + @Test + public void testSimple() { + final KafkaProducer producer = mock(KafkaProducer.class); + when(producer.send((ProducerRecord)any(), (Callback)any())).thenAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + Callback c = (Callback)invocation.getArguments()[1]; + c.onCompletion(null, null); + return null; + } + }); + KafkaBolt bolt = new KafkaBolt() { + @Override + protected KafkaProducer mkProducer(Properties props) { + return producer; + } + }; + bolt.withTopicSelector("MY_TOPIC"); + + OutputCollector collector = mock(OutputCollector.class); + TopologyContext context = mock(TopologyContext.class); + Map conf = new HashMap<>(); + bolt.prepare(conf, context, collector); + MkTupleParam param = new MkTupleParam(); + param.setFields("key", "message"); + Tuple testTuple = Testing.testTuple(Arrays.asList("KEY", "VALUE"), param); + bolt.execute(testTuple); + verify(producer).send(argThat(new ArgumentMatcher>() { + @Override + public boolean matches(Object argument) { + LOG.info("GOT {} ->", argument); + ProducerRecord arg = (ProducerRecord) argument; + LOG.info(" {} {} {}", arg.topic(), arg.key(), arg.value()); + return "MY_TOPIC".equals(arg.topic()) && + "KEY".equals(arg.key()) && + "VALUE".equals(arg.value()); + } + }), any(Callback.class)); + verify(collector).ack(testTuple); + } + +} diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/ByTopicRecordTranslatorTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/ByTopicRecordTranslatorTest.java new file mode 100644 index 00000000000..ea0b6e73f37 --- /dev/null +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/ByTopicRecordTranslatorTest.java @@ -0,0 +1,93 @@ +/* + * 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; + +import static org.junit.Assert.*; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; +import org.junit.Test; + +public class ByTopicRecordTranslatorTest { + public static Func, List> JUST_KEY_FUNC = new Func, List>() { + @Override + public List apply(ConsumerRecord record) { + return new Values(record.key()); + } + }; + + public static Func, List> JUST_VALUE_FUNC = new Func, List>() { + @Override + public List apply(ConsumerRecord record) { + return new Values(record.value()); + } + }; + + public static Func, List> KEY_VALUE_FUNC = new Func, List>() { + @Override + public List apply(ConsumerRecord record) { + return new Values(record.key(), record.value()); + } + }; + + @Test + public void testBasic() { + ByTopicRecordTranslator trans = + new ByTopicRecordTranslator<>(JUST_KEY_FUNC, new Fields("key")); + trans.forTopic("TOPIC 1", JUST_VALUE_FUNC, new Fields("value"), "value-stream"); + trans.forTopic("TOPIC 2", KEY_VALUE_FUNC, new Fields("key", "value"), "key-value-stream"); + HashSet expectedStreams = new HashSet<>(); + expectedStreams.add("default"); + expectedStreams.add("value-stream"); + expectedStreams.add("key-value-stream"); + assertEquals(expectedStreams, new HashSet<>(trans.streams())); + + ConsumerRecord cr1 = new ConsumerRecord<>("TOPIC OTHER", 100, 100, "THE KEY", "THE VALUE"); + assertEquals(new Fields("key"), trans.getFieldsFor("default")); + assertEquals(Arrays.asList("THE KEY"), trans.apply(cr1)); + + ConsumerRecord cr2 = new ConsumerRecord<>("TOPIC 1", 100, 100, "THE KEY", "THE VALUE"); + assertEquals(new Fields("value"), trans.getFieldsFor("value-stream")); + assertEquals(Arrays.asList("THE VALUE"), trans.apply(cr2)); + + ConsumerRecord cr3 = new ConsumerRecord<>("TOPIC 2", 100, 100, "THE KEY", "THE VALUE"); + assertEquals(new Fields("key", "value"), trans.getFieldsFor("key-value-stream")); + assertEquals(Arrays.asList("THE KEY", "THE VALUE"), trans.apply(cr3)); + } + + @Test(expected = IllegalArgumentException.class) + public void testFieldCollision() { + ByTopicRecordTranslator trans = + new ByTopicRecordTranslator<>(JUST_KEY_FUNC, new Fields("key")); + trans.forTopic("foo", JUST_VALUE_FUNC, new Fields("value")); + } + + @Test(expected = IllegalStateException.class) + public void testTopicCollision() { + ByTopicRecordTranslator trans = + new ByTopicRecordTranslator<>(JUST_KEY_FUNC, new Fields("key")); + trans.forTopic("foo", JUST_VALUE_FUNC, new Fields("value"), "foo1"); + trans.forTopic("foo", KEY_VALUE_FUNC, new Fields("key", "value"), "foo2"); + } + +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderWildcardTopics.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/DefaultRecordTranslatorTest.java similarity index 58% rename from external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderWildcardTopics.java rename to external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/DefaultRecordTranslatorTest.java index 85d48098119..f4275e49d10 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderWildcardTopics.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/DefaultRecordTranslatorTest.java @@ -15,22 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.storm.kafka.spout; -import org.apache.kafka.clients.consumer.ConsumerRecord; - -import java.util.List; +import static org.junit.Assert.*; -public class KafkaSpoutTuplesBuilderWildcardTopics implements KafkaSpoutTuplesBuilder { - private KafkaSpoutTupleBuilder tupleBuilder; +import java.util.Arrays; - public KafkaSpoutTuplesBuilderWildcardTopics(KafkaSpoutTupleBuilder tupleBuilder) { - this.tupleBuilder = tupleBuilder; - } +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.storm.tuple.Fields; +import org.junit.Test; - @Override - public List buildTuple(ConsumerRecord consumerRecord) { - return tupleBuilder.buildTuple(consumerRecord); +public class DefaultRecordTranslatorTest { + @Test + public void testBasic() { + DefaultRecordTranslator trans = new DefaultRecordTranslator<>(); + assertEquals(Arrays.asList("default"), trans.streams()); + assertEquals(new Fields("topic", "partition", "offset", "key", "value"), trans.getFieldsFor("default")); + ConsumerRecord cr = new ConsumerRecord<>("TOPIC", 100, 100, "THE KEY", "THE VALUE"); + assertEquals(Arrays.asList("TOPIC", 100, 100l, "THE KEY", "THE VALUE"), trans.apply(cr)); } } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreams.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutConfigTest.java similarity index 51% rename from external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreams.java rename to external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutConfigTest.java index d4178a9f52a..08220dd207e 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreams.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutConfigTest.java @@ -15,24 +15,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.storm.kafka.spout; -import org.apache.storm.spout.SpoutOutputCollector; -import org.apache.storm.topology.OutputFieldsDeclarer; -import org.apache.storm.tuple.Fields; +import static org.junit.Assert.*; -import java.io.Serializable; -import java.util.List; +import java.util.HashMap; -/** - * Represents the {@link KafkaSpoutStream} associated with each topic or topic pattern (wildcard), and provides - * a public API to declare output streams and emmit tuples, on the appropriate stream, for all the topics specified. - */ -public interface KafkaSpoutStreams extends Serializable { - void declareOutputFields(OutputFieldsDeclarer declarer); +import org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy; +import org.junit.Test; - void emit(SpoutOutputCollector collector, List tuple, KafkaSpoutMessageId messageId); +public class KafkaSpoutConfigTest { - Fields getOutputFields(); + @Test + public void testBasic() { + KafkaSpoutConfig conf = KafkaSpoutConfig.builder("localhost:1234", "topic").build(); + assertEquals(FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST, conf.getFirstPollOffsetStrategy()); + assertNull(conf.getConsumerGroupId()); + assertTrue(conf.getTranslator() instanceof DefaultRecordTranslator); + HashMap expected = new HashMap<>(); + expected.put("bootstrap.servers", "localhost:1234"); + expected.put("enable.auto.commit", "false"); + assertEquals(expected, conf.getKafkaProps()); + } } diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java index 2d3eb569936..9969d84eada 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java @@ -15,11 +15,25 @@ */ package org.apache.storm.kafka.spout; +import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfig; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.Matchers.hasKey; +import static org.junit.Assert.assertThat; +import static org.mockito.Matchers.anyCollection; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; + import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -27,37 +41,16 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration; - -import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutStreams; - import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory; import org.apache.storm.spout.SpoutOutputCollector; import org.apache.storm.task.TopologyContext; +import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; - -import static org.mockito.Matchers.anyCollection; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.anyObject; -import static org.mockito.Mockito.when; - -import org.junit.Before; import org.mockito.Captor; - -import static org.mockito.Mockito.reset; - +import org.mockito.Mockito; import org.mockito.MockitoAnnotations; -import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfig; -import static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.Matchers.hasKey; -import static org.junit.Assert.assertThat; -import static org.mockito.Matchers.anyList; -import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; - public class KafkaSpoutRebalanceTest { @Captor @@ -65,7 +58,7 @@ public class KafkaSpoutRebalanceTest { private TopologyContext contextMock; private SpoutOutputCollector collectorMock; - private Map conf; + private Map conf; private KafkaConsumer consumerMock; private KafkaConsumerFactory consumerFactoryMock; @@ -113,11 +106,11 @@ private List emitOneMessagePerPartitionThenRevokeOnePartiti //Emit the messages spout.nextTuple(); ArgumentCaptor messageIdForRevokedPartition = ArgumentCaptor.forClass(KafkaSpoutMessageId.class); - verify(collectorMock).emit(anyString(), anyList(), messageIdForRevokedPartition.capture()); + verify(collectorMock).emit(Mockito.anyString(), Mockito.anyList(), messageIdForRevokedPartition.capture()); reset(collectorMock); spout.nextTuple(); ArgumentCaptor messageIdForAssignedPartition = ArgumentCaptor.forClass(KafkaSpoutMessageId.class); - verify(collectorMock).emit(anyString(), anyList(), messageIdForAssignedPartition.capture()); + verify(collectorMock).emit(Mockito.anyString(), Mockito.anyList(), messageIdForAssignedPartition.capture()); //Now rebalance consumerRebalanceListener.onPartitionsRevoked(assignedPartitions); @@ -132,7 +125,7 @@ private List emitOneMessagePerPartitionThenRevokeOnePartiti @Test public void spoutMustIgnoreAcksForTuplesItIsNotAssignedAfterRebalance() throws Exception { //Acking tuples for partitions that are no longer assigned is useless since the spout will not be allowed to commit them - KafkaSpout spout = new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams(), -1, 10), consumerFactoryMock); + KafkaSpout spout = new KafkaSpout<>(getKafkaSpoutConfig(-1, 10), consumerFactoryMock); String topic = SingleTopicKafkaSpoutConfiguration.TOPIC; TopicPartition partitionThatWillBeRevoked = new TopicPartition(topic, 1); TopicPartition assignedPartition = new TopicPartition(topic, 2); @@ -161,7 +154,7 @@ public void spoutMustIgnoreAcksForTuplesItIsNotAssignedAfterRebalance() throws E public void spoutMustIgnoreFailsForTuplesItIsNotAssignedAfterRebalance() throws Exception { //Failing tuples for partitions that are no longer assigned is useless since the spout will not be allowed to commit them if they later pass KafkaSpoutRetryService retryServiceMock = mock(KafkaSpoutRetryService.class); - KafkaSpout spout = new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams(), -1, 10, retryServiceMock), consumerFactoryMock); + KafkaSpout spout = new KafkaSpout<>(getKafkaSpoutConfig(-1, 10, retryServiceMock), consumerFactoryMock); String topic = SingleTopicKafkaSpoutConfiguration.TOPIC; TopicPartition partitionThatWillBeRevoked = new TopicPartition(topic, 1); TopicPartition assignedPartition = new TopicPartition(topic, 2); diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopicsTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopicsTest.java deleted file mode 100644 index 723f5fdf437..00000000000 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopicsTest.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Copyright 2016 The Apache Software Foundation. - * - * Licensed 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; - -import org.apache.storm.tuple.Fields; -import org.junit.Assert; -import org.junit.Test; - -public class KafkaSpoutStreamsNamedTopicsTest { - - @Test - public void testGetOutputFields() { - Fields outputFields = new Fields("b","a"); - String[] topics = new String[]{"testTopic"}; - String streamId = "test"; - KafkaSpoutStreamsNamedTopics build = new KafkaSpoutStreamsNamedTopics.Builder(outputFields, streamId, topics) - .addStream(outputFields, streamId, topics) - .build(); - Fields actualFields = build.getOutputFields(); - Assert.assertEquals(outputFields.get(0), actualFields.get(0)); - Assert.assertEquals(outputFields.get(1), actualFields.get(1)); - - } - -} diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java index 6983160673d..f457b5968ac 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java @@ -70,7 +70,7 @@ SpoutContext initializeSpout(int msgCount) { SpoutOutputCollector collector = mock(SpoutOutputCollector.class); Map conf = mock(Map.class); - KafkaSpout spout = new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams(), kafkaPort)); + KafkaSpout spout = new KafkaSpout<>(getKafkaSpoutConfig(kafkaPort)); spout.open(conf, topology, collector); spout.activate(); return new SpoutContext(spout, collector); diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/SingleTopicKafkaSpoutConfiguration.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/SingleTopicKafkaSpoutConfiguration.java index 95b2199b3f9..99bd3de15e0 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/SingleTopicKafkaSpoutConfiguration.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/SingleTopicKafkaSpoutConfiguration.java @@ -17,17 +17,21 @@ */ package org.apache.storm.kafka.spout.builders; +import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; + +import java.util.List; + +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.storm.Config; import org.apache.storm.generated.StormTopology; -import org.apache.storm.kafka.spout.*; +import org.apache.storm.kafka.spout.Func; +import org.apache.storm.kafka.spout.KafkaSpout; +import org.apache.storm.kafka.spout.KafkaSpoutConfig; +import org.apache.storm.kafka.spout.KafkaSpoutRetryService; import org.apache.storm.kafka.spout.test.KafkaSpoutTestBolt; import org.apache.storm.topology.TopologyBuilder; import org.apache.storm.tuple.Fields; - -import java.util.HashMap; -import java.util.Map; - -import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; +import org.apache.storm.tuple.Values; public class SingleTopicKafkaSpoutConfiguration { @@ -42,53 +46,41 @@ public static Config getConfig() { public static StormTopology getTopologyKafkaSpout(int port) { final TopologyBuilder tp = new TopologyBuilder(); - tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams(), port)), 1); + tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig(port)), 1); tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAM); return tp.createTopology(); } - public static KafkaSpoutConfig getKafkaSpoutConfig(KafkaSpoutStreams kafkaSpoutStreams, int port) { - return getKafkaSpoutConfig(kafkaSpoutStreams, port, 10_000); + static public KafkaSpoutConfig getKafkaSpoutConfig(int port) { + return getKafkaSpoutConfig(port, 10_000); } - public static KafkaSpoutConfig getKafkaSpoutConfig(KafkaSpoutStreams kafkaSpoutStreams, int port, long offsetCommitPeriodMs) { - return getKafkaSpoutConfig(kafkaSpoutStreams, port, offsetCommitPeriodMs, getRetryService()); + static public KafkaSpoutConfig getKafkaSpoutConfig(int port, long offsetCommitPeriodMs) { + return getKafkaSpoutConfig(port, offsetCommitPeriodMs, getRetryService()); } - public static KafkaSpoutConfig getKafkaSpoutConfig(KafkaSpoutStreams kafkaSpoutStreams, int port, long offsetCommitPeriodMs, KafkaSpoutRetryService retryService) { - return new KafkaSpoutConfig.Builder<>(getKafkaConsumerProps(port), kafkaSpoutStreams, getTuplesBuilder(), retryService) - .setOffsetCommitPeriodMs(offsetCommitPeriodMs) - .setFirstPollOffsetStrategy(EARLIEST) - .setMaxUncommittedOffsets(250) - .setPollTimeoutMs(1000) - .build(); + private static Func, List> TOPIC_KEY_VALUE_FUNC = new Func, List>() { + @Override + public List apply(ConsumerRecord r) { + return new Values(r.topic(), r.key(), r.value()); + } + }; + + static public KafkaSpoutConfig getKafkaSpoutConfig(int port, long offsetCommitPeriodMs, KafkaSpoutRetryService retryService) { + return KafkaSpoutConfig.builder("127.0.0.1:" + port, TOPIC) + .setRecordTranslator(TOPIC_KEY_VALUE_FUNC, + new Fields("topic", "key", "value"), STREAM) + .setGroupId("kafkaSpoutTestGroup") + .setMaxPollRecords(5) + .setRetry(retryService) + .setOffsetCommitPeriodMs(offsetCommitPeriodMs) + .setFirstPollOffsetStrategy(EARLIEST) + .setMaxUncommittedOffsets(250) + .setPollTimeoutMs(1000) + .build(); } - + protected static KafkaSpoutRetryService getRetryService() { - return new KafkaSpoutRetryExponentialBackoff(KafkaSpoutRetryExponentialBackoff.TimeInterval.microSeconds(0), - KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(0), Integer.MAX_VALUE, KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(0)); - - } - - protected static Map getKafkaConsumerProps(int port) { - Map props = new HashMap<>(); - props.put(KafkaSpoutConfig.Consumer.BOOTSTRAP_SERVERS, "127.0.0.1:" + port); - props.put(KafkaSpoutConfig.Consumer.GROUP_ID, "kafkaSpoutTestGroup"); - props.put(KafkaSpoutConfig.Consumer.KEY_DESERIALIZER, "org.apache.kafka.common.serialization.StringDeserializer"); - props.put(KafkaSpoutConfig.Consumer.VALUE_DESERIALIZER, "org.apache.kafka.common.serialization.StringDeserializer"); - props.put("max.poll.records", "5"); - return props; - } - - protected static KafkaSpoutTuplesBuilder getTuplesBuilder() { - return new KafkaSpoutTuplesBuilderNamedTopics.Builder<>( - new TopicKeyValueTupleBuilder(TOPIC)) - .build(); - } - - public static KafkaSpoutStreams getKafkaSpoutStreams() { - final Fields outputFields = new Fields("topic", "key", "value"); - return new KafkaSpoutStreamsNamedTopics.Builder(outputFields, STREAM, new String[]{TOPIC}) // contents of topics test sent to test_stream - .build(); + return KafkaSpoutConfig.UNIT_TEST_RETRY_SERVICE; } } diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java index d9818b7608b..6e59d420ae5 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java @@ -18,32 +18,32 @@ package org.apache.storm.kafka.spout.test; +import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.List; + +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.storm.Config; import org.apache.storm.LocalCluster; import org.apache.storm.StormSubmitter; import org.apache.storm.generated.StormTopology; +import org.apache.storm.kafka.spout.ByTopicRecordTranslator; +import org.apache.storm.kafka.spout.Func; import org.apache.storm.kafka.spout.KafkaSpout; import org.apache.storm.kafka.spout.KafkaSpoutConfig; import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff; import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff.TimeInterval; import org.apache.storm.kafka.spout.KafkaSpoutRetryService; -import org.apache.storm.kafka.spout.KafkaSpoutStreams; -import org.apache.storm.kafka.spout.KafkaSpoutStreamsNamedTopics; -import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilder; -import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilderNamedTopics; import org.apache.storm.topology.TopologyBuilder; import org.apache.storm.tuple.Fields; - -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.util.HashMap; -import java.util.Map; - -import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; +import org.apache.storm.tuple.Values; public class KafkaSpoutTopologyMainNamedTopics { - private static final String[] STREAMS = new String[]{"test_stream","test1_stream","test2_stream"}; + private static final String TOPIC_2_STREAM = "test_2_stream"; + private static final String TOPIC_0_1_STREAM = "test_0_1_stream"; private static final String[] TOPICS = new String[]{"test","test1","test2"}; @@ -88,14 +88,32 @@ protected Config getConfig() { protected StormTopology getTopolgyKafkaSpout() { final TopologyBuilder tp = new TopologyBuilder(); - tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams())), 1); - tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAMS[0]); - tp.setBolt("kafka_bolt_1", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAMS[2]); + tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig()), 1); + tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt()) + .shuffleGrouping("kafka_spout", TOPIC_0_1_STREAM) + .shuffleGrouping("kafka_spout", TOPIC_2_STREAM); + tp.setBolt("kafka_bolt_1", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", TOPIC_2_STREAM); return tp.createTopology(); } - protected KafkaSpoutConfig getKafkaSpoutConfig(KafkaSpoutStreams kafkaSpoutStreams) { - return new KafkaSpoutConfig.Builder(getKafkaConsumerProps(), kafkaSpoutStreams, getTuplesBuilder(), getRetryService()) + public static Func, List> TOPIC_PART_OFF_KEY_VALUE_FUNC = new Func, List>() { + @Override + public List apply(ConsumerRecord r) { + return new Values(r.topic(), r.partition(), r.offset(), r.key(), r.value()); + } + }; + + protected KafkaSpoutConfig getKafkaSpoutConfig() { + ByTopicRecordTranslator trans = new ByTopicRecordTranslator<>( + TOPIC_PART_OFF_KEY_VALUE_FUNC, + new Fields("topic", "partition", "offset", "key", "value"), TOPIC_0_1_STREAM); + trans.forTopic(TOPICS[2], + TOPIC_PART_OFF_KEY_VALUE_FUNC, + new Fields("topic", "partition", "offset", "key", "value"), TOPIC_2_STREAM); + return KafkaSpoutConfig.builder("127.0.0.1:9092", TOPICS) + .setGroupId("kafkaSpoutTestGroup") + .setRetry(getRetryService()) + .setRecordTranslator(trans) .setOffsetCommitPeriodMs(10_000) .setFirstPollOffsetStrategy(EARLIEST) .setMaxUncommittedOffsets(250) @@ -106,30 +124,4 @@ protected KafkaSpoutRetryService getRetryService() { return new KafkaSpoutRetryExponentialBackoff(TimeInterval.microSeconds(500), TimeInterval.milliSeconds(2), Integer.MAX_VALUE, TimeInterval.seconds(10)); } - - protected Map getKafkaConsumerProps() { - Map props = new HashMap<>(); -// props.put(KafkaSpoutConfig.Consumer.ENABLE_AUTO_COMMIT, "true"); - props.put(KafkaSpoutConfig.Consumer.BOOTSTRAP_SERVERS, "127.0.0.1:9092"); - props.put(KafkaSpoutConfig.Consumer.GROUP_ID, "kafkaSpoutTestGroup"); - props.put(KafkaSpoutConfig.Consumer.KEY_DESERIALIZER, "org.apache.kafka.common.serialization.StringDeserializer"); - props.put(KafkaSpoutConfig.Consumer.VALUE_DESERIALIZER, "org.apache.kafka.common.serialization.StringDeserializer"); - return props; - } - - protected KafkaSpoutTuplesBuilder getTuplesBuilder() { - return new KafkaSpoutTuplesBuilderNamedTopics.Builder<>( - new TopicsTest0Test1TupleBuilder(TOPICS[0], TOPICS[1]), - new TopicTest2TupleBuilder(TOPICS[2])) - .build(); - } - - protected KafkaSpoutStreams getKafkaSpoutStreams() { - final Fields outputFields = new Fields("topic", "partition", "offset", "key", "value"); - final Fields outputFields1 = new Fields("topic", "partition", "offset"); - return new KafkaSpoutStreamsNamedTopics.Builder(outputFields, STREAMS[0], new String[]{TOPICS[0], TOPICS[1]}) // contents of topics test, test1, sent to test_stream - .addStream(outputFields, STREAMS[0], new String[]{TOPICS[2]}) // contents of topic test2 sent to test_stream - .addStream(outputFields1, STREAMS[2], new String[]{TOPICS[2]}) // contents of topic test2 sent to test2_stream - .build(); - } } diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java index c362a2b9f6c..8b967fad484 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java @@ -18,22 +18,23 @@ package org.apache.storm.kafka.spout.test; +import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; + +import java.util.List; +import java.util.regex.Pattern; + +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.storm.generated.StormTopology; +import org.apache.storm.kafka.spout.Func; import org.apache.storm.kafka.spout.KafkaSpout; -import org.apache.storm.kafka.spout.KafkaSpoutStream; -import org.apache.storm.kafka.spout.KafkaSpoutStreams; -import org.apache.storm.kafka.spout.KafkaSpoutStreamsWildcardTopics; -import org.apache.storm.kafka.spout.KafkaSpoutTupleBuilder; -import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilder; -import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilderWildcardTopics; +import org.apache.storm.kafka.spout.KafkaSpoutConfig; import org.apache.storm.topology.TopologyBuilder; import org.apache.storm.tuple.Fields; - -import java.util.regex.Pattern; +import org.apache.storm.tuple.Values; public class KafkaSpoutTopologyMainWildcardTopics extends KafkaSpoutTopologyMainNamedTopics { private static final String STREAM = "test_wildcard_stream"; - private static final String TOPIC_WILDCARD_PATTERN = "test[1|2]"; + private static final Pattern TOPIC_WILDCARD_PATTERN = Pattern.compile("test[1|2]"); public static void main(String[] args) throws Exception { new KafkaSpoutTopologyMainWildcardTopics().runMain(args); @@ -41,22 +42,27 @@ public static void main(String[] args) throws Exception { protected StormTopology getTopolgyKafkaSpout() { final TopologyBuilder tp = new TopologyBuilder(); - tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams())), 1); + tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig()), 1); tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAM); return tp.createTopology(); } - protected KafkaSpoutTuplesBuilder getTuplesBuilder() { - return new KafkaSpoutTuplesBuilderWildcardTopics<>(getTupleBuilder()); - } - - protected KafkaSpoutTupleBuilder getTupleBuilder() { - return new TopicsTest0Test1TupleBuilder<>(TOPIC_WILDCARD_PATTERN); - } - - protected KafkaSpoutStreams getKafkaSpoutStreams() { - final Fields outputFields = new Fields("topic", "partition", "offset", "key", "value"); - final KafkaSpoutStream kafkaSpoutStream = new KafkaSpoutStream(outputFields, STREAM, Pattern.compile(TOPIC_WILDCARD_PATTERN)); - return new KafkaSpoutStreamsWildcardTopics(kafkaSpoutStream); + public static Func, List> TOPIC_PART_OFF_KEY_VALUE_FUNC = new Func, List>() { + @Override + public List apply(ConsumerRecord r) { + return new Values(r.topic(), r.partition(), r.offset(), r.key(), r.value()); + } + }; + + protected KafkaSpoutConfig getKafkaSpoutConfig() { + return KafkaSpoutConfig.builder("127.0.0.1:9092", TOPIC_WILDCARD_PATTERN) + .setGroupId("kafkaSpoutTestGroup") + .setRetry(getRetryService()) + .setRecordTranslator(TOPIC_PART_OFF_KEY_VALUE_FUNC, + new Fields("topic", "partition", "offset", "key", "value"), STREAM) + .setOffsetCommitPeriodMs(10_000) + .setFirstPollOffsetStrategy(EARLIEST) + .setMaxUncommittedOffsets(250) + .build(); } } diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicTest2TupleBuilder.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicTest2TupleBuilder.java deleted file mode 100644 index ca65177951b..00000000000 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicTest2TupleBuilder.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.test; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.storm.kafka.spout.KafkaSpoutTupleBuilder; -import org.apache.storm.tuple.Values; - -import java.util.List; - -public class TopicTest2TupleBuilder extends KafkaSpoutTupleBuilder { - /** - * @param topics list of topics that use this implementation to build tuples - */ - public TopicTest2TupleBuilder(String... topics) { - super(topics); - } - - @Override - public List buildTuple(ConsumerRecord consumerRecord) { - return new Values(consumerRecord.topic(), - consumerRecord.partition(), - consumerRecord.offset()); - } -} \ No newline at end of file diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicsTest0Test1TupleBuilder.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicsTest0Test1TupleBuilder.java deleted file mode 100644 index 4c55aa19c9e..00000000000 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicsTest0Test1TupleBuilder.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.test; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.storm.kafka.spout.KafkaSpoutTupleBuilder; -import org.apache.storm.tuple.Values; - -import java.util.List; - -public class TopicsTest0Test1TupleBuilder extends KafkaSpoutTupleBuilder { - /** - * @param topics list of topics that use this implementation to build tuples - */ - public TopicsTest0Test1TupleBuilder(String... topics) { - super(topics); - } - - @Override - public List buildTuple(ConsumerRecord consumerRecord) { - return new Values(consumerRecord.topic(), - consumerRecord.partition(), - consumerRecord.offset(), - consumerRecord.key(), - consumerRecord.value()); - } -} \ No newline at end of file diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java index 19f04527883..4c5dba5a369 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java @@ -41,7 +41,7 @@ public ConnectionInfo(SimpleConsumer consumer) { } } - Map _connections = new HashMap(); + Map _connections = new HashMap<>(); KafkaConfig _config; IBrokerReader _reader; diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java index d2bd313bd1d..4608963660a 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java @@ -61,7 +61,7 @@ public KafkaSpout(SpoutConfig spoutConf) { public void open(Map conf, final TopologyContext context, final SpoutOutputCollector collector) { _collector = collector; String topologyInstanceId = context.getStormId(); - Map stateConf = new HashMap(conf); + Map stateConf = new HashMap<>(conf); List zkServers = _spoutConfig.zkServers; if (zkServers == null) { zkServers = (List) conf.get(Config.STORM_ZOOKEEPER_SERVERS); diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java index 8c22118c607..f23c873ddf4 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java @@ -110,7 +110,7 @@ private class TopicMetrics { @Override public Object getValueAndReset() { try { - HashMap ret = new HashMap(); + HashMap ret = new HashMap<>(); if (_partitions != null && _partitions.size() == _partitionToOffset.size()) { Map topicMetricsMap = new TreeMap(); for (Map.Entry e : _partitionToOffset.entrySet()) { diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java index 79e7c3d223b..db5558d38d4 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java @@ -133,7 +133,7 @@ public PartitionManager(DynamicPartitionConnections connections, String topology } public Map getMetricsDataMap() { - Map ret = new HashMap(); + Map ret = new HashMap<>(); ret.put(_partition + "/fetchAPILatencyMax", _fetchAPILatencyMax.getValueAndReset()); ret.put(_partition + "/fetchAPILatencyMean", _fetchAPILatencyMean.getValueAndReset()); ret.put(_partition + "/fetchAPICallCount", _fetchAPICallCount.getValueAndReset()); diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java index bdbc44d2b15..628bfc0ac59 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java @@ -24,7 +24,7 @@ public class StaticCoordinator implements PartitionCoordinator { Map _managers = new HashMap(); - List _allManagers = new ArrayList(); + List _allManagers = new ArrayList<>(); public StaticCoordinator(DynamicPartitionConnections connections, Map stormConf, SpoutConfig config, ZkState state, int taskIndex, int totalTasks, String topologyInstanceId) { StaticHosts hosts = (StaticHosts) config.hosts; @@ -34,7 +34,7 @@ public StaticCoordinator(DynamicPartitionConnections connections, Map stormConf, for (Partition myPartition : myPartitions) { _managers.put(myPartition, new PartitionManager(connections, topologyInstanceId, state, stormConf, config, myPartition)); } - _allManagers = new ArrayList(_managers.values()); + _allManagers = new ArrayList<>(_managers.values()); } @Override diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java index 31eaac5bf1c..b5bb124ce0e 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java @@ -51,7 +51,9 @@ * This bolt uses 0.8.2 Kafka Producer API. *

* It works for sending tuples to older Kafka version (0.8.1). + * @deprecated Please use the KafkaBolt in storm-kafka-client */ +@Deprecated public class KafkaBolt extends BaseRichBolt { private static final Logger LOG = LoggerFactory.getLogger(KafkaBolt.class); diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java index 33632529310..46cc60d0f49 100644 --- a/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java +++ b/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java @@ -54,7 +54,7 @@ public class DynamicBrokersReaderTest { public void setUp() throws Exception { server = new TestingServer(); String connectionString = server.getConnectString(); - Map conf = new HashMap(); + Map conf = new HashMap<>(); conf.put(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT, 1000); conf.put(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT, 1000); conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 4); @@ -64,7 +64,7 @@ public void setUp() throws Exception { zookeeper = CuratorFrameworkFactory.newClient(connectionString, retryPolicy); dynamicBrokersReader = new DynamicBrokersReader(conf, connectionString, masterPath, topic); - Map conf2 = new HashMap(); + Map conf2 = new HashMap<>(); conf2.putAll(conf); conf2.put("kafka.topic.wildcard.match",true); @@ -240,7 +240,7 @@ public void testSwitchHostForPartition() throws Exception { @Test(expected = NullPointerException.class) public void testErrorLogsWhenConfigIsMissing() throws Exception { String connectionString = server.getConnectString(); - Map conf = new HashMap(); + Map conf = new HashMap<>(); conf.put(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT, 1000); // conf.put(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT, 1000); conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 4); diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java index 7a6073aa3d5..864eaa90cbd 100644 --- a/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java +++ b/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java @@ -42,7 +42,7 @@ public class TridentKafkaTest { public void setup() { broker = new KafkaTestBroker(); simpleConsumer = TestUtils.getKafkaConsumer(broker); - TridentTupleToKafkaMapper mapper = new FieldNameBasedTupleToKafkaMapper("key", "message"); + TridentTupleToKafkaMapper mapper = new FieldNameBasedTupleToKafkaMapper("key", "message"); KafkaTopicSelector topicSelector = new DefaultTopicSelector(TestUtils.TOPIC); state = new TridentKafkaState() .withKafkaTopicSelector(topicSelector) diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java index 65bf0b45ee1..364da3314f3 100644 --- a/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java +++ b/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java @@ -124,7 +124,7 @@ private void assertPartitionsAreDifferent(List partitionManage } private List> getPartitionManagers(List coordinatorList) { - List> partitions = new ArrayList(); + List> partitions = new ArrayList<>(); for (ZkCoordinator coordinator : coordinatorList) { partitions.add(coordinator.getMyManagedPartitions()); } diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java index 180828ecf95..58e52e8934e 100644 --- a/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java +++ b/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java @@ -52,6 +52,7 @@ import java.nio.ByteBuffer; import java.util.List; +import java.util.Map; import java.util.HashMap; import java.util.Properties; import java.util.concurrent.Future; @@ -287,7 +288,8 @@ private boolean verifyMessage(String key, String message) { private Tuple generateTestTuple(Object key, Object message) { TopologyBuilder builder = new TopologyBuilder(); - GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(), new Config(), new HashMap(), new HashMap(), new HashMap(), "") { + GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(), new Config(), new HashMap(), + new HashMap>(), new HashMap>(), "") { @Override public Fields getComponentOutputFields(String componentId, String streamId) { return new Fields("key", "message"); @@ -298,7 +300,8 @@ public Fields getComponentOutputFields(String componentId, String streamId) { private Tuple generateTestTuple(Object message) { TopologyBuilder builder = new TopologyBuilder(); - GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(), new Config(), new HashMap(), new HashMap(), new HashMap(), "") { + GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(), new Config(), new HashMap(), + new HashMap>(), new HashMap>(), "") { @Override public Fields getComponentOutputFields(String componentId, String streamId) { return new Fields("message"); diff --git a/storm-core/src/jvm/org/apache/storm/tuple/Fields.java b/storm-core/src/jvm/org/apache/storm/tuple/Fields.java index 840b2d32e2e..a7717482b8b 100644 --- a/storm-core/src/jvm/org/apache/storm/tuple/Fields.java +++ b/storm-core/src/jvm/org/apache/storm/tuple/Fields.java @@ -29,6 +29,7 @@ * Collection of unique named fields using in an ITuple */ public class Fields implements Iterable, Serializable { + private static final long serialVersionUID = -3377931843059975424L; private List _fields; private Map _index = new HashMap<>(); @@ -122,5 +123,20 @@ private void index() { @Override public String toString() { return _fields.toString(); - } + } + + @Override + public boolean equals(Object other) { + if (this == other) return true; + if (other instanceof Fields) { + Fields of = (Fields)other; + return _fields.equals(of._fields); + } + return false; + } + + @Override + public int hashCode() { + return _fields.hashCode(); + } }