You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by revans2 <gi...@git.apache.org> on 2016/12/01 14:34:36 UTC

[GitHub] storm pull request #1808: STORM-1997: copy state/bolt from storm-kafka to st...

GitHub user revans2 opened a pull request:

    https://github.com/apache/storm/pull/1808

    STORM-1997: copy state/bolt from storm-kafka to storm-kafka-client

    
    STORM-2228: removed ability to request a single topic go to multiple streams

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/revans2/incubator-storm STORM-2225

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/storm/pull/1808.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1808
    
----
commit 32da09c1b205e0e2822e959674557b7c61842a40
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Date:   2016-11-30T03:39:26Z

    STORM-1997: copy state/bolt from storm-kafka to storm-kafka-client
    STORM-2225: change spout config to be simpler.
    STORM-2228: removed ability to request a single topic go to multiple streams

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96756536
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Subscription.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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 <K, V> void subscribe(KafkaConsumer<K,V> consumer, ConsumerRebalanceListener listener, TopologyContext context) {
    +    	subscribe(consumer, listener);
    +    }
    +
    +    /**
    +     * Subscribe the KafkaConsumer to the proper topics
    +     * @param consumer the Consumer to get.
    +     * @param listener the rebalance listener to include in the subscription
    +     * @deprecated please use the version with the TopologyContext in it
    +     */
    +    public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceListener listener) {
    --- End diff --
    
    Why have deprecated methods in the first implementation of this class ? Shouldn't first implementation be `@deprecated` free ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    @srdo I think I addressed all of your review comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r97187057
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java ---
    @@ -1,16 +1,35 @@
    +/*
    + * 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.common.TopicPartition;
    -
     import java.util.Comparator;
     
    +import org.apache.kafka.common.TopicPartition;
    +
     public class TopicPartitionComparator implements Comparator<TopicPartition> {
    -    @Override
    -    public int compare(TopicPartition o1, TopicPartition o2) {
    -        if (!o1.topic().equals(o2.topic())) {
    -            return o1.topic().compareTo(o2.topic());
    -        } else {
    -            return o1.partition() - o2.partition();
    -        }
    -    }
    +	public static final TopicPartitionComparator INSTANCE = new TopicPartitionComparator();
    --- End diff --
    
    @srdo that is not true. See Effective Java Item 3. Yes, private constructor would work.
    
    @revans2 either way is fine. My comment was motivated because I believe that either we should make it a singleton or not. If not, let the users simple instantiate it with new and that's it. I am a believer in consistency.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96721101
  
    --- Diff: external/storm-kafka-client/README.md ---
    @@ -1,191 +1,5 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Apache Kafka integration using the kafka-client jar (This inclused the new Apache Kafka consumer API)
    --- End diff --
    
    inclused -> includes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96923940
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java ---
    @@ -0,0 +1,114 @@
    +/**
    + * 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.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) {
    --- End diff --
    
    This is code that was "moved" like with the KafkaBolt for storm-kafka to storm-kafka-client.  If we really want to make it immutable we can, but I think that is beyond the scope of this JIRA  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    The test failures are unrelated and are around the integration tests that always seem to fail lately.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91364222
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RecordTranslator.java ---
    @@ -0,0 +1,59 @@
    +/*
    + * 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.Arrays;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.function.Function;
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.storm.tuple.Fields;
    +
    +/**
    + * Translates a Kafka ConsumerRecord into a tuple
    --- End diff --
    
    Consider adding a period at the end here, when shown as javadoc this is going to look weird otherwise.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    The test failures are unrelated


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96884451
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -225,11 +213,7 @@ public void nextTuple() {
                     }
     
                     if (poll()) {
    -                    try {
    -                        setWaitingToEmit(pollKafkaBroker());
    -                    } catch (RetriableException e) {
    -                        LOG.error("Failed to poll from kafka.", e);
    -                    }
    +                    setWaitingToEmit(pollKafkaBroker());
    --- End diff --
    
    That's right it was a merge error.  Great catch though.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91351636
  
    --- Diff: examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java ---
    @@ -18,87 +18,48 @@
     
     package org.apache.storm.kafka.trident;
     
    -import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
    +
    +import java.util.Arrays;
    +import java.util.concurrent.TimeUnit;
    +
     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.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<String, String> newKafkaTridentSpoutOpaque() {
    -        return new KafkaTridentSpoutOpaque<>(new KafkaTridentSpoutManager<>(
    -                        newKafkaSpoutConfig(
    -                        newKafkaSpoutStreams())));
    +        return new KafkaTridentSpoutOpaque<>(newKafkaSpoutConfig());
         }
     
    -    private KafkaSpoutConfig<String,String> newKafkaSpoutConfig(KafkaSpoutStreams kafkaSpoutStreams) {
    -        return new KafkaSpoutConfig.Builder<>(newKafkaConsumerProps(),
    -                    kafkaSpoutStreams, newTuplesBuilder(), newRetryService())
    +    protected KafkaSpoutConfig<String,String> newKafkaSpoutConfig() {
    +        return KafkaSpoutConfig.builder("127.0.0.1:9092", TOPIC_1, TOPIC_2)
    --- End diff --
    
    Consider using KAFKA_LOCAL_BROKER here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91354645
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and 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` use to support decided which topic should to push message from tuple.
    +User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
    +When the topic name not found , `KafkaBolt` will write messages into default topic .
    --- End diff --
    
    Nit: I think it would be clearer if this referred to Field*TopicSelector instead of KafkaBolt, I misunderstood it on first read.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96694593
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,232 @@
    -#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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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.
    +
    +There are a few key configs to pay attention to.
    +
    +`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
    +
    + * `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 `ConsumerRecord` into a Tuple and which stream that tuple will go to.  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
    +
    +#### Create a Simple Insecure Spout
    +The following will consume all events published to "topic" and send them to MyBolt as "topic", "partition", "offset", "key", "value".
     ```java
    -KafkaSpoutStreams kafkaSpoutStreams = new KafkaSpoutStreamsWildcardTopics(
    -            new KafkaSpoutStream(outputFields, STREAM, Pattern.compile(TOPIC_WILDCARD_PATTERN)));
     
    -KafkaSpoutTuplesBuilder<String, String> tuplesBuilder = new TopicsTest0Test1TupleBuilder<>(TOPIC_WILDCARD_PATTERN);
    +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");
    +...
     
    -String STREAM = "test_wildcard_stream";
    -String TOPIC_WILDCARD_PATTERN = "test[1|2]";
    +```
    +
    +#### Wildcard Topics
    +Wildcard topics will consume from all topics that are on the configured broakers and match the pattern.  So in the following example
    +"topic", "topic_foo" and "topic_bar" will all match the pattern, but "not_my_topic" would not match. 
    --- End diff --
    
    the pattern "topic.*"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96762352
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedSubscription.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.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<String> topics;
    +    
    +    public NamedSubscription(Collection<String> topics) {
    +        super();
    +        this.topics = Collections.unmodifiableCollection(new ArrayList<>(topics));
    +    }
    +    
    +    public NamedSubscription(String ... topics) {
    +        this(Arrays.asList(topics));
    +    }
    +
    +    @Override
    +    public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceListener listener, TopologyContext unused) {
    +        consumer.subscribe(topics, listener);
    +        LOG.info("Kafka consumer subscribed topics {}", topics);
    +    }
    +
    +    @Override
    +    public String getTopicsString() {
    --- End diff --
    
    Javadoc saying that it is a comma separated String of all the topic names


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    @srdo as part of backporting this to 1.x I am going to need to make a change to not use Function directly, because it is only in java 8.  So to maintain compatibility between 1.x and 2.x I am going to need to make a few changes in this patch too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96773771
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedSubscription.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.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<String> topics;
    +    
    +    public NamedSubscription(Collection<String> topics) {
    +        super();
    --- End diff --
    
    super not necessary


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96730185
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -235,41 +355,116 @@ public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStrea
                 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<K, V> setPartitionRefreshPeriodMs(long partitionRefreshPeriodMs) {
    -            this.partitionRefreshPeriodMs = partitionRefreshPeriodMs;
    +        public Builder<K, V> setRetry(KafkaSpoutRetryService retryService) {
    +            if (retryService == null) {
    +                throw new NullPointerException("retryService cannot be null");
    +            }
    +            this.retryService = retryService;
                 return this;
             }
     
    +        public Builder<K, V> setRecordTranslator(RecordTranslator<K, V> translator) {
    +            this.translator = translator;
    +            return this;
    +        }
    +        
    +        public Builder<K, V> setRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
    +            return setRecordTranslator(new SimpleRecordTranslator<>(func, fields));
    +        }
    +        
    +        public Builder<K, V> setRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
    +            return setRecordTranslator(new SimpleRecordTranslator<>(func, fields, stream));
    +        }
    +        
             /**
    -         * 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.
    +         * Sets partition refresh period in milliseconds. This is how often the subscription is refreshed
    +         * For most subscriptions that go through the KafkaConsumer.subscribe this is ignored.
    +         * @param partitionRefreshPeriodMs time in milliseconds
    +         * @return the builder (this)
              */
    -        public Builder<K, V> setManualPartitionAssignment(boolean manualPartitionAssignment) {
    -            this.manualPartitionAssignment = manualPartitionAssignment;
    -            return this;
    +        public Builder<K, V> setPartitionRefreshPeriodMs(long partitionRefreshPeriodMs) {
    +        	this.partitionRefreshPeriodMs = partitionRefreshPeriodMs;
    +        	return this;
             }
    -
    +        
             public KafkaSpoutConfig<K,V> build() {
                 return new KafkaSpoutConfig<>(this);
             }
         }
     
    +    // Kafka consumer configuration
    +    private final Map<String, Object> kafkaProps;
    +    private final Subscription subscription;
    +    private final SerializableDeserializer<K> keyDes;
    +    private final Class<? extends Deserializer<K>> keyDesClazz;
    +    private final SerializableDeserializer<V> valueDes;
    +    private final Class<? extends Deserializer<V>> valueDesClazz;
    +    private final long pollTimeoutMs;
    +
    +    // Kafka spout configuration
    +    private final RecordTranslator<K, V> 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<K,V> 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<String, Object> getKafkaProps() {
             return kafkaProps;
         }
     
         public Deserializer<K> getKeyDeserializer() {
    -        return keyDeserializer;
    +    	if (keyDesClazz != null) {
    +    		try {
    --- End diff --
    
    Tabs


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96743744
  
    --- Diff: 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 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) {
    --- End diff --
    
    This logic could be incorporated in the `FieldIndexTopicSelector`. It should be possible to delete this class, rename `FieldIndexTopicSelector` to `FieldTopicSelector`, and provide two constructors. One with `fieldName` and another with `fieldIndex`. And then use `Tuple#fieldIndex(String field)` method to extract one from the other.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96774279
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RecordTranslator.java ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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 ConsumerRecord to a tuple.
    + */
    +public interface RecordTranslator<K, V> extends Serializable, Func<ConsumerRecord<K, V>, List<Object>> {
    +    public static final List<String> DEFAULT_STREAM = Collections.singletonList("default");
    +    
    +    /**
    +     * Do the translation.
    --- End diff --
    
    Creates a tuple from a Kafka ConsumerRecord.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96766970
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedSubscription.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.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<String> topics;
    +    
    +    public NamedSubscription(Collection<String> topics) {
    +        super();
    --- End diff --
    
    super not necessary


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91362618
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -60,123 +68,197 @@
          * If no offset has been committed, it behaves as LATEST.</li>
          * </ul>
          * */
    -    public enum FirstPollOffsetStrategy {
    +    public static enum FirstPollOffsetStrategy {
             EARLIEST,
             LATEST,
             UNCOMMITTED_EARLIEST,
             UNCOMMITTED_LATEST }
    -
    -    // Kafka consumer configuration
    -    private final Map<String, Object> kafkaProps;
    -    private final Deserializer<K> keyDeserializer;
    -    private final Deserializer<V> valueDeserializer;
    -    private final long pollTimeoutMs;
    -
    -    // Kafka spout configuration
    -    private final long offsetCommitPeriodMs;
    -    private final int maxRetries;
    -    private final int maxUncommittedOffsets;
    -    private final FirstPollOffsetStrategy firstPollOffsetStrategy;
    -    private final KafkaSpoutStreams kafkaSpoutStreams;
    -    private final KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;
    -    private final KafkaSpoutRetryService retryService;
    -
    -    private KafkaSpoutConfig(Builder<K,V> 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.tuplesBuilder = builder.tuplesBuilder;
    -        this.retryService = builder.retryService;
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, String ... topics) {
    +        return new Builder<>(bootstrapServers, new StringDeserializer(), new StringDeserializer(), topics);
         }
    -
    -    private Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Collection<String> topics) {
    +        return new Builder<>(bootstrapServers, new StringDeserializer(), new StringDeserializer(), topics);
    +    }
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Pattern topics) {
    +        return new Builder<>(bootstrapServers, new StringDeserializer(), new StringDeserializer(), topics);
    +    }
    +    
    +    private static Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
             // set defaults for properties not specified
    -        if (!kafkaProps.containsKey(Consumer.ENABLE_AUTO_COMMIT)) {
    -            kafkaProps.put(Consumer.ENABLE_AUTO_COMMIT, "false");
    +        if (!kafkaProps.containsKey(ENABLE_AUTO_COMMIT_CONF)) {
    +            kafkaProps.put(ENABLE_AUTO_COMMIT_CONF, "false");
             }
             return kafkaProps;
         }
    -
    +    
         public static class Builder<K,V> {
             private final Map<String, Object> kafkaProps;
    -        private Deserializer<K> keyDeserializer;
    -        private Deserializer<V> valueDeserializer;
    +        private Subscription subscription;
    +        private final Deserializer<K> keyDes;
    +        private final Deserializer<V> valueDes;
    +        private RecordTranslator<K, V> 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 final KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;
    -        private final KafkaSpoutRetryService retryService;
    -
    -        /**
    -         * Please refer to javadoc in {@link #Builder(Map, KafkaSpoutStreams, KafkaSpoutTuplesBuilder, KafkaSpoutRetryService)}.<p/>
    -         * This constructor uses by the default the following implementation for {@link KafkaSpoutRetryService}:<p/>
    -         * {@code new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -         *           DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)))}
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> tuplesBuilder) {
    -            this(kafkaProps, kafkaSpoutStreams, tuplesBuilder,
    -                    new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -                            DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)));
    +        private KafkaSpoutRetryService retryService = DEFAULT_RETRY_SERVICE;
    +        
    +        public Builder(String bootstrapServers, Deserializer<K> keyDes, Deserializer<V> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
             }
    -
    -        /***
    -         * 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 <a href="http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html">KafkaConsumer</a>
    -         * @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
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> 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");
    +        
    +        public Builder(String bootstrapServers, Deserializer<K> keyDes, Deserializer<V> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Deserializer<K> keyDes, Deserializer<V> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Deserializer<K> keyDes, Deserializer<V> valDes, Subscription subscription) {
    +            kafkaProps = new HashMap<>();
    +            if (bootstrapServers == null || bootstrapServers.isEmpty()) {
    +                throw new IllegalArgumentException("bootstrap servers cannot be null");
                 }
    +            kafkaProps.put(BOOTSTRAP_SERVERS_CONF, bootstrapServers);
    +            this.keyDes = keyDes;
    +            this.valueDes = valDes;
    +            this.subscription = subscription;
    +            this.translator = new DefaultRecordTranslator<K,V>();
    +        }
     
    -            this.kafkaProps = kafkaProps;
    -            this.kafkaSpoutStreams = kafkaSpoutStreams;
    -            this.tuplesBuilder = tuplesBuilder;
    -            this.retryService = retryService;
    +        private Builder(Builder<?, ?> builder, Deserializer<K> keyDes, Deserializer<V> valueDes) {
    +            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 this is the only way to really maintain API compatibility
    +            this.translator = (RecordTranslator<K, V>) builder.translator;
    +            this.retryService = builder.retryService;
    +            this.keyDes = keyDes;
    +            this.valueDes = valueDes;
             }
     
             /**
              * Specifying this key deserializer overrides the property key.deserializer
              */
    -        public Builder<K,V> setKeyDeserializer(Deserializer<K> keyDeserializer) {
    -            this.keyDeserializer = keyDeserializer;
    -            return this;
    +        public <NK> Builder<NK,V> setKeyDeserializer(Deserializer<NK> keyDeserializer) {
    +            return new Builder<>(this, keyDeserializer, valueDes);
             }
     
             /**
              * Specifying this value deserializer overrides the property value.deserializer
              */
    -        public Builder<K,V> setValueDeserializer(Deserializer<V> valueDeserializer) {
    -            this.valueDeserializer = valueDeserializer;
    +        public <NV> Builder<K,NV> setValueDeserializer(Deserializer<NV> valueDeserializer) {
    +            return new Builder<>(this, this.keyDes, valueDeserializer);
    +        }
    +        
    +        /**
    +         * Set a Kafka property config
    +         */
    +        public Builder<K,V> setProp(String key, Object value) {
    +            kafkaProps.put(key, value);
                 return this;
             }
    +        
    +        /**
    +         * Set multiple Kafka property configs
    +         */
    +        public Builder<K,V> setProp(Map<String, Object> props) {
    +            kafkaProps.putAll(props);
    +            return this;
    +        }
    +        
    +        /**
    +         * Set multiple Kafka property configs
    +         */
    +        public Builder<K,V> 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<K,V> setGroupId(String id) {
    +            return setProp("group.id", id);
    +        }
    +        
    +        /**
    +         * reset the bootstrap servers for the Consumer
    +         */
    +        public Builder<K,V> setBootstrapServers(String servers) {
    +            return setProp(BOOTSTRAP_SERVERS_CONF, servers);
    +        }
    +        
    +        /**
    +         * The minimum amount of data the broker should return for a fetch request.
    +         */
    +        public Builder<K,V> setFetchMinBytes(int bytes) {
    +            return setProp("fetch.min.bytes", bytes);
    --- End diff --
    
    Nit: Consider using some of the constants in `org.apache.kafka.clients.consumer.ConsumerConfig` instead


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96897291
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -325,15 +310,19 @@ private boolean emitTupleIfNotEmitted(ConsumerRecord<K, V> 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<Object> tuple = tuplesBuilder.buildTuple(record);
    -                kafkaSpoutStreams.emit(collector, tuple, msgId);
    +                final List<Object> tuple = kafkaSpoutConfig.getTranslator().apply(record);
    +                if (tuple instanceof KafkaTuple) {
    --- End diff --
    
    The javadocs for RecordTranslator state.
    ```
         * @return the objects in the tuple.  Return a {@link KafkaTuple}
         * if you want to route the tuple to a non-default stream
    ```
    
    All of the provided implementations support this.  Both Simple and ByTopic, by way of the SimpleRecordTranslator.
    
    I will add a section in the docs to talk about this, but it only really matters if you are writing your own record translator from scratch, instead of using the built in ones that should cover the vast majority of the use cases.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91367555
  
    --- Diff: external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/ByTopicRecordTranslatorTest.java ---
    @@ -0,0 +1,72 @@
    +/*
    + * 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 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 {
    +
    +    @Test
    +    public void testBasic() {
    +        ByTopicRecordTranslator<String, String> trans = 
    +                new ByTopicRecordTranslator<>((r) -> new Values(r.key()), new Fields("key"));
    +        trans.forTopic("TOPIC 1", (r) -> new Values(r.value()), new Fields("value"), "value-stream");
    +        trans.forTopic("TOPIC 2", (r) -> new Values(r.key(), r.value()), new Fields("key", "value"), "key-value-stream");
    +        HashSet<String> expectedStreams = new HashSet<>();
    +        expectedStreams.add("default");
    +        expectedStreams.add("value-stream");
    +        expectedStreams.add("key-value-stream");
    +        assertEquals(expectedStreams, new HashSet<>(trans.streams()));
    +
    +        ConsumerRecord<String, String> 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<String, String> 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<String, String> 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 testFiledCollision() {
    --- End diff --
    
    filed -> field


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96894587
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ByTopicRecordTranslator.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 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;
    +
    +public class ByTopicRecordTranslator<K, V> implements RecordTranslator<K, V> {
    +    private static final long serialVersionUID = -121699733778988688L;
    +    private final RecordTranslator<K,V> defaultTranslator;
    +    private final Map<String, RecordTranslator<K,V>> topicToTranslator = new HashMap<>();
    +    private final Map<String, Fields> streamToFields = new HashMap<>();
    +    
    +    public ByTopicRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
    +        this(new SimpleRecordTranslator<>(func, fields, stream));
    +    }
    +    
    +    public ByTopicRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
    +        this(new SimpleRecordTranslator<>(func, fields));
    +    }
    +    
    +    public ByTopicRecordTranslator(RecordTranslator<K,V> defaultTranslator) {
    +        this.defaultTranslator = defaultTranslator;
    +        cacheNCheckFields(defaultTranslator);
    +    }
    +    
    +    public ByTopicRecordTranslator<K, V> forTopic(String topic, Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
    +        return forTopic(topic, new SimpleRecordTranslator<>(func, fields));
    +    }
    +    
    +    public ByTopicRecordTranslator<K, V> forTopic(String topic, Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
    +        return forTopic(topic, new SimpleRecordTranslator<>(func, fields, stream));
    +    }
    +    
    +    public ByTopicRecordTranslator<K, V> forTopic(String topic, RecordTranslator<K,V> translator) {
    +        if (topicToTranslator.containsKey(topic)) {
    +            throw new IllegalStateException("Topic " + topic + " is already registered");
    +        }
    +        topicToTranslator.put(topic, translator);
    +        cacheNCheckFields(translator);
    +        return this;
    +    }
    +    
    +    private void cacheNCheckFields(RecordTranslator<K, V> 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);
    --- End diff --
    
    In this case it is the argument being passed in that is bad and we are rejecting it.  They could come back and switch it to a new stream which would be fine.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    I like the new configuration design. Backward compatibility is not an issue for us. I'm wondering if enough people have even switched to the new spout yet to make backward compatibility for 1.x a must.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91345785
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
    --- End diff --
    
    Nitpick: Compatibility is misspelled


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96908491
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedSubscription.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.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<String> topics;
    +    
    +    public NamedSubscription(Collection<String> topics) {
    +        super();
    +        this.topics = Collections.unmodifiableCollection(new ArrayList<>(topics));
    +    }
    +    
    +    public NamedSubscription(String ... topics) {
    +        this(Arrays.asList(topics));
    +    }
    +
    +    @Override
    +    public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceListener listener, TopologyContext unused) {
    +        consumer.subscribe(topics, listener);
    +        LOG.info("Kafka consumer subscribed topics {}", topics);
    +    }
    +
    +    @Override
    +    public String getTopicsString() {
    --- End diff --
    
    It is in the parent javadocs.  That is around the override


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96721536
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    --- End diff --
    
    Is kafka.broker.properties still being used somewhere? It might be good to mention that new code should prefer withTopicSelector over using 'topic' in storm conf.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96908178
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java ---
    @@ -1,16 +1,35 @@
    +/*
    + * 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.common.TopicPartition;
    -
     import java.util.Comparator;
     
    +import org.apache.kafka.common.TopicPartition;
    +
     public class TopicPartitionComparator implements Comparator<TopicPartition> {
    -    @Override
    -    public int compare(TopicPartition o1, TopicPartition o2) {
    -        if (!o1.topic().equals(o2.topic())) {
    -            return o1.topic().compareTo(o2.topic());
    -        } else {
    -            return o1.partition() - o2.partition();
    -        }
    -    }
    +	public static final TopicPartitionComparator INSTANCE = new TopicPartitionComparator();
    --- End diff --
    
    I'm not sure we need to guarantee it.  The old code had the INSTANCE where it was used, I moved it here in hopes that others might use it.  I could make the constructor private if we really want it to be a singleton, but I don't think it is a requirement.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91350461
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and 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` use to support decided which topic should to push message from tuple.
    +User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
    +When the topic name not found , `KafkaBolt` will write messages into default topic .
    +Please make sure the default topic have created .
    +
    +### Specifying Kafka producer properties
    +You can provide all the produce 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.
    +
    +###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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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 prooperties, but it is preferable to be explicet here, again to maintain 
    +type safety with the generics.
    +
    +There are a few key configs to pay attention to.
    +
    +`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
    +
    + * `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 `ConsumerRecord` into a Tuple and which stream that tuple will go to.  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 convinenece method.
    --- End diff --
    
    convinenece -> convenience


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    +1
    
    My only suggestion at this point is that we file an umbrella JIRA covering some of the follow up comments that haven't been addressed, such as minor refactoring, improved (java) docs, immutability, etc. Basically to just capture the most meaningful comments that makes sense handle later.
    
    Thanks @revans2 for all your patience addressing the many review comments


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r97333183
  
    --- Diff: 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 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) {
    --- End diff --
    
    I filed https://issues.apache.org/jira/browse/STORM-2318 for this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96726868
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -412,8 +401,12 @@ 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);
    +
    +        // Initial poll to get the consumer registration process going.
    +        // KafkaSpoutConsumerRebalanceListener will be called following this poll, upon partition registration
    +        kafkaConsumer.poll(0);
    --- End diff --
    
    Doesn't this belong in the automatic subscriptions? It's not necessary if using manual partition assignment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96719617
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    --- End diff --
    
    Can you please add a comment saying to look at the comments in the set method? I missed those comments at first.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r97332427
  
    --- Diff: 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.
      * <p/>
      * It works for sending tuples to older Kafka version (0.8.1).
    + * @deprecated Please use the KafkaBolt in storm-kafka-client
      */
    +@Deprecated
    --- End diff --
    
    I filed https://issues.apache.org/jira/browse/STORM-2317 to do this and I assigned it to myself.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91350132
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and 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` use to support decided which topic should to push message from tuple.
    +User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
    +When the topic name not found , `KafkaBolt` will write messages into default topic .
    +Please make sure the default topic have created .
    +
    +### Specifying Kafka producer properties
    +You can provide all the produce 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.
    +
    +###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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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 prooperties, but it is preferable to be explicet here, again to maintain 
    --- End diff --
    
    explicet -> explicit


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91351118
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and 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` use to support decided which topic should to push message from tuple.
    +User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
    +When the topic name not found , `KafkaBolt` will write messages into default topic .
    +Please make sure the default topic have created .
    +
    +### Specifying Kafka producer properties
    +You can provide all the produce 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.
    +
    +###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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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 prooperties, but it is preferable to be explicet here, again to maintain 
    +type safety with the generics.
    +
    +There are a few key configs to pay attention to.
    +
    +`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
    +
    + * `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 `ConsumerRecord` into a Tuple and which stream that tuple will go to.  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 convinenece 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
    +
    +#### Create a Simple Insecure Spout
    +
     ```java
    -KafkaSpoutStreams kafkaSpoutStreams = new KafkaSpoutStreamsWildcardTopics(
    -            new KafkaSpoutStream(outputFields, STREAM, Pattern.compile(TOPIC_WILDCARD_PATTERN)));
     
    -KafkaSpoutTuplesBuilder<String, String> tuplesBuilder = new TopicsTest0Test1TupleBuilder<>(TOPIC_WILDCARD_PATTERN);
    +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");
    +...
    +
    +```
    +
    +#### Wildcard Topics
    +
    +```java
     
    -String STREAM = "test_wildcard_stream";
    -String TOPIC_WILDCARD_PATTERN = "test[1|2]";
    +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");
    +...
     
    -Fields outputFields = new Fields("topic", "partition", "offset", "key", "value");
     ```
     
    -### Create a simple Toplogy using the Kafka Spout:
    +#### Multiple Streams
    +```java
    +
    +final TopologyBuilder tp = new TopologyBuilder();
     
    +//By default all topics not covered by another rule will be emitted to "STREAM_1" as "topic", "key", and "value"
    +ByTopicRecordTranslator<String, String> 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, Pattern.compile("topic_.*")).build()), 1);
    --- End diff --
    
    Nit: Won't this only consume from topic_2 because the pattern includes _?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91348040
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and 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` use to support decided which topic should to push message from tuple.
    +User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
    +When the topic name not found , `KafkaBolt` will write messages into default topic .
    --- End diff --
    
    "not found" -> "is not found"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    I think I have addressed all of the review comments so far.  I will try to get my 1.x version of the patch up shortly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91349569
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and 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` use to support decided which topic should to push message from tuple.
    +User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
    +When the topic name not found , `KafkaBolt` will write messages into default topic .
    +Please make sure the default topic have created .
    +
    +### Specifying Kafka producer properties
    +You can provide all the produce 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.
    +
    +###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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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");
    --- End diff --
    
    Nit: The property names have constants defined in `org.apache.kafka.clients.producer.ProducerConfig`, might be good to point people to them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91366137
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java ---
    @@ -152,7 +152,9 @@ private long seek(TopicPartition tp, KafkaTridentSpoutBatchMetadata<K, V> lastBa
         }
     
         private Collection<TopicPartition> toArrayList(final TopicPartition tp) {
    -        return new ArrayList<TopicPartition>(1){{add(tp);}};
    +        ArrayList<TopicPartition> ret = new ArrayList<>(1);
    --- End diff --
    
    Nit: Same as in KafkaSpout


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    Everyone,
    
    Sorry this took so long.  I kept getting pulled off on critical issues.  I have rebased/squashed everything (except the code revert) and also fixed the issue with tabs vs spaces.  Since it was just white space issues I don't think I need more +1s but if someone wants to take a look again please feel free to.  I will do the same for the 1.x version, and then file the umbrella jira that @hmcl requested.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    All outstanding review comments should be done now.  This and the 1.x port at #1868 should be ready for a final pass and hopefully being merged in.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    Sure, that makes sense. Maybe storm-kafka-client should have been marked unstable since it's fairly new, it's still in a phase where the kinks are being worked out, even if the version number says differently.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96717629
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    +        this.boltSpecfiedProperties = producerProperties;
    +        return this;
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        //for backward compatibility.
    +        if(mapper == null) {
    +            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
    +        }
    +
    +        //for backward compatibility.
    +        if(topicSelector == null) {
    +            if(stormConf.containsKey(TOPIC)) {
    +                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
    +            } else {
    +                throw new IllegalArgumentException("topic should be specified in bolt's configuration");
    --- End diff --
    
    `IllegalStateException` ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96723152
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    +        this.boltSpecfiedProperties = producerProperties;
    +        return this;
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        //for backward compatibility.
    +        if(mapper == null) {
    +            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
    +        }
    +
    +        //for backward compatibility.
    +        if(topicSelector == null) {
    +            if(stormConf.containsKey(TOPIC)) {
    +                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
    +            } else {
    +                throw new IllegalArgumentException("topic should be specified in bolt's configuration");
    +            }
    +        }
    +
    +        producer = mkProducer(boltSpecfiedProperties);
    +        this.collector = collector;
    +    }
    +    
    +    /**
    +     * Intended to be overridden for tests.  Make the producer from props
    +     */
    +    protected KafkaProducer<K, V> 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<RecordMetadata> result = producer.send(new ProducerRecord<K, V>(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 bold will assume that sending a message to kafka will succeed and will ack
    --- End diff --
    
    bold -> bolt


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96721821
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    +        this.boltSpecfiedProperties = producerProperties;
    +        return this;
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        //for backward compatibility.
    +        if(mapper == null) {
    +            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
    +        }
    +
    +        //for backward compatibility.
    +        if(topicSelector == null) {
    +            if(stormConf.containsKey(TOPIC)) {
    +                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
    +            } else {
    +                throw new IllegalArgumentException("topic should be specified in bolt's configuration");
    +            }
    +        }
    +
    +        producer = mkProducer(boltSpecfiedProperties);
    +        this.collector = collector;
    +    }
    +    
    +    /**
    +     * Intended to be overridden for tests.  Make the producer from props
    +     */
    +    protected KafkaProducer<K, V> 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<RecordMetadata> result = producer.send(new ProducerRecord<K, V>(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 bold 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 we successfully sent to kafka or
    --- End diff --
    
    it was successfully


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91365059
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RecordTranslator.java ---
    @@ -0,0 +1,59 @@
    +/*
    + * 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.Arrays;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.function.Function;
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.storm.tuple.Fields;
    +
    +/**
    + * Translates a Kafka ConsumerRecord into a tuple
    + * use {@link KafkaTuple} if you want to route your tuples
    --- End diff --
    
    Nit: I think it's a little clearer to say "Translate to KafkaTuple if..." or "Return a KafkaTuple if..."


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96779395
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java ---
    @@ -44,29 +43,30 @@
         // Bookkeeping
         private final KafkaSpoutConfig<K, V> 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<K, V> tuplesBuilder;        // Object that contains the logic to build tuples for each ConsumerRecord
    +    private final Fields fields;
     
         public KafkaTridentSpoutManager(KafkaSpoutConfig<K, V> kafkaSpoutConfig) {
             this.kafkaSpoutConfig = kafkaSpoutConfig;
    -        kafkaSpoutStreams = kafkaSpoutConfig.getKafkaSpoutStreams();
    -        tuplesBuilder = kafkaSpoutConfig.getTuplesBuilder();
    +        RecordTranslator<K, V> translator = kafkaSpoutConfig.getTranslator();
    +        Fields fields = null;
    --- End diff --
    
    I would put all of this logic in a method called `setFields`. However, this is going to make this class hard to unit test. So we should even create a static factory method just wrapping this logic, and the constructor with two parameters, `KafkaTridentSpoutManager(KafkaSpoutConfig<K, V> kafkaSpoutConfig, Fields fields)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r97334512
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -447,7 +440,10 @@ private void shutdown() {
     
         @Override
         public void declareOutputFields(OutputFieldsDeclarer declarer) {
    -        kafkaSpoutStreams.declareOutputFields(declarer);
    +        RecordTranslator<K, V> translator = kafkaSpoutConfig.getTranslator();
    --- End diff --
    
    I agree I think we need a fairly standard way to Tuple->X and X->Tuple.  But really it would just end up looking like
    
    ```java
    public interface <X> TupleToX {
        public X fromTuple(Tuple t);
    }
    
    public interface X XToTuple {
        public Tuple toTuple(X record);
    }
    ```
    
    But I don't know how much code reuse there could be, because X is so different for each library we are going to interface with.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96923481
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java ---
    @@ -44,29 +43,30 @@
         // Bookkeeping
         private final KafkaSpoutConfig<K, V> 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<K, V> tuplesBuilder;        // Object that contains the logic to build tuples for each ConsumerRecord
    +    private final Fields fields;
     
         public KafkaTridentSpoutManager(KafkaSpoutConfig<K, V> kafkaSpoutConfig) {
             this.kafkaSpoutConfig = kafkaSpoutConfig;
    -        kafkaSpoutStreams = kafkaSpoutConfig.getKafkaSpoutStreams();
    -        tuplesBuilder = kafkaSpoutConfig.getTuplesBuilder();
    +        RecordTranslator<K, V> translator = kafkaSpoutConfig.getTranslator();
    +        Fields fields = null;
    --- End diff --
    
    I'm a bit confused about what benefit that would give.  The Fields come from the SpoutConfig, by way of the RecordTranslator.  Why would you want to override the Fields but not the RecordTranslator that must conform to those fields?  I just don't see value it separating two things that are highly coupled together.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r95055015
  
    --- Diff: external/storm-kafka-client/README.md ---
    @@ -1,191 +1,5 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +Spouts and Bolts used to interact with Kafka thought the kafka-client library.
    --- End diff --
    
    Nit: "Thought" should be "through"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91622857
  
    --- Diff: 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<String, String> producer = mock(KafkaProducer.class);
    +        when(producer.send(any(), any())).thenAnswer(new Answer<Object>() {
    +            @Override
    +            public Object answer(InvocationOnMock invocation) throws Throwable {
    +                Callback c = (Callback)invocation.getArguments()[1];
    +                c.onCompletion(null, null);
    +                return null;
    +            }
    +        });
    +        KafkaBolt<String, String> bolt = new KafkaBolt<String, String>() {
    --- End diff --
    
    Right, can't do that. Sorry :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96893867
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ByTopicRecordTranslator.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 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;
    +
    +public class ByTopicRecordTranslator<K, V> implements RecordTranslator<K, V> {
    +    private static final long serialVersionUID = -121699733778988688L;
    +    private final RecordTranslator<K,V> defaultTranslator;
    +    private final Map<String, RecordTranslator<K,V>> topicToTranslator = new HashMap<>();
    +    private final Map<String, Fields> streamToFields = new HashMap<>();
    +    
    +    public ByTopicRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
    +        this(new SimpleRecordTranslator<>(func, fields, stream));
    +    }
    +    
    +    public ByTopicRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
    +        this(new SimpleRecordTranslator<>(func, fields));
    +    }
    +    
    +    public ByTopicRecordTranslator(RecordTranslator<K,V> defaultTranslator) {
    +        this.defaultTranslator = defaultTranslator;
    +        cacheNCheckFields(defaultTranslator);
    +    }
    +    
    +    public ByTopicRecordTranslator<K, V> forTopic(String topic, Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
    +        return forTopic(topic, new SimpleRecordTranslator<>(func, fields));
    +    }
    +    
    +    public ByTopicRecordTranslator<K, V> forTopic(String topic, Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
    +        return forTopic(topic, new SimpleRecordTranslator<>(func, fields, stream));
    +    }
    +    
    +    public ByTopicRecordTranslator<K, V> forTopic(String topic, RecordTranslator<K,V> translator) {
    +        if (topicToTranslator.containsKey(topic)) {
    +            throw new IllegalStateException("Topic " + topic + " is already registered");
    +        }
    +        topicToTranslator.put(topic, translator);
    +        cacheNCheckFields(translator);
    +        return this;
    +    }
    +    
    +    private void cacheNCheckFields(RecordTranslator<K, V> translator) {
    +        for (String stream : translator.streams()) {
    +            Fields fromTrans = translator.getFieldsFor(stream);
    +            Fields cached = streamToFields.get(stream);
    +            if (cached != null && !fromTrans.equals(cached)) {
    --- End diff --
    
    ```
    ByTopicRecordTranslator<A,B> trans = new ByTopicRecordTranslator((rec) -> Arrays.asList(rec.offset()), new Fields("offset"), "default");
    trans.forTopic("specialTopic", (rec) -> Arrays.asList(rec.offset(), rec.message()), new Fields("offset", "message"), "default");
    ```
    At this point we have tried to declare that the "default" stream has Fields ["offset"], and ["offset", "message"]  This is not supported by storm so we should not allow anyone to configure the spout to do this.
    
    streamToFields is not yet updated for the new translator we are adding it yet.  We do it after we have verified that the Fields match for anything we have done already.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91351206
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and 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` use to support decided which topic should to push message from tuple.
    +User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
    +When the topic name not found , `KafkaBolt` will write messages into default topic .
    +Please make sure the default topic have created .
    +
    +### Specifying Kafka producer properties
    +You can provide all the produce 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.
    +
    +###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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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 prooperties, but it is preferable to be explicet here, again to maintain 
    +type safety with the generics.
    +
    +There are a few key configs to pay attention to.
    +
    +`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
    +
    + * `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 `ConsumerRecord` into a Tuple and which stream that tuple will go to.  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 convinenece 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
    +
    +#### Create a Simple Insecure Spout
    +
     ```java
    -KafkaSpoutStreams kafkaSpoutStreams = new KafkaSpoutStreamsWildcardTopics(
    -            new KafkaSpoutStream(outputFields, STREAM, Pattern.compile(TOPIC_WILDCARD_PATTERN)));
     
    -KafkaSpoutTuplesBuilder<String, String> tuplesBuilder = new TopicsTest0Test1TupleBuilder<>(TOPIC_WILDCARD_PATTERN);
    +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");
    +...
    +
    +```
    +
    +#### Wildcard Topics
    +
    +```java
     
    -String STREAM = "test_wildcard_stream";
    -String TOPIC_WILDCARD_PATTERN = "test[1|2]";
    +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");
    +...
     
    -Fields outputFields = new Fields("topic", "partition", "offset", "key", "value");
     ```
     
    -### Create a simple Toplogy using the Kafka Spout:
    +#### Multiple Streams
    +```java
    +
    +final TopologyBuilder tp = new TopologyBuilder();
     
    +//By default all topics not covered by another rule will be emitted to "STREAM_1" as "topic", "key", and "value"
    +ByTopicRecordTranslator<String, String> 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, Pattern.compile("topic_.*")).build()), 1);
    +tp.setBolt("bolt", new myBolt()).shuffleGrouping("kafka_spout", "STREAM_1");
    +tp.setBolt("another", new myOtherBolt()).shuffleGrouping("kafka_spout", "STREAM_2");
    +...
    +
    +```
    +
    +#### Trident
     
     ```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();
    +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)
    +...
    +
     ```
     
    -# 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.
    +Trident does not support multiple streams and will ignore any streams set for output.  If however the Fields are not idential for each
    --- End diff --
    
    idential -> identical


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91618249
  
    --- Diff: 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<String, String> producer = mock(KafkaProducer.class);
    +        when(producer.send(any(), any())).thenAnswer(new Answer<Object>() {
    +            @Override
    +            public Object answer(InvocationOnMock invocation) throws Throwable {
    +                Callback c = (Callback)invocation.getArguments()[1];
    +                c.onCompletion(null, null);
    +                return null;
    +            }
    +        });
    +        KafkaBolt<String, String> bolt = new KafkaBolt<String, String>() {
    --- End diff --
    
    In this case because I am subclassing KafkaBolt it the compiler(at least in eclipse) actually complains that I am not allowed to do it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91346392
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and instance of FieldNameBasedTupleToKafkaMapper.
    --- End diff --
    
    Nitpick: and -> an


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96887342
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -61,129 +66,244 @@
          * If no offset has been committed, it behaves as LATEST.</li>
          * </ul>
          * */
    -    public enum FirstPollOffsetStrategy {
    +    public static enum FirstPollOffsetStrategy {
             EARLIEST,
             LATEST,
             UNCOMMITTED_EARLIEST,
             UNCOMMITTED_LATEST }
    -
    -    // Kafka consumer configuration
    -    private final Map<String, Object> kafkaProps;
    -    private final Deserializer<K> keyDeserializer;
    -    private final Deserializer<V> 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<K, V> tuplesBuilder;
    -    private final KafkaSpoutRetryService retryService;
    -
    -    private KafkaSpoutConfig(Builder<K,V> 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<String, String> builder(String bootstrapServers, String ... topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
         }
    -
    -    private Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Collection<String> topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
    +    }
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Pattern topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
    +    }
    +    
    +    private static Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> 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<K,V> {
             private final Map<String, Object> kafkaProps;
    -        private SerializableDeserializer<K> keyDeserializer;
    -        private SerializableDeserializer<V> valueDeserializer;
    +        private Subscription subscription;
    +        private final SerializableDeserializer<K> keyDes;
    +        private final Class<? extends Deserializer<K>> keyDesClazz;
    +        private final SerializableDeserializer<V> valueDes;
    +        private final Class<? extends Deserializer<V>> valueDesClazz;
    +        private RecordTranslator<K, V> 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<K, V> tuplesBuilder;
    -        private final KafkaSpoutRetryService retryService;
    -
    -        /**
    -         * Please refer to javadoc in {@link #Builder(Map, KafkaSpoutStreams, KafkaSpoutTuplesBuilder, KafkaSpoutRetryService)}.<p/>
    -         * This constructor uses by the default the following implementation for {@link KafkaSpoutRetryService}:<p/>
    -         * {@code new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -         *           DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)))}
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> tuplesBuilder) {
    -            this(kafkaProps, kafkaSpoutStreams, tuplesBuilder,
    -                    new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -                            DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)));
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
             }
    -
    -        /***
    -         * 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 <a href="http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html">KafkaConsumer</a>
    -         * @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
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> 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");
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Subscription subscription) {
    +        	this(bootstrapServers, keyDes, null, valDes, null, subscription);
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Subscription subscription) {
    +        	this(bootstrapServers, null, keyDes, null, valDes, subscription);
    +        }
    +        
    +        private Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
    +        		SerializableDeserializer<V> valDes, Class<? extends Deserializer<V>> 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<K,V>();
    +        }
     
    -            this.kafkaProps = kafkaProps;
    -            this.kafkaSpoutStreams = kafkaSpoutStreams;
    -            this.tuplesBuilder = tuplesBuilder;
    -            this.retryService = retryService;
    +        private Builder(Builder<?, ?> builder, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
    +        		SerializableDeserializer<V> valueDes, Class<? extends Deserializer<V>> 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 this is the only way to really maintain API compatibility
    +            this.translator = (RecordTranslator<K, V>) builder.translator;
    +            this.retryService = builder.retryService;
    +            this.keyDes = keyDes;
    +            this.keyDesClazz = keyDesClazz;
    +            this.valueDes = valueDes;
    +            this.valueDesClazz = valueDesClazz;
             }
     
             /**
              * Specifying this key deserializer overrides the property key.deserializer
              */
    -        public Builder<K,V> setKeyDeserializer(SerializableDeserializer<K> keyDeserializer) {
    -            this.keyDeserializer = keyDeserializer;
    -            return this;
    +        public <NK> Builder<NK,V> setKey(SerializableDeserializer<NK> keyDeserializer) {
    --- End diff --
    
    Because we are changing the key type of the deserializer.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96879307
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    +        this.boltSpecfiedProperties = producerProperties;
    +        return this;
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        //for backward compatibility.
    +        if(mapper == null) {
    +            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
    +        }
    +
    +        //for backward compatibility.
    +        if(topicSelector == null) {
    +            if(stormConf.containsKey(TOPIC)) {
    +                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
    +            } else {
    +                throw new IllegalArgumentException("topic should be specified in bolt's configuration");
    +            }
    +        }
    +
    +        producer = mkProducer(boltSpecfiedProperties);
    +        this.collector = collector;
    +    }
    +    
    +    /**
    +     * Intended to be overridden for tests.  Make the producer from props
    --- End diff --
    
    backwards compatibility


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96720421
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    --- End diff --
    
    Can you please add a comment saying to look at the comments in the set method? I missed those comments at first.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    There are still a few files with tabs mixed in to spaces, so autoformatting all the files might be good. I also still don't really understand why the KafkaSpoutConfig builder needs to support setting a deserializer with a different generic type to the one the builder was instantiated with.
    
    +1 though.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96718850
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    +        this.boltSpecfiedProperties = producerProperties;
    +        return this;
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        //for backward compatibility.
    +        if(mapper == null) {
    +            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
    +        }
    +
    +        //for backward compatibility.
    +        if(topicSelector == null) {
    +            if(stormConf.containsKey(TOPIC)) {
    +                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
    --- End diff --
    
    perhaps log here
    LOG.info("TopicSelector not specified. Using [{}] for topic [{}] specified in bolt configuration, DefaultTopicSelector.class.getSimpleName(), stormConf.get(TOPIC));


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96895148
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Subscription.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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 <K, V> void subscribe(KafkaConsumer<K,V> consumer, ConsumerRebalanceListener listener, TopologyContext context) {
    +    	subscribe(consumer, listener);
    +    }
    +
    +    /**
    +     * Subscribe the KafkaConsumer to the proper topics
    +     * @param consumer the Consumer to get.
    +     * @param listener the rebalance listener to include in the subscription
    +     * @deprecated please use the version with the TopologyContext in it
    +     */
    +    public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceListener listener) {
    --- End diff --
    
    I didn't feel good about deprecating it in the first place and will remove it.  It just means that I will also pull in the changes I made for STORM-2236 back to 1.1 too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91364347
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RecordTranslator.java ---
    @@ -0,0 +1,59 @@
    +/*
    + * 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.Arrays;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.function.Function;
    +
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.storm.tuple.Fields;
    +
    +/**
    + * Translates a Kafka ConsumerRecord into a tuple
    + * use {@link KafkaTuple} if you want to route your tuples
    + * to different streams.
    + */
    +public interface RecordTranslator<K, V> extends Serializable, Function<ConsumerRecord<K, V>, List<Object>> {
    +    public static final List<String> DEFAULT_STREAM = Collections.unmodifiableList(Arrays.asList("default"));
    --- End diff --
    
    Collections.singletonList is an easier way to do this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96925279
  
    --- Diff: 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<K, V> 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);
    --- End diff --
    
    It is possible but that is the point of the generics.  To try and reduce the likelihood of it happening.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r97367473
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java ---
    @@ -1,16 +1,35 @@
    +/*
    + * 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.common.TopicPartition;
    -
     import java.util.Comparator;
     
    +import org.apache.kafka.common.TopicPartition;
    +
     public class TopicPartitionComparator implements Comparator<TopicPartition> {
    -    @Override
    -    public int compare(TopicPartition o1, TopicPartition o2) {
    -        if (!o1.topic().equals(o2.topic())) {
    -            return o1.topic().compareTo(o2.topic());
    -        } else {
    -            return o1.partition() - o2.partition();
    -        }
    -    }
    +	public static final TopicPartitionComparator INSTANCE = new TopicPartitionComparator();
    --- End diff --
    
    @hmcl Neat, hadn't considered those arguments for using enum.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96886556
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -61,129 +66,244 @@
          * If no offset has been committed, it behaves as LATEST.</li>
          * </ul>
          * */
    -    public enum FirstPollOffsetStrategy {
    +    public static enum FirstPollOffsetStrategy {
             EARLIEST,
             LATEST,
             UNCOMMITTED_EARLIEST,
             UNCOMMITTED_LATEST }
    -
    -    // Kafka consumer configuration
    -    private final Map<String, Object> kafkaProps;
    -    private final Deserializer<K> keyDeserializer;
    -    private final Deserializer<V> 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<K, V> tuplesBuilder;
    -    private final KafkaSpoutRetryService retryService;
    -
    -    private KafkaSpoutConfig(Builder<K,V> 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<String, String> builder(String bootstrapServers, String ... topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
         }
    -
    -    private Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Collection<String> topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
    +    }
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Pattern topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
    +    }
    +    
    +    private static Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> 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<K,V> {
             private final Map<String, Object> kafkaProps;
    -        private SerializableDeserializer<K> keyDeserializer;
    -        private SerializableDeserializer<V> valueDeserializer;
    +        private Subscription subscription;
    +        private final SerializableDeserializer<K> keyDes;
    +        private final Class<? extends Deserializer<K>> keyDesClazz;
    +        private final SerializableDeserializer<V> valueDes;
    +        private final Class<? extends Deserializer<V>> valueDesClazz;
    +        private RecordTranslator<K, V> 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<K, V> tuplesBuilder;
    -        private final KafkaSpoutRetryService retryService;
    -
    -        /**
    -         * Please refer to javadoc in {@link #Builder(Map, KafkaSpoutStreams, KafkaSpoutTuplesBuilder, KafkaSpoutRetryService)}.<p/>
    -         * This constructor uses by the default the following implementation for {@link KafkaSpoutRetryService}:<p/>
    -         * {@code new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -         *           DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)))}
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> tuplesBuilder) {
    -            this(kafkaProps, kafkaSpoutStreams, tuplesBuilder,
    -                    new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -                            DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)));
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
             }
    -
    -        /***
    -         * 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 <a href="http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html">KafkaConsumer</a>
    -         * @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
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> 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");
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Subscription subscription) {
    +        	this(bootstrapServers, keyDes, null, valDes, null, subscription);
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Subscription subscription) {
    +        	this(bootstrapServers, null, keyDes, null, valDes, subscription);
    +        }
    +        
    +        private Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
    +        		SerializableDeserializer<V> valDes, Class<? extends Deserializer<V>> 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<K,V>();
    +        }
     
    -            this.kafkaProps = kafkaProps;
    -            this.kafkaSpoutStreams = kafkaSpoutStreams;
    -            this.tuplesBuilder = tuplesBuilder;
    -            this.retryService = retryService;
    +        private Builder(Builder<?, ?> builder, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
    +        		SerializableDeserializer<V> valueDes, Class<? extends Deserializer<V>> 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,
    --- End diff --
    
    Sorry the comment was not updated.  I will do that.  Originally it was for API compatibility, but now it is to not force someone to reset the translator when they change a key or value type.  Specifically the default translator that should work on any types.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91361471
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -60,123 +68,197 @@
          * If no offset has been committed, it behaves as LATEST.</li>
          * </ul>
          * */
    -    public enum FirstPollOffsetStrategy {
    +    public static enum FirstPollOffsetStrategy {
             EARLIEST,
             LATEST,
             UNCOMMITTED_EARLIEST,
             UNCOMMITTED_LATEST }
    -
    -    // Kafka consumer configuration
    -    private final Map<String, Object> kafkaProps;
    -    private final Deserializer<K> keyDeserializer;
    -    private final Deserializer<V> valueDeserializer;
    -    private final long pollTimeoutMs;
    -
    -    // Kafka spout configuration
    -    private final long offsetCommitPeriodMs;
    -    private final int maxRetries;
    -    private final int maxUncommittedOffsets;
    -    private final FirstPollOffsetStrategy firstPollOffsetStrategy;
    -    private final KafkaSpoutStreams kafkaSpoutStreams;
    -    private final KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;
    -    private final KafkaSpoutRetryService retryService;
    -
    -    private KafkaSpoutConfig(Builder<K,V> 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.tuplesBuilder = builder.tuplesBuilder;
    -        this.retryService = builder.retryService;
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, String ... topics) {
    +        return new Builder<>(bootstrapServers, new StringDeserializer(), new StringDeserializer(), topics);
         }
    -
    -    private Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Collection<String> topics) {
    +        return new Builder<>(bootstrapServers, new StringDeserializer(), new StringDeserializer(), topics);
    --- End diff --
    
    I don't think this works when the topology is serialized. See https://github.com/apache/storm/pull/1696/files#diff-d2c28214a421176fceab415ffe4c95c0R24


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96725508
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ByTopicRecordTranslator.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 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;
    +
    +public class ByTopicRecordTranslator<K, V> implements RecordTranslator<K, V> {
    +    private static final long serialVersionUID = -121699733778988688L;
    +    private final RecordTranslator<K,V> defaultTranslator;
    +    private final Map<String, RecordTranslator<K,V>> topicToTranslator = new HashMap<>();
    +    private final Map<String, Fields> streamToFields = new HashMap<>();
    +    
    +    public ByTopicRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
    --- End diff --
    
    Nit: This class could do with some javadoc


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96951710
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java ---
    @@ -0,0 +1,114 @@
    +/**
    + * 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.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) {
    +        if (mapper == null) throw new NullPointerException("mapper can not be null");
    --- End diff --
    
    @srdo you are right.
    
    ```
                    <artifactId>maven-compiler-plugin</artifactId>
                    <configuration>
                        <source>1.7</source>
                        <target>1.7</target>
                    </configuration>
    ```
    I'll update it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91592134
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and 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` use to support decided which topic should to push message from tuple.
    +User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
    +When the topic name not found , `KafkaBolt` will write messages into default topic .
    +Please make sure the default topic have created .
    +
    +### Specifying Kafka producer properties
    +You can provide all the produce 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.
    +
    +###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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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 prooperties, but it is preferable to be explicet here, again to maintain 
    +type safety with the generics.
    +
    +There are a few key configs to pay attention to.
    +
    +`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
    +
    + * `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 `ConsumerRecord` into a Tuple and which stream that tuple will go to.  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 convinenece 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
    +
    +#### Create a Simple Insecure Spout
    +
     ```java
    -KafkaSpoutStreams kafkaSpoutStreams = new KafkaSpoutStreamsWildcardTopics(
    -            new KafkaSpoutStream(outputFields, STREAM, Pattern.compile(TOPIC_WILDCARD_PATTERN)));
     
    -KafkaSpoutTuplesBuilder<String, String> tuplesBuilder = new TopicsTest0Test1TupleBuilder<>(TOPIC_WILDCARD_PATTERN);
    +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");
    +...
    +
    +```
    +
    +#### Wildcard Topics
    +
    +```java
     
    -String STREAM = "test_wildcard_stream";
    -String TOPIC_WILDCARD_PATTERN = "test[1|2]";
    +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");
    +...
     
    -Fields outputFields = new Fields("topic", "partition", "offset", "key", "value");
     ```
     
    -### Create a simple Toplogy using the Kafka Spout:
    +#### Multiple Streams
    +```java
    +
    +final TopologyBuilder tp = new TopologyBuilder();
     
    +//By default all topics not covered by another rule will be emitted to "STREAM_1" as "topic", "key", and "value"
    +ByTopicRecordTranslator<String, String> 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, Pattern.compile("topic_.*")).build()), 1);
    --- End diff --
    
    That all depends on the topics defined in the kafka cluster.  If it only has `topic_2` and `topic` defined then yes, but if it has `topic_1` and `topic_2` defined then it will match both of them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96875424
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,232 @@
    -#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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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.
    +
    +There are a few key configs to pay attention to.
    +
    +`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
    +
    + * `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 `ConsumerRecord` into a Tuple and which stream that tuple will go to.  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.
    --- End diff --
    
    We are configuring the kafka consumer, so I would assume that it was implied.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96877333
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    --- End diff --
    
    I don't want to rename public facing APIs right now, because this is a copy of what is in external/storm-kafka.  The other one is deprecated, but I want to maintain compatibility if possible.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96744488
  
    --- Diff: 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;
    --- End diff --
    
    Why does this class and other classes (e.g. ByTopicRecordTranslator) have `serialVersionUID` but others (e.g. DefaultTopicSelector, FieldNameTopicSelector) don't?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96754681
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ByTopicRecordTranslator.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 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;
    +
    +public class ByTopicRecordTranslator<K, V> implements RecordTranslator<K, V> {
    +    private static final long serialVersionUID = -121699733778988688L;
    +    private final RecordTranslator<K,V> defaultTranslator;
    +    private final Map<String, RecordTranslator<K,V>> topicToTranslator = new HashMap<>();
    +    private final Map<String, Fields> streamToFields = new HashMap<>();
    +    
    +    public ByTopicRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
    +        this(new SimpleRecordTranslator<>(func, fields, stream));
    +    }
    +    
    +    public ByTopicRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
    +        this(new SimpleRecordTranslator<>(func, fields));
    +    }
    +    
    +    public ByTopicRecordTranslator(RecordTranslator<K,V> defaultTranslator) {
    +        this.defaultTranslator = defaultTranslator;
    +        cacheNCheckFields(defaultTranslator);
    +    }
    +    
    +    public ByTopicRecordTranslator<K, V> forTopic(String topic, Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
    +        return forTopic(topic, new SimpleRecordTranslator<>(func, fields));
    +    }
    +    
    +    public ByTopicRecordTranslator<K, V> forTopic(String topic, Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
    +        return forTopic(topic, new SimpleRecordTranslator<>(func, fields, stream));
    +    }
    +    
    +    public ByTopicRecordTranslator<K, V> forTopic(String topic, RecordTranslator<K,V> translator) {
    +        if (topicToTranslator.containsKey(topic)) {
    +            throw new IllegalStateException("Topic " + topic + " is already registered");
    +        }
    +        topicToTranslator.put(topic, translator);
    +        cacheNCheckFields(translator);
    +        return this;
    +    }
    +    
    +    private void cacheNCheckFields(RecordTranslator<K, V> translator) {
    +        for (String stream : translator.streams()) {
    +            Fields fromTrans = translator.getFieldsFor(stream);
    +            Fields cached = streamToFields.get(stream);
    +            if (cached != null && !fromTrans.equals(cached)) {
    --- End diff --
    
    In which scenario can this be true? streamToFields is always set using `streamToFields.put(stream, fromTrans);`  Doesn't it guarantee that  this if is always false ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96713059
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,232 @@
    -#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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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.
    +
    +There are a few key configs to pay attention to.
    +
    +`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
    +
    + * `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 `ConsumerRecord` into a Tuple and which stream that tuple will go to.  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
    +
    +#### Create a Simple Insecure Spout
    +The following will consume all events published to "topic" and send them to MyBolt as "topic", "partition", "offset", "key", "value".
    --- End diff --
    
    ... MyBolt using the fields "topic" ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96775776
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RoundRobinManualPartitioner.java ---
    @@ -15,14 +15,26 @@
      *   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.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
     
    -import org.apache.kafka.clients.consumer.KafkaConsumer;
     import org.apache.kafka.common.TopicPartition;
    +import org.apache.storm.task.TopologyContext;
     
    -import java.util.List;
    +public class RoundRobinManualPartitioner implements ManualPartitioner {
     
    -public interface KafkaPartitionReader {
    -    List<TopicPartition> readPartitions(KafkaConsumer<?, ?> consumer);
    +	@Override
    +	public List<TopicPartition> partition(List<TopicPartition> allPartitions, TopologyContext context) {
    --- End diff --
    
    Can you please add a comment explaining the logic doing this round robin partitioning.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96909498
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTuple.java ---
    @@ -15,22 +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.util.List;
    -
    -public class KafkaSpoutTuplesBuilderWildcardTopics<K,V> implements KafkaSpoutTuplesBuilder<K,V> {
    -    private KafkaSpoutTupleBuilder<K, V> tupleBuilder;
    +import org.apache.storm.tuple.Values;
     
    -    public KafkaSpoutTuplesBuilderWildcardTopics(KafkaSpoutTupleBuilder<K, V> tupleBuilder) {
    -        this.tupleBuilder = tupleBuilder;
    +/**
    + * A list of Values in a tuple that can be routed 
    + * to a given stream.
    + */
    +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) {
    --- End diff --
    
    This is because the constructor is varadic following the Values parent class. It is ambiguous to have
    
    ```
        public KafkaTuple(Object... vals)
    ```
    
    and any other constructor.  Java complains.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96925318
  
    --- Diff: 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<K, V> 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) {
    --- End diff --
    
    It is possible but that is the point of the generics. To try and reduce the likelihood of it happening.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96717295
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    --- End diff --
    
    Is this method intended to set KafkaProducerProperties only? If so, wouldn't it be beneficial to call this method `withKafkaProducerProps`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91613603
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,194 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + * <p/>
    + * This bolt uses 0.8.2 Kafka Producer API.
    + * <p/>
    + * It works for sending tuples to older Kafka version (0.8.1).
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    /**
    +     * With default setting for fireAndForget and async, the callback is called when the sending succeeds.
    +     * By setting fireAndForget true, the send will not wait at all for kafka to ack.
    +     * "acks" setting in 0.8.2 Producer API config doesn't matter if fireAndForget is set.
    +     * By setting async false, synchronous sending is used. 
    +     */
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    +        this.boltSpecfiedProperties = producerProperties;
    +        return this;
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        //for backward compatibility.
    +        if(mapper == null) {
    +            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
    +        }
    +
    +        //for backward compatibility.
    +        if(topicSelector == null) {
    +            if(stormConf.containsKey(TOPIC)) {
    +                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
    +            } else {
    +                throw new IllegalArgumentException("topic should be specified in bolt's configuration");
    +            }
    +        }
    +
    +        producer = mkProducer(boltSpecfiedProperties);
    +        this.collector = collector;
    +    }
    +    
    +    /**
    +     * Intended to be overridden for tests.  Make the producer from props
    +     */
    +    protected KafkaProducer<K, V> 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);
    --- End diff --
    
    collector could always be called from background threads.  There, however, was a race in the original shuffle grouping that could crash your worker if you didn't synchronize around calls to the collector.  I fixed that a long time ago, but people are still very nervous about it, or are still confused about it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91348211
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and 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` use to support decided which topic should to push message from tuple.
    +User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
    +When the topic name not found , `KafkaBolt` will write messages into default topic .
    +Please make sure the default topic have created .
    +
    +### Specifying Kafka producer properties
    +You can provide all the produce properties in your Storm topology by calling `KafkaBolt.withProducerProperties()` and `TridentKafkaStateFactory.withProducerProperties()`. Please see  http://kafka.apache.org/documentation.html#newproducerconfigs
    --- End diff --
    
    produce -> producer


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91592178
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and 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` use to support decided which topic should to push message from tuple.
    +User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
    +When the topic name not found , `KafkaBolt` will write messages into default topic .
    +Please make sure the default topic have created .
    +
    +### Specifying Kafka producer properties
    +You can provide all the produce 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.
    +
    +###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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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 prooperties, but it is preferable to be explicet here, again to maintain 
    +type safety with the generics.
    +
    +There are a few key configs to pay attention to.
    +
    +`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
    +
    + * `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 `ConsumerRecord` into a Tuple and which stream that tuple will go to.  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 convinenece 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
    +
    +#### Create a Simple Insecure Spout
    +
     ```java
    -KafkaSpoutStreams kafkaSpoutStreams = new KafkaSpoutStreamsWildcardTopics(
    -            new KafkaSpoutStream(outputFields, STREAM, Pattern.compile(TOPIC_WILDCARD_PATTERN)));
     
    -KafkaSpoutTuplesBuilder<String, String> tuplesBuilder = new TopicsTest0Test1TupleBuilder<>(TOPIC_WILDCARD_PATTERN);
    +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");
    +...
    +
    +```
    +
    +#### Wildcard Topics
    +
    +```java
     
    -String STREAM = "test_wildcard_stream";
    -String TOPIC_WILDCARD_PATTERN = "test[1|2]";
    +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");
    +...
     
    -Fields outputFields = new Fields("topic", "partition", "offset", "key", "value");
     ```
     
    -### Create a simple Toplogy using the Kafka Spout:
    +#### Multiple Streams
    +```java
    +
    +final TopologyBuilder tp = new TopologyBuilder();
     
    +//By default all topics not covered by another rule will be emitted to "STREAM_1" as "topic", "key", and "value"
    +ByTopicRecordTranslator<String, String> 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, Pattern.compile("topic_.*")).build()), 1);
    --- End diff --
    
    I'll try to make the docs clearer


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91352188
  
    --- Diff: external/storm-kafka-client/README.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
    --- End diff --
    
    Why is this file both in docs/ and here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96764322
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTuple.java ---
    @@ -15,22 +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.util.List;
    -
    -public class KafkaSpoutTuplesBuilderWildcardTopics<K,V> implements KafkaSpoutTuplesBuilder<K,V> {
    -    private KafkaSpoutTupleBuilder<K, V> tupleBuilder;
    +import org.apache.storm.tuple.Values;
     
    -    public KafkaSpoutTuplesBuilderWildcardTopics(KafkaSpoutTupleBuilder<K, V> tupleBuilder) {
    -        this.tupleBuilder = tupleBuilder;
    +/**
    + * A list of Values in a tuple that can be routed 
    + * to a given stream.
    + */
    +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) {
    --- End diff --
    
    Why not make this object immutable and use two constructors one with and one without stream ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    @revans2 I would like to take a look at this before this gets merged in.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96734787
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -61,129 +66,244 @@
          * If no offset has been committed, it behaves as LATEST.</li>
          * </ul>
          * */
    -    public enum FirstPollOffsetStrategy {
    +    public static enum FirstPollOffsetStrategy {
             EARLIEST,
             LATEST,
             UNCOMMITTED_EARLIEST,
             UNCOMMITTED_LATEST }
    -
    -    // Kafka consumer configuration
    -    private final Map<String, Object> kafkaProps;
    -    private final Deserializer<K> keyDeserializer;
    -    private final Deserializer<V> 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<K, V> tuplesBuilder;
    -    private final KafkaSpoutRetryService retryService;
    -
    -    private KafkaSpoutConfig(Builder<K,V> 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<String, String> builder(String bootstrapServers, String ... topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
         }
    -
    -    private Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Collection<String> topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
    +    }
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Pattern topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
    +    }
    +    
    +    private static Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> 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<K,V> {
             private final Map<String, Object> kafkaProps;
    -        private SerializableDeserializer<K> keyDeserializer;
    -        private SerializableDeserializer<V> valueDeserializer;
    +        private Subscription subscription;
    +        private final SerializableDeserializer<K> keyDes;
    +        private final Class<? extends Deserializer<K>> keyDesClazz;
    +        private final SerializableDeserializer<V> valueDes;
    +        private final Class<? extends Deserializer<V>> valueDesClazz;
    +        private RecordTranslator<K, V> 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<K, V> tuplesBuilder;
    -        private final KafkaSpoutRetryService retryService;
    -
    -        /**
    -         * Please refer to javadoc in {@link #Builder(Map, KafkaSpoutStreams, KafkaSpoutTuplesBuilder, KafkaSpoutRetryService)}.<p/>
    -         * This constructor uses by the default the following implementation for {@link KafkaSpoutRetryService}:<p/>
    -         * {@code new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -         *           DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)))}
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> tuplesBuilder) {
    -            this(kafkaProps, kafkaSpoutStreams, tuplesBuilder,
    -                    new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -                            DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)));
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
             }
    -
    -        /***
    -         * 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 <a href="http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html">KafkaConsumer</a>
    -         * @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
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> 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");
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Subscription subscription) {
    +        	this(bootstrapServers, keyDes, null, valDes, null, subscription);
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Subscription subscription) {
    +        	this(bootstrapServers, null, keyDes, null, valDes, subscription);
    +        }
    +        
    +        private Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
    +        		SerializableDeserializer<V> valDes, Class<? extends Deserializer<V>> 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<K,V>();
    +        }
     
    -            this.kafkaProps = kafkaProps;
    -            this.kafkaSpoutStreams = kafkaSpoutStreams;
    -            this.tuplesBuilder = tuplesBuilder;
    -            this.retryService = retryService;
    +        private Builder(Builder<?, ?> builder, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
    +        		SerializableDeserializer<V> valueDes, Class<? extends Deserializer<V>> 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 this is the only way to really maintain API compatibility
    +            this.translator = (RecordTranslator<K, V>) builder.translator;
    +            this.retryService = builder.retryService;
    +            this.keyDes = keyDes;
    +            this.keyDesClazz = keyDesClazz;
    +            this.valueDes = valueDes;
    +            this.valueDesClazz = valueDesClazz;
             }
     
             /**
              * Specifying this key deserializer overrides the property key.deserializer
              */
    -        public Builder<K,V> setKeyDeserializer(SerializableDeserializer<K> keyDeserializer) {
    -            this.keyDeserializer = keyDeserializer;
    -            return this;
    +        public <NK> Builder<NK,V> setKey(SerializableDeserializer<NK> keyDeserializer) {
    +            return new Builder<>(this, keyDeserializer, null, valueDes, valueDesClazz);
    +        }
    +        
    +        /**
    +         * Specify a class that can be instantiated to create a key.deserializer
    +         * This is the same as setting key.deserializer, but overrides it.
    +         */
    +        public <NK> Builder<NK, V> setKey(Class<? extends Deserializer<NK>> clazz) {
    +            return new Builder<>(this, null, clazz, valueDes, valueDesClazz);
             }
     
             /**
              * Specifying this value deserializer overrides the property value.deserializer
              */
    -        public Builder<K,V> setValueDeserializer(SerializableDeserializer<V> valueDeserializer) {
    -            this.valueDeserializer = valueDeserializer;
    +        public <NV> Builder<K,NV> setValue(SerializableDeserializer<NV> 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.
    +         */
    +        public <NV> Builder<K,NV> setValue(Class<? extends Deserializer<NV>> clazz) {
    +            return new Builder<>(this, keyDes, keyDesClazz, null, clazz);
    +        }
    +        
    +        /**
    +         * Set a Kafka property config
    +         */
    +        public Builder<K,V> setProp(String key, Object value) {
    +            kafkaProps.put(key, value);
    +            return this;
    +        }
    +        
    +        /**
    +         * Set multiple Kafka property configs
    +         */
    +        public Builder<K,V> setProp(Map<String, Object> props) {
    +            kafkaProps.putAll(props);
    +            return this;
    +        }
    +        
    +        /**
    +         * Set multiple Kafka property configs
    +         */
    +        public Builder<K,V> 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<K,V> setGroupId(String id) {
    +            return setProp("group.id", id);
    +        }
    +        
    +        /**
    +         * reset the bootstrap servers for the Consumer
    +         */
    +        public Builder<K,V> 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<K,V> setFetchMinBytes(int bytes) {
    +            return setProp(ConsumerConfig.FETCH_MIN_BYTES_CONFIG, bytes);
    +        }
    +        
    +        /**
    +         * The maximum amount of data per-partition the broker will return.
    +         */
    +        public Builder<K,V> 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<K,V> setMaxPoolRecords(int records) {
    --- End diff --
    
    maxPool -> maxPoll


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96889298
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Subscription.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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 <K, V> void subscribe(KafkaConsumer<K,V> consumer, ConsumerRebalanceListener listener, TopologyContext context) {
    +    	subscribe(consumer, listener);
    +    }
    +
    +    /**
    +     * Subscribe the KafkaConsumer to the proper topics
    +     * @param consumer the Consumer to get.
    +     * @param listener the rebalance listener to include in the subscription
    +     * @deprecated please use the version with the TopologyContext in it
    +     */
    +    public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceListener listener) {
    +    	throw new IllegalStateException("At least one subscribe method must be overwritten");
    +    }
    +    
    +    /**
    +     * @return a string representing the subscribed topics.
    --- End diff --
    
    I didn't know either, but the only place I can see it used is in logging so that is what I did.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96995077
  
    --- Diff: 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);
    --- End diff --
    
    IMHO we need to try avoiding JDK 8 feature for public API unless we decided to release minor version only for 2.0.0. But anyway I also think this is beyond of the scope.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96903456
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RoundRobinManualPartitioner.java ---
    @@ -15,14 +15,26 @@
      *   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.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
     
    -import org.apache.kafka.clients.consumer.KafkaConsumer;
     import org.apache.kafka.common.TopicPartition;
    +import org.apache.storm.task.TopologyContext;
     
    -import java.util.List;
    +public class RoundRobinManualPartitioner implements ManualPartitioner {
     
    -public interface KafkaPartitionReader {
    -    List<TopicPartition> readPartitions(KafkaConsumer<?, ?> consumer);
    +	@Override
    +	public List<TopicPartition> partition(List<TopicPartition> allPartitions, TopologyContext context) {
    +		int thisTaskIndex = context.getThisTaskIndex();
    +		int totalTaskCount = context.getComponentTasks(context.getThisComponentId()).size();
    +		Set<TopicPartition> myPartitions = new HashSet<>(allPartitions.size()/totalTaskCount+1);
    +		for (int i = thisTaskIndex; i < allPartitions.size(); i += totalTaskCount) {
    --- End diff --
    
    I don't think so. Either i is in [0, allPartitions.size()-1] or the loop is over.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    I see some duplicate code and documentation.  Is the idea to move from /external/storm-kafka to /external/storm-kafka-client?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96726512
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -225,11 +213,7 @@ public void nextTuple() {
                     }
     
                     if (poll()) {
    -                    try {
    -                        setWaitingToEmit(pollKafkaBroker());
    -                    } catch (RetriableException e) {
    -                        LOG.error("Failed to poll from kafka.", e);
    -                    }
    +                    setWaitingToEmit(pollKafkaBroker());
    --- End diff --
    
    I think @liurenjie1024 had decent reasoning for catching this, see https://github.com/apache/storm/pull/1835#discussion_r93105952.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    Thanks for the great work and patience. +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91348095
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and 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` use to support decided which topic should to push message from tuple.
    +User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
    +When the topic name not found , `KafkaBolt` will write messages into default topic .
    +Please make sure the default topic have created .
    --- End diff --
    
    have -> has been


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    I updated the docs and the examples.  This is a non-backwards compatible change from the 1.x release.  I have some code (not included here) that can maintain most compatibility if we really want to, but because of STORM-2228 I decided it would be better to just not support that at all.  If we want to add that compatibility for 1.x  but deprecated I am OK with that, but would like feedback on it before putting up some kind of a pull request.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96713362
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,232 @@
    -#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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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.
    +
    +There are a few key configs to pay attention to.
    +
    +`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
    +
    + * `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 `ConsumerRecord` into a Tuple and which stream that tuple will go to.  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
    +
    +#### Create a Simple Insecure Spout
    +The following will consume all events published to "topic" and send them to MyBolt as "topic", "partition", "offset", "key", "value".
     ```java
    -KafkaSpoutStreams kafkaSpoutStreams = new KafkaSpoutStreamsWildcardTopics(
    -            new KafkaSpoutStream(outputFields, STREAM, Pattern.compile(TOPIC_WILDCARD_PATTERN)));
     
    -KafkaSpoutTuplesBuilder<String, String> tuplesBuilder = new TopicsTest0Test1TupleBuilder<>(TOPIC_WILDCARD_PATTERN);
    +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");
    +...
     
    -String STREAM = "test_wildcard_stream";
    -String TOPIC_WILDCARD_PATTERN = "test[1|2]";
    +```
    +
    +#### Wildcard Topics
    +Wildcard topics will consume from all topics that exist in the specified broakers list and match the pattern.  So in the following example
    --- End diff --
    
    typo - brokers


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96719192
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    +        this.boltSpecfiedProperties = producerProperties;
    +        return this;
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        //for backward compatibility.
    +        if(mapper == null) {
    +            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
    +        }
    +
    +        //for backward compatibility.
    +        if(topicSelector == null) {
    +            if(stormConf.containsKey(TOPIC)) {
    +                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
    +            } else {
    +                throw new IllegalArgumentException("topic should be specified in bolt's configuration");
    +            }
    +        }
    +
    +        producer = mkProducer(boltSpecfiedProperties);
    +        this.collector = collector;
    +    }
    +    
    +    /**
    +     * Intended to be overridden for tests.  Make the producer from props
    --- End diff --
    
    Create producer with props. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96733517
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Subscription.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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 <K, V> void subscribe(KafkaConsumer<K,V> consumer, ConsumerRebalanceListener listener, TopologyContext context) {
    +    	subscribe(consumer, listener);
    +    }
    +
    +    /**
    +     * Subscribe the KafkaConsumer to the proper topics
    +     * @param consumer the Consumer to get.
    +     * @param listener the rebalance listener to include in the subscription
    +     * @deprecated please use the version with the TopologyContext in it
    +     */
    +    public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceListener listener) {
    +    	throw new IllegalStateException("At least one subscribe method must be overwritten");
    +    }
    +    
    +    /**
    +     * @return a string representing the subscribed topics.
    --- End diff --
    
    Can this string be in any format? It seems to only be used in getComponentConfiguration, not really sure where that's being read.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96727056
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -609,4 +593,7 @@ public String toString() {
                         '}';
             }
         }
    +
    +    // =========== Timer ===========
    --- End diff --
    
    This doesn't seem like it should be here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96879248
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    +        this.boltSpecfiedProperties = producerProperties;
    +        return this;
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        //for backward compatibility.
    +        if(mapper == null) {
    +            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
    +        }
    +
    +        //for backward compatibility.
    +        if(topicSelector == null) {
    +            if(stormConf.containsKey(TOPIC)) {
    +                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
    +            } else {
    +                throw new IllegalArgumentException("topic should be specified in bolt's configuration");
    +            }
    +        }
    +
    +        producer = mkProducer(boltSpecfiedProperties);
    +        this.collector = collector;
    +    }
    +    
    +    /**
    +     * Intended to be overridden for tests.  Make the producer from props
    +     */
    +    protected KafkaProducer<K, V> mkProducer(Properties props) {
    --- End diff --
    
    backwards compatibility



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    @hmcl github like to truncate long messages.
    
    ```
    $ git log STORM-2225
    commit c9f9348e3de0917020a8c903f4c9d026b7dc6204
    Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
    Date:   Tue Nov 29 21:39:26 2016 -0600
    
        STORM-1997: copy state/bolt from storm-kafka to storm-kafka-client
        STORM-2225: change spout config to be simpler.
        STORM-2228: removed ability to request a single topic go to multiple streams
        STORM-2236: Reimplemented manual partition management on top of STORM-2225
    
    commit 74a77aa6494957b941df55d66ff64b6561eee4a8
    Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
    Date:   Wed Jan 18 10:34:30 2017 -0600
    
        Revert "Merge branch 'STORM-2236' of https://github.com/MediaV/storm into STORM-2236"
    
        This reverts commit e2f87c304297ce95f01708e0934c44ee385fa0bb, reversing
        changes made to 00fed518d14b485e2a9e9cdd91ade3cc9d68dc8d.
    
    ```
    
    If you really want me to squash the code change with the revert and only reference a single JIRA in the commit messages I can.  But I thought it would be better if I am addressing 4 separate (but very closely related) JIRA I should keep them all together.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96711515
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,232 @@
    -#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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    --- End diff --
    
    This is to guarantee type safety through the use of Java generics


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96775680
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RoundRobinManualPartitioner.java ---
    @@ -15,14 +15,26 @@
      *   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.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
     
    -import org.apache.kafka.clients.consumer.KafkaConsumer;
     import org.apache.kafka.common.TopicPartition;
    +import org.apache.storm.task.TopologyContext;
     
    -import java.util.List;
    +public class RoundRobinManualPartitioner implements ManualPartitioner {
     
    -public interface KafkaPartitionReader {
    -    List<TopicPartition> readPartitions(KafkaConsumer<?, ?> consumer);
    +	@Override
    +	public List<TopicPartition> partition(List<TopicPartition> allPartitions, TopologyContext context) {
    +		int thisTaskIndex = context.getThisTaskIndex();
    +		int totalTaskCount = context.getComponentTasks(context.getThisComponentId()).size();
    +		Set<TopicPartition> myPartitions = new HashSet<>(allPartitions.size()/totalTaskCount+1);
    +		for (int i = thisTaskIndex; i < allPartitions.size(); i += totalTaskCount) {
    --- End diff --
    
    can't this code cause `IndexOutOfBoundsException` ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    @revans2 this is perfect. These 4 commits make it much easier to track down. I confused the way GitHub truncates the messages - my bad.
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96877744
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    --- End diff --
    
    This is "moving" existing code so I want to maintain compatibility if possible.  But we are doing a copy + deprecation because the two libraries are compiled with different versions of Kafka so combining the two libraries in a single topology is difficult in some cases.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96926665
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java ---
    @@ -0,0 +1,114 @@
    +/**
    + * 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.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) {
    +        if (mapper == null) throw new NullPointerException("mapper can not be null");
    --- End diff --
    
    I thought 1.x was targeting at least Java 7?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96722977
  
    --- Diff: 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;
    +
    +/**
    + * as the really verbose name suggests this interface maps a storm tuple to kafka key and message.
    --- End diff --
    
    I would delete "as the really verbose name suggests this " and put "interface defining a mapping from storm tuple to kafka key and message"
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96902715
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java ---
    @@ -1,16 +1,35 @@
    +/*
    + * 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.common.TopicPartition;
    -
     import java.util.Comparator;
     
    +import org.apache.kafka.common.TopicPartition;
    +
     public class TopicPartitionComparator implements Comparator<TopicPartition> {
    -    @Override
    -    public int compare(TopicPartition o1, TopicPartition o2) {
    -        if (!o1.topic().equals(o2.topic())) {
    -            return o1.topic().compareTo(o2.topic());
    -        } else {
    -            return o1.partition() - o2.partition();
    -        }
    -    }
    +	public static final TopicPartitionComparator INSTANCE = new TopicPartitionComparator();
    --- End diff --
    
    Enums for things that are not an enumeration feels a little weird. Wouldn't you get the same result by just adding a private no-args constructor?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96774008
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RecordTranslator.java ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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 ConsumerRecord to a tuple.
    --- End diff --
    
    {@link org.apache.kafka.clients.consumer.ConsumerRecord}


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96904191
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -61,129 +66,244 @@
          * If no offset has been committed, it behaves as LATEST.</li>
          * </ul>
          * */
    -    public enum FirstPollOffsetStrategy {
    +    public static enum FirstPollOffsetStrategy {
             EARLIEST,
             LATEST,
             UNCOMMITTED_EARLIEST,
             UNCOMMITTED_LATEST }
    -
    -    // Kafka consumer configuration
    -    private final Map<String, Object> kafkaProps;
    -    private final Deserializer<K> keyDeserializer;
    -    private final Deserializer<V> 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<K, V> tuplesBuilder;
    -    private final KafkaSpoutRetryService retryService;
    -
    -    private KafkaSpoutConfig(Builder<K,V> 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<String, String> builder(String bootstrapServers, String ... topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
         }
    -
    -    private Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Collection<String> topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
    +    }
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Pattern topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
    +    }
    +    
    +    private static Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> 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<K,V> {
             private final Map<String, Object> kafkaProps;
    -        private SerializableDeserializer<K> keyDeserializer;
    -        private SerializableDeserializer<V> valueDeserializer;
    +        private Subscription subscription;
    +        private final SerializableDeserializer<K> keyDes;
    +        private final Class<? extends Deserializer<K>> keyDesClazz;
    +        private final SerializableDeserializer<V> valueDes;
    +        private final Class<? extends Deserializer<V>> valueDesClazz;
    +        private RecordTranslator<K, V> 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<K, V> tuplesBuilder;
    -        private final KafkaSpoutRetryService retryService;
    -
    -        /**
    -         * Please refer to javadoc in {@link #Builder(Map, KafkaSpoutStreams, KafkaSpoutTuplesBuilder, KafkaSpoutRetryService)}.<p/>
    -         * This constructor uses by the default the following implementation for {@link KafkaSpoutRetryService}:<p/>
    -         * {@code new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -         *           DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)))}
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> tuplesBuilder) {
    -            this(kafkaProps, kafkaSpoutStreams, tuplesBuilder,
    -                    new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -                            DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)));
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
             }
    -
    -        /***
    -         * 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 <a href="http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html">KafkaConsumer</a>
    -         * @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
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> 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");
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Subscription subscription) {
    +        	this(bootstrapServers, keyDes, null, valDes, null, subscription);
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Subscription subscription) {
    +        	this(bootstrapServers, null, keyDes, null, valDes, subscription);
    +        }
    +        
    +        private Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
    +        		SerializableDeserializer<V> valDes, Class<? extends Deserializer<V>> 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<K,V>();
    +        }
     
    -            this.kafkaProps = kafkaProps;
    -            this.kafkaSpoutStreams = kafkaSpoutStreams;
    -            this.tuplesBuilder = tuplesBuilder;
    -            this.retryService = retryService;
    +        private Builder(Builder<?, ?> builder, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
    +        		SerializableDeserializer<V> valueDes, Class<? extends Deserializer<V>> 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 this is the only way to really maintain API compatibility
    +            this.translator = (RecordTranslator<K, V>) builder.translator;
    +            this.retryService = builder.retryService;
    +            this.keyDes = keyDes;
    +            this.keyDesClazz = keyDesClazz;
    +            this.valueDes = valueDes;
    +            this.valueDesClazz = valueDesClazz;
             }
     
             /**
              * Specifying this key deserializer overrides the property key.deserializer
              */
    -        public Builder<K,V> setKeyDeserializer(SerializableDeserializer<K> keyDeserializer) {
    -            this.keyDeserializer = keyDeserializer;
    -            return this;
    +        public <NK> Builder<NK,V> setKey(SerializableDeserializer<NK> keyDeserializer) {
    --- End diff --
    
    I think what I was asking was "Why would you need to create a Builder with key type A and then later set a deserializer for keys of type B"? :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r97331194
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java ---
    @@ -1,16 +1,35 @@
    +/*
    + * 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.common.TopicPartition;
    -
     import java.util.Comparator;
     
    +import org.apache.kafka.common.TopicPartition;
    +
     public class TopicPartitionComparator implements Comparator<TopicPartition> {
    -    @Override
    -    public int compare(TopicPartition o1, TopicPartition o2) {
    -        if (!o1.topic().equals(o2.topic())) {
    -            return o1.topic().compareTo(o2.topic());
    -        } else {
    -            return o1.partition() - o2.partition();
    -        }
    -    }
    +	public static final TopicPartitionComparator INSTANCE = new TopicPartitionComparator();
    --- End diff --
    
    @hmcl I agree I went half way to making it a true singleton and I should finish the job.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/storm/pull/1808


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96763257
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -609,4 +593,7 @@ public String toString() {
                         '}';
             }
         }
    +
    +    // =========== Timer ===========
    --- End diff --
    
    `Timer` was moved to its own parent class. We can remove this comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96712816
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,232 @@
    -#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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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.
    +
    +There are a few key configs to pay attention to.
    +
    +`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
    +
    + * `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 `ConsumerRecord` into a Tuple and which stream that tuple will go to.  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.
    --- End diff --
    
    "Optional" -  should we rename this method to `setKafkaProp` or `setKafkaBrokerProp`  ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96731003
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -235,41 +355,116 @@ public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStrea
                 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<K, V> setPartitionRefreshPeriodMs(long partitionRefreshPeriodMs) {
    -            this.partitionRefreshPeriodMs = partitionRefreshPeriodMs;
    +        public Builder<K, V> setRetry(KafkaSpoutRetryService retryService) {
    +            if (retryService == null) {
    +                throw new NullPointerException("retryService cannot be null");
    +            }
    +            this.retryService = retryService;
                 return this;
             }
     
    +        public Builder<K, V> setRecordTranslator(RecordTranslator<K, V> translator) {
    +            this.translator = translator;
    +            return this;
    +        }
    +        
    +        public Builder<K, V> setRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
    +            return setRecordTranslator(new SimpleRecordTranslator<>(func, fields));
    +        }
    +        
    +        public Builder<K, V> setRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
    +            return setRecordTranslator(new SimpleRecordTranslator<>(func, fields, stream));
    +        }
    +        
             /**
    -         * 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.
    +         * Sets partition refresh period in milliseconds. This is how often the subscription is refreshed
    +         * For most subscriptions that go through the KafkaConsumer.subscribe this is ignored.
    --- End diff --
    
    This seems vague, and requires the user to go look at the subscription source. Maybe either link the method in Subscription that must be overridden to make this apply, or list the subscriptions that don't ignore this setting


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96780567
  
    --- Diff: 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<K, V> 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);
    --- End diff --
    
    ClassCastException?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    @srdo for me backwards compatibility for 1.x is more a question of violating out versioning than anything else.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96719100
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    +        this.boltSpecfiedProperties = producerProperties;
    +        return this;
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        //for backward compatibility.
    +        if(mapper == null) {
    +            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
    +        }
    +
    +        //for backward compatibility.
    +        if(topicSelector == null) {
    +            if(stormConf.containsKey(TOPIC)) {
    +                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
    +            } else {
    +                throw new IllegalArgumentException("topic should be specified in bolt's configuration");
    +            }
    +        }
    +
    +        producer = mkProducer(boltSpecfiedProperties);
    +        this.collector = collector;
    +    }
    +    
    +    /**
    +     * Intended to be overridden for tests.  Make the producer from props
    +     */
    +    protected KafkaProducer<K, V> mkProducer(Properties props) {
    --- End diff --
    
    `newProducer` ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91359755
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -290,8 +285,13 @@ private boolean emitTupleIfNotEmitted(ConsumerRecord<K, V> record) {
             } else if (emitted.contains(msgId)) {   // has been emitted and it's pending ack or fail
                 LOG.trace("Tuple for record [{}] has already been emitted. Skipping", record);
             } else if (!retryService.isScheduled(msgId) || retryService.isReady(msgId)) {   // not scheduled <=> never failed (i.e. never emitted) or ready to be retried
    -            final List<Object> tuple = tuplesBuilder.buildTuple(record);
    -            kafkaSpoutStreams.emit(collector, tuple, msgId);
    +            //TODO cache translator
    --- End diff --
    
    Can't we get rid of this TODO now by putting the translator in a field?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96911529
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RoundRobinManualPartitioner.java ---
    @@ -15,14 +15,26 @@
      *   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.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
     
    -import org.apache.kafka.clients.consumer.KafkaConsumer;
     import org.apache.kafka.common.TopicPartition;
    +import org.apache.storm.task.TopologyContext;
     
    -import java.util.List;
    +public class RoundRobinManualPartitioner implements ManualPartitioner {
     
    -public interface KafkaPartitionReader {
    -    List<TopicPartition> readPartitions(KafkaConsumer<?, ?> consumer);
    +	@Override
    +	public List<TopicPartition> partition(List<TopicPartition> allPartitions, TopologyContext context) {
    +		int thisTaskIndex = context.getThisTaskIndex();
    +		int totalTaskCount = context.getComponentTasks(context.getThisComponentId()).size();
    +		Set<TopicPartition> myPartitions = new HashSet<>(allPartitions.size()/totalTaskCount+1);
    +		for (int i = thisTaskIndex; i < allPartitions.size(); i += totalTaskCount) {
    --- End diff --
    
    No.  `i < allPartitions.size()` guarantees that we will never call get on allPartitions with an index that is out of bounds.  The Set is just setting the initial size to avoid more memory allocation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96710739
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,232 @@
    -#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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    --- End diff --
    
    ... expression, which specifies that any topics ... 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96734680
  
    --- Diff: 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.
      * <p/>
      * It works for sending tuples to older Kafka version (0.8.1).
    + * @deprecated Please use the KafkaBolt in storm-kafka-client
      */
    +@Deprecated
    --- End diff --
    
    Maybe it would be fine to just remove it here immediately and deprecate it in 1.1.0?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96907232
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -325,15 +310,19 @@ private boolean emitTupleIfNotEmitted(ConsumerRecord<K, V> 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<Object> tuple = tuplesBuilder.buildTuple(record);
    -                kafkaSpoutStreams.emit(collector, tuple, msgId);
    +                final List<Object> tuple = kafkaSpoutConfig.getTranslator().apply(record);
    +                if (tuple instanceof KafkaTuple) {
    --- End diff --
    
    Yes, and it is documented in RecordTranslator.  I will add in more documentation on it though.
    
    The reason specifically for this was because the spout is not able to keep track of a single message being emitted to multiple streams.  It would get confused and ack it before it was truly done.  This makes it impossible for that to happen.  What is more the built in record translators should cover 99% of the use cases, so the fact that it is not super well documented should be more of a corner case.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96925159
  
    --- Diff: 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);
    --- End diff --
    
    File another JIRA and we can look into it.  I think it is beyond the scope of this one.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91347833
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,222 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
    +##Compatability
     
    -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 (please see [Apache Kafka Version Compatibility] (#compatibility)). 
     
    -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 and 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` use to support decided which topic should to push message from tuple.
    --- End diff --
    
    Phrasing is a little awkward here. Maybe something like "... can be used to decide which topic a message should go to based on a field in the tuple" instead?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91363994
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedSubscription.java ---
    @@ -0,0 +1,66 @@
    +/*
    + * 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.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;
    +    private final Collection<String> topics;
    +    
    +    public NamedSubscription( Collection<String> topics) {
    +        super();
    +        this.topics = Collections.unmodifiableCollection(new ArrayList<>(topics));
    +    }
    +    
    +    public NamedSubscription(String ... topics) {
    +        this(Arrays.asList(topics));
    +    }
    +
    +    @Override
    +    public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceListener listener) {
    +        consumer.subscribe(topics, listener);
    +        LOG.info("Kafka consumer subscribed topics {}", topics);
    +    }
    +
    +    @Override
    +    public String getTopicsString() {
    +        StringBuilder ret = new StringBuilder();
    --- End diff --
    
    Why not use https://docs.oracle.com/javase/8/docs/api/java/lang/String.html#join-java.lang.CharSequence-java.lang.Iterable-?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91355497
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,194 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + * <p/>
    + * This bolt uses 0.8.2 Kafka Producer API.
    + * <p/>
    + * It works for sending tuples to older Kafka version (0.8.1).
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    /**
    +     * With default setting for fireAndForget and async, the callback is called when the sending succeeds.
    +     * By setting fireAndForget true, the send will not wait at all for kafka to ack.
    +     * "acks" setting in 0.8.2 Producer API config doesn't matter if fireAndForget is set.
    +     * By setting async false, synchronous sending is used. 
    +     */
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    +        this.boltSpecfiedProperties = producerProperties;
    +        return this;
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        //for backward compatibility.
    +        if(mapper == null) {
    +            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
    +        }
    +
    +        //for backward compatibility.
    +        if(topicSelector == null) {
    +            if(stormConf.containsKey(TOPIC)) {
    +                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
    +            } else {
    +                throw new IllegalArgumentException("topic should be specified in bolt's configuration");
    +            }
    +        }
    +
    +        producer = mkProducer(boltSpecfiedProperties);
    +        this.collector = collector;
    +    }
    +    
    +    /**
    +     * Intended to be overridden for tests.  Make the producer from props
    +     */
    +    protected KafkaProducer<K, V> 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);
    --- End diff --
    
    I thought the OutputCollector could only be used from the thread calling execute()? Is this no longer the case?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96889563
  
    --- Diff: 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.
      * <p/>
      * It works for sending tuples to older Kafka version (0.8.1).
    + * @deprecated Please use the KafkaBolt in storm-kafka-client
      */
    +@Deprecated
    --- End diff --
    
    I would rather do that in a follow on JIRA.  Most of the code is the same between the two so getting the rework from here to make there is a lot simpler if I can do a cherry pick


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    I think my only remaining nit is the spelling error in KafkaBolt. Thanks @revans2. I am +1 on this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96780204
  
    --- Diff: 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);
    --- End diff --
    
    since in some scenarios it's OK that this value returns null, wouldn't it be much clearer to return `Optional<String>` instead?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96740920
  
    --- Diff: 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 " + fieldIndex + " Out of bound . Using default topic " + defaultTopicName);
    --- End diff --
    
    bounds. Returning default ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96879498
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    +        this.boltSpecfiedProperties = producerProperties;
    +        return this;
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        //for backward compatibility.
    +        if(mapper == null) {
    +            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
    +        }
    +
    +        //for backward compatibility.
    +        if(topicSelector == null) {
    +            if(stormConf.containsKey(TOPIC)) {
    +                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
    +            } else {
    +                throw new IllegalArgumentException("topic should be specified in bolt's configuration");
    +            }
    +        }
    +
    +        producer = mkProducer(boltSpecfiedProperties);
    +        this.collector = collector;
    +    }
    +    
    +    /**
    +     * Intended to be overridden for tests.  Make the producer from props
    --- End diff --
    
    No sorry wrong place for that comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91367220
  
    --- Diff: 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<String, String> producer = mock(KafkaProducer.class);
    +        when(producer.send(any(), any())).thenAnswer(new Answer<Object>() {
    +            @Override
    +            public Object answer(InvocationOnMock invocation) throws Throwable {
    +                Callback c = (Callback)invocation.getArguments()[1];
    +                c.onCompletion(null, null);
    +                return null;
    +            }
    +        });
    +        KafkaBolt<String, String> bolt = new KafkaBolt<String, String>() {
    --- End diff --
    
    Nit: Can leave off the contents of `<>` on the right hand side in a few places here 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    @srdo @harshach sorry to do this to you, but I just fixed the conflicts with STORM-2236.  Sadly the fastest way I could do it was to revert the original code and implement similar functionality, which is the latest commit.  Could you please take a look at it?  I created a few new Subscription implementations that can do the manual partition management.  It only needed a small change to the Spout to support a timeout.  I will try to look at adding some documentation and also the impact to the trident spout. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91362663
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -60,123 +68,197 @@
          * If no offset has been committed, it behaves as LATEST.</li>
          * </ul>
          * */
    -    public enum FirstPollOffsetStrategy {
    +    public static enum FirstPollOffsetStrategy {
             EARLIEST,
             LATEST,
             UNCOMMITTED_EARLIEST,
             UNCOMMITTED_LATEST }
    -
    -    // Kafka consumer configuration
    -    private final Map<String, Object> kafkaProps;
    -    private final Deserializer<K> keyDeserializer;
    -    private final Deserializer<V> valueDeserializer;
    -    private final long pollTimeoutMs;
    -
    -    // Kafka spout configuration
    -    private final long offsetCommitPeriodMs;
    -    private final int maxRetries;
    -    private final int maxUncommittedOffsets;
    -    private final FirstPollOffsetStrategy firstPollOffsetStrategy;
    -    private final KafkaSpoutStreams kafkaSpoutStreams;
    -    private final KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;
    -    private final KafkaSpoutRetryService retryService;
    -
    -    private KafkaSpoutConfig(Builder<K,V> 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.tuplesBuilder = builder.tuplesBuilder;
    -        this.retryService = builder.retryService;
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, String ... topics) {
    +        return new Builder<>(bootstrapServers, new StringDeserializer(), new StringDeserializer(), topics);
         }
    -
    -    private Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Collection<String> topics) {
    +        return new Builder<>(bootstrapServers, new StringDeserializer(), new StringDeserializer(), topics);
    +    }
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Pattern topics) {
    +        return new Builder<>(bootstrapServers, new StringDeserializer(), new StringDeserializer(), topics);
    +    }
    +    
    +    private static Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
             // set defaults for properties not specified
    -        if (!kafkaProps.containsKey(Consumer.ENABLE_AUTO_COMMIT)) {
    -            kafkaProps.put(Consumer.ENABLE_AUTO_COMMIT, "false");
    +        if (!kafkaProps.containsKey(ENABLE_AUTO_COMMIT_CONF)) {
    +            kafkaProps.put(ENABLE_AUTO_COMMIT_CONF, "false");
             }
             return kafkaProps;
         }
    -
    +    
         public static class Builder<K,V> {
             private final Map<String, Object> kafkaProps;
    -        private Deserializer<K> keyDeserializer;
    -        private Deserializer<V> valueDeserializer;
    +        private Subscription subscription;
    +        private final Deserializer<K> keyDes;
    +        private final Deserializer<V> valueDes;
    +        private RecordTranslator<K, V> 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 final KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;
    -        private final KafkaSpoutRetryService retryService;
    -
    -        /**
    -         * Please refer to javadoc in {@link #Builder(Map, KafkaSpoutStreams, KafkaSpoutTuplesBuilder, KafkaSpoutRetryService)}.<p/>
    -         * This constructor uses by the default the following implementation for {@link KafkaSpoutRetryService}:<p/>
    -         * {@code new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -         *           DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)))}
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> tuplesBuilder) {
    -            this(kafkaProps, kafkaSpoutStreams, tuplesBuilder,
    -                    new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -                            DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)));
    +        private KafkaSpoutRetryService retryService = DEFAULT_RETRY_SERVICE;
    +        
    +        public Builder(String bootstrapServers, Deserializer<K> keyDes, Deserializer<V> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
             }
    -
    -        /***
    -         * 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 <a href="http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html">KafkaConsumer</a>
    -         * @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
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> 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");
    +        
    +        public Builder(String bootstrapServers, Deserializer<K> keyDes, Deserializer<V> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Deserializer<K> keyDes, Deserializer<V> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Deserializer<K> keyDes, Deserializer<V> valDes, Subscription subscription) {
    +            kafkaProps = new HashMap<>();
    +            if (bootstrapServers == null || bootstrapServers.isEmpty()) {
    +                throw new IllegalArgumentException("bootstrap servers cannot be null");
                 }
    +            kafkaProps.put(BOOTSTRAP_SERVERS_CONF, bootstrapServers);
    +            this.keyDes = keyDes;
    +            this.valueDes = valDes;
    +            this.subscription = subscription;
    +            this.translator = new DefaultRecordTranslator<K,V>();
    +        }
     
    -            this.kafkaProps = kafkaProps;
    -            this.kafkaSpoutStreams = kafkaSpoutStreams;
    -            this.tuplesBuilder = tuplesBuilder;
    -            this.retryService = retryService;
    +        private Builder(Builder<?, ?> builder, Deserializer<K> keyDes, Deserializer<V> valueDes) {
    +            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 this is the only way to really maintain API compatibility
    +            this.translator = (RecordTranslator<K, V>) builder.translator;
    +            this.retryService = builder.retryService;
    +            this.keyDes = keyDes;
    +            this.valueDes = valueDes;
             }
     
             /**
              * Specifying this key deserializer overrides the property key.deserializer
              */
    -        public Builder<K,V> setKeyDeserializer(Deserializer<K> keyDeserializer) {
    -            this.keyDeserializer = keyDeserializer;
    -            return this;
    +        public <NK> Builder<NK,V> setKeyDeserializer(Deserializer<NK> keyDeserializer) {
    +            return new Builder<>(this, keyDeserializer, valueDes);
             }
     
             /**
              * Specifying this value deserializer overrides the property value.deserializer
              */
    -        public Builder<K,V> setValueDeserializer(Deserializer<V> valueDeserializer) {
    -            this.valueDeserializer = valueDeserializer;
    +        public <NV> Builder<K,NV> setValueDeserializer(Deserializer<NV> valueDeserializer) {
    +            return new Builder<>(this, this.keyDes, valueDeserializer);
    +        }
    +        
    +        /**
    +         * Set a Kafka property config
    +         */
    +        public Builder<K,V> setProp(String key, Object value) {
    +            kafkaProps.put(key, value);
                 return this;
             }
    +        
    +        /**
    +         * Set multiple Kafka property configs
    +         */
    +        public Builder<K,V> setProp(Map<String, Object> props) {
    +            kafkaProps.putAll(props);
    +            return this;
    +        }
    +        
    +        /**
    +         * Set multiple Kafka property configs
    +         */
    +        public Builder<K,V> 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<K,V> setGroupId(String id) {
    +            return setProp("group.id", id);
    +        }
    +        
    +        /**
    +         * reset the bootstrap servers for the Consumer
    +         */
    +        public Builder<K,V> setBootstrapServers(String servers) {
    +            return setProp(BOOTSTRAP_SERVERS_CONF, servers);
    +        }
    +        
    +        /**
    +         * The minimum amount of data the broker should return for a fetch request.
    +         */
    +        public Builder<K,V> setFetchMinBytes(int bytes) {
    +            return setProp("fetch.min.bytes", bytes);
    +        }
    +        
    +        /**
    +         * The maximum amount of data per-partition the broker will return.
    +         */
    +        public Builder<K,V> setMaxPartitionFectchBytes(int bytes) {
    +            return setProp("max.partition.fetch.bytes", bytes);
    +        }
    +        
    +        /**
    +         * The maximum number of records a poll will return.
    +         * Only will work with Kafak 0.10.0 and above.
    +         */
    +        public Builder<K,V> setMaxPoolRecords(int records) {
    +            return setProp("max.poll.recordes", records);
    --- End diff --
    
    recordes -> records


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96779787
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java ---
    @@ -0,0 +1,114 @@
    +/**
    + * 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.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) {
    +        if (mapper == null) throw new NullPointerException("mapper can not be null");
    --- End diff --
    
    Objects.requireNonNull("mapper cannot be null");
    ..


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r94917566
  
    --- Diff: external/storm-kafka-client/README.md ---
    @@ -1,191 +1,3 @@
    -#Storm Kafka Spout with New Kafka Consumer API
    +#Storm Kafka integration using the kafka-client jar
     
    -Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version 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`.
    -
    -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<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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<String, String> 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
    -<plugin>
    -    <groupId>org.apache.maven.plugins</groupId>
    -    <artifactId>maven-shade-plugin</artifactId>
    -    <version>2.4.1</version>
    -    <executions>
    -        <execution>
    -            <phase>package</phase>
    -            <goals>
    -                <goal>shade</goal>
    -            </goals>
    -            <configuration>
    -                <transformers>
    -                    <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
    -                        <mainClass>org.apache.storm.kafka.spout.test.KafkaSpoutTopologyMain</mainClass>
    -                    </transformer>
    -                </transformers>
    -            </configuration>
    -        </execution>
    -    </executions>
    -</plugin>
    -```
    -
    -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
    -        <dependency>
    -            <groupId>org.apache.kafka</groupId>
    -            <artifactId>kafka-clients</artifactId>
    -            <version>0.10.0.0</version>
    -        </dependency>
    -```
    -
    -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
    -<br/>
    -
    -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: 
    -
    -* \u201cfetch.min.bytes\u201d
    -* \u201cfetch.max.wait.ms\u201d
    -* [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)
    -<br/>
    -
    -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
    -<br/>
    -
    -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)
    --- End diff --
    
    We still need to keep the description because this file will be included to binary dist whereas docs is not.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96728300
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -61,129 +66,244 @@
          * If no offset has been committed, it behaves as LATEST.</li>
          * </ul>
          * */
    -    public enum FirstPollOffsetStrategy {
    +    public static enum FirstPollOffsetStrategy {
             EARLIEST,
             LATEST,
             UNCOMMITTED_EARLIEST,
             UNCOMMITTED_LATEST }
    -
    -    // Kafka consumer configuration
    -    private final Map<String, Object> kafkaProps;
    -    private final Deserializer<K> keyDeserializer;
    -    private final Deserializer<V> 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<K, V> tuplesBuilder;
    -    private final KafkaSpoutRetryService retryService;
    -
    -    private KafkaSpoutConfig(Builder<K,V> 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<String, String> builder(String bootstrapServers, String ... topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
         }
    -
    -    private Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Collection<String> topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
    +    }
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Pattern topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
    +    }
    +    
    +    private static Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> 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<K,V> {
             private final Map<String, Object> kafkaProps;
    -        private SerializableDeserializer<K> keyDeserializer;
    -        private SerializableDeserializer<V> valueDeserializer;
    +        private Subscription subscription;
    +        private final SerializableDeserializer<K> keyDes;
    +        private final Class<? extends Deserializer<K>> keyDesClazz;
    +        private final SerializableDeserializer<V> valueDes;
    +        private final Class<? extends Deserializer<V>> valueDesClazz;
    +        private RecordTranslator<K, V> 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<K, V> tuplesBuilder;
    -        private final KafkaSpoutRetryService retryService;
    -
    -        /**
    -         * Please refer to javadoc in {@link #Builder(Map, KafkaSpoutStreams, KafkaSpoutTuplesBuilder, KafkaSpoutRetryService)}.<p/>
    -         * This constructor uses by the default the following implementation for {@link KafkaSpoutRetryService}:<p/>
    -         * {@code new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -         *           DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)))}
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> tuplesBuilder) {
    -            this(kafkaProps, kafkaSpoutStreams, tuplesBuilder,
    -                    new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -                            DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)));
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
             }
    -
    -        /***
    -         * 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 <a href="http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html">KafkaConsumer</a>
    -         * @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
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> 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");
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Subscription subscription) {
    +        	this(bootstrapServers, keyDes, null, valDes, null, subscription);
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Subscription subscription) {
    +        	this(bootstrapServers, null, keyDes, null, valDes, subscription);
    +        }
    +        
    +        private Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
    +        		SerializableDeserializer<V> valDes, Class<? extends Deserializer<V>> 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<K,V>();
    +        }
     
    -            this.kafkaProps = kafkaProps;
    -            this.kafkaSpoutStreams = kafkaSpoutStreams;
    -            this.tuplesBuilder = tuplesBuilder;
    -            this.retryService = retryService;
    +        private Builder(Builder<?, ?> builder, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
    +        		SerializableDeserializer<V> valueDes, Class<? extends Deserializer<V>> 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,
    --- End diff --
    
    I'm wondering if the API compatibility workarounds are necessary for master, or if they can be limited to 1.x?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91357255
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.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.selector;
    +
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Uses field index to select topic name from tuple .
    --- End diff --
    
    Nit: Maybe a little clearer to say "Uses the field with index..."


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96751858
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ByTopicRecordTranslator.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 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;
    +
    +public class ByTopicRecordTranslator<K, V> implements RecordTranslator<K, V> {
    +    private static final long serialVersionUID = -121699733778988688L;
    +    private final RecordTranslator<K,V> defaultTranslator;
    +    private final Map<String, RecordTranslator<K,V>> topicToTranslator = new HashMap<>();
    +    private final Map<String, Fields> streamToFields = new HashMap<>();
    +    
    +    public ByTopicRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
    +        this(new SimpleRecordTranslator<>(func, fields, stream));
    +    }
    +    
    +    public ByTopicRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
    +        this(new SimpleRecordTranslator<>(func, fields));
    +    }
    +    
    +    public ByTopicRecordTranslator(RecordTranslator<K,V> defaultTranslator) {
    +        this.defaultTranslator = defaultTranslator;
    +        cacheNCheckFields(defaultTranslator);
    +    }
    +    
    +    public ByTopicRecordTranslator<K, V> forTopic(String topic, Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
    +        return forTopic(topic, new SimpleRecordTranslator<>(func, fields));
    +    }
    +    
    +    public ByTopicRecordTranslator<K, V> forTopic(String topic, Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
    +        return forTopic(topic, new SimpleRecordTranslator<>(func, fields, stream));
    +    }
    +    
    +    public ByTopicRecordTranslator<K, V> forTopic(String topic, RecordTranslator<K,V> translator) {
    +        if (topicToTranslator.containsKey(topic)) {
    +            throw new IllegalStateException("Topic " + topic + " is already registered");
    +        }
    +        topicToTranslator.put(topic, translator);
    +        cacheNCheckFields(translator);
    +        return this;
    +    }
    +    
    +    private void cacheNCheckFields(RecordTranslator<K, V> 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);
    --- End diff --
    
    `IllegalStateException` ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96767444
  
    --- Diff: 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 parter;
    --- End diff --
    
    I think `ptioner` is a better abbreviation than `parter`. `partitioner` would make code more readable.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91360488
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -18,34 +18,42 @@
     
     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.function.Function;
     import java.util.regex.Pattern;
     
    +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<K, V> implements Serializable {
    +    private static final long serialVersionUID = 141902646130682494L;
    +    private static final String BOOTSTRAP_SERVERS_CONF = "bootstrap.servers";
    --- End diff --
    
    Nit: These constants are already defined in `org.apache.kafka.clients.consumer.ConsumerConfig`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    @hmcl I think I addressed all of your review comments.  I also re-based to deal with the merge conflict. If everything looks good I will squash some of the commits.  There are a lot.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91353717
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,194 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + * <p/>
    + * This bolt uses 0.8.2 Kafka Producer API.
    + * <p/>
    + * It works for sending tuples to older Kafka version (0.8.1).
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    --- End diff --
    
    Specfied -> Specified


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    Thanks @revans2. Can you please just amend the commit log [messages](https://github.com/apache/storm/pull/1808/commits) as I believe they do not really reflect the changes in this patch. One of the messages has STORM-1997 title, and the other has a cryptic merge log message.
    
    Also, can you please do the same for 1.x-branch [patch](https://github.com/apache/storm/pull/1868)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96762055
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java ---
    @@ -1,16 +1,35 @@
    +/*
    + * 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.common.TopicPartition;
    -
     import java.util.Comparator;
     
    +import org.apache.kafka.common.TopicPartition;
    +
     public class TopicPartitionComparator implements Comparator<TopicPartition> {
    -    @Override
    -    public int compare(TopicPartition o1, TopicPartition o2) {
    -        if (!o1.topic().equals(o2.topic())) {
    -            return o1.topic().compareTo(o2.topic());
    -        } else {
    -            return o1.partition() - o2.partition();
    -        }
    -    }
    +	public static final TopicPartitionComparator INSTANCE = new TopicPartitionComparator();
    --- End diff --
    
    Probably the best way to guarantee that INSTANCE is always called is to make this an `Enum`, which will also make the code thread safe. Currently nothing prevents the user from creating her own instance of this class.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91621754
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,194 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + * <p/>
    + * This bolt uses 0.8.2 Kafka Producer API.
    + * <p/>
    + * It works for sending tuples to older Kafka version (0.8.1).
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    /**
    +     * With default setting for fireAndForget and async, the callback is called when the sending succeeds.
    +     * By setting fireAndForget true, the send will not wait at all for kafka to ack.
    +     * "acks" setting in 0.8.2 Producer API config doesn't matter if fireAndForget is set.
    +     * By setting async false, synchronous sending is used. 
    +     */
    +    private boolean fireAndForget = false;
    +    private boolean async = true;
    +
    +    public KafkaBolt() {}
    +
    +    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K, V> withTopicSelector(String topic) {
    +        return withTopicSelector(new DefaultTopicSelector(topic));
    +    }
    +    
    +    public KafkaBolt<K,V> withTopicSelector(KafkaTopicSelector selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    public KafkaBolt<K,V> withProducerProperties(Properties producerProperties) {
    +        this.boltSpecfiedProperties = producerProperties;
    +        return this;
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        //for backward compatibility.
    +        if(mapper == null) {
    +            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
    +        }
    +
    +        //for backward compatibility.
    +        if(topicSelector == null) {
    +            if(stormConf.containsKey(TOPIC)) {
    +                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
    +            } else {
    +                throw new IllegalArgumentException("topic should be specified in bolt's configuration");
    +            }
    +        }
    +
    +        producer = mkProducer(boltSpecfiedProperties);
    +        this.collector = collector;
    +    }
    +    
    +    /**
    +     * Intended to be overridden for tests.  Make the producer from props
    +     */
    +    protected KafkaProducer<K, V> 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);
    --- End diff --
    
    Sounds good then, thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96780580
  
    --- Diff: 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<K, V> 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) {
    --- End diff --
    
    ClassCastException?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96776152
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Subscription.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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 <K, V> void subscribe(KafkaConsumer<K,V> consumer, ConsumerRebalanceListener listener, TopologyContext context) {
    +    	subscribe(consumer, listener);
    +    }
    +
    +    /**
    +     * Subscribe the KafkaConsumer to the proper topics
    +     * @param consumer the Consumer to get.
    +     * @param listener the rebalance listener to include in the subscription
    +     * @deprecated please use the version with the TopologyContext in it
    +     */
    +    public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceListener listener) {
    +    	throw new IllegalStateException("At least one subscribe method must be overwritten");
    +    }
    +    
    +    /**
    +     * @return a string representing the subscribed topics.
    +     */
    +    public abstract String getTopicsString();
    +    
    +    /**
    +     * Refresh any assignments if needed.  Kafka usually will handle this for you.
    --- End diff --
    
    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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96730015
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -235,41 +355,116 @@ public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStrea
                 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<K, V> setPartitionRefreshPeriodMs(long partitionRefreshPeriodMs) {
    -            this.partitionRefreshPeriodMs = partitionRefreshPeriodMs;
    +        public Builder<K, V> setRetry(KafkaSpoutRetryService retryService) {
    +            if (retryService == null) {
    +                throw new NullPointerException("retryService cannot be null");
    +            }
    +            this.retryService = retryService;
                 return this;
             }
     
    +        public Builder<K, V> setRecordTranslator(RecordTranslator<K, V> translator) {
    +            this.translator = translator;
    +            return this;
    +        }
    +        
    +        public Builder<K, V> setRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
    --- End diff --
    
    Would probably be good to put javadoc on this one and the one below, it's not obvious from the interface/parameter names what it does


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96924990
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java ---
    @@ -0,0 +1,114 @@
    +/**
    + * 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.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) {
    +        if (mapper == null) throw new NullPointerException("mapper can not be null");
    --- End diff --
    
    Objects does not exist in java 6 and I would prefer to keep the code compatible as mush as possible to avoid extra rework when pulling these changes back.  If you insist I will do it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96767284
  
    --- Diff: 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 parter;
    +	private Set<TopicPartition> currentAssignment = null;
    --- End diff --
    
    all these `= null` are redundant


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91366483
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java ---
    @@ -44,29 +40,30 @@
         // Bookkeeping
         private final KafkaSpoutConfig<K, V> 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<K, V> tuplesBuilder;        // Object that contains the logic to build tuples for each ConsumerRecord
    +    private final Fields fields;
     
         public KafkaTridentSpoutManager(KafkaSpoutConfig<K, V> kafkaSpoutConfig) {
             this.kafkaSpoutConfig = kafkaSpoutConfig;
    -        kafkaSpoutStreams = kafkaSpoutConfig.getKafkaSpoutStreams();
    -        tuplesBuilder = kafkaSpoutConfig.getTuplesBuilder();
    +        RecordTranslator<K, V> 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 nut support multiple output Fields");
    --- End diff --
    
    Nit: nut -> not. Also maybe rephrase as "must have the same fields for all streams"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96891568
  
    --- Diff: 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;
    --- End diff --
    
    Because I missed them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1808
  
    The travis test failures are unrelated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96712487
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,232 @@
    -#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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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.
    +
    +There are a few key configs to pay attention to.
    +
    +`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
    +
    + * `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 `ConsumerRecord` into a Tuple and which stream that tuple will go to.  By default the "topic",
    --- End diff --
    
    Kafka `Consumer Record` into a Tuple, and which stream that tuple will be published into.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96716770
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,194 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + * <p/>
    + * This bolt uses 0.8.2 Kafka Producer API.
    + * <p/>
    + * It works for sending tuples to older Kafka version (0.8.1).
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    --- End diff --
    
    nit: `boltSpecifiedProps` or `boltSpecificProps`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96763944
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTuple.java ---
    @@ -15,22 +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.util.List;
    -
    -public class KafkaSpoutTuplesBuilderWildcardTopics<K,V> implements KafkaSpoutTuplesBuilder<K,V> {
    -    private KafkaSpoutTupleBuilder<K, V> tupleBuilder;
    +import org.apache.storm.tuple.Values;
     
    -    public KafkaSpoutTuplesBuilderWildcardTopics(KafkaSpoutTupleBuilder<K, V> tupleBuilder) {
    -        this.tupleBuilder = tupleBuilder;
    +/**
    + * A list of Values in a tuple that can be routed 
    + * to a given stream.
    --- End diff --
    
    If the `KafkaSpout` remains as is, I would suggest that we add a comment here saying that the method `org.apache.storm.kafka.spout.RecordTranslator#apply` must necessarily return `KafkaTuple` if the user desires to write to multiple streams.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96729446
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -61,129 +66,244 @@
          * If no offset has been committed, it behaves as LATEST.</li>
          * </ul>
          * */
    -    public enum FirstPollOffsetStrategy {
    +    public static enum FirstPollOffsetStrategy {
             EARLIEST,
             LATEST,
             UNCOMMITTED_EARLIEST,
             UNCOMMITTED_LATEST }
    -
    -    // Kafka consumer configuration
    -    private final Map<String, Object> kafkaProps;
    -    private final Deserializer<K> keyDeserializer;
    -    private final Deserializer<V> 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<K, V> tuplesBuilder;
    -    private final KafkaSpoutRetryService retryService;
    -
    -    private KafkaSpoutConfig(Builder<K,V> 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<String, String> builder(String bootstrapServers, String ... topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
         }
    -
    -    private Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Collection<String> topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
    +    }
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Pattern topics) {
    +        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
    +    }
    +    
    +    private static Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> 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<K,V> {
             private final Map<String, Object> kafkaProps;
    -        private SerializableDeserializer<K> keyDeserializer;
    -        private SerializableDeserializer<V> valueDeserializer;
    +        private Subscription subscription;
    +        private final SerializableDeserializer<K> keyDes;
    +        private final Class<? extends Deserializer<K>> keyDesClazz;
    +        private final SerializableDeserializer<V> valueDes;
    +        private final Class<? extends Deserializer<V>> valueDesClazz;
    +        private RecordTranslator<K, V> 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<K, V> tuplesBuilder;
    -        private final KafkaSpoutRetryService retryService;
    -
    -        /**
    -         * Please refer to javadoc in {@link #Builder(Map, KafkaSpoutStreams, KafkaSpoutTuplesBuilder, KafkaSpoutRetryService)}.<p/>
    -         * This constructor uses by the default the following implementation for {@link KafkaSpoutRetryService}:<p/>
    -         * {@code new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -         *           DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)))}
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> tuplesBuilder) {
    -            this(kafkaProps, kafkaSpoutStreams, tuplesBuilder,
    -                    new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -                            DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)));
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
             }
    -
    -        /***
    -         * 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 <a href="http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html">KafkaConsumer</a>
    -         * @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
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> 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");
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Subscription subscription) {
    +        	this(bootstrapServers, keyDes, null, valDes, null, subscription);
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Subscription subscription) {
    +        	this(bootstrapServers, null, keyDes, null, valDes, subscription);
    +        }
    +        
    +        private Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
    +        		SerializableDeserializer<V> valDes, Class<? extends Deserializer<V>> 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<K,V>();
    +        }
     
    -            this.kafkaProps = kafkaProps;
    -            this.kafkaSpoutStreams = kafkaSpoutStreams;
    -            this.tuplesBuilder = tuplesBuilder;
    -            this.retryService = retryService;
    +        private Builder(Builder<?, ?> builder, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
    +        		SerializableDeserializer<V> valueDes, Class<? extends Deserializer<V>> 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 this is the only way to really maintain API compatibility
    +            this.translator = (RecordTranslator<K, V>) builder.translator;
    +            this.retryService = builder.retryService;
    +            this.keyDes = keyDes;
    +            this.keyDesClazz = keyDesClazz;
    +            this.valueDes = valueDes;
    +            this.valueDesClazz = valueDesClazz;
             }
     
             /**
              * Specifying this key deserializer overrides the property key.deserializer
              */
    -        public Builder<K,V> setKeyDeserializer(SerializableDeserializer<K> keyDeserializer) {
    -            this.keyDeserializer = keyDeserializer;
    -            return this;
    +        public <NK> Builder<NK,V> setKey(SerializableDeserializer<NK> keyDeserializer) {
    --- End diff --
    
    Why do we need to sometimes change the key type of a builder?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91362733
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java ---
    @@ -60,123 +68,197 @@
          * If no offset has been committed, it behaves as LATEST.</li>
          * </ul>
          * */
    -    public enum FirstPollOffsetStrategy {
    +    public static enum FirstPollOffsetStrategy {
             EARLIEST,
             LATEST,
             UNCOMMITTED_EARLIEST,
             UNCOMMITTED_LATEST }
    -
    -    // Kafka consumer configuration
    -    private final Map<String, Object> kafkaProps;
    -    private final Deserializer<K> keyDeserializer;
    -    private final Deserializer<V> valueDeserializer;
    -    private final long pollTimeoutMs;
    -
    -    // Kafka spout configuration
    -    private final long offsetCommitPeriodMs;
    -    private final int maxRetries;
    -    private final int maxUncommittedOffsets;
    -    private final FirstPollOffsetStrategy firstPollOffsetStrategy;
    -    private final KafkaSpoutStreams kafkaSpoutStreams;
    -    private final KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;
    -    private final KafkaSpoutRetryService retryService;
    -
    -    private KafkaSpoutConfig(Builder<K,V> 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.tuplesBuilder = builder.tuplesBuilder;
    -        this.retryService = builder.retryService;
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, String ... topics) {
    +        return new Builder<>(bootstrapServers, new StringDeserializer(), new StringDeserializer(), topics);
         }
    -
    -    private Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Collection<String> topics) {
    +        return new Builder<>(bootstrapServers, new StringDeserializer(), new StringDeserializer(), topics);
    +    }
    +    
    +    public static Builder<String, String> builder(String bootstrapServers, Pattern topics) {
    +        return new Builder<>(bootstrapServers, new StringDeserializer(), new StringDeserializer(), topics);
    +    }
    +    
    +    private static Map<String, Object> setDefaultsAndGetKafkaProps(Map<String, Object> kafkaProps) {
             // set defaults for properties not specified
    -        if (!kafkaProps.containsKey(Consumer.ENABLE_AUTO_COMMIT)) {
    -            kafkaProps.put(Consumer.ENABLE_AUTO_COMMIT, "false");
    +        if (!kafkaProps.containsKey(ENABLE_AUTO_COMMIT_CONF)) {
    +            kafkaProps.put(ENABLE_AUTO_COMMIT_CONF, "false");
             }
             return kafkaProps;
         }
    -
    +    
         public static class Builder<K,V> {
             private final Map<String, Object> kafkaProps;
    -        private Deserializer<K> keyDeserializer;
    -        private Deserializer<V> valueDeserializer;
    +        private Subscription subscription;
    +        private final Deserializer<K> keyDes;
    +        private final Deserializer<V> valueDes;
    +        private RecordTranslator<K, V> 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 final KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;
    -        private final KafkaSpoutRetryService retryService;
    -
    -        /**
    -         * Please refer to javadoc in {@link #Builder(Map, KafkaSpoutStreams, KafkaSpoutTuplesBuilder, KafkaSpoutRetryService)}.<p/>
    -         * This constructor uses by the default the following implementation for {@link KafkaSpoutRetryService}:<p/>
    -         * {@code new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -         *           DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)))}
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> tuplesBuilder) {
    -            this(kafkaProps, kafkaSpoutStreams, tuplesBuilder,
    -                    new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
    -                            DEFAULT_MAX_RETRIES, TimeInterval.seconds(10)));
    +        private KafkaSpoutRetryService retryService = DEFAULT_RETRY_SERVICE;
    +        
    +        public Builder(String bootstrapServers, Deserializer<K> keyDes, Deserializer<V> valDes, String ... topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
             }
    -
    -        /***
    -         * 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 <a href="http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html">KafkaConsumer</a>
    -         * @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
    -         */
    -        public Builder(Map<String, Object> kafkaProps, KafkaSpoutStreams kafkaSpoutStreams,
    -                       KafkaSpoutTuplesBuilder<K,V> 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");
    +        
    +        public Builder(String bootstrapServers, Deserializer<K> keyDes, Deserializer<V> valDes, Collection<String> topics) {
    +            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Deserializer<K> keyDes, Deserializer<V> valDes, Pattern topics) {
    +            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
    +        }
    +        
    +        public Builder(String bootstrapServers, Deserializer<K> keyDes, Deserializer<V> valDes, Subscription subscription) {
    +            kafkaProps = new HashMap<>();
    +            if (bootstrapServers == null || bootstrapServers.isEmpty()) {
    +                throw new IllegalArgumentException("bootstrap servers cannot be null");
                 }
    +            kafkaProps.put(BOOTSTRAP_SERVERS_CONF, bootstrapServers);
    +            this.keyDes = keyDes;
    +            this.valueDes = valDes;
    +            this.subscription = subscription;
    +            this.translator = new DefaultRecordTranslator<K,V>();
    +        }
     
    -            this.kafkaProps = kafkaProps;
    -            this.kafkaSpoutStreams = kafkaSpoutStreams;
    -            this.tuplesBuilder = tuplesBuilder;
    -            this.retryService = retryService;
    +        private Builder(Builder<?, ?> builder, Deserializer<K> keyDes, Deserializer<V> valueDes) {
    +            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 this is the only way to really maintain API compatibility
    +            this.translator = (RecordTranslator<K, V>) builder.translator;
    +            this.retryService = builder.retryService;
    +            this.keyDes = keyDes;
    +            this.valueDes = valueDes;
             }
     
             /**
              * Specifying this key deserializer overrides the property key.deserializer
              */
    -        public Builder<K,V> setKeyDeserializer(Deserializer<K> keyDeserializer) {
    -            this.keyDeserializer = keyDeserializer;
    -            return this;
    +        public <NK> Builder<NK,V> setKeyDeserializer(Deserializer<NK> keyDeserializer) {
    +            return new Builder<>(this, keyDeserializer, valueDes);
             }
     
             /**
              * Specifying this value deserializer overrides the property value.deserializer
              */
    -        public Builder<K,V> setValueDeserializer(Deserializer<V> valueDeserializer) {
    -            this.valueDeserializer = valueDeserializer;
    +        public <NV> Builder<K,NV> setValueDeserializer(Deserializer<NV> valueDeserializer) {
    +            return new Builder<>(this, this.keyDes, valueDeserializer);
    +        }
    +        
    +        /**
    +         * Set a Kafka property config
    +         */
    +        public Builder<K,V> setProp(String key, Object value) {
    +            kafkaProps.put(key, value);
                 return this;
             }
    +        
    +        /**
    +         * Set multiple Kafka property configs
    +         */
    +        public Builder<K,V> setProp(Map<String, Object> props) {
    +            kafkaProps.putAll(props);
    +            return this;
    +        }
    +        
    +        /**
    +         * Set multiple Kafka property configs
    +         */
    +        public Builder<K,V> 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<K,V> setGroupId(String id) {
    +            return setProp("group.id", id);
    +        }
    +        
    +        /**
    +         * reset the bootstrap servers for the Consumer
    +         */
    +        public Builder<K,V> setBootstrapServers(String servers) {
    +            return setProp(BOOTSTRAP_SERVERS_CONF, servers);
    +        }
    +        
    +        /**
    +         * The minimum amount of data the broker should return for a fetch request.
    +         */
    +        public Builder<K,V> setFetchMinBytes(int bytes) {
    +            return setProp("fetch.min.bytes", bytes);
    +        }
    +        
    +        /**
    +         * The maximum amount of data per-partition the broker will return.
    +         */
    +        public Builder<K,V> setMaxPartitionFectchBytes(int bytes) {
    +            return setProp("max.partition.fetch.bytes", bytes);
    +        }
    +        
    +        /**
    +         * The maximum number of records a poll will return.
    +         * Only will work with Kafak 0.10.0 and above.
    --- End diff --
    
    kafak -> kafka, and "will only work" instead of "only will work"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96694426
  
    --- Diff: docs/storm-kafka-client.md ---
    @@ -1,90 +1,232 @@
    -#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);
     
    -# Usage Examples
    +```
     
    -### Create a Kafka Spout
    +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
     
    -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.
    +###Putting it all together
     
    +For the bolt :
     ```java
    -KafkaSpout<String, String> kafkaSpout = new KafkaSpout<>(kafkaSpoutConfig);
    -
    -KafkaSpoutConfig kafkaSpoutConfig = new KafkaSpoutConfig.Builder<String, String>(kafkaConsumerProps, kafkaSpoutStreams, tuplesBuilder, retryService)
    -        .setOffsetCommitPeriodMs(10_000)
    -        .setFirstPollOffsetStrategy(EARLIEST)
    -        .setMaxUncommittedOffsets(250)
    -        .build();
    -
    -Map<String, Object> 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));
    +        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());
     ```
     
    -### Named Topics
    +For Trident:
    +
     ```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<String, String> tuplesBuilder = new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
    -            new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
    -            new TopicTest2TupleBuilder<String, String>(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");
    +        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());
     ```
     
    -### Topic Wildcards
    +## 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` and any
    +topics that match that regular expression will be consumed.
    +
    +In the case of the Constructors you may also need to specify a key deserializer and a value deserializer.  This is to help make the java generics happy
    +and help maintain type safety.  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.
    +
    +There are a few key configs to pay attention to.
    +
    +`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
    +
    + * `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 `ConsumerRecord` into a Tuple and which stream that tuple will go to.  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
    +
    +#### Create a Simple Insecure Spout
    +The following will consume all events published to "topic" and send them to MyBolt as "topic", "partition", "offset", "key", "value".
     ```java
    -KafkaSpoutStreams kafkaSpoutStreams = new KafkaSpoutStreamsWildcardTopics(
    -            new KafkaSpoutStream(outputFields, STREAM, Pattern.compile(TOPIC_WILDCARD_PATTERN)));
     
    -KafkaSpoutTuplesBuilder<String, String> tuplesBuilder = new TopicsTest0Test1TupleBuilder<>(TOPIC_WILDCARD_PATTERN);
    +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");
    +...
     
    -String STREAM = "test_wildcard_stream";
    -String TOPIC_WILDCARD_PATTERN = "test[1|2]";
    +```
    +
    +#### Wildcard Topics
    +Wildcard topics will consume from all topics that are on the configured broakers and match the pattern.  So in the following example
    --- End diff --
    
    ... that exist in the brokers list specified and...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91356935
  
    --- Diff: 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;
    +
    +/**
    + * as the really verbose name suggests this interface mapps a storm tuple to kafka key and message.
    --- End diff --
    
    mapps -> maps


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96725703
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -80,16 +76,14 @@
         private transient KafkaSpoutRetryService retryService;              // Class that has the logic to handle tuple failure
         private transient Timer commitTimer;                                // timer == null for auto commit mode
         private transient boolean initialized;                              // Flag indicating that the spout is still undergoing initialization process.
    -    private transient KafkaRecordsFetcher<K, V> recordsFetcher;         // Class that encapsulates the logic of managing partitions and fetching records
         // Initialization is only complete after the first call to  KafkaSpoutConsumerRebalanceListener.onPartitionsAssigned()
     
    -    private KafkaSpoutStreams kafkaSpoutStreams;                        // Object that wraps all the logic to declare output fields and emit tuples
    -    private transient KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;      // Object that contains the logic to build tuples for each ConsumerRecord
    -
         transient Map<TopicPartition, OffsetEntry> 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<KafkaSpoutMessageId> emitted;                 // Tuples that have been emitted but that are "on the wire", i.e. pending being acked or failed
         private transient Iterator<ConsumerRecord<K, V>> 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;
    --- End diff --
    
    Nit: The rest of the file uses spaces for indentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96891946
  
    --- Diff: 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 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) {
    --- End diff --
    
    I agree but both for backwards compatibility and to limit the scope of this JIRA I would rather see it done in a follow on JIRA


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91365926
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -185,7 +178,9 @@ private long doSeek(TopicPartition tp, OffsetAndMetadata committedOffset) {
         }
     
         private Collection<TopicPartition> toArrayList(final TopicPartition tp) {
    -        return new ArrayList<TopicPartition>(1){{add(tp);}};
    +        ArrayList<TopicPartition> ret = new ArrayList<>(1);
    --- End diff --
    
    Nit: Unless you need to modify the return value later, you can just return Collections.singleton(tp)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96779676
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java ---
    @@ -0,0 +1,114 @@
    +/**
    + * 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.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) {
    --- End diff --
    
    Wouldn't it be preferable the class to be immutable ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r91353321
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java ---
    @@ -0,0 +1,194 @@
    +/**
    + * 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
    + * <p/>
    + * It expects the producer configuration and topic in storm config under
    + * <p/>
    + * 'kafka.broker.properties' and 'topic'
    + * <p/>
    + * respectively.
    + * <p/>
    + * This bolt uses 0.8.2 Kafka Producer API.
    + * <p/>
    + * It works for sending tuples to older Kafka version (0.8.1).
    + */
    +public class KafkaBolt<K, V> 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<K, V> producer;
    +    private OutputCollector collector;
    +    private TupleToKafkaMapper<K,V> mapper;
    +    private KafkaTopicSelector topicSelector;
    +    private Properties boltSpecfiedProperties = new Properties();
    +    /**
    +     * With default setting for fireAndForget and async, the callback is called when the sending succeeds.
    --- End diff --
    
    This seems a little hard to parse unless you also have the code open. I think it's clearer to describe this in terms of when tuples are acked/failed, rather than when the callback is called.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96763012
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -447,7 +440,10 @@ private void shutdown() {
     
         @Override
         public void declareOutputFields(OutputFieldsDeclarer declarer) {
    -        kafkaSpoutStreams.declareOutputFields(declarer);
    +        RecordTranslator<K, V> translator = kafkaSpoutConfig.getTranslator();
    --- End diff --
    
    I agree with this. I just think that this pattern is used many times in many Bolt implementations. I think that it would be useful to have an abstraction as part of stomr, similar to what `KafkaSpoutStreams` was doing, that would allow Bolts to declare the set of fields per stream in a pluggable way. This would make it uniform for every Bolt, and avoid having to do similar work for every Bolt that is writing to multiple streams.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1808: STORM-2225: change spout config to be simpler.

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1808#discussion_r96761049
  
    --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java ---
    @@ -325,15 +310,19 @@ private boolean emitTupleIfNotEmitted(ConsumerRecord<K, V> 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<Object> tuple = tuplesBuilder.buildTuple(record);
    -                kafkaSpoutStreams.emit(collector, tuple, msgId);
    +                final List<Object> tuple = kafkaSpoutConfig.getTranslator().apply(record);
    +                if (tuple instanceof KafkaTuple) {
    --- End diff --
    
    Doesn't this require that for a tuple to go to a specific stream, that tuple has to necessarily be a `KafkaTuple`? If that is the case, it is not obvious for an implementer of `RecordTranslator` that he has to do so. If I understand it correctly, currently this code only works as intended with `SimpleRecordTranslator`.
    
    The reasoning behind the `KafkaSpoutStreams` abstraction was to allow the user to clearly define the stream and associated fields without ambiguity.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---