You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2017/02/01 15:55:37 UTC

[1/7] storm git commit: Revert "Merge branch 'STORM-2236' of https://github.com/MediaV/storm into STORM-2236"

Repository: storm
Updated Branches:
  refs/heads/master 22cecb087 -> 251cb8876


Revert "Merge branch 'STORM-2236' of https://github.com/MediaV/storm into STORM-2236"

This reverts commit e2f87c304297ce95f01708e0934c44ee385fa0bb, reversing
changes made to 00fed518d14b485e2a9e9cdd91ade3cc9d68dc8d.


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/74a77aa6
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/74a77aa6
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/74a77aa6

Branch: refs/heads/master
Commit: 74a77aa6494957b941df55d66ff64b6561eee4a8
Parents: 4c5e34e
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Jan 18 10:34:30 2017 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Mon Jan 30 14:17:40 2017 -0600

----------------------------------------------------------------------
 .../apache/storm/kafka/spout/KafkaSpout.java    |  95 ++++++++++++----
 .../storm/kafka/spout/KafkaSpoutConfig.java     |  34 ------
 .../kafka/spout/TopicPartitionComparator.java   |  34 ------
 .../storm/kafka/spout/internal/Timer.java       |  75 -------------
 .../fetcher/AutomaticKafkaRecordsFetcher.java   |  67 ------------
 .../internal/fetcher/KafkaRecordsFetcher.java   |  25 -----
 .../internal/fetcher/KafkaRecordsFetchers.java  |  54 ----------
 .../fetcher/ManualKafkaRecordsFetcher.java      | 108 -------------------
 .../partition/KafkaPartitionReader.java         |  28 -----
 .../partition/KafkaPartitionReaders.java        |  45 --------
 .../partition/NamedTopicPartitionReader.java    |  47 --------
 .../partition/WildcardTopicPartitionReader.java |  51 ---------
 12 files changed, 77 insertions(+), 586 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/74a77aa6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java
index dcb245c..d405c4d 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java
@@ -24,14 +24,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecords;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.InterruptException;
-import org.apache.kafka.common.errors.RetriableException;
 import org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy;
-import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
-import org.apache.storm.kafka.spout.internal.KafkaConsumerFactoryDefault;
-import org.apache.storm.kafka.spout.internal.Timer;
-import org.apache.storm.kafka.spout.internal.fetcher.KafkaRecordsFetcher;
-import org.apache.storm.kafka.spout.internal.fetcher.KafkaRecordsFetchers;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
@@ -53,11 +46,16 @@ import java.util.NavigableSet;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
 
 import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
 import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.LATEST;
 import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST;
 import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_LATEST;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactoryDefault;
+
+import org.apache.kafka.common.errors.InterruptException;
 
 public class KafkaSpout<K, V> extends BaseRichSpout {
     private static final Logger LOG = LoggerFactory.getLogger(KafkaSpout.class);
@@ -65,7 +63,6 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
 
     // Storm
     protected SpoutOutputCollector collector;
-    private TopologyContext topologyContext;
 
     // Kafka
     private final KafkaSpoutConfig<K, V> kafkaSpoutConfig;
@@ -80,7 +77,6 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
     private transient KafkaSpoutRetryService retryService;              // Class that has the logic to handle tuple failure
     private transient Timer commitTimer;                                // timer == null for auto commit mode
     private transient boolean initialized;                              // Flag indicating that the spout is still undergoing initialization process.
-    private transient KafkaRecordsFetcher<K, V> recordsFetcher;         // Class that encapsulates the logic of managing partitions and fetching records
     // Initialization is only complete after the first call to  KafkaSpoutConsumerRebalanceListener.onPartitionsAssigned()
 
     private KafkaSpoutStreams kafkaSpoutStreams;                        // Object that wraps all the logic to declare output fields and emit tuples
@@ -106,9 +102,9 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
     @Override
     public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
         initialized = false;
+
         // Spout internals
         this.collector = collector;
-        this.topologyContext = context;
         maxRetries = kafkaSpoutConfig.getMaxTupleRetries();
         numUncommittedOffsets = 0;
 
@@ -225,11 +221,7 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
                 }
 
                 if (poll()) {
-                    try {
-                        setWaitingToEmit(pollKafkaBroker());
-                    } catch (RetriableException e) {
-                        LOG.error("Failed to poll from kafka.", e);
-                    }
+                    setWaitingToEmit(pollKafkaBroker());
                 }
 
                 if (waitingToEmit()) {
@@ -285,7 +277,7 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
     private ConsumerRecords<K, V> pollKafkaBroker() {
         doSeekRetriableTopicPartitions();
 
-        final ConsumerRecords<K, V> consumerRecords = recordsFetcher.fetchRecords(kafkaSpoutConfig.getPollTimeoutMs());
+        final ConsumerRecords<K, V> consumerRecords = kafkaConsumer.poll(kafkaSpoutConfig.getPollTimeoutMs());
         final int numPolledRecords = consumerRecords.count();
         LOG.debug("Polled [{}] records from Kafka. [{}] uncommitted offsets across all topic partitions", numPolledRecords, numUncommittedOffsets);
         return consumerRecords;
@@ -412,8 +404,19 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
 
     private void subscribeKafkaConsumer() {
         kafkaConsumer = kafkaConsumerFactory.createConsumer(kafkaSpoutConfig);
-        recordsFetcher = KafkaRecordsFetchers.create(kafkaSpoutConfig, kafkaConsumer, topologyContext,
-            new KafkaSpoutConsumerRebalanceListener());
+
+        if (kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics) {
+            final List<String> topics = ((KafkaSpoutStreamsNamedTopics) kafkaSpoutStreams).getTopics();
+            kafkaConsumer.subscribe(topics, new KafkaSpoutConsumerRebalanceListener());
+            LOG.info("Kafka consumer subscribed topics {}", topics);
+        } else if (kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics) {
+            final Pattern pattern = ((KafkaSpoutStreamsWildcardTopics) kafkaSpoutStreams).getTopicWildcardPattern();
+            kafkaConsumer.subscribe(pattern, new KafkaSpoutConsumerRebalanceListener());
+            LOG.info("Kafka consumer subscribed topics matching wildcard pattern [{}]", pattern);
+        }
+        // Initial poll to get the consumer registration process going.
+        // KafkaSpoutConsumerRebalanceListener will be called following this poll, upon partition registration
+        kafkaConsumer.poll(0);
     }
 
     @Override
@@ -609,4 +612,60 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
                     '}';
         }
     }
+
+    // =========== Timer ===========
+
+    private class Timer {
+        private final long delay;
+        private final long period;
+        private final TimeUnit timeUnit;
+        private final long periodNanos;
+        private long start;
+
+        /**
+         * Creates a class that mimics a single threaded timer that expires periodically. If a call to {@link
+         * #isExpiredResetOnTrue()} occurs later than {@code period} since the timer was initiated or reset, this method returns
+         * true. Each time the method returns true the counter is reset. The timer starts with the specified time delay.
+         *
+         * @param delay    the initial delay before the timer starts
+         * @param period   the period between calls {@link #isExpiredResetOnTrue()}
+         * @param timeUnit the time unit of delay and period
+         */
+        public Timer(long delay, long period, TimeUnit timeUnit) {
+            this.delay = delay;
+            this.period = period;
+            this.timeUnit = timeUnit;
+
+            periodNanos = timeUnit.toNanos(period);
+            start = System.nanoTime() + timeUnit.toNanos(delay);
+        }
+
+        public long period() {
+            return period;
+        }
+
+        public long delay() {
+            return delay;
+        }
+
+        public TimeUnit getTimeUnit() {
+            return timeUnit;
+        }
+
+        /**
+         * Checks if a call to this method occurs later than {@code period} since the timer was initiated or reset. If that is the
+         * case the method returns true, otherwise it returns false. Each time this method returns true, the counter is reset
+         * (re-initiated) and a new cycle will start.
+         *
+         * @return true if the time elapsed since the last call returning true is greater than {@code period}. Returns false
+         * otherwise.
+         */
+        public boolean isExpiredResetOnTrue() {
+            final boolean expired = System.nanoTime() - start > periodNanos;
+            if (expired) {
+                start = System.nanoTime();
+            }
+            return expired;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/74a77aa6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java
index 5daf13f..8aa525b 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java
@@ -36,7 +36,6 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
     public static final long DEFAULT_OFFSET_COMMIT_PERIOD_MS = 30_000;   // 30s
     public static final int DEFAULT_MAX_RETRIES = Integer.MAX_VALUE;     // Retry forever
     public static final int DEFAULT_MAX_UNCOMMITTED_OFFSETS = 10_000_000;    // 10,000,000 records => 80MBs of memory footprint in the worst case
-    public static final long DEFAULT_PARTITION_REFRESH_PERIOD_MS = 2_000; // 2s
 
     // Kafka property names
     public interface Consumer {
@@ -77,8 +76,6 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
     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;
@@ -94,8 +91,6 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
         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;
     }
@@ -118,8 +113,6 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
         private FirstPollOffsetStrategy firstPollOffsetStrategy = FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST;
         private final KafkaSpoutStreams kafkaSpoutStreams;
         private int maxUncommittedOffsets = DEFAULT_MAX_UNCOMMITTED_OFFSETS;
-        private long partitionRefreshPeriodMs = DEFAULT_PARTITION_REFRESH_PERIOD_MS;
-        private boolean manualPartitionAssignment = false;
         private final KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;
         private final KafkaSpoutRetryService retryService;
 
@@ -236,25 +229,6 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
             return this;
         }
 
-        /**
-         * Sets partition refresh period in milliseconds in manual partition assignment model. Default is 2s.
-         * @param partitionRefreshPeriodMs time in milliseconds
-         */
-        public Builder<K, V> setPartitionRefreshPeriodMs(long partitionRefreshPeriodMs) {
-            this.partitionRefreshPeriodMs = partitionRefreshPeriodMs;
-            return this;
-        }
-
-        /**
-         * Defines whether the consumer manages partition manually.
-         * If set to true, the consumer manage partition manually, otherwise it will rely on kafka to do partition assignment.
-         * @param manualPartitionAssignment True if using manual partition assignment.
-         */
-        public Builder<K, V> setManualPartitionAssignment(boolean manualPartitionAssignment) {
-            this.manualPartitionAssignment = manualPartitionAssignment;
-            return this;
-        }
-
         public KafkaSpoutConfig<K,V> build() {
             return new KafkaSpoutConfig<>(this);
         }
@@ -333,14 +307,6 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
         return retryService;
     }
 
-    public long getPartitionRefreshPeriodMs() {
-        return partitionRefreshPeriodMs;
-    }
-
-    public boolean isManualPartitionAssignment() {
-        return manualPartitionAssignment;
-    }
-
     @Override
     public String toString() {
         return "KafkaSpoutConfig{" +

http://git-wip-us.apache.org/repos/asf/storm/blob/74a77aa6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java
deleted file mode 100644
index b908001..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout;
-
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.Comparator;
-
-public class TopicPartitionComparator implements Comparator<TopicPartition> {
-    @Override
-    public int compare(TopicPartition o1, TopicPartition o2) {
-        if (!o1.topic().equals(o2.topic())) {
-            return o1.topic().compareTo(o2.topic());
-        } else {
-            return o1.partition() - o2.partition();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/74a77aa6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java
deleted file mode 100644
index 45ae330..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout.internal;
-
-import java.util.concurrent.TimeUnit;
-
-public class Timer {
-    private final long delay;
-    private final long period;
-    private final TimeUnit timeUnit;
-    private final long periodNanos;
-    private long start;
-
-    /**
-     * Creates a class that mimics a single threaded timer that expires periodically. If a call to {@link
-     * #isExpiredResetOnTrue()} occurs later than {@code period} since the timer was initiated or reset, this method returns
-     * true. Each time the method returns true the counter is reset. The timer starts with the specified time delay.
-     *
-     * @param delay    the initial delay before the timer starts
-     * @param period   the period between calls {@link #isExpiredResetOnTrue()}
-     * @param timeUnit the time unit of delay and period
-     */
-    public Timer(long delay, long period, TimeUnit timeUnit) {
-        this.delay = delay;
-        this.period = period;
-        this.timeUnit = timeUnit;
-
-        periodNanos = timeUnit.toNanos(period);
-        start = System.nanoTime() + timeUnit.toNanos(delay);
-    }
-
-    public long period() {
-        return period;
-    }
-
-    public long delay() {
-        return delay;
-    }
-
-    public TimeUnit getTimeUnit() {
-        return timeUnit;
-    }
-
-    /**
-     * Checks if a call to this method occurs later than {@code period} since the timer was initiated or reset. If that is the
-     * case the method returns true, otherwise it returns false. Each time this method returns true, the counter is reset
-     * (re-initiated) and a new cycle will start.
-     *
-     * @return true if the time elapsed since the last call returning true is greater than {@code period}. Returns false
-     * otherwise.
-     */
-    public boolean isExpiredResetOnTrue() {
-        final boolean expired = System.nanoTime() - start > periodNanos;
-        if (expired) {
-            start = System.nanoTime();
-        }
-        return expired;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/74a77aa6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/AutomaticKafkaRecordsFetcher.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/AutomaticKafkaRecordsFetcher.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/AutomaticKafkaRecordsFetcher.java
deleted file mode 100644
index 8ba7098..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/AutomaticKafkaRecordsFetcher.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout.internal.fetcher;
-
-import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.storm.kafka.spout.KafkaSpoutStreams;
-import org.apache.storm.kafka.spout.KafkaSpoutStreamsNamedTopics;
-import org.apache.storm.kafka.spout.KafkaSpoutStreamsWildcardTopics;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-import java.util.regex.Pattern;
-
-public class AutomaticKafkaRecordsFetcher<K, V> implements KafkaRecordsFetcher<K, V> {
-    private static final Logger LOG = LoggerFactory.getLogger(AutomaticKafkaRecordsFetcher.class);
-
-    private final KafkaConsumer<K, V> kafkaConsumer;
-    private final ConsumerRebalanceListener consumerRebalanceListener;
-
-    public AutomaticKafkaRecordsFetcher(KafkaConsumer<K, V> kafkaConsumer,
-                                        ConsumerRebalanceListener consumerRebalanceListener,
-                                        KafkaSpoutStreams kafkaSpoutStreams) {
-        this.kafkaConsumer = kafkaConsumer;
-        this.consumerRebalanceListener = consumerRebalanceListener;
-
-        subscribe(kafkaSpoutStreams);
-    }
-
-    private void subscribe(KafkaSpoutStreams kafkaSpoutStreams) {
-        if (kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics) {
-            final List<String> topics = ((KafkaSpoutStreamsNamedTopics) kafkaSpoutStreams).getTopics();
-            kafkaConsumer.subscribe(topics, consumerRebalanceListener);
-            LOG.info("Kafka consumer subscribed topics {}", topics);
-        } else if (kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics) {
-            final Pattern pattern = ((KafkaSpoutStreamsWildcardTopics) kafkaSpoutStreams).getTopicWildcardPattern();
-            kafkaConsumer.subscribe(pattern, consumerRebalanceListener);
-            LOG.info("Kafka consumer subscribed topics matching wildcard pattern [{}]", pattern);
-        }
-        // Initial poll to get the consumer registration process going.
-        // KafkaSpoutConsumerRebalanceListener will be called following this poll, upon partition registration
-        kafkaConsumer.poll(0);
-    }
-
-    @Override
-    public ConsumerRecords<K, V> fetchRecords(long fetchTimeoutMs) {
-        return kafkaConsumer.poll(fetchTimeoutMs);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/74a77aa6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetcher.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetcher.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetcher.java
deleted file mode 100644
index 47a61c1..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetcher.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout.internal.fetcher;
-
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-
-public interface KafkaRecordsFetcher<K, V> {
-    ConsumerRecords<K, V> fetchRecords(long fetchTimeoutMs);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/74a77aa6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetchers.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetchers.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetchers.java
deleted file mode 100644
index e3b7cd6..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/KafkaRecordsFetchers.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout.internal.fetcher;
-
-import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.storm.kafka.spout.KafkaSpoutConfig;
-import org.apache.storm.kafka.spout.internal.Timer;
-import org.apache.storm.kafka.spout.internal.partition.KafkaPartitionReader;
-import org.apache.storm.kafka.spout.internal.partition.KafkaPartitionReaders;
-import org.apache.storm.task.TopologyContext;
-
-import java.util.concurrent.TimeUnit;
-
-public final class KafkaRecordsFetchers {
-    public static <K, V> KafkaRecordsFetcher<K, V> create(KafkaSpoutConfig kafkaSpoutConfig,
-                                                          KafkaConsumer<K, V> consumer,
-                                                          TopologyContext context,
-                                                          ConsumerRebalanceListener rebalanceListener) {
-        if (kafkaSpoutConfig.isManualPartitionAssignment()) {
-            int thisTaskIndex = context.getThisTaskIndex();
-            int totalTaskCount = context.getComponentTasks(context.getThisComponentId()).size();
-            KafkaPartitionReader partitionReader = KafkaPartitionReaders.create(
-                kafkaSpoutConfig.getKafkaSpoutStreams());
-            Timer partitionRefreshTimer = new Timer(500,
-                kafkaSpoutConfig.getPartitionRefreshPeriodMs(), TimeUnit.MILLISECONDS);
-
-            ManualKafkaRecordsFetcher.PartitionAssignmentChangeListener partitionAssignmentChangeListener =
-                ManualKafkaRecordsFetcher.listenerOf(rebalanceListener);
-
-            return new ManualKafkaRecordsFetcher<>(consumer, thisTaskIndex, totalTaskCount, partitionReader,
-                partitionRefreshTimer, partitionAssignmentChangeListener);
-        } else {
-            return new AutomaticKafkaRecordsFetcher<>(consumer, rebalanceListener,
-                kafkaSpoutConfig.getKafkaSpoutStreams());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/74a77aa6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/ManualKafkaRecordsFetcher.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/ManualKafkaRecordsFetcher.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/ManualKafkaRecordsFetcher.java
deleted file mode 100644
index d02a59a..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/fetcher/ManualKafkaRecordsFetcher.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout.internal.fetcher;
-
-import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.storm.kafka.spout.internal.Timer;
-import org.apache.storm.kafka.spout.internal.partition.KafkaPartitionReader;
-import org.apache.storm.kafka.spout.TopicPartitionComparator;
-
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-
-public class ManualKafkaRecordsFetcher<K, V> implements KafkaRecordsFetcher<K, V> {
-    private static final Comparator<TopicPartition> KAFKA_TOPIC_PARTITION_COMPARATOR = new TopicPartitionComparator();
-
-    private final KafkaConsumer<K, V> consumer;
-    private final int thisTaskIndex;
-    private final int totalTaskCount;
-    private final KafkaPartitionReader partitionReader;
-    private final Timer partitionRefreshTimer;
-    private final PartitionAssignmentChangeListener partitionAssignmentChangeListener;
-    private Set<TopicPartition> myPartitions = Collections.emptySet();
-
-    public ManualKafkaRecordsFetcher(KafkaConsumer<K, V> consumer,
-                                     int thisTaskIndex,
-                                     int totalTaskCount,
-                                     KafkaPartitionReader partitionReader,
-                                     Timer partitionRefreshTimer,
-                                     PartitionAssignmentChangeListener partitionAssignmentChangeListener) {
-        this.consumer = consumer;
-        this.thisTaskIndex = thisTaskIndex;
-        this.totalTaskCount = totalTaskCount;
-        this.partitionReader = partitionReader;
-        this.partitionRefreshTimer = partitionRefreshTimer;
-        this.partitionAssignmentChangeListener = partitionAssignmentChangeListener;
-
-        doRefreshMyPartitions();
-    }
-
-    private void refreshMyPartitionsIfNeeded() {
-        if (!partitionRefreshTimer.isExpiredResetOnTrue()) {
-            return;
-        }
-
-        doRefreshMyPartitions();
-    }
-
-    private void doRefreshMyPartitions() {
-        List<TopicPartition> topicPartitions = partitionReader.readPartitions(consumer);
-        Collections.sort(topicPartitions, KAFKA_TOPIC_PARTITION_COMPARATOR);
-
-        Set<TopicPartition> curPartitions = new HashSet<>(topicPartitions.size()/totalTaskCount+1);
-        for (int i=thisTaskIndex; i<topicPartitions.size(); i+=totalTaskCount) {
-            curPartitions.add(topicPartitions.get(i));
-        }
-
-        if (!myPartitions.equals(curPartitions) && myPartitions!=null) {
-            partitionAssignmentChangeListener.onPartitionAssignmentChange(myPartitions, curPartitions);
-        }
-
-        myPartitions = curPartitions;
-
-        consumer.assign(myPartitions);
-    }
-
-    @Override
-    public ConsumerRecords<K, V> fetchRecords(long fetchTimeoutMs) {
-        refreshMyPartitionsIfNeeded();
-
-        return consumer.poll(fetchTimeoutMs);
-    }
-
-    @FunctionalInterface
-    public interface PartitionAssignmentChangeListener {
-        void onPartitionAssignmentChange(Set<TopicPartition> oldPartitions, Set<TopicPartition> newPartitions);
-    }
-
-    public static PartitionAssignmentChangeListener listenerOf(final ConsumerRebalanceListener consumerRebalanceListener) {
-        return (oldPartitions, newPartitions) -> {
-            consumerRebalanceListener.onPartitionsRevoked(oldPartitions);
-            consumerRebalanceListener.onPartitionsAssigned(newPartitions);
-        };
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/74a77aa6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReader.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReader.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReader.java
deleted file mode 100644
index e3480ac..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReader.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout.internal.partition;
-
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.List;
-
-public interface KafkaPartitionReader {
-    List<TopicPartition> readPartitions(KafkaConsumer<?, ?> consumer);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/74a77aa6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReaders.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReaders.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReaders.java
deleted file mode 100644
index 4e51c1e..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/KafkaPartitionReaders.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout.internal.partition;
-
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.storm.kafka.spout.KafkaSpoutStreams;
-import org.apache.storm.kafka.spout.KafkaSpoutStreamsNamedTopics;
-import org.apache.storm.kafka.spout.KafkaSpoutStreamsWildcardTopics;
-
-import java.util.HashSet;
-
-public final class KafkaPartitionReaders {
-    public static KafkaPartitionReader create(KafkaSpoutStreams kafkaSpoutStreams) {
-        if (kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics) {
-            return new NamedTopicPartitionReader(new HashSet<>(
-                KafkaSpoutStreamsNamedTopics.class.cast(kafkaSpoutStreams).getTopics()));
-        } else if (kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics) {
-            return new WildcardTopicPartitionReader(
-                KafkaSpoutStreamsWildcardTopics.class.cast(kafkaSpoutStreams).getTopicWildcardPattern());
-        } else {
-            throw new IllegalArgumentException("Unrecognized kafka spout stream: " + kafkaSpoutStreams.getClass());
-        }
-    }
-
-    public static TopicPartition toTopicPartition(PartitionInfo partitionInfo) {
-        return new TopicPartition(partitionInfo.topic(), partitionInfo.partition());
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/74a77aa6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/NamedTopicPartitionReader.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/NamedTopicPartitionReader.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/NamedTopicPartitionReader.java
deleted file mode 100644
index 41db169..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/NamedTopicPartitionReader.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout.internal.partition;
-
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-
-public class NamedTopicPartitionReader implements KafkaPartitionReader {
-    private final Set<String > topics;
-
-    public NamedTopicPartitionReader(Set<String> topics) {
-        this.topics = topics;
-    }
-
-    @Override
-    public List<TopicPartition> readPartitions(KafkaConsumer<?, ?> consumer) {
-        List<TopicPartition> topicPartitions = new ArrayList<>();
-        for (String topic : topics) {
-            for (PartitionInfo partitionInfo: consumer.partitionsFor(topic)) {
-                topicPartitions.add(KafkaPartitionReaders.toTopicPartition(partitionInfo));
-            }
-        }
-
-        return topicPartitions;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/74a77aa6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/WildcardTopicPartitionReader.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/WildcardTopicPartitionReader.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/WildcardTopicPartitionReader.java
deleted file mode 100644
index fcac1c1..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/partition/WildcardTopicPartitionReader.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout.internal.partition;
-
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Pattern;
-
-public class WildcardTopicPartitionReader implements KafkaPartitionReader {
-    private final Pattern wildcardTopicPattern;
-
-    public WildcardTopicPartitionReader(Pattern wildcardTopicPattern) {
-        this.wildcardTopicPattern = wildcardTopicPattern;
-    }
-
-    @Override
-    public List<TopicPartition> readPartitions(KafkaConsumer<?, ?> consumer) {
-        List<TopicPartition> topicPartitions = new ArrayList<>();
-
-        for(Map.Entry<String, List<PartitionInfo>> entry: consumer.listTopics().entrySet()) {
-            if (wildcardTopicPattern.matcher(entry.getKey()).matches()) {
-                for (PartitionInfo partitionInfo: entry.getValue()) {
-                    topicPartitions.add(KafkaPartitionReaders.toTopicPartition(partitionInfo));
-                }
-            }
-        }
-
-        return topicPartitions;
-    }
-}


[3/7] storm git commit: 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 partit

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitioner.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitioner.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitioner.java
new file mode 100644
index 0000000..88803f8
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitioner.java
@@ -0,0 +1,40 @@
+/*
+ * 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.List;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.task.TopologyContext;
+
+/**
+ * A function used to assign partitions to this spout.
+ * WARNING if this is not done correctly you can really mess things up, like not reading data in some partitions.
+ * The complete TopologyContext is passed in, but it is suggested that you use the index of the spout and the total
+ * number of spouts to avoid missing partitions or double assigning partitions.
+ */
+@FunctionalInterface
+public interface ManualPartitioner {
+    /**
+     * Get the partitions for this assignment
+     * @param allPartitions all of the partitions that the set of spouts want to subscribe to, in a strict ordering
+     * @param context the context of the topology
+     * @return the subset of the partitions that this spout should use.
+     */
+    public List<TopicPartition> partition(List<TopicPartition> allPartitions, TopologyContext context);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedSubscription.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedSubscription.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedSubscription.java
new file mode 100644
index 0000000..6eba566
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedSubscription.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+package org.apache.storm.kafka.spout;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.storm.task.TopologyContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Subscribe to all topics that follow a given list of values
+ */
+public class NamedSubscription extends Subscription {
+    private static final Logger LOG = LoggerFactory.getLogger(NamedSubscription.class);
+    private static final long serialVersionUID = 3438543305215813839L;
+    protected final Collection<String> topics;
+    
+    public NamedSubscription(Collection<String> topics) {
+        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);
+        
+        // Initial poll to get the consumer registration process going.
+        // KafkaSpoutConsumerRebalanceListener will be called following this poll, upon partition registration
+        consumer.poll(0);
+    }
+
+    @Override
+    public String getTopicsString() {
+        return String.join(",", topics);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/PatternSubscription.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/PatternSubscription.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/PatternSubscription.java
new file mode 100644
index 0000000..9a8de0f
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/PatternSubscription.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+package org.apache.storm.kafka.spout;
+
+import java.util.regex.Pattern;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.storm.task.TopologyContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Subscribe to all topics that match a given pattern
+ */
+public class PatternSubscription extends Subscription {
+    private static final Logger LOG = LoggerFactory.getLogger(PatternSubscription.class);
+    private static final long serialVersionUID = 3438543305215813839L;
+    protected final Pattern pattern;
+    
+    public PatternSubscription(Pattern pattern) {
+        this.pattern = pattern;
+    }
+
+    @Override
+    public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceListener listener, TopologyContext unused) {
+        consumer.subscribe(pattern, listener);
+        LOG.info("Kafka consumer subscribed topics matching wildcard pattern [{}]", pattern);
+        
+        // Initial poll to get the consumer registration process going.
+        // KafkaSpoutConsumerRebalanceListener will be called following this poll, upon partition registration
+        consumer.poll(0);
+    }
+
+    @Override
+    public String getTopicsString() {
+        return pattern.pattern();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RecordTranslator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RecordTranslator.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RecordTranslator.java
new file mode 100644
index 0000000..f2d52ec
--- /dev/null
+++ b/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 {@link org.apache.kafka.clients.consumer.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");
+    
+    /**
+     * Translate the ConsumerRecord into a list of objects that can be emitted
+     * @param record the record to translate
+     * @return the objects in the tuple.  Return a {@link KafkaTuple}
+     * if you want to route the tuple to a non-default stream
+     */
+    List<Object> apply(ConsumerRecord<K,V> record);
+    
+    /**
+     * Get the fields associated with a stream.  The streams passed in are
+     * returned by the {@link RecordTranslator.streams} method.
+     * @param stream the stream the fields are for
+     * @return the fields for that stream.
+     */
+    Fields getFieldsFor(String stream);
+    
+    /**
+     * @return the list of streams that this will handle.
+     */
+    default List<String> streams() {
+        return DEFAULT_STREAM;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RoundRobinManualPartitioner.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RoundRobinManualPartitioner.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RoundRobinManualPartitioner.java
new file mode 100644
index 0000000..e23e2dc
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/RoundRobinManualPartitioner.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+package org.apache.storm.kafka.spout;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.task.TopologyContext;
+
+/**
+ * Assign partitions in a round robin fashion for all spouts,
+ * not just the ones that are alive. Because the parallelism of 
+ * the spouts does not typically change while running this makes
+ * the assignments more stable in the face of crashing spouts.
+ * 
+ * Round Robin means that first spout of N spouts will get the first
+ * partition, and the N+1th partition... The second spout will get the second partition and
+ * N+2th partition etc.
+ */
+public class RoundRobinManualPartitioner implements ManualPartitioner {
+
+    @Override
+    public List<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) {
+            myPartitions.add(allPartitions.get(i));
+        }
+        return new ArrayList<>(myPartitions);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/SimpleRecordTranslator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/SimpleRecordTranslator.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/SimpleRecordTranslator.java
new file mode 100644
index 0000000..46c2849
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/SimpleRecordTranslator.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+package org.apache.storm.kafka.spout;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.storm.tuple.Fields;
+
+public class SimpleRecordTranslator<K, V> implements RecordTranslator<K, V> {
+    private static final long serialVersionUID = 4678369144122009596L;
+    private final Fields fields;
+    private final Func<ConsumerRecord<K, V>, List<Object>> func;
+    private final String stream;
+
+    public SimpleRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
+        this(func, fields, "default");
+    }
+    
+    public SimpleRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
+        this.func = func;
+        this.fields = fields;
+        this.stream = stream;
+    }
+    
+    @Override
+    public List<Object> apply(ConsumerRecord<K, V> record) {
+        KafkaTuple ret = new KafkaTuple();
+        ret.addAll(func.apply(record));
+        return ret.routedTo(stream);
+    }
+
+    @Override
+    public Fields getFieldsFor(String stream) {
+        return fields;
+    }
+    
+    @Override
+    public List<String> streams() {
+        return Arrays.asList(stream);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Subscription.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Subscription.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Subscription.java
new file mode 100644
index 0000000..db2a7bb
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Subscription.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+package org.apache.storm.kafka.spout;
+
+import java.io.Serializable;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.storm.task.TopologyContext;
+
+/**
+ * A subscription to kafka.
+ */
+public abstract class Subscription implements Serializable {
+    private static final long serialVersionUID = -216136367240198716L;
+
+    /**
+     * Subscribe the KafkaConsumer to the proper topics
+     * @param consumer the Consumer to get.
+     * @param listener the rebalance listener to include in the subscription
+     */
+    public abstract <K, V> void subscribe(KafkaConsumer<K,V> consumer, ConsumerRebalanceListener listener, TopologyContext context);
+    
+    /**
+     * @return a string representing the subscribed topics.
+     */
+    public abstract String getTopicsString();
+    
+    /**
+     * NOOP is the default behavior, which means that Kafka will internally handle partition assignment.
+     * If you wish to do manual partition management, you must provide an implementation of this method
+     * that will check with kafka for any changes and call the ConsumerRebalanceListener from subscribe
+     * to inform the rest of the system of those changes.
+     */
+    public void refreshAssignment() {
+        //NOOP
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.java
new file mode 100644
index 0000000..dafb97c
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/TopicPartitionComparator.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.spout;
+
+import java.util.Comparator;
+
+import org.apache.kafka.common.TopicPartition;
+
+/**
+ * Singleton comparator of TopicPartitions.  Topics have precedence over partitions.
+ * Topics are compared through String.compare and partitions are compared
+ * numerically.
+ * 
+ * Use INSTANCE for all sorting.
+ */
+public class TopicPartitionComparator implements Comparator<TopicPartition> {
+    public static final TopicPartitionComparator INSTANCE = new TopicPartitionComparator();
+    
+    /**
+     * Private to make it a singleton
+     */
+    private TopicPartitionComparator() {
+        //Empty
+    }
+    
+    @Override
+    public int compare(TopicPartition o1, TopicPartition o2) {
+        if (!o1.topic().equals(o2.topic())) {
+            return o1.topic().compareTo(o2.topic());
+        } else {
+            return o1.partition() - o2.partition();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java
new file mode 100644
index 0000000..d51104d
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+package org.apache.storm.kafka.spout.internal;
+
+import java.util.concurrent.TimeUnit;
+
+public class Timer {
+    private final long delay;
+    private final long period;
+    private final TimeUnit timeUnit;
+    private final long periodNanos;
+    private long start;
+
+    /**
+     * Creates a class that mimics a single threaded timer that expires periodically. If a call to {@link
+     * #isExpiredResetOnTrue()} occurs later than {@code period} since the timer was initiated or reset, this method returns
+     * true. Each time the method returns true the counter is reset. The timer starts with the specified time delay.
+     *
+     * @param delay    the initial delay before the timer starts
+     * @param period   the period between calls {@link #isExpiredResetOnTrue()}
+     * @param timeUnit the time unit of delay and period
+     */
+    public Timer(long delay, long period, TimeUnit timeUnit) {
+        this.delay = delay;
+        this.period = period;
+        this.timeUnit = timeUnit;
+
+        periodNanos = timeUnit.toNanos(period);
+        start = System.nanoTime() + timeUnit.toNanos(delay);
+    }
+
+    public long period() {
+        return period;
+    }
+
+    public long delay() {
+        return delay;
+    }
+
+    public TimeUnit getTimeUnit() {
+        return timeUnit;
+    }
+
+    /**
+     * Checks if a call to this method occurs later than {@code period} since the timer was initiated or reset. If that is the
+     * case the method returns true, otherwise it returns false. Each time this method returns true, the counter is reset
+     * (re-initiated) and a new cycle will start.
+     *
+     * @return true if the time elapsed since the last call returning true is greater than {@code period}. Returns false
+     * otherwise.
+     */
+    public boolean isExpiredResetOnTrue() {
+        final boolean expired = System.nanoTime() - start > periodNanos;
+        if (expired) {
+            start = System.nanoTime();
+        }
+        return expired;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java
index 0218f07..19b4f01 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java
@@ -18,33 +18,38 @@
 
 package org.apache.storm.kafka.spout.trident;
 
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.LATEST;
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST;
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_LATEST;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecords;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.storm.kafka.spout.KafkaSpoutConfig;
-import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilder;
+import org.apache.storm.kafka.spout.RecordTranslator;
+import org.apache.storm.kafka.spout.internal.Timer;
+import org.apache.storm.task.TopologyContext;
 import org.apache.storm.trident.operation.TridentCollector;
 import org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout;
 import org.apache.storm.trident.topology.TransactionAttempt;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
-import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.LATEST;
-import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST;
-import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_LATEST;
-
 public class KafkaTridentSpoutEmitter<K,V> implements IOpaquePartitionedTridentSpout.Emitter<List<TopicPartition>, KafkaTridentSpoutTopicPartition, KafkaTridentSpoutBatchMetadata<K,V>>, Serializable {
+    private static final long serialVersionUID = -7343927794834130435L;
+
     private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutEmitter.class);
 
     // Kafka
@@ -53,18 +58,20 @@ public class KafkaTridentSpoutEmitter<K,V> implements IOpaquePartitionedTridentS
     // Bookkeeping
     private final KafkaTridentSpoutManager<K, V> kafkaManager;
     // Declare some KafkaTridentSpoutManager references for convenience
-    private final KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;
     private final long pollTimeoutMs;
     private final KafkaSpoutConfig.FirstPollOffsetStrategy firstPollOffsetStrategy;
+    private final RecordTranslator<K, V> translator;
+    private final Timer refreshSubscriptionTimer;
 
-    public KafkaTridentSpoutEmitter(KafkaTridentSpoutManager<K,V> kafkaManager) {
+    public KafkaTridentSpoutEmitter(KafkaTridentSpoutManager<K,V> kafkaManager, TopologyContext context) {
         this.kafkaManager = kafkaManager;
-        this.kafkaManager.subscribeKafkaConsumer();
+        this.kafkaManager.subscribeKafkaConsumer(context);
+        refreshSubscriptionTimer = new Timer(500, kafkaManager.getKafkaSpoutConfig().getPartitionRefreshPeriodMs(), TimeUnit.MILLISECONDS);
 
         //must subscribeKafkaConsumer before this line
         kafkaConsumer = kafkaManager.getKafkaConsumer();
+        translator = kafkaManager.getKafkaSpoutConfig().getTranslator();
 
-        tuplesBuilder = kafkaManager.getTuplesBuilder();
         final KafkaSpoutConfig<K, V> kafkaSpoutConfig = kafkaManager.getKafkaSpoutConfig();
         pollTimeoutMs = kafkaSpoutConfig.getPollTimeoutMs();
         firstPollOffsetStrategy = kafkaSpoutConfig.getFirstPollOffsetStrategy();
@@ -88,6 +95,9 @@ public class KafkaTridentSpoutEmitter<K,V> implements IOpaquePartitionedTridentS
             seek(topicPartition, lastBatch);
 
             // poll
+            if (refreshSubscriptionTimer.isExpiredResetOnTrue()) {
+                kafkaManager.getKafkaSpoutConfig().getSubscription().refreshAssignment();
+            }
             final ConsumerRecords<K, V> records = kafkaConsumer.poll(pollTimeoutMs);
             LOG.debug("Polled [{}] records from Kafka.", records.count());
 
@@ -106,7 +116,7 @@ public class KafkaTridentSpoutEmitter<K,V> implements IOpaquePartitionedTridentS
 
     private void emitTuples(TridentCollector collector, ConsumerRecords<K, V> records) {
         for (ConsumerRecord<K, V> record : records) {
-            final List<Object> tuple = tuplesBuilder.buildTuple(record);
+            final List<Object> tuple = translator.apply(record);
             collector.emit(tuple);
             LOG.debug("Emitted tuple [{}] for record: [{}]", tuple, record);
         }
@@ -131,18 +141,18 @@ public class KafkaTridentSpoutEmitter<K,V> implements IOpaquePartitionedTridentS
             final OffsetAndMetadata committedOffset = kafkaConsumer.committed(tp);
             if (committedOffset != null) {             // offset was committed for this TopicPartition
                 if (firstPollOffsetStrategy.equals(EARLIEST)) {
-                    kafkaConsumer.seekToBeginning(toArrayList(tp));
+                    kafkaConsumer.seekToBeginning(Collections.singleton(tp));
                 } else if (firstPollOffsetStrategy.equals(LATEST)) {
-                    kafkaConsumer.seekToEnd(toArrayList(tp));
+                    kafkaConsumer.seekToEnd(Collections.singleton(tp));
                 } else {
                     // By default polling starts at the last committed offset. +1 to point fetch to the first uncommitted offset.
                     kafkaConsumer.seek(tp, committedOffset.offset() + 1);
                 }
             } else {    // no commits have ever been done, so start at the beginning or end depending on the strategy
                 if (firstPollOffsetStrategy.equals(EARLIEST) || firstPollOffsetStrategy.equals(UNCOMMITTED_EARLIEST)) {
-                    kafkaConsumer.seekToBeginning(toArrayList(tp));
+                    kafkaConsumer.seekToBeginning(Collections.singleton(tp));
                 } else if (firstPollOffsetStrategy.equals(LATEST) || firstPollOffsetStrategy.equals(UNCOMMITTED_LATEST)) {
-                    kafkaConsumer.seekToEnd(toArrayList(tp));
+                    kafkaConsumer.seekToEnd(Collections.singleton(tp));
                 }
             }
         }
@@ -151,10 +161,6 @@ public class KafkaTridentSpoutEmitter<K,V> implements IOpaquePartitionedTridentS
         return fetchOffset;
     }
 
-    private Collection<TopicPartition> toArrayList(final TopicPartition tp) {
-        return new ArrayList<TopicPartition>(1){{add(tp);}};
-    }
-
     // returns paused topic partitions
     private Collection<TopicPartition> pauseTopicPartitions(TopicPartition excludedTp) {
         final Set<TopicPartition> pausedTopicPartitions  = new HashSet<>(kafkaConsumer.assignment());

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java
index d2c1700..4b60f33 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java
@@ -18,23 +18,22 @@
 
 package org.apache.storm.kafka.spout.trident;
 
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.storm.kafka.spout.KafkaSpoutConfig;
-import org.apache.storm.kafka.spout.KafkaSpoutStreams;
-import org.apache.storm.kafka.spout.KafkaSpoutStreamsNamedTopics;
-import org.apache.storm.kafka.spout.KafkaSpoutStreamsWildcardTopics;
-import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilder;
+import org.apache.storm.kafka.spout.RecordTranslator;
+import org.apache.storm.kafka.spout.internal.Timer;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.Fields;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import java.util.regex.Pattern;
-
 public class KafkaTridentSpoutManager<K, V> implements Serializable {
     private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutManager.class);
 
@@ -44,29 +43,30 @@ public class KafkaTridentSpoutManager<K, V> implements Serializable {
     // 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 not support multiple output Fields");
+                }
+            }
+        }
+        this.fields = fields;
         LOG.debug("Created {}", this);
     }
 
-    void subscribeKafkaConsumer() {
+    void subscribeKafkaConsumer(TopologyContext context) {
         kafkaConsumer = new KafkaConsumer<>(kafkaSpoutConfig.getKafkaProps(),
                 kafkaSpoutConfig.getKeyDeserializer(), kafkaSpoutConfig.getValueDeserializer());
 
-        if (kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics) {
-            final List<String> subTopics = kafkaSpoutConfig.getSubscribedTopics();
-            kafkaConsumer.subscribe(subTopics, new KafkaSpoutConsumerRebalanceListener());
-            LOG.info("Kafka consumer subscribed topics {}", subTopics);
-        } else if (kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics) {
-            final Pattern pattern = kafkaSpoutConfig.getTopicWildcardPattern();
-            kafkaConsumer.subscribe(pattern, new KafkaSpoutConsumerRebalanceListener());
-            LOG.info("Kafka consumer subscribed topics matching wildcard pattern [{}]", pattern);
-        }
+        kafkaSpoutConfig.getSubscription().subscribe(kafkaConsumer, new KafkaSpoutConsumerRebalanceListener(), context);
 
         // Initial poll to get the consumer registration process going.
         // KafkaSpoutConsumerRebalanceListener will be called following this poll, upon partition registration
@@ -93,16 +93,12 @@ public class KafkaTridentSpoutManager<K, V> implements Serializable {
         return kafkaConsumer;
     }
 
-    KafkaSpoutTuplesBuilder<K, V> getTuplesBuilder() {
-        return tuplesBuilder;
-    }
-
     Set<TopicPartition> getTopicPartitions() {
         return KafkaTridentSpoutTopicPartitionRegistry.INSTANCE.getTopicPartitions();
     }
-
-    KafkaSpoutStreams getKafkaSpoutStreams() {
-        return kafkaSpoutStreams;
+    
+    Fields getFields() {
+        return fields;
     }
 
     KafkaSpoutConfig<K, V> getKafkaSpoutConfig() {

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java
index b77ad11..5c5856c 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java
@@ -19,6 +19,7 @@
 package org.apache.storm.kafka.spout.trident;
 
 import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.kafka.spout.KafkaSpoutConfig;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout;
 import org.apache.storm.tuple.Fields;
@@ -29,12 +30,19 @@ import java.util.List;
 import java.util.Map;
 
 public class KafkaTridentSpoutOpaque<K,V> implements IOpaquePartitionedTridentSpout<List<TopicPartition>, KafkaTridentSpoutTopicPartition, KafkaTridentSpoutBatchMetadata<K,V>> {
+    private static final long serialVersionUID = -8003272486566259640L;
+
     private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutOpaque.class);
 
     private KafkaTridentSpoutManager<K, V> kafkaManager;
     private KafkaTridentSpoutEmitter<K, V> kafkaTridentSpoutEmitter;
     private KafkaTridentSpoutOpaqueCoordinator<K, V> coordinator;
 
+    
+    public KafkaTridentSpoutOpaque(KafkaSpoutConfig<K, V> conf) {
+        this(new KafkaTridentSpoutManager<>(conf));
+    }
+    
     public KafkaTridentSpoutOpaque(KafkaTridentSpoutManager<K, V> kafkaManager) {
         this.kafkaManager = kafkaManager;
         LOG.debug("Created {}", this);
@@ -44,7 +52,7 @@ public class KafkaTridentSpoutOpaque<K,V> implements IOpaquePartitionedTridentSp
     public Emitter<List<TopicPartition>, KafkaTridentSpoutTopicPartition, KafkaTridentSpoutBatchMetadata<K,V>> getEmitter(Map conf, TopologyContext context) {
         // Instance is created on first call rather than in constructor to avoid NotSerializableException caused by KafkaConsumer
         if (kafkaTridentSpoutEmitter == null) {
-            kafkaTridentSpoutEmitter = new KafkaTridentSpoutEmitter<>(kafkaManager);
+            kafkaTridentSpoutEmitter = new KafkaTridentSpoutEmitter<>(kafkaManager, context);
         }
         return kafkaTridentSpoutEmitter;
     }
@@ -64,7 +72,7 @@ public class KafkaTridentSpoutOpaque<K,V> implements IOpaquePartitionedTridentSp
 
     @Override
     public Fields getOutputFields() {
-        final Fields outputFields = kafkaManager.getKafkaSpoutStreams().getOutputFields();
+        final Fields outputFields = kafkaManager.getFields();
         LOG.debug("OutputFields = {}", outputFields);
         return outputFields;
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java
new file mode 100644
index 0000000..f24fed5
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaState.java
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.trident;
+
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.topology.FailedException;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.storm.kafka.trident.mapper.TridentTupleToKafkaMapper;
+import org.apache.storm.kafka.trident.selector.KafkaTopicSelector;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.state.State;
+import org.apache.storm.trident.tuple.TridentTuple;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+public class TridentKafkaState implements State {
+    private static final Logger LOG = LoggerFactory.getLogger(TridentKafkaState.class);
+
+    private KafkaProducer producer;
+    private OutputCollector collector;
+
+    private TridentTupleToKafkaMapper mapper;
+    private KafkaTopicSelector topicSelector;
+
+    public TridentKafkaState withTridentTupleToKafkaMapper(TridentTupleToKafkaMapper mapper) {
+        this.mapper = mapper;
+        return this;
+    }
+
+    public TridentKafkaState withKafkaTopicSelector(KafkaTopicSelector selector) {
+        this.topicSelector = selector;
+        return this;
+    }
+
+    @Override
+    public void beginCommit(Long txid) {
+        LOG.debug("beginCommit is Noop.");
+    }
+
+    @Override
+    public void commit(Long txid) {
+        LOG.debug("commit is Noop.");
+    }
+
+    public void prepare(Properties options) {
+        Objects.requireNonNull(mapper, "mapper can not be null");
+        Objects.requireNonNull(topicSelector, "topicSelector can not be null");
+        producer = new KafkaProducer(options);
+    }
+
+    public void updateState(List<TridentTuple> tuples, TridentCollector collector) {
+        String topic = null;
+        try {
+            List<Future<RecordMetadata>> futures = new ArrayList<>(tuples.size());
+            for (TridentTuple tuple : tuples) {
+                topic = topicSelector.getTopic(tuple);
+
+                if(topic != null) {
+                    Future<RecordMetadata> result = producer.send(new ProducerRecord(topic,
+                            mapper.getKeyFromTuple(tuple), mapper.getMessageFromTuple(tuple)));
+                    futures.add(result);
+                } else {
+                    LOG.warn("skipping key = " + mapper.getKeyFromTuple(tuple) + ", topic selector returned null.");
+                }
+            }
+
+            List<ExecutionException> exceptions = new ArrayList<>(futures.size());
+            for (Future<RecordMetadata> future : futures) {
+                try {
+                    future.get();
+                } catch (ExecutionException e) {
+                    exceptions.add(e);
+                }
+            }
+
+            if(exceptions.size() > 0){
+                String errorMsg = "Could not retrieve result for messages " + tuples + " from topic = " + topic 
+                        + " because of the following exceptions: \n";
+                for (ExecutionException exception : exceptions) {
+                    errorMsg = errorMsg + exception.getMessage() + "\n";
+                }
+                LOG.error(errorMsg);
+                throw new FailedException(errorMsg);
+            }
+        } catch (Exception ex) {
+            String errorMsg = "Could not send messages " + tuples + " to topic = " + topic;
+            LOG.warn(errorMsg, ex);
+            throw new FailedException(errorMsg, ex);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaStateFactory.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaStateFactory.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaStateFactory.java
new file mode 100644
index 0000000..f564510
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaStateFactory.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.trident;
+
+import org.apache.storm.task.IMetricsContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.storm.kafka.trident.mapper.TridentTupleToKafkaMapper;
+import org.apache.storm.kafka.trident.selector.KafkaTopicSelector;
+import org.apache.storm.trident.state.State;
+import org.apache.storm.trident.state.StateFactory;
+
+import java.util.Map;
+import java.util.Properties;
+
+public class TridentKafkaStateFactory implements StateFactory {
+
+    private static final Logger LOG = LoggerFactory.getLogger(TridentKafkaStateFactory.class);
+
+    private TridentTupleToKafkaMapper mapper;
+    private KafkaTopicSelector topicSelector;
+    private Properties producerProperties = new Properties();
+
+    public TridentKafkaStateFactory withTridentTupleToKafkaMapper(TridentTupleToKafkaMapper mapper) {
+        this.mapper = mapper;
+        return this;
+    }
+
+    public TridentKafkaStateFactory withKafkaTopicSelector(KafkaTopicSelector selector) {
+        this.topicSelector = selector;
+        return this;
+    }
+
+    public TridentKafkaStateFactory withProducerProperties(Properties props) {
+        this.producerProperties = props;
+        return this;
+    }
+
+    @Override
+    public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
+        LOG.info("makeState(partitonIndex={}, numpartitions={}", partitionIndex, numPartitions);
+        TridentKafkaState state = new TridentKafkaState()
+                .withKafkaTopicSelector(this.topicSelector)
+                .withTridentTupleToKafkaMapper(this.mapper);
+        state.prepare(producerProperties);
+        return state;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/FieldNameBasedTupleToKafkaMapper.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/FieldNameBasedTupleToKafkaMapper.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/FieldNameBasedTupleToKafkaMapper.java
new file mode 100644
index 0000000..2d04971
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/FieldNameBasedTupleToKafkaMapper.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.trident.mapper;
+
+import org.apache.storm.trident.tuple.TridentTuple;
+
+public class FieldNameBasedTupleToKafkaMapper<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) {
+        return (V) tuple.getValueByField(msgFieldName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/TridentTupleToKafkaMapper.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/TridentTupleToKafkaMapper.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/TridentTupleToKafkaMapper.java
new file mode 100644
index 0000000..28c6c89
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/mapper/TridentTupleToKafkaMapper.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.trident.mapper;
+
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.trident.tuple.TridentTuple;
+
+import java.io.Serializable;
+
+public interface TridentTupleToKafkaMapper<K,V>  extends Serializable {
+    K getKeyFromTuple(TridentTuple tuple);
+    V getMessageFromTuple(TridentTuple tuple);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/DefaultTopicSelector.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/DefaultTopicSelector.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/DefaultTopicSelector.java
new file mode 100644
index 0000000..607c996
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/DefaultTopicSelector.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.trident.selector;
+
+import org.apache.storm.trident.tuple.TridentTuple;
+
+public class DefaultTopicSelector implements KafkaTopicSelector {
+    private static final long serialVersionUID = -1172454882072591493L;
+    private final String topicName;
+
+    public DefaultTopicSelector(final String topicName) {
+        this.topicName = topicName;
+    }
+
+    @Override
+    public String getTopic(TridentTuple tuple) {
+        return topicName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/KafkaTopicSelector.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/KafkaTopicSelector.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/KafkaTopicSelector.java
new file mode 100644
index 0000000..012a6c7
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/selector/KafkaTopicSelector.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.trident.selector;
+
+import org.apache.storm.trident.tuple.TridentTuple;
+
+import java.io.Serializable;
+
+public interface KafkaTopicSelector extends Serializable {
+    String getTopic(TridentTuple tuple);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/bolt/KafkaBoltTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/bolt/KafkaBoltTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/bolt/KafkaBoltTest.java
new file mode 100644
index 0000000..4d5f75d
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/bolt/KafkaBoltTest.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.bolt;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.storm.Testing;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.testing.MkTupleParam;
+import org.apache.storm.tuple.Tuple;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class KafkaBoltTest {
+    private static final Logger LOG = LoggerFactory.getLogger(KafkaBoltTest.class);
+    
+    @SuppressWarnings({ "unchecked", "serial" })
+    @Test
+    public void testSimple() {
+        final KafkaProducer<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>() {
+            @Override
+            protected KafkaProducer<String, String> mkProducer(Properties props) {
+                return producer;
+            }
+        };
+        bolt.withTopicSelector("MY_TOPIC");
+        
+        OutputCollector collector = mock(OutputCollector.class);
+        TopologyContext context = mock(TopologyContext.class);
+        Map<String, Object> conf = new HashMap<>();
+        bolt.prepare(conf, context, collector);
+        MkTupleParam param = new MkTupleParam();
+        param.setFields("key", "message");
+        Tuple testTuple = Testing.testTuple(Arrays.asList("KEY", "VALUE"), param);
+        bolt.execute(testTuple);
+        verify(producer).send(argThat(new ArgumentMatcher<ProducerRecord<String, String>>() {
+            @Override
+            public boolean matches(Object argument) {
+                LOG.info("GOT {} ->", argument);
+                ProducerRecord<String, String> arg = (ProducerRecord<String, String>) argument;
+                LOG.info("  {} {} {}", arg.topic(), arg.key(), arg.value());
+                return "MY_TOPIC".equals(arg.topic()) &&
+                        "KEY".equals(arg.key()) &&
+                        "VALUE".equals(arg.value());
+            }
+        }), any(Callback.class));
+        verify(collector).ack(testTuple);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/ByTopicRecordTranslatorTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/ByTopicRecordTranslatorTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/ByTopicRecordTranslatorTest.java
new file mode 100644
index 0000000..fd53b15
--- /dev/null
+++ b/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 testFieldCollision() {
+        ByTopicRecordTranslator<String, String> trans = 
+                new ByTopicRecordTranslator<>((r) -> new Values(r.key()), new Fields("key"));
+        trans.forTopic("foo", (r) -> new Values(r.value()), new Fields("value"));
+    }
+    
+    @Test(expected = IllegalStateException.class)
+    public void testTopicCollision() {
+        ByTopicRecordTranslator<String, String> trans = 
+                new ByTopicRecordTranslator<>((r) -> new Values(r.key()), new Fields("key"));
+        trans.forTopic("foo", (r) -> new Values(r.value()), new Fields("value"), "foo1");
+        trans.forTopic("foo", (r) -> new Values(r.key(), r.value()), new Fields("key", "value"), "foo2");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/DefaultRecordTranslatorTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/DefaultRecordTranslatorTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/DefaultRecordTranslatorTest.java
new file mode 100644
index 0000000..f4275e4
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/DefaultRecordTranslatorTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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 org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.storm.tuple.Fields;
+import org.junit.Test;
+
+public class DefaultRecordTranslatorTest {
+    @Test
+    public void testBasic() {
+        DefaultRecordTranslator<String, String> trans = new DefaultRecordTranslator<>();
+        assertEquals(Arrays.asList("default"), trans.streams());
+        assertEquals(new Fields("topic", "partition", "offset", "key", "value"), trans.getFieldsFor("default"));
+        ConsumerRecord<String, String> cr = new ConsumerRecord<>("TOPIC", 100, 100, "THE KEY", "THE VALUE");
+        assertEquals(Arrays.asList("TOPIC", 100, 100l, "THE KEY", "THE VALUE"), trans.apply(cr));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutConfigTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutConfigTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutConfigTest.java
new file mode 100644
index 0000000..08220dd
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutConfigTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.HashMap;
+
+import org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy;
+import org.junit.Test;
+
+public class KafkaSpoutConfigTest {
+
+    @Test
+    public void testBasic() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic").build();
+        assertEquals(FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST, conf.getFirstPollOffsetStrategy());
+        assertNull(conf.getConsumerGroupId());
+        assertTrue(conf.getTranslator() instanceof DefaultRecordTranslator);
+        HashMap<String, Object> expected = new HashMap<>();
+        expected.put("bootstrap.servers", "localhost:1234");
+        expected.put("enable.auto.commit", "false");
+        assertEquals(expected, conf.getKafkaProps());
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java
index 3e077ab..68fd4a6 100644
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java
@@ -15,11 +15,25 @@
  */
 package org.apache.storm.kafka.spout;
 
+import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfig;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.Matchers.hasKey;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Matchers.anyCollection;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+
 import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecords;
@@ -27,27 +41,14 @@ import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
-import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutStreams;
 import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
+import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
-import static org.mockito.Matchers.anyCollection;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Mockito.when;
-import org.junit.Before;
 import org.mockito.Captor;
-import static org.mockito.Mockito.reset;
 import org.mockito.MockitoAnnotations;
-import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfig;
-import static org.hamcrest.CoreMatchers.not;
-import static org.hamcrest.Matchers.hasKey;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
 
 public class KafkaSpoutRebalanceTest {
 
@@ -56,7 +57,7 @@ public class KafkaSpoutRebalanceTest {
 
     private TopologyContext contextMock;
     private SpoutOutputCollector collectorMock;
-    private Map conf;
+    private Map<String, Object> conf;
     private KafkaConsumer<String, String> consumerMock;
     private KafkaConsumerFactory<String, String> consumerFactoryMock;
 
@@ -118,7 +119,7 @@ public class KafkaSpoutRebalanceTest {
     @Test
     public void spoutMustIgnoreAcksForTuplesItIsNotAssignedAfterRebalance() throws Exception {
         //Acking tuples for partitions that are no longer assigned is useless since the spout will not be allowed to commit them
-        KafkaSpout<String, String> spout = new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams(), -1, 10), consumerFactoryMock);
+        KafkaSpout<String, String> spout = new KafkaSpout<>(getKafkaSpoutConfig(-1, 10), consumerFactoryMock);
         String topic = SingleTopicKafkaSpoutConfiguration.TOPIC;
         TopicPartition partitionThatWillBeRevoked = new TopicPartition(topic, 1);
         TopicPartition assignedPartition = new TopicPartition(topic, 2);
@@ -147,7 +148,7 @@ public class KafkaSpoutRebalanceTest {
     public void spoutMustIgnoreFailsForTuplesItIsNotAssignedAfterRebalance() throws Exception {
         //Failing tuples for partitions that are no longer assigned is useless since the spout will not be allowed to commit them if they later pass
         KafkaSpoutRetryService retryServiceMock = mock(KafkaSpoutRetryService.class);
-        KafkaSpout<String, String> spout = new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams(), -1, 10, retryServiceMock), consumerFactoryMock);
+        KafkaSpout<String, String> spout = new KafkaSpout<>(getKafkaSpoutConfig(-1, 10, retryServiceMock), consumerFactoryMock);
         String topic = SingleTopicKafkaSpoutConfiguration.TOPIC;
         TopicPartition partitionThatWillBeRevoked = new TopicPartition(topic, 1);
         TopicPartition assignedPartition = new TopicPartition(topic, 2);

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopicsTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopicsTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopicsTest.java
deleted file mode 100644
index 723f5fd..0000000
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopicsTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Copyright 2016 The Apache Software Foundation.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.kafka.spout;
-
-import org.apache.storm.tuple.Fields;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class KafkaSpoutStreamsNamedTopicsTest {
-
-	@Test
-	public void testGetOutputFields() {
-		Fields outputFields = new Fields("b","a");
-		String[] topics = new String[]{"testTopic"};
-		String streamId = "test";
-		KafkaSpoutStreamsNamedTopics build = new KafkaSpoutStreamsNamedTopics.Builder(outputFields, streamId, topics)
-        .addStream(outputFields, streamId, topics)
-        .build();
-		Fields actualFields = build.getOutputFields();
-		Assert.assertEquals(outputFields.get(0), actualFields.get(0));
-		Assert.assertEquals(outputFields.get(1), actualFields.get(1));
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java
index 8fa7b80..c5e4e31 100644
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java
@@ -71,7 +71,7 @@ public class SingleTopicKafkaSpoutTest {
         SpoutOutputCollector collector = mock(SpoutOutputCollector.class);
         Map conf = mock(Map.class);
 
-        KafkaSpout<String, String> spout = new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams(), kafkaPort));
+        KafkaSpout<String, String> spout = new KafkaSpout<>(getKafkaSpoutConfig(kafkaPort));
         spout.open(conf, topology, collector);
         spout.activate();
         return new SpoutContext(spout, collector);

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/SingleTopicKafkaSpoutConfiguration.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/SingleTopicKafkaSpoutConfiguration.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/SingleTopicKafkaSpoutConfiguration.java
index 6921f7c..d04b16a 100644
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/SingleTopicKafkaSpoutConfiguration.java
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/SingleTopicKafkaSpoutConfiguration.java
@@ -17,17 +17,17 @@
  */
 package org.apache.storm.kafka.spout.builders;
 
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
+
 import org.apache.storm.Config;
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.kafka.spout.*;
+import org.apache.storm.kafka.spout.KafkaSpout;
+import org.apache.storm.kafka.spout.KafkaSpoutConfig;
+import org.apache.storm.kafka.spout.KafkaSpoutRetryService;
 import org.apache.storm.kafka.spout.test.KafkaSpoutTestBolt;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.tuple.Fields;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
+import org.apache.storm.tuple.Values;
 
 public class SingleTopicKafkaSpoutConfiguration {
     public static final String STREAM = "test_stream";
@@ -41,53 +41,34 @@ public class SingleTopicKafkaSpoutConfiguration {
 
     public static StormTopology getTopologyKafkaSpout(int port) {
         final TopologyBuilder tp = new TopologyBuilder();
-        tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams(), port)), 1);
+        tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig(port)), 1);
         tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAM);
         return tp.createTopology();
     }
 
-    static public KafkaSpoutConfig<String, String> getKafkaSpoutConfig(KafkaSpoutStreams kafkaSpoutStreams, int port) {
-        return getKafkaSpoutConfig(kafkaSpoutStreams, port, 10_000);
+    static public KafkaSpoutConfig<String, String> getKafkaSpoutConfig(int port) {
+        return getKafkaSpoutConfig(port, 10_000);
     }
 
-    static public KafkaSpoutConfig<String, String> getKafkaSpoutConfig(KafkaSpoutStreams kafkaSpoutStreams, int port, long offsetCommitPeriodMs) {
-        return getKafkaSpoutConfig(kafkaSpoutStreams, port, offsetCommitPeriodMs, getRetryService());
+    static public KafkaSpoutConfig<String, String> getKafkaSpoutConfig(int port, long offsetCommitPeriodMs) {
+        return getKafkaSpoutConfig(port, offsetCommitPeriodMs, getRetryService());
     }
-    
-    static public KafkaSpoutConfig<String,String> getKafkaSpoutConfig(KafkaSpoutStreams kafkaSpoutStreams, int port, long offsetCommitPeriodMs, KafkaSpoutRetryService retryService) {
-        return new KafkaSpoutConfig.Builder<>(getKafkaConsumerProps(port), kafkaSpoutStreams, getTuplesBuilder(), retryService)
+
+    static public KafkaSpoutConfig<String,String> getKafkaSpoutConfig(int port, long offsetCommitPeriodMs, KafkaSpoutRetryService retryService) {
+        return KafkaSpoutConfig.builder("127.0.0.1:" + port, TOPIC)
+                .setRecordTranslator((r) -> new Values(r.topic(), r.key(), r.value()),
+                        new Fields("topic", "key", "value"), STREAM)
+                .setGroupId("kafkaSpoutTestGroup")
+                .setMaxPollRecords(5)
+                .setRetry(retryService)
                 .setOffsetCommitPeriodMs(offsetCommitPeriodMs)
                 .setFirstPollOffsetStrategy(EARLIEST)
                 .setMaxUncommittedOffsets(250)
                 .setPollTimeoutMs(1000)
                 .build();
     }
-
+        
     protected static KafkaSpoutRetryService getRetryService() {
-        return new KafkaSpoutRetryExponentialBackoff(KafkaSpoutRetryExponentialBackoff.TimeInterval.microSeconds(0),
-                KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(0), Integer.MAX_VALUE, KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(0));
-
-    }
-
-    protected static Map<String,Object> getKafkaConsumerProps(int port) {
-        Map<String, Object> props = new HashMap<>();
-        props.put(KafkaSpoutConfig.Consumer.BOOTSTRAP_SERVERS, "127.0.0.1:" + port);
-        props.put(KafkaSpoutConfig.Consumer.GROUP_ID, "kafkaSpoutTestGroup");
-        props.put(KafkaSpoutConfig.Consumer.KEY_DESERIALIZER, "org.apache.kafka.common.serialization.StringDeserializer");
-        props.put(KafkaSpoutConfig.Consumer.VALUE_DESERIALIZER, "org.apache.kafka.common.serialization.StringDeserializer");
-        props.put("max.poll.records", "5");
-        return props;
-    }
-
-    protected static KafkaSpoutTuplesBuilder<String, String> getTuplesBuilder() {
-        return new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
-                new TopicKeyValueTupleBuilder<String, String>(TOPIC))
-                .build();
-    }
-
-    public static KafkaSpoutStreams getKafkaSpoutStreams() {
-        final Fields outputFields = new Fields("topic", "key", "value");
-        return new KafkaSpoutStreamsNamedTopics.Builder(outputFields, STREAM, new String[]{TOPIC})  // contents of topics test sent to test_stream
-                .build();
+        return KafkaSpoutConfig.UNIT_TEST_RETRY_SERVICE;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/TopicKeyValueTupleBuilder.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/TopicKeyValueTupleBuilder.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/TopicKeyValueTupleBuilder.java
deleted file mode 100644
index 4f20b58..0000000
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/builders/TopicKeyValueTupleBuilder.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-package org.apache.storm.kafka.spout.builders;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.storm.kafka.spout.KafkaSpoutTupleBuilder;
-import org.apache.storm.tuple.Values;
-
-import java.util.List;
-
-public class TopicKeyValueTupleBuilder<K, V> extends KafkaSpoutTupleBuilder<K,V> {
-    /**
-     * @param topics list of topics that use this implementation to build tuples
-     */
-    public TopicKeyValueTupleBuilder(String... topics) {
-        super(topics);
-    }
-
-    @Override
-    public List<Object> buildTuple(ConsumerRecord<K, V> consumerRecord) {
-        return new Values(consumerRecord.topic(),
-                consumerRecord.key(),
-                consumerRecord.value());
-    }
-}


[6/7] storm git commit: Merge branch 'STORM-2225' of https://github.com/revans2/incubator-storm into STORM-2225

Posted by bo...@apache.org.
Merge branch 'STORM-2225' of https://github.com/revans2/incubator-storm into STORM-2225

STORM-2225: change spout config to be simpler.


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/0d037042
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/0d037042
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/0d037042

Branch: refs/heads/master
Commit: 0d037042b95486c631f8e407f19f4a2c9c9ab095
Parents: 22cecb0 c9f9348
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Feb 1 09:15:04 2017 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Feb 1 09:15:04 2017 -0600

----------------------------------------------------------------------
 docs/storm-kafka-client.md                      | 293 ++++++++---
 .../TridentKafkaClientWordCountNamedTopics.java |  69 +--
 ...identKafkaClientWordCountWildcardTopics.java |  31 +-
 external/storm-kafka-client/README.md           | 192 +------
 .../org/apache/storm/kafka/bolt/KafkaBolt.java  | 223 ++++++++
 .../FieldNameBasedTupleToKafkaMapper.java       |  48 ++
 .../kafka/bolt/mapper/TupleToKafkaMapper.java   |  32 ++
 .../bolt/selector/DefaultTopicSelector.java     |  34 ++
 .../bolt/selector/FieldIndexTopicSelector.java  |  52 ++
 .../bolt/selector/FieldNameTopicSelector.java   |  49 ++
 .../kafka/bolt/selector/KafkaTopicSelector.java |  26 +
 .../kafka/spout/ByTopicRecordTranslator.java    | 146 ++++++
 .../kafka/spout/DefaultRecordTranslator.java    |  42 ++
 .../java/org/apache/storm/kafka/spout/Func.java |  26 +
 .../apache/storm/kafka/spout/KafkaSpout.java    | 117 ++---
 .../storm/kafka/spout/KafkaSpoutConfig.java     | 506 +++++++++++++------
 .../storm/kafka/spout/KafkaSpoutMessageId.java  |   2 +-
 .../storm/kafka/spout/KafkaSpoutStream.java     | 121 -----
 .../storm/kafka/spout/KafkaSpoutStreams.java    |  38 --
 .../spout/KafkaSpoutStreamsNamedTopics.java     | 165 ------
 .../spout/KafkaSpoutStreamsWildcardTopics.java  |  67 ---
 .../kafka/spout/KafkaSpoutTupleBuilder.java     |  58 ---
 .../kafka/spout/KafkaSpoutTuplesBuilder.java    |  32 --
 .../KafkaSpoutTuplesBuilderNamedTopics.java     |  78 ---
 .../KafkaSpoutTuplesBuilderWildcardTopics.java  |  36 --
 .../apache/storm/kafka/spout/KafkaTuple.java    |  47 ++
 .../spout/ManualPartitionNamedSubscription.java |  78 +++
 .../ManualPartitionPatternSubscription.java     |  76 +++
 .../storm/kafka/spout/ManualPartitioner.java    |  40 ++
 .../storm/kafka/spout/NamedSubscription.java    |  61 +++
 .../storm/kafka/spout/PatternSubscription.java  |  54 ++
 .../storm/kafka/spout/RecordTranslator.java     |  55 ++
 .../spout/RoundRobinManualPartitioner.java      |  50 ++
 .../kafka/spout/SimpleRecordTranslator.java     |  58 +++
 .../apache/storm/kafka/spout/Subscription.java  |  53 ++
 .../kafka/spout/TopicPartitionComparator.java   |  21 +-
 .../storm/kafka/spout/internal/Timer.java       |   3 +-
 .../fetcher/AutomaticKafkaRecordsFetcher.java   |  67 ---
 .../internal/fetcher/KafkaRecordsFetcher.java   |  25 -
 .../internal/fetcher/KafkaRecordsFetchers.java  |  54 --
 .../fetcher/ManualKafkaRecordsFetcher.java      | 108 ----
 .../partition/KafkaPartitionReader.java         |  28 -
 .../partition/KafkaPartitionReaders.java        |  45 --
 .../partition/NamedTopicPartitionReader.java    |  47 --
 .../partition/WildcardTopicPartitionReader.java |  51 --
 .../spout/trident/KafkaTridentSpoutEmitter.java |  60 ++-
 .../spout/trident/KafkaTridentSpoutManager.java |  58 +--
 .../spout/trident/KafkaTridentSpoutOpaque.java  |  12 +-
 .../storm/kafka/trident/TridentKafkaState.java  | 115 +++++
 .../kafka/trident/TridentKafkaStateFactory.java |  63 +++
 .../FieldNameBasedTupleToKafkaMapper.java       |  41 ++
 .../mapper/TridentTupleToKafkaMapper.java       |  28 +
 .../trident/selector/DefaultTopicSelector.java  |  34 ++
 .../trident/selector/KafkaTopicSelector.java    |  26 +
 .../apache/storm/kafka/bolt/KafkaBoltTest.java  |  91 ++++
 .../spout/ByTopicRecordTranslatorTest.java      |  72 +++
 .../spout/DefaultRecordTranslatorTest.java      |  37 ++
 .../storm/kafka/spout/KafkaSpoutConfigTest.java |  40 ++
 .../kafka/spout/KafkaSpoutRebalanceTest.java    |  35 +-
 .../spout/KafkaSpoutStreamsNamedTopicsTest.java |  38 --
 .../kafka/spout/SingleTopicKafkaSpoutTest.java  |   2 +-
 .../SingleTopicKafkaSpoutConfiguration.java     |  61 +--
 .../builders/TopicKeyValueTupleBuilder.java     |  40 --
 .../test/KafkaSpoutTopologyMainNamedTopics.java |  70 +--
 .../KafkaSpoutTopologyMainWildcardTopics.java   |  40 +-
 .../spout/test/TopicTest2TupleBuilder.java      |  40 --
 .../test/TopicsTest0Test1TupleBuilder.java      |  42 --
 .../kafka/DynamicPartitionConnections.java      |   2 +-
 .../jvm/org/apache/storm/kafka/KafkaSpout.java  |   2 +-
 .../jvm/org/apache/storm/kafka/KafkaUtils.java  |   4 +-
 .../apache/storm/kafka/PartitionManager.java    |   2 +-
 .../apache/storm/kafka/StaticCoordinator.java   |   4 +-
 .../org/apache/storm/kafka/bolt/KafkaBolt.java  |   2 +
 .../storm/kafka/DynamicBrokersReaderTest.java   |   6 +-
 .../apache/storm/kafka/TridentKafkaTest.java    |   2 +-
 .../apache/storm/kafka/ZkCoordinatorTest.java   |   2 +-
 .../apache/storm/kafka/bolt/KafkaBoltTest.java  |   4 +-
 .../src/jvm/org/apache/storm/tuple/Fields.java  |  18 +-
 78 files changed, 2661 insertions(+), 1936 deletions(-)
----------------------------------------------------------------------



[7/7] storm git commit: Added STORM-2225 to Changelog

Posted by bo...@apache.org.
Added STORM-2225 to Changelog


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/251cb887
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/251cb887
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/251cb887

Branch: refs/heads/master
Commit: 251cb8876b0dbab1211a69e35c460aeeb07a20e2
Parents: 0d03704
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Feb 1 09:15:34 2017 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Feb 1 09:15:34 2017 -0600

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/251cb887/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 1fde2c6..e2f7cdd 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -195,6 +195,7 @@
  * STORM-1769: Added a test to check local nimbus with notifier plugin
 
 ## 1.1.0
+ * STORM-2225: change spout config to be simpler.
  * STORM-2330: Fix storm sql code generation for UDAF with non standard sql types
  * STORM-2298: Don't kill Nimbus when ClusterMetricsConsumer is failed to initialize
  * STORM-2301: [storm-cassandra] upgrade cassandra driver to 3.1.2


[4/7] storm git commit: 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 partit

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java
index d405c4d..67ff62a 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java
@@ -18,21 +18,11 @@
 
 package org.apache.storm.kafka.spout;
 
-import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy;
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseRichSpout;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.LATEST;
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST;
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_LATEST;
 
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -46,18 +36,28 @@ import java.util.NavigableSet;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
-import java.util.regex.Pattern;
 
-import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
-import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.LATEST;
-import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST;
-import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_LATEST;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy;
 import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
 import org.apache.storm.kafka.spout.internal.KafkaConsumerFactoryDefault;
-
-import org.apache.kafka.common.errors.InterruptException;
+import org.apache.storm.kafka.spout.internal.Timer;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class KafkaSpout<K, V> extends BaseRichSpout {
+    private static final long serialVersionUID = 4151921085047987154L;
     private static final Logger LOG = LoggerFactory.getLogger(KafkaSpout.class);
     private static final Comparator<KafkaSpoutMessageId> OFFSET_COMPARATOR = new OffsetComparator();
 
@@ -79,13 +79,12 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
     private transient boolean initialized;                              // Flag indicating that the spout is still undergoing initialization process.
     // 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;
+    private transient Timer refreshSubscriptionTimer;                   // Used to say when a subscription should be refreshed
 
 
     public KafkaSpout(KafkaSpoutConfig<K, V> kafkaSpoutConfig) {
@@ -95,13 +94,13 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
     //This constructor is here for testing
     KafkaSpout(KafkaSpoutConfig<K, V> kafkaSpoutConfig, KafkaConsumerFactory<K, V> kafkaConsumerFactory) {
         this.kafkaSpoutConfig = kafkaSpoutConfig;                 // Pass in configuration
-        this.kafkaSpoutStreams = kafkaSpoutConfig.getKafkaSpoutStreams();
         this.kafkaConsumerFactory = kafkaConsumerFactory;
     }
 
     @Override
     public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
         initialized = false;
+        this.context = context;
 
         // Spout internals
         this.collector = collector;
@@ -115,12 +114,10 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
         // Retries management
         retryService = kafkaSpoutConfig.getRetryService();
 
-        // Tuples builder delegate
-        tuplesBuilder = kafkaSpoutConfig.getTuplesBuilder();
-
         if (!consumerAutoCommitMode) {     // If it is auto commit, no need to commit offsets manually
             commitTimer = new Timer(500, kafkaSpoutConfig.getOffsetsCommitPeriodMs(), TimeUnit.MILLISECONDS);
         }
+        refreshSubscriptionTimer = new Timer(500, kafkaSpoutConfig.getPartitionRefreshPeriodMs(), TimeUnit.MILLISECONDS);
 
         acked = new HashMap<>();
         emitted = new HashSet<>();
@@ -158,7 +155,7 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
             retryService.retainAll(partitions);
             
             //Emitted messages for partitions that are no longer assigned to this spout can't be acked, and they shouldn't be retried. Remove them from emitted.
-            Set<TopicPartition> partitionsSet = new HashSet(partitions);
+            Set<TopicPartition> partitionsSet = new HashSet<>(partitions);
             emitted.removeIf((msgId) -> !partitionsSet.contains(msgId.getTopicPartition()));
 
             for (TopicPartition tp : partitions) {
@@ -177,10 +174,10 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
             long fetchOffset;
             if (committedOffset != null) {             // offset was committed for this TopicPartition
                 if (firstPollOffsetStrategy.equals(EARLIEST)) {
-                    kafkaConsumer.seekToBeginning(toArrayList(tp));
+                    kafkaConsumer.seekToBeginning(Collections.singleton(tp));
                     fetchOffset = kafkaConsumer.position(tp);
                 } else if (firstPollOffsetStrategy.equals(LATEST)) {
-                    kafkaConsumer.seekToEnd(toArrayList(tp));
+                    kafkaConsumer.seekToEnd(Collections.singleton(tp));
                     fetchOffset = kafkaConsumer.position(tp);
                 } else {
                     // By default polling starts at the last committed offset. +1 to point fetch to the first uncommitted offset.
@@ -189,9 +186,9 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
                 }
             } else {    // no commits have ever been done, so start at the beginning or end depending on the strategy
                 if (firstPollOffsetStrategy.equals(EARLIEST) || firstPollOffsetStrategy.equals(UNCOMMITTED_EARLIEST)) {
-                    kafkaConsumer.seekToBeginning(toArrayList(tp));
+                    kafkaConsumer.seekToBeginning(Collections.singleton(tp));
                 } else if (firstPollOffsetStrategy.equals(LATEST) || firstPollOffsetStrategy.equals(UNCOMMITTED_LATEST)) {
-                    kafkaConsumer.seekToEnd(toArrayList(tp));
+                    kafkaConsumer.seekToEnd(Collections.singleton(tp));
                 }
                 fetchOffset = kafkaConsumer.position(tp);
             }
@@ -199,10 +196,6 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
         }
     }
 
-    private Collection<TopicPartition> toArrayList(final TopicPartition tp) {
-        return new ArrayList<TopicPartition>(1){{add(tp);}};
-    }
-
     private void setAcked(TopicPartition tp, long fetchOffset) {
         // If this partition was previously assigned to this spout, leave the acked offsets as they were to resume where it left off
         if (!consumerAutoCommitMode && !acked.containsKey(tp)) {
@@ -221,7 +214,11 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
                 }
 
                 if (poll()) {
-                    setWaitingToEmit(pollKafkaBroker());
+                    try {
+                        setWaitingToEmit(pollKafkaBroker());
+                    } catch (RetriableException e) {
+                        LOG.error("Failed to poll from kafka.", e);
+                    }
                 }
 
                 if (waitingToEmit()) {
@@ -276,7 +273,9 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
     // ======== poll =========
     private ConsumerRecords<K, V> pollKafkaBroker() {
         doSeekRetriableTopicPartitions();
-
+        if (refreshSubscriptionTimer.isExpiredResetOnTrue()) {
+            kafkaSpoutConfig.getSubscription().refreshAssignment();
+        }
         final ConsumerRecords<K, V> consumerRecords = kafkaConsumer.poll(kafkaSpoutConfig.getPollTimeoutMs());
         final int numPolledRecords = consumerRecords.count();
         LOG.debug("Polled [{}] records from Kafka. [{}] uncommitted offsets across all topic partitions", numPolledRecords, numUncommittedOffsets);
@@ -303,7 +302,6 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
         }
     }
 
-
     //Emits one tuple per record
     //@return true if tuple was emitted
     private boolean emitTupleIfNotEmitted(ConsumerRecord<K, V> record) {
@@ -317,15 +315,19 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
         } 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) {
+                    collector.emit(((KafkaTuple)tuple).getStream(), tuple, msgId);
+                } else {
+                    collector.emit(tuple, msgId);
+                }
                 emitted.add(msgId);
                 numUncommittedOffsets++;
                 if (isScheduled) { // Was scheduled for retry, now being re-emitted. Remove from schedule.
                     retryService.remove(msgId);
                 }
                 LOG.trace("Emitted tuple [{}] for record [{}]", tuple, record);
-               return true;
+                return true;
             }
         }
         return false;
@@ -405,18 +407,7 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
     private void subscribeKafkaConsumer() {
         kafkaConsumer = kafkaConsumerFactory.createConsumer(kafkaSpoutConfig);
 
-        if (kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics) {
-            final List<String> topics = ((KafkaSpoutStreamsNamedTopics) kafkaSpoutStreams).getTopics();
-            kafkaConsumer.subscribe(topics, new KafkaSpoutConsumerRebalanceListener());
-            LOG.info("Kafka consumer subscribed topics {}", topics);
-        } else if (kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics) {
-            final Pattern pattern = ((KafkaSpoutStreamsWildcardTopics) kafkaSpoutStreams).getTopicWildcardPattern();
-            kafkaConsumer.subscribe(pattern, new KafkaSpoutConsumerRebalanceListener());
-            LOG.info("Kafka consumer subscribed topics matching wildcard pattern [{}]", pattern);
-        }
-        // Initial poll to get the consumer registration process going.
-        // KafkaSpoutConsumerRebalanceListener will be called following this poll, upon partition registration
-        kafkaConsumer.poll(0);
+        kafkaSpoutConfig.getSubscription().subscribe(kafkaConsumer, new KafkaSpoutConsumerRebalanceListener(), context);
     }
 
     @Override
@@ -450,7 +441,10 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
 
     @Override
     public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        kafkaSpoutStreams.declareOutputFields(declarer);
+        RecordTranslator<K, V> translator = kafkaSpoutConfig.getTranslator();
+        for (String stream: translator.streams()) {
+            declarer.declareStream(stream, translator.getFieldsFor(stream));
+        }
     }
 
     @Override
@@ -469,11 +463,7 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
         }
         String configKeyPrefix = "config.";
 
-        if (kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics) {
-            configuration.put(configKeyPrefix + "topics", getNamedTopics());
-        } else if (kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics) {
-            configuration.put(configKeyPrefix + "topics", getWildCardTopics());
-        }
+        configuration.put(configKeyPrefix + "topics", getTopicsString());
 
         configuration.put(configKeyPrefix + "groupid", kafkaSpoutConfig.getConsumerGroupId());
         configuration.put(configKeyPrefix + "bootstrap.servers", kafkaSpoutConfig.getKafkaProps().get("bootstrap.servers"));
@@ -481,16 +471,8 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
         return configuration;
     }
 
-    private String getNamedTopics() {
-        StringBuilder topics = new StringBuilder();
-        for (String topic: kafkaSpoutConfig.getSubscribedTopics()) {
-            topics.append(topic).append(",");
-        }
-        return topics.toString();
-    }
-
-    private String getWildCardTopics() {
-        return kafkaSpoutConfig.getTopicWildcardPattern().toString();
+    private String getTopicsString() {
+        return kafkaSpoutConfig.getSubscription().getTopicsString();
     }
 
     // ======= Offsets Commit Management ==========
@@ -594,7 +576,7 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
             return ackedMsgs.isEmpty();
         }
 
-        public boolean contains(ConsumerRecord record) {
+        public boolean contains(ConsumerRecord<K, V> record) {
             return contains(new KafkaSpoutMessageId(record));
         }
 
@@ -612,60 +594,4 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
                     '}';
         }
     }
-
-    // =========== Timer ===========
-
-    private class Timer {
-        private final long delay;
-        private final long period;
-        private final TimeUnit timeUnit;
-        private final long periodNanos;
-        private long start;
-
-        /**
-         * Creates a class that mimics a single threaded timer that expires periodically. If a call to {@link
-         * #isExpiredResetOnTrue()} occurs later than {@code period} since the timer was initiated or reset, this method returns
-         * true. Each time the method returns true the counter is reset. The timer starts with the specified time delay.
-         *
-         * @param delay    the initial delay before the timer starts
-         * @param period   the period between calls {@link #isExpiredResetOnTrue()}
-         * @param timeUnit the time unit of delay and period
-         */
-        public Timer(long delay, long period, TimeUnit timeUnit) {
-            this.delay = delay;
-            this.period = period;
-            this.timeUnit = timeUnit;
-
-            periodNanos = timeUnit.toNanos(period);
-            start = System.nanoTime() + timeUnit.toNanos(delay);
-        }
-
-        public long period() {
-            return period;
-        }
-
-        public long delay() {
-            return delay;
-        }
-
-        public TimeUnit getTimeUnit() {
-            return timeUnit;
-        }
-
-        /**
-         * Checks if a call to this method occurs later than {@code period} since the timer was initiated or reset. If that is the
-         * case the method returns true, otherwise it returns false. Each time this method returns true, the counter is reset
-         * (re-initiated) and a new cycle will start.
-         *
-         * @return true if the time elapsed since the last call returning true is greater than {@code period}. Returns false
-         * otherwise.
-         */
-        public boolean isExpiredResetOnTrue() {
-            final boolean expired = System.nanoTime() - start > periodNanos;
-            if (expired) {
-                start = System.nanoTime();
-            }
-            return expired;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java
index 8aa525b..db07fda 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java
@@ -18,34 +18,40 @@
 
 package org.apache.storm.kafka.spout;
 
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.common.serialization.Deserializer;
-import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff.TimeInterval;
-
 import java.io.Serializable;
-import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.regex.Pattern;
 
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff.TimeInterval;
+import org.apache.storm.tuple.Fields;
+
 /**
  * KafkaSpoutConfig defines the required configuration to connect a consumer to a consumer group, as well as the subscribing topics
  */
 public class KafkaSpoutConfig<K, V> implements Serializable {
+    private static final long serialVersionUID = 141902646130682494L;
     public static final long DEFAULT_POLL_TIMEOUT_MS = 200;            // 200ms
     public static final long DEFAULT_OFFSET_COMMIT_PERIOD_MS = 30_000;   // 30s
     public static final int DEFAULT_MAX_RETRIES = Integer.MAX_VALUE;     // Retry forever
     public static final int DEFAULT_MAX_UNCOMMITTED_OFFSETS = 10_000_000;    // 10,000,000 records => 80MBs of memory footprint in the worst case
-
-    // Kafka property names
-    public interface Consumer {
-        String GROUP_ID = "group.id";
-        String BOOTSTRAP_SERVERS = "bootstrap.servers";
-        String ENABLE_AUTO_COMMIT = "enable.auto.commit";
-        String AUTO_COMMIT_INTERVAL_MS = "auto.commit.interval.ms";
-        String KEY_DESERIALIZER = "key.deserializer";
-        String VALUE_DESERIALIZER = "value.deserializer";
-    }
+    public static final long DEFAULT_PARTITION_REFRESH_PERIOD_MS = 2_000; // 2s
+    public static final KafkaSpoutRetryService DEFAULT_RETRY_SERVICE =  
+            new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
+                    DEFAULT_MAX_RETRIES, TimeInterval.seconds(10));
+    /**
+     * Retry in a tight loop (keep unit tests fasts) do not use in production.
+     */
+    public static final KafkaSpoutRetryService UNIT_TEST_RETRY_SERVICE = 
+    new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(0),
+            DEFAULT_MAX_RETRIES, TimeInterval.milliSeconds(0));
 
     /**
      * The offset used by the Kafka spout in the first poll to Kafka broker. The choice of this parameter will
@@ -60,123 +66,254 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
      * 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, 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 final KafkaSpoutTuplesBuilder<K, V> tuplesBuilder;
-        private final KafkaSpoutRetryService retryService;
+        private KafkaSpoutRetryService retryService = DEFAULT_RETRY_SERVICE;
+        private long partitionRefreshPeriodMs = DEFAULT_PARTITION_REFRESH_PERIOD_MS;
+        
+        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, String ... topics) {
+            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
+        }
+        
+        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>();
+        }
+
+        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 because some translators will work no matter what the generics
+            // are I thought it best not to force someone to reset the translator
+            // when they change the key/value types.
+            this.translator = (RecordTranslator<K, V>) builder.translator;
+            this.retryService = builder.retryService;
+            this.keyDes = keyDes;
+            this.keyDesClazz = keyDesClazz;
+            this.valueDes = valueDes;
+            this.valueDesClazz = valueDesClazz;
+        }
 
         /**
-         * Please refer to javadoc in {@link #Builder(Map, KafkaSpoutStreams, KafkaSpoutTuplesBuilder, KafkaSpoutRetryService)}.<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)))}
+         * Specifying this key deserializer overrides the property key.deserializer. If you have
+         * set a custom RecordTranslator before calling this it may result in class cast
+         * exceptions at runtime.
          */
-        public Builder(Map<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 <NK> Builder<NK,V> setKey(SerializableDeserializer<NK> keyDeserializer) {
+            return new Builder<>(this, keyDeserializer, null, valueDes, valueDesClazz);
         }
-
-        /***
-         * KafkaSpoutConfig defines the required configuration to connect a consumer to a consumer group, as well as the subscribing topics
-         * The optional configuration can be specified using the set methods of this builder
-         * @param kafkaProps    properties defining consumer connection to Kafka broker as specified in @see <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
+        
+        /**
+         * Specify a class that can be instantiated to create a key.deserializer
+         * This is the same as setting key.deserializer, but overrides it. If you have
+         * set a custom RecordTranslator before calling this it may result in class cast
+         * exceptions at runtime.
          */
-        public Builder(Map<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");
-            }
-
-            this.kafkaProps = kafkaProps;
-            this.kafkaSpoutStreams = kafkaSpoutStreams;
-            this.tuplesBuilder = tuplesBuilder;
-            this.retryService = retryService;
+        public <NK> Builder<NK, V> setKey(Class<? extends Deserializer<NK>> clazz) {
+            return new Builder<>(this, null, clazz, valueDes, valueDesClazz);
         }
 
         /**
-         * Specifying this key deserializer overrides the property key.deserializer
+         * Specifying this value deserializer overrides the property value.deserializer.  If you have
+         * set a custom RecordTranslator before calling this it may result in class cast
+         * exceptions at runtime.
          */
-        public Builder<K,V> setKeyDeserializer(SerializableDeserializer<K> keyDeserializer) {
-            this.keyDeserializer = keyDeserializer;
+        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.  If you have
+         * set a custom RecordTranslator before calling this it may result in class cast
+         * exceptions at runtime.
+         */
+        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;
         }
-
+        
         /**
-         * Specifying this value deserializer overrides the property value.deserializer
+         * Set multiple Kafka property configs
          */
-        public Builder<K,V> setValueDeserializer(SerializableDeserializer<V> valueDeserializer) {
-            this.valueDeserializer = valueDeserializer;
+        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> setMaxPollRecords(int records) {
+            //to avoid issues with 0.9 versions that technically still work
+            // with this we do not use ConsumerConfig.MAX_POLL_RECORDS_CONFIG
+            return setProp("max.poll.records", records);
+        }
+        
+        //Security Related Configs
+        
+        /**
+         * Configure the SSL Keystore for mutual authentication
+         */
+        public Builder<K,V> setSSLKeystore(String location, String password) {
+            return setProp("ssl.keystore.location", location)
+                    .setProp("ssl.keystore.password", password);
+        }
+       
+        /**
+         * Configure the SSL Keystore for mutual authentication
+         */
+        public Builder<K,V> setSSLKeystore(String location, String password, String keyPassword) {
+            return setProp("ssl.key.password", keyPassword)
+                    .setSSLKeystore(location, password);
+        }
+        
+        /**
+         * Configure the SSL Truststore to authenticate with the brokers
+         */
+        public Builder<K,V> setSSLTruststore(String location, String password) {
+            return setSecurityProtocol("SSL")
+                    .setProp("ssl.truststore.location", location)
+                    .setProp("ssl.truststore.password", password);
+        }
+        
+        /**
+         * Protocol used to communicate with brokers. 
+         * Valid values are: PLAINTEXT, SSL, SASL_PLAINTEXT, SASL_SSL.
+         */
+        public Builder<K, V> setSecurityProtocol(String protocol) {
+            return setProp("security.protocol", protocol);
+        }
 
+        //Spout Settings
         /**
          * Specifies the time, in milliseconds, spent waiting in poll if data is not available. Default is 2s
          * @param pollTimeoutMs time in ms
@@ -228,22 +365,131 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
             this.firstPollOffsetStrategy = firstPollOffsetStrategy;
             return this;
         }
+        
+        /**
+         * Sets the retry service for the spout to use.
+         * @param retryService the new retry service
+         * @return the builder (this).
+         */
+        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;
+        }
+        
+        /**
+         * Configure a translator with tuples to be emitted on the default stream.
+         * @param func extracts and turns a Kafka ConsumerRecord into a list of objects to be emitted
+         * @param fields the names of the fields extracted
+         * @return this to be able to chain configuration
+         */
+        public Builder<K, V> setRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
+            return setRecordTranslator(new SimpleRecordTranslator<>(func, fields));
+        }
+        
+        /**
+         * Configure a translator with tuples to be emitted to a given stream.
+         * @param func extracts and turns a Kafka ConsumerRecord into a list of objects to be emitted
+         * @param fields the names of the fields extracted
+         * @param stream the stream to emit the tuples on
+         * @return this to be able to chain configuration
+         */
+        public Builder<K, V> setRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
+            return setRecordTranslator(new SimpleRecordTranslator<>(func, fields, stream));
+        }
+        
+        /**
+         * Sets partition refresh period in milliseconds. This is how often kafka will be polled
+         * to check for new topics and/or new partitions.
+         * This is mostly for Subscription implementations that manually assign partitions. NamedSubscription and
+         * PatternSubscription rely on kafka to handle this instead.
+         * @param partitionRefreshPeriodMs time in milliseconds
+         * @return the builder (this)
+         */
+        public Builder<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 {
+                return keyDesClazz.newInstance();
+            } catch (InstantiationException | IllegalAccessException e) {
+                throw new RuntimeException("Could not instantiate key deserializer " + keyDesClazz);
+            }
+        }
+        return keyDes;
     }
 
     public Deserializer<V> getValueDeserializer() {
-        return valueDeserializer;
+        if (valueDesClazz != null) {
+            try {
+                return valueDesClazz.newInstance();
+            } catch (InstantiationException | IllegalAccessException e) {
+                throw new RuntimeException("Could not instantiate value deserializer " + valueDesClazz);
+            }
+        }
+        return valueDes;
+    }
+    
+    public Subscription getSubscription() {
+        return subscription;
+    }
+    
+    public RecordTranslator<K,V> getTranslator() {
+        return translator;
     }
 
     public long getPollTimeoutMs() {
@@ -255,32 +501,12 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
     }
 
     public boolean isConsumerAutoCommitMode() {
-        return kafkaProps.get(Consumer.ENABLE_AUTO_COMMIT) == null     // default is true
-                || Boolean.valueOf((String)kafkaProps.get(Consumer.ENABLE_AUTO_COMMIT));
+        return kafkaProps.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG) == null     // default is true
+                || Boolean.valueOf((String)kafkaProps.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG));
     }
 
     public String getConsumerGroupId() {
-        return (String) kafkaProps.get(Consumer.GROUP_ID);
-    }
-
-    /**
-     * @return list of topics subscribed and emitting tuples to a stream as configured by {@link KafkaSpoutStream},
-     * or null if this stream is associated with a wildcard pattern topic
-     */
-    public List<String> getSubscribedTopics() {
-        return kafkaSpoutStreams instanceof KafkaSpoutStreamsNamedTopics ?
-            new ArrayList<>(((KafkaSpoutStreamsNamedTopics) kafkaSpoutStreams).getTopics()) :
-            null;
-    }
-
-    /**
-     * @return the wildcard pattern topic associated with this {@link KafkaSpoutStream}, or null
-     * if this stream is associated with a specific named topic
-     */
-    public Pattern getTopicWildcardPattern() {
-        return kafkaSpoutStreams instanceof KafkaSpoutStreamsWildcardTopics ?
-                ((KafkaSpoutStreamsWildcardTopics)kafkaSpoutStreams).getTopicWildcardPattern() :
-                null;
+        return (String) kafkaProps.get(ConsumerConfig.GROUP_ID_CONFIG);
     }
 
     public int getMaxTupleRetries() {
@@ -291,38 +517,32 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
         return firstPollOffsetStrategy;
     }
 
-    public KafkaSpoutStreams getKafkaSpoutStreams() {
-        return kafkaSpoutStreams;
-    }
-
     public int getMaxUncommittedOffsets() {
         return maxUncommittedOffsets;
     }
 
-    public KafkaSpoutTuplesBuilder<K, V> getTuplesBuilder() {
-        return tuplesBuilder;
-    }
-
     public KafkaSpoutRetryService getRetryService() {
         return retryService;
     }
+    
+    public long getPartitionRefreshPeriodMs() {
+        return partitionRefreshPeriodMs;
+    }
 
     @Override
     public String toString() {
         return "KafkaSpoutConfig{" +
                 "kafkaProps=" + kafkaProps +
-                ", keyDeserializer=" + keyDeserializer +
-                ", valueDeserializer=" + valueDeserializer +
+                ", key=" + getKeyDeserializer() +
+                ", value=" + getValueDeserializer() +
                 ", pollTimeoutMs=" + pollTimeoutMs +
                 ", offsetCommitPeriodMs=" + offsetCommitPeriodMs +
                 ", maxRetries=" + maxRetries +
                 ", maxUncommittedOffsets=" + maxUncommittedOffsets +
                 ", firstPollOffsetStrategy=" + firstPollOffsetStrategy +
-                ", kafkaSpoutStreams=" + kafkaSpoutStreams +
-                ", tuplesBuilder=" + tuplesBuilder +
+                ", subscription=" + subscription +
+                ", translator=" + translator +
                 ", retryService=" + retryService +
-                ", topics=" + getSubscribedTopics() +
-                ", topicWildcardPattern=" + getTopicWildcardPattern() +
                 '}';
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutMessageId.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutMessageId.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutMessageId.java
index 71f8327..3cfad9d 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutMessageId.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutMessageId.java
@@ -26,7 +26,7 @@ public class KafkaSpoutMessageId {
     private transient long offset;
     private transient int numFails = 0;
 
-    public KafkaSpoutMessageId(ConsumerRecord consumerRecord) {
+    public KafkaSpoutMessageId(ConsumerRecord<?, ?> consumerRecord) {
         this(new TopicPartition(consumerRecord.topic(), consumerRecord.partition()), consumerRecord.offset());
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStream.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStream.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStream.java
deleted file mode 100644
index 0f444b4..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStream.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout;
-
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.regex.Pattern;
-
-/**
- * Represents the stream and output fields used by a topic
- */
-public class KafkaSpoutStream implements Serializable {
-    private static final Logger LOG = LoggerFactory.getLogger(KafkaSpoutStream.class);
-
-    private final Fields outputFields;
-    private final String streamId;
-    private final String topic;
-    private Pattern topicWildcardPattern;
-
-    /** Represents the specified outputFields and topic with the default stream */
-    public KafkaSpoutStream(Fields outputFields, String topic) {
-        this(outputFields, Utils.DEFAULT_STREAM_ID, topic);
-    }
-
-    /** Represents the specified outputFields and topic with the specified stream */
-    public KafkaSpoutStream(Fields outputFields, String streamId, String topic) {
-        if (outputFields == null || streamId == null || topic == null) {
-            throw new IllegalArgumentException(String.format("Constructor parameters cannot be null. " +
-                    "[outputFields=%s, streamId=%s, topic=%s]", outputFields, streamId, topic));
-        }
-        this.outputFields = outputFields;
-        this.streamId = streamId;
-        this.topic = topic;
-        this.topicWildcardPattern = null;
-    }
-
-    /** Represents the specified outputFields and topic wild card with the default stream */
-    public KafkaSpoutStream(Fields outputFields, Pattern topicWildcardPattern) {
-        this(outputFields, Utils.DEFAULT_STREAM_ID, topicWildcardPattern);
-    }
-
-    /** Represents the specified outputFields and topic wild card with the specified stream */
-    public KafkaSpoutStream(Fields outputFields, String streamId, Pattern topicWildcardPattern) {
-
-        if (outputFields == null || streamId == null || topicWildcardPattern == null) {
-            throw new IllegalArgumentException(String.format("Constructor parameters cannot be null. " +
-                    "[outputFields=%s, streamId=%s, topicWildcardPattern=%s]", outputFields, streamId, topicWildcardPattern));
-        }
-        this.outputFields = outputFields;
-        this.streamId = streamId;
-        this.topic = null;
-        this.topicWildcardPattern = topicWildcardPattern;
-    }
-
-    public void emit(SpoutOutputCollector collector, List<Object> tuple, KafkaSpoutMessageId messageId) {
-        collector.emit(streamId, tuple, messageId);
-    }
-
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        LOG.info("Declared [streamId = {}], [outputFields = {}] for [topic = {}]", streamId, outputFields, topic);
-        declarer.declareStream(streamId, outputFields);
-    }
-
-
-    public Fields getOutputFields() {
-        return outputFields;
-    }
-
-    public String getStreamId() {
-        return streamId;
-    }
-
-    /**
-     * @return the topic associated with this {@link KafkaSpoutStream}, or null
-     * if this stream is associated with a wildcard pattern topic
-     */
-    public String getTopic() {
-        return topic;
-    }
-
-    /**
-     * @return the wildcard pattern topic associated with this {@link KafkaSpoutStream}, or null
-     * if this stream is associated with a specific named topic
-     */
-    public Pattern getTopicWildcardPattern() {
-        return topicWildcardPattern;
-    }
-
-    @Override
-    public String toString() {
-        return "KafkaSpoutStream{" +
-                "outputFields=" + outputFields +
-                ", streamId='" + streamId + '\'' +
-                ", topic='" + topic + '\'' +
-                ", topicWildcardPattern=" + topicWildcardPattern +
-                '}';
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreams.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreams.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreams.java
deleted file mode 100644
index d4178a9..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreams.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout;
-
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.tuple.Fields;
-
-import java.io.Serializable;
-import java.util.List;
-
-/**
- * Represents the {@link KafkaSpoutStream} associated with each topic or topic pattern (wildcard), and provides
- * a public API to declare output streams and emmit tuples, on the appropriate stream, for all the topics specified.
- */
-public interface KafkaSpoutStreams extends Serializable {
-    void declareOutputFields(OutputFieldsDeclarer declarer);
-
-    void emit(SpoutOutputCollector collector, List<Object> tuple, KafkaSpoutMessageId messageId);
-
-    Fields getOutputFields();
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopics.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopics.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopics.java
deleted file mode 100644
index bc2426a..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsNamedTopics.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.OutputFieldsGetter;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Represents the {@link KafkaSpoutStream} associated with each topic, and provides a public API to
- * declare output streams and emmit tuples, on the appropriate stream, for all the topics specified.
- */
-public class KafkaSpoutStreamsNamedTopics implements KafkaSpoutStreams {
-    private static final Logger LOG = LoggerFactory.getLogger(KafkaSpoutStreamsNamedTopics.class);
-
-    private final Map<String, KafkaSpoutStream> topicToStream;
-
-    private KafkaSpoutStreamsNamedTopics(Builder builder) {
-        this.topicToStream = builder.topicToStream;
-        LOG.debug("Built {}", this);
-    }
-
-    /**
-     * @param topic the topic for which to get output fields
-     * @return the declared output fields
-     */
-    public Fields getOutputFields(String topic) {
-        if (topicToStream.containsKey(topic)) {
-            final Fields outputFields = topicToStream.get(topic).getOutputFields();
-            LOG.trace("Topic [{}] has output fields [{}]", topic, outputFields);
-            return outputFields;
-        }
-        throw new IllegalStateException(this.getClass().getName() + " not configured for topic: " + topic);
-    }
-
-    @Override
-    public Fields getOutputFields() {
-        final Set<String> allFields = new LinkedHashSet<>();
-        for (KafkaSpoutStream kafkaSpoutStream : topicToStream.values()) {
-            allFields.addAll(kafkaSpoutStream.getOutputFields().toList());
-        }
-        return new Fields(new ArrayList<>(allFields));
-    }
-
-    /**
-     * @param topic the topic to for which to get the stream id
-     * @return the id of the stream to where the tuples are emitted
-     */
-    public KafkaSpoutStream getStream(String topic) {
-        if (topicToStream.containsKey(topic)) {
-            return topicToStream.get(topic);
-        }
-        throw new IllegalStateException(this.getClass().getName() + " not configured for topic: " + topic);
-    }
-
-    /**
-     * @return list of topics subscribed and emitting tuples to a stream as configured by {@link KafkaSpoutStream}
-     */
-    public List<String> getTopics() {
-        return new ArrayList<>(topicToStream.keySet());
-    }
-
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        for (KafkaSpoutStream stream : topicToStream.values()) {
-            if (!((OutputFieldsGetter)declarer).getFieldsDeclaration().containsKey(stream.getStreamId())) {
-                stream.declareOutputFields(declarer);
-            }
-        }
-    }
-
-    public void emit(SpoutOutputCollector collector, List<Object> tuple, KafkaSpoutMessageId messageId) {
-        getStream(messageId.topic()).emit(collector, tuple, messageId);
-    }
-
-    @Override
-    public String toString() {
-        return "KafkaSpoutStreamsNamedTopics{" +
-                "topicToStream=" + topicToStream +
-                '}';
-    }
-
-    public static class Builder {
-        private final Map<String, KafkaSpoutStream> topicToStream = new HashMap<>();;
-
-        /**
-         * Creates a {@link KafkaSpoutStream} with the given output Fields for each topic specified.
-         * All topics will have the default stream id and the same output fields.
-         */
-        public Builder(Fields outputFields, String... topics) {
-            addStream(outputFields, topics);
-        }
-
-        /**
-         * Creates a {@link KafkaSpoutStream} with this particular stream for each topic specified.
-         * All the topics will have the specified stream id and the same output fields.
-         */
-        public Builder (Fields outputFields, String streamId, String... topics) {
-            addStream(outputFields, streamId, topics);
-        }
-
-        /**
-         * Adds this stream to the state representing the streams associated with each topic
-         */
-        public Builder(KafkaSpoutStream stream) {
-            addStream(stream);
-        }
-
-        /**
-         * Adds this stream to the state representing the streams associated with each topic
-         */
-        public Builder addStream(KafkaSpoutStream stream) {
-            topicToStream.put(stream.getTopic(), stream);
-            return this;
-        }
-
-        /**
-         * Please refer to javadoc in {@link #Builder(Fields, String...)}
-         */
-        public Builder addStream(Fields outputFields, String... topics) {
-            addStream(outputFields, Utils.DEFAULT_STREAM_ID, topics);
-            return this;
-        }
-
-        /**
-         * Please refer to javadoc in {@link #Builder(Fields, String, String...)}
-         */
-        public Builder addStream(Fields outputFields, String streamId, String... topics) {
-            for (String topic : topics) {
-                topicToStream.put(topic, new KafkaSpoutStream(outputFields, streamId, topic));
-            }
-            return this;
-        }
-
-        public KafkaSpoutStreamsNamedTopics build() {
-            return new KafkaSpoutStreamsNamedTopics(this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsWildcardTopics.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsWildcardTopics.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsWildcardTopics.java
deleted file mode 100644
index 64132b3..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutStreamsWildcardTopics.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout;
-
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.tuple.Fields;
-
-import java.util.List;
-import java.util.regex.Pattern;
-
-public class KafkaSpoutStreamsWildcardTopics implements KafkaSpoutStreams {
-    private KafkaSpoutStream kafkaSpoutStream;
-
-    public KafkaSpoutStreamsWildcardTopics(KafkaSpoutStream kafkaSpoutStream) {
-        this.kafkaSpoutStream = kafkaSpoutStream;
-        if (kafkaSpoutStream.getTopicWildcardPattern() == null) {
-            throw new IllegalStateException("KafkaSpoutStream must be configured for wildcard topic");
-        }
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        kafkaSpoutStream.declareOutputFields(declarer);
-    }
-
-    @Override
-    public void emit(SpoutOutputCollector collector, List<Object> tuple, KafkaSpoutMessageId messageId) {
-        kafkaSpoutStream.emit(collector, tuple, messageId);
-    }
-
-    @Override
-    public Fields getOutputFields() {
-        return kafkaSpoutStream.getOutputFields();
-    }
-
-    public KafkaSpoutStream getStream() {
-        return kafkaSpoutStream;
-    }
-
-    public Pattern getTopicWildcardPattern() {
-        return kafkaSpoutStream.getTopicWildcardPattern();
-    }
-
-    @Override
-    public String toString() {
-        return "KafkaSpoutStreamsWildcardTopics{" +
-                "kafkaSpoutStream=" + kafkaSpoutStream +
-                '}';
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTupleBuilder.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTupleBuilder.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTupleBuilder.java
deleted file mode 100644
index 3bb71a8..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTupleBuilder.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Implementations of {@link KafkaSpoutTupleBuilder} contain the logic to build tuples from {@link ConsumerRecord}s.
- * Users must subclass this abstract class to provide their implementation. See also {@link KafkaSpoutTuplesBuilder}
- */
-public abstract class KafkaSpoutTupleBuilder<K,V> implements Serializable {
-    private List<String> topics;
-
-    /**
-     * @param topics list of topics that use this implementation to build tuples
-     */
-    public KafkaSpoutTupleBuilder(String... topics) {
-        if (topics == null || topics.length == 0) {
-            throw new IllegalArgumentException("Must specify at least one topic. It cannot be null or empty");
-        }
-        this.topics = Arrays.asList(topics);
-    }
-
-    /**
-     * @return list of topics that use this implementation to build tuples
-     */
-    public List<String> getTopics() {
-        return Collections.unmodifiableList(topics);
-    }
-
-    /**
-     * Builds a list of tuples using the ConsumerRecord specified as parameter
-     * @param consumerRecord whose contents are used to build tuples
-     * @return list of tuples
-     */
-    public abstract List<Object> buildTuple(ConsumerRecord<K, V> consumerRecord);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilder.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilder.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilder.java
deleted file mode 100644
index 2ba0a79..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilder.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import java.io.Serializable;
-import java.util.List;
-
-/**
- * {@link KafkaSpoutTuplesBuilder} wraps all the logic that builds tuples from {@link ConsumerRecord}s.
- * The logic is provided by the user by implementing the appropriate number of {@link KafkaSpoutTupleBuilder} instances
- */
-public interface KafkaSpoutTuplesBuilder<K,V> extends Serializable {
-    List<Object> buildTuple(ConsumerRecord<K, V> consumerRecord);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderNamedTopics.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderNamedTopics.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderNamedTopics.java
deleted file mode 100644
index 80fe543..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderNamedTopics.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class KafkaSpoutTuplesBuilderNamedTopics<K,V> implements KafkaSpoutTuplesBuilder<K,V> {
-    private static final Logger LOG = LoggerFactory.getLogger(KafkaSpoutTuplesBuilderNamedTopics.class);
-
-    private Map<String, KafkaSpoutTupleBuilder<K, V>> topicToTupleBuilders;
-
-    private KafkaSpoutTuplesBuilderNamedTopics(Builder<K,V> builder) {
-        this.topicToTupleBuilders = builder.topicToTupleBuilders;
-        LOG.debug("Instantiated {}", this);
-    }
-
-    public static class Builder<K,V> {
-        private List<KafkaSpoutTupleBuilder<K, V>> tupleBuilders;
-        private Map<String, KafkaSpoutTupleBuilder<K, V>> topicToTupleBuilders;
-
-        @SafeVarargs
-        public Builder(KafkaSpoutTupleBuilder<K,V>... tupleBuilders) {
-            if (tupleBuilders == null || tupleBuilders.length == 0) {
-                throw new IllegalArgumentException("Must specify at last one tuple builder per topic declared in KafkaSpoutStreams");
-            }
-
-            this.tupleBuilders = Arrays.asList(tupleBuilders);
-            topicToTupleBuilders = new HashMap<>();
-        }
-
-        public KafkaSpoutTuplesBuilderNamedTopics<K,V> build() {
-            for (KafkaSpoutTupleBuilder<K, V> tupleBuilder : tupleBuilders) {
-                for (String topic : tupleBuilder.getTopics()) {
-                    if (!topicToTupleBuilders.containsKey(topic)) {
-                        topicToTupleBuilders.put(topic, tupleBuilder);
-                    }
-                }
-            }
-            return new KafkaSpoutTuplesBuilderNamedTopics<>(this);
-        }
-    }
-
-    public List<Object>buildTuple(ConsumerRecord<K,V> consumerRecord) {
-        final String topic = consumerRecord.topic();
-        return topicToTupleBuilders.get(topic).buildTuple(consumerRecord);
-    }
-
-    @Override
-    public String toString() {
-        return "KafkaSpoutTuplesBuilderNamedTopics {" +
-                "topicToTupleBuilders=" + topicToTupleBuilders +
-                '}';
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderWildcardTopics.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderWildcardTopics.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderWildcardTopics.java
deleted file mode 100644
index 85d4809..0000000
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutTuplesBuilderWildcardTopics.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.spout;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import java.util.List;
-
-public class KafkaSpoutTuplesBuilderWildcardTopics<K,V> implements KafkaSpoutTuplesBuilder<K,V> {
-    private KafkaSpoutTupleBuilder<K, V> tupleBuilder;
-
-    public KafkaSpoutTuplesBuilderWildcardTopics(KafkaSpoutTupleBuilder<K, V> tupleBuilder) {
-        this.tupleBuilder = tupleBuilder;
-    }
-
-    @Override
-    public List<Object> buildTuple(ConsumerRecord<K, V> consumerRecord) {
-        return tupleBuilder.buildTuple(consumerRecord);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTuple.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTuple.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTuple.java
new file mode 100644
index 0000000..f5953ad
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTuple.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+package org.apache.storm.kafka.spout;
+
+import org.apache.storm.tuple.Values;
+
+/**
+ * A list of Values in a tuple that can be routed 
+ * to a given stream. {@see org.apache.storm.kafka.spout.RecordTranslator#apply}
+ */
+public class KafkaTuple extends Values {
+    private static final long serialVersionUID = 4803794470450587992L;
+    private String stream = null;
+    
+    public KafkaTuple() {
+        super();
+    }
+    
+    public KafkaTuple(Object... vals) {
+        super(vals);
+    }
+    
+    public KafkaTuple routedTo(String stream) {
+        assert(this.stream == null);
+        this.stream = stream;
+        return this;
+    }
+
+    public String getStream() {
+        return stream;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionNamedSubscription.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionNamedSubscription.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionNamedSubscription.java
new file mode 100644
index 0000000..df3e800
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionNamedSubscription.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+package org.apache.storm.kafka.spout;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.task.TopologyContext;
+
+public class ManualPartitionNamedSubscription extends NamedSubscription {
+    private static final long serialVersionUID = 5633018073527583826L;
+    private final ManualPartitioner partitioner;
+    private Set<TopicPartition> currentAssignment = null;
+    private KafkaConsumer<?, ?> consumer = null;
+    private ConsumerRebalanceListener listener = null;
+    private TopologyContext context = null;
+
+    public ManualPartitionNamedSubscription(ManualPartitioner parter, Collection<String> topics) {
+        super(topics);
+        this.partitioner = parter;
+    }
+    
+    public ManualPartitionNamedSubscription(ManualPartitioner parter, String ... topics) {
+        this(parter, Arrays.asList(topics));
+    }
+    
+    @Override
+    public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceListener listener, TopologyContext context) {
+        this.consumer = consumer;
+        this.listener = listener;
+        this.context = context;
+        refreshAssignment();
+    }
+    
+    @Override
+    public void refreshAssignment() {
+        List<TopicPartition> allPartitions = new ArrayList<>();
+        for (String topic : topics) {
+            for (PartitionInfo partitionInfo: consumer.partitionsFor(topic)) {
+                allPartitions.add(new TopicPartition(partitionInfo.topic(), partitionInfo.partition()));
+            }
+        }
+        Collections.sort(allPartitions, TopicPartitionComparator.INSTANCE);
+        Set<TopicPartition> newAssignment = new HashSet<>(partitioner.partition(allPartitions, context));
+        if (!newAssignment.equals(currentAssignment)) {
+            if (currentAssignment != null) {
+                listener.onPartitionsRevoked(currentAssignment);
+                listener.onPartitionsAssigned(newAssignment);
+            }
+            currentAssignment = newAssignment;
+            consumer.assign(currentAssignment);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionPatternSubscription.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionPatternSubscription.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionPatternSubscription.java
new file mode 100644
index 0000000..cf4dfcb
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionPatternSubscription.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+package org.apache.storm.kafka.spout;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.task.TopologyContext;
+
+public class ManualPartitionPatternSubscription extends PatternSubscription {
+    private static final long serialVersionUID = 5633018073527583826L;
+    private final ManualPartitioner parter;
+    private Set<TopicPartition> currentAssignment = null;
+    private KafkaConsumer<?, ?> consumer = null;
+    private ConsumerRebalanceListener listener = null;
+    private TopologyContext context = null;
+
+    public ManualPartitionPatternSubscription(ManualPartitioner parter, Pattern pattern) {
+        super(pattern);
+        this.parter = parter;
+    }
+    
+    @Override
+    public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceListener listener, TopologyContext context) {
+        this.consumer = consumer;
+        this.listener = listener;
+        this.context = context;
+        refreshAssignment();
+    }
+    
+    @Override
+    public void refreshAssignment() {
+        List<TopicPartition> allPartitions = new ArrayList<>();
+        for(Map.Entry<String, List<PartitionInfo>> entry: consumer.listTopics().entrySet()) {
+            if (pattern.matcher(entry.getKey()).matches()) {
+                for (PartitionInfo partitionInfo: entry.getValue()) {
+                    allPartitions.add(new TopicPartition(partitionInfo.topic(), partitionInfo.partition()));
+                }
+            }
+        }
+        Collections.sort(allPartitions, TopicPartitionComparator.INSTANCE);
+        Set<TopicPartition> newAssignment = new HashSet<>(parter.partition(allPartitions, context));
+        if (!newAssignment.equals(currentAssignment)) {
+            if (currentAssignment != null) {
+                listener.onPartitionsRevoked(currentAssignment);
+                listener.onPartitionsAssigned(newAssignment);
+            }
+            currentAssignment = newAssignment;
+            consumer.assign(currentAssignment);
+        }
+    }
+}
\ No newline at end of file


[2/7] storm git commit: 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 partit

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java
index 4cf4824..2aeeb95 100644
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java
@@ -18,32 +18,29 @@
 
 package org.apache.storm.kafka.spout.test;
 
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
 import org.apache.storm.Config;
 import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
+import org.apache.storm.kafka.spout.ByTopicRecordTranslator;
 import org.apache.storm.kafka.spout.KafkaSpout;
 import org.apache.storm.kafka.spout.KafkaSpoutConfig;
 import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff;
 import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff.TimeInterval;
 import org.apache.storm.kafka.spout.KafkaSpoutRetryService;
-import org.apache.storm.kafka.spout.KafkaSpoutStreams;
-import org.apache.storm.kafka.spout.KafkaSpoutStreamsNamedTopics;
-import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilder;
-import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilderNamedTopics;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.tuple.Fields;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
+import org.apache.storm.tuple.Values;
 
 public class KafkaSpoutTopologyMainNamedTopics {
-    private static final String[] STREAMS = new String[]{"test_stream","test1_stream","test2_stream"};
+    private static final String TOPIC_2_STREAM = "test_2_stream";
+    private static final String TOPIC_0_1_STREAM = "test_0_1_stream";
     private static final String[] TOPICS = new String[]{"test","test1","test2"};
 
 
@@ -87,14 +84,25 @@ public class KafkaSpoutTopologyMainNamedTopics {
 
     protected StormTopology getTopolgyKafkaSpout() {
         final TopologyBuilder tp = new TopologyBuilder();
-        tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams())), 1);
-        tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAMS[0]);
-        tp.setBolt("kafka_bolt_1", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAMS[2]);
+        tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig()), 1);
+        tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt())
+          .shuffleGrouping("kafka_spout", TOPIC_0_1_STREAM)
+          .shuffleGrouping("kafka_spout", TOPIC_2_STREAM);
+        tp.setBolt("kafka_bolt_1", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", TOPIC_2_STREAM);
         return tp.createTopology();
     }
 
-    protected KafkaSpoutConfig<String,String> getKafkaSpoutConfig(KafkaSpoutStreams kafkaSpoutStreams) {
-        return new KafkaSpoutConfig.Builder<String, String>(getKafkaConsumerProps(), kafkaSpoutStreams, getTuplesBuilder(), getRetryService())
+    protected KafkaSpoutConfig<String,String> getKafkaSpoutConfig() {
+        ByTopicRecordTranslator<String, String> trans = new ByTopicRecordTranslator<>(
+                (r) -> new Values(r.topic(), r.partition(), r.offset(), r.key(), r.value()),
+                new Fields("topic", "partition", "offset", "key", "value"), TOPIC_0_1_STREAM);
+        trans.forTopic(TOPICS[2], 
+                (r) -> new Values(r.topic(), r.partition(), r.offset(), r.key(), r.value()),
+                new Fields("topic", "partition", "offset", "key", "value"), TOPIC_2_STREAM);
+        return KafkaSpoutConfig.builder("127.0.0.1:9092", TOPICS)
+                .setGroupId("kafkaSpoutTestGroup")
+                .setRetry(getRetryService())
+                .setRecordTranslator(trans)
                 .setOffsetCommitPeriodMs(10_000)
                 .setFirstPollOffsetStrategy(EARLIEST)
                 .setMaxUncommittedOffsets(250)
@@ -105,30 +113,4 @@ public class KafkaSpoutTopologyMainNamedTopics {
             return new KafkaSpoutRetryExponentialBackoff(TimeInterval.microSeconds(500),
                     TimeInterval.milliSeconds(2), Integer.MAX_VALUE, TimeInterval.seconds(10));
     }
-
-    protected Map<String,Object> getKafkaConsumerProps() {
-        Map<String, Object> props = new HashMap<>();
-//        props.put(KafkaSpoutConfig.Consumer.ENABLE_AUTO_COMMIT, "true");
-        props.put(KafkaSpoutConfig.Consumer.BOOTSTRAP_SERVERS, "127.0.0.1:9092");
-        props.put(KafkaSpoutConfig.Consumer.GROUP_ID, "kafkaSpoutTestGroup");
-        props.put(KafkaSpoutConfig.Consumer.KEY_DESERIALIZER, "org.apache.kafka.common.serialization.StringDeserializer");
-        props.put(KafkaSpoutConfig.Consumer.VALUE_DESERIALIZER, "org.apache.kafka.common.serialization.StringDeserializer");
-        return props;
-    }
-
-    protected KafkaSpoutTuplesBuilder<String, String> getTuplesBuilder() {
-        return new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
-                new TopicsTest0Test1TupleBuilder<String, String>(TOPICS[0], TOPICS[1]),
-                new TopicTest2TupleBuilder<String, String>(TOPICS[2]))
-                .build();
-    }
-
-    protected KafkaSpoutStreams getKafkaSpoutStreams() {
-        final Fields outputFields = new Fields("topic", "partition", "offset", "key", "value");
-        final Fields outputFields1 = new Fields("topic", "partition", "offset");
-        return new KafkaSpoutStreamsNamedTopics.Builder(outputFields, STREAMS[0], new String[]{TOPICS[0], TOPICS[1]})  // contents of topics test, test1, sent to test_stream
-                .addStream(outputFields, STREAMS[0], new String[]{TOPICS[2]})  // contents of topic test2 sent to test_stream
-                .addStream(outputFields1, STREAMS[2], new String[]{TOPICS[2]})  // contents of topic test2 sent to test2_stream
-                .build();
-    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java
index c362a2b..d0376e6 100644
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java
@@ -18,22 +18,20 @@
 
 package org.apache.storm.kafka.spout.test;
 
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
+
+import java.util.regex.Pattern;
+
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.kafka.spout.KafkaSpout;
-import org.apache.storm.kafka.spout.KafkaSpoutStream;
-import org.apache.storm.kafka.spout.KafkaSpoutStreams;
-import org.apache.storm.kafka.spout.KafkaSpoutStreamsWildcardTopics;
-import org.apache.storm.kafka.spout.KafkaSpoutTupleBuilder;
-import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilder;
-import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilderWildcardTopics;
+import org.apache.storm.kafka.spout.KafkaSpoutConfig;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.tuple.Fields;
-
-import java.util.regex.Pattern;
+import org.apache.storm.tuple.Values;
 
 public class KafkaSpoutTopologyMainWildcardTopics extends KafkaSpoutTopologyMainNamedTopics {
     private static final String STREAM = "test_wildcard_stream";
-    private static final String TOPIC_WILDCARD_PATTERN = "test[1|2]";
+    private static final Pattern TOPIC_WILDCARD_PATTERN = Pattern.compile("test[1|2]");
 
     public static void main(String[] args) throws Exception {
         new KafkaSpoutTopologyMainWildcardTopics().runMain(args);
@@ -41,22 +39,20 @@ public class KafkaSpoutTopologyMainWildcardTopics extends KafkaSpoutTopologyMain
 
     protected StormTopology getTopolgyKafkaSpout() {
         final TopologyBuilder tp = new TopologyBuilder();
-        tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams())), 1);
+        tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig()), 1);
         tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAM);
         return tp.createTopology();
     }
 
-    protected KafkaSpoutTuplesBuilder<String, String> getTuplesBuilder() {
-        return new KafkaSpoutTuplesBuilderWildcardTopics<>(getTupleBuilder());
-    }
-
-    protected KafkaSpoutTupleBuilder<String, String> getTupleBuilder() {
-        return new TopicsTest0Test1TupleBuilder<>(TOPIC_WILDCARD_PATTERN);
-    }
-
-    protected KafkaSpoutStreams getKafkaSpoutStreams() {
-        final Fields outputFields = new Fields("topic", "partition", "offset", "key", "value");
-        final KafkaSpoutStream kafkaSpoutStream = new KafkaSpoutStream(outputFields, STREAM, Pattern.compile(TOPIC_WILDCARD_PATTERN));
-        return new KafkaSpoutStreamsWildcardTopics(kafkaSpoutStream);
+    protected KafkaSpoutConfig<String,String> getKafkaSpoutConfig() {
+        return KafkaSpoutConfig.builder("127.0.0.1:9092", TOPIC_WILDCARD_PATTERN)
+                .setGroupId("kafkaSpoutTestGroup")
+                .setRetry(getRetryService())
+                .setRecordTranslator((r) -> new Values(r.topic(), r.partition(), r.offset(), r.key(), r.value()),
+                        new Fields("topic", "partition", "offset", "key", "value"), STREAM)
+                .setOffsetCommitPeriodMs(10_000)
+                .setFirstPollOffsetStrategy(EARLIEST)
+                .setMaxUncommittedOffsets(250)
+                .build();
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicTest2TupleBuilder.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicTest2TupleBuilder.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicTest2TupleBuilder.java
deleted file mode 100644
index ca65177..0000000
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicTest2TupleBuilder.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-package org.apache.storm.kafka.spout.test;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.storm.kafka.spout.KafkaSpoutTupleBuilder;
-import org.apache.storm.tuple.Values;
-
-import java.util.List;
-
-public class TopicTest2TupleBuilder<K, V> extends KafkaSpoutTupleBuilder<K,V> {
-    /**
-     * @param topics list of topics that use this implementation to build tuples
-     */
-    public TopicTest2TupleBuilder(String... topics) {
-        super(topics);
-    }
-
-    @Override
-    public List<Object> buildTuple(ConsumerRecord<K, V> consumerRecord) {
-        return new Values(consumerRecord.topic(),
-                consumerRecord.partition(),
-                consumerRecord.offset());
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicsTest0Test1TupleBuilder.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicsTest0Test1TupleBuilder.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicsTest0Test1TupleBuilder.java
deleted file mode 100644
index 4c55aa1..0000000
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/TopicsTest0Test1TupleBuilder.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-package org.apache.storm.kafka.spout.test;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.storm.kafka.spout.KafkaSpoutTupleBuilder;
-import org.apache.storm.tuple.Values;
-
-import java.util.List;
-
-public class TopicsTest0Test1TupleBuilder<K, V> extends KafkaSpoutTupleBuilder<K,V> {
-    /**
-     * @param topics list of topics that use this implementation to build tuples
-     */
-    public TopicsTest0Test1TupleBuilder(String... topics) {
-        super(topics);
-    }
-
-    @Override
-    public List<Object> buildTuple(ConsumerRecord<K, V> consumerRecord) {
-        return new Values(consumerRecord.topic(),
-                consumerRecord.partition(),
-                consumerRecord.offset(),
-                consumerRecord.key(),
-                consumerRecord.value());
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java
index 19f0452..4c5dba5 100644
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java
+++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java
@@ -41,7 +41,7 @@ public class DynamicPartitionConnections {
         }
     }
 
-    Map<Broker, ConnectionInfo> _connections = new HashMap();
+    Map<Broker, ConnectionInfo> _connections = new HashMap<>();
     KafkaConfig _config;
     IBrokerReader _reader;
 

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java
index d2bd313..4608963 100644
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java
+++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java
@@ -61,7 +61,7 @@ public class KafkaSpout extends BaseRichSpout {
     public void open(Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
         _collector = collector;
         String topologyInstanceId = context.getStormId();
-        Map stateConf = new HashMap(conf);
+        Map<String, Object> stateConf = new HashMap<>(conf);
         List<String> zkServers = _spoutConfig.zkServers;
         if (zkServers == null) {
             zkServers = (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS);

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java
index 53961d4..2072df3 100644
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java
+++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java
@@ -114,7 +114,7 @@ public class KafkaUtils {
         @Override
         public Object getValueAndReset() {
             try {
-                HashMap ret = new HashMap();
+                HashMap<String, Long> ret = new HashMap<>();
                 if (_partitions != null && _partitions.size() == _partitionToOffset.size()) {
                     Map<String,TopicMetrics> topicMetricsMap = new TreeMap<String, TopicMetrics>();
                     for (Map.Entry<Partition, PartitionManager.OffsetData> e : _partitionToOffset.entrySet()) {
@@ -195,7 +195,7 @@ public class KafkaUtils {
         int partitionId = partition.partition;
         FetchRequestBuilder builder = new FetchRequestBuilder();
         FetchRequest fetchRequest = builder.addFetch(topic, partitionId, offset, config.fetchSizeBytes).
-        		clientId(config.clientId).maxWait(config.fetchMaxWait).minBytes(config.minFetchByte).build();
+                clientId(config.clientId).maxWait(config.fetchMaxWait).minBytes(config.minFetchByte).build();
         FetchResponse fetchResponse;
         try {
             fetchResponse = consumer.fetch(fetchRequest);

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java
index 793d227..8d608d9 100644
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java
+++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java
@@ -133,7 +133,7 @@ public class PartitionManager {
     }
 
     public Map getMetricsDataMap() {
-        Map ret = new HashMap();
+        Map<String, Object> ret = new HashMap<>();
         ret.put(_partition + "/fetchAPILatencyMax", _fetchAPILatencyMax.getValueAndReset());
         ret.put(_partition + "/fetchAPILatencyMean", _fetchAPILatencyMean.getValueAndReset());
         ret.put(_partition + "/fetchAPICallCount", _fetchAPICallCount.getValueAndReset());

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java
index bdbc44d..628bfc0 100644
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java
+++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java
@@ -24,7 +24,7 @@ import java.util.*;
 
 public class StaticCoordinator implements PartitionCoordinator {
     Map<Partition, PartitionManager> _managers = new HashMap<Partition, PartitionManager>();
-    List<PartitionManager> _allManagers = new ArrayList();
+    List<PartitionManager> _allManagers = new ArrayList<>();
 
     public StaticCoordinator(DynamicPartitionConnections connections, Map stormConf, SpoutConfig config, ZkState state, int taskIndex, int totalTasks, String topologyInstanceId) {
         StaticHosts hosts = (StaticHosts) config.hosts;
@@ -34,7 +34,7 @@ public class StaticCoordinator implements PartitionCoordinator {
         for (Partition myPartition : myPartitions) {
             _managers.put(myPartition, new PartitionManager(connections, topologyInstanceId, state, stormConf, config, myPartition));
         }
-        _allManagers = new ArrayList(_managers.values());
+        _allManagers = new ArrayList<>(_managers.values());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java
index 31eaac5..b5bb124 100644
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java
+++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java
@@ -51,7 +51,9 @@ import java.util.Properties;
  * 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
 public class KafkaBolt<K, V> extends BaseRichBolt {
 
     private static final Logger LOG = LoggerFactory.getLogger(KafkaBolt.class);

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java
index 3363252..46cc60d 100644
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java
+++ b/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java
@@ -54,7 +54,7 @@ public class DynamicBrokersReaderTest {
     public void setUp() throws Exception {
         server = new TestingServer();
         String connectionString = server.getConnectString();
-        Map conf = new HashMap();
+        Map<String, Object> conf = new HashMap<>();
         conf.put(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT, 1000);
         conf.put(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT, 1000);
         conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 4);
@@ -64,7 +64,7 @@ public class DynamicBrokersReaderTest {
         zookeeper = CuratorFrameworkFactory.newClient(connectionString, retryPolicy);
         dynamicBrokersReader = new DynamicBrokersReader(conf, connectionString, masterPath, topic);
 
-        Map conf2 = new HashMap();
+        Map<String, Object> conf2 = new HashMap<>();
         conf2.putAll(conf);
         conf2.put("kafka.topic.wildcard.match",true);
 
@@ -240,7 +240,7 @@ public class DynamicBrokersReaderTest {
     @Test(expected = NullPointerException.class)
     public void testErrorLogsWhenConfigIsMissing() throws Exception {
         String connectionString = server.getConnectString();
-        Map conf = new HashMap();
+        Map<String, Object> conf = new HashMap<>();
         conf.put(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT, 1000);
 //        conf.put(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT, 1000);
         conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 4);

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java
index 7a6073a..864eaa9 100644
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java
+++ b/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java
@@ -42,7 +42,7 @@ public class TridentKafkaTest {
     public void setup() {
         broker = new KafkaTestBroker();
         simpleConsumer = TestUtils.getKafkaConsumer(broker);
-        TridentTupleToKafkaMapper mapper = new FieldNameBasedTupleToKafkaMapper("key", "message");
+        TridentTupleToKafkaMapper<Object, Object> mapper = new FieldNameBasedTupleToKafkaMapper<Object, Object>("key", "message");
         KafkaTopicSelector topicSelector = new DefaultTopicSelector(TestUtils.TOPIC);
         state = new TridentKafkaState()
                 .withKafkaTopicSelector(topicSelector)

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java
index 65bf0b4..364da33 100644
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java
+++ b/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java
@@ -124,7 +124,7 @@ public class ZkCoordinatorTest {
     }
 
     private List<List<PartitionManager>> getPartitionManagers(List<ZkCoordinator> coordinatorList) {
-        List<List<PartitionManager>> partitions = new ArrayList();
+        List<List<PartitionManager>> partitions = new ArrayList<>();
         for (ZkCoordinator coordinator : coordinatorList) {
             partitions.add(coordinator.getMyManagedPartitions());
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java
index 180828e..cccbce0 100644
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java
+++ b/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java
@@ -287,7 +287,7 @@ public class KafkaBoltTest {
 
     private Tuple generateTestTuple(Object key, Object message) {
         TopologyBuilder builder = new TopologyBuilder();
-        GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(), new Config(), new HashMap(), new HashMap(), new HashMap(), "") {
+        GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(), new Config(), new HashMap<>(), new HashMap<>(), new HashMap<>(), "") {
             @Override
             public Fields getComponentOutputFields(String componentId, String streamId) {
                 return new Fields("key", "message");
@@ -298,7 +298,7 @@ public class KafkaBoltTest {
 
     private Tuple generateTestTuple(Object message) {
         TopologyBuilder builder = new TopologyBuilder();
-        GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(), new Config(), new HashMap(), new HashMap(), new HashMap(), "") {
+        GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(), new Config(), new HashMap<>(), new HashMap<>(), new HashMap<>(), "") {
             @Override
             public Fields getComponentOutputFields(String componentId, String streamId) {
                 return new Fields("message");

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/storm-core/src/jvm/org/apache/storm/tuple/Fields.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/tuple/Fields.java b/storm-core/src/jvm/org/apache/storm/tuple/Fields.java
index 840b2d3..a771748 100644
--- a/storm-core/src/jvm/org/apache/storm/tuple/Fields.java
+++ b/storm-core/src/jvm/org/apache/storm/tuple/Fields.java
@@ -29,6 +29,7 @@ import java.io.Serializable;
  * Collection of unique named fields using in an ITuple
  */
 public class Fields implements Iterable<String>, Serializable {
+    private static final long serialVersionUID = -3377931843059975424L;
     private List<String> _fields;
     private Map<String, Integer> _index = new HashMap<>();
     
@@ -122,5 +123,20 @@ public class Fields implements Iterable<String>, Serializable {
     @Override
     public String toString() {
         return _fields.toString();
-    }    
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (this == other) return true;
+        if (other instanceof Fields) {
+            Fields of = (Fields)other;
+            return _fields.equals(of._fields);
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        return _fields.hashCode();
+    }
 }


[5/7] storm git commit: 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 partit

Posted by bo...@apache.org.
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


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/c9f9348e
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/c9f9348e
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/c9f9348e

Branch: refs/heads/master
Commit: c9f9348e3de0917020a8c903f4c9d026b7dc6204
Parents: 74a77aa
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Tue Nov 29 21:39:26 2016 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Mon Jan 30 14:42:10 2017 -0600

----------------------------------------------------------------------
 docs/storm-kafka-client.md                      | 293 ++++++++---
 .../TridentKafkaClientWordCountNamedTopics.java |  69 +--
 ...identKafkaClientWordCountWildcardTopics.java |  31 +-
 external/storm-kafka-client/README.md           | 192 +------
 .../org/apache/storm/kafka/bolt/KafkaBolt.java  | 223 +++++++++
 .../FieldNameBasedTupleToKafkaMapper.java       |  48 ++
 .../kafka/bolt/mapper/TupleToKafkaMapper.java   |  32 ++
 .../bolt/selector/DefaultTopicSelector.java     |  34 ++
 .../bolt/selector/FieldIndexTopicSelector.java  |  52 ++
 .../bolt/selector/FieldNameTopicSelector.java   |  49 ++
 .../kafka/bolt/selector/KafkaTopicSelector.java |  26 +
 .../kafka/spout/ByTopicRecordTranslator.java    | 146 ++++++
 .../kafka/spout/DefaultRecordTranslator.java    |  42 ++
 .../java/org/apache/storm/kafka/spout/Func.java |  26 +
 .../apache/storm/kafka/spout/KafkaSpout.java    | 182 ++-----
 .../storm/kafka/spout/KafkaSpoutConfig.java     | 498 +++++++++++++------
 .../storm/kafka/spout/KafkaSpoutMessageId.java  |   2 +-
 .../storm/kafka/spout/KafkaSpoutStream.java     | 121 -----
 .../storm/kafka/spout/KafkaSpoutStreams.java    |  38 --
 .../spout/KafkaSpoutStreamsNamedTopics.java     | 165 ------
 .../spout/KafkaSpoutStreamsWildcardTopics.java  |  67 ---
 .../kafka/spout/KafkaSpoutTupleBuilder.java     |  58 ---
 .../kafka/spout/KafkaSpoutTuplesBuilder.java    |  32 --
 .../KafkaSpoutTuplesBuilderNamedTopics.java     |  78 ---
 .../KafkaSpoutTuplesBuilderWildcardTopics.java  |  36 --
 .../apache/storm/kafka/spout/KafkaTuple.java    |  47 ++
 .../spout/ManualPartitionNamedSubscription.java |  78 +++
 .../ManualPartitionPatternSubscription.java     |  76 +++
 .../storm/kafka/spout/ManualPartitioner.java    |  40 ++
 .../storm/kafka/spout/NamedSubscription.java    |  61 +++
 .../storm/kafka/spout/PatternSubscription.java  |  54 ++
 .../storm/kafka/spout/RecordTranslator.java     |  55 ++
 .../spout/RoundRobinManualPartitioner.java      |  50 ++
 .../kafka/spout/SimpleRecordTranslator.java     |  58 +++
 .../apache/storm/kafka/spout/Subscription.java  |  53 ++
 .../kafka/spout/TopicPartitionComparator.java   |  49 ++
 .../storm/kafka/spout/internal/Timer.java       |  74 +++
 .../spout/trident/KafkaTridentSpoutEmitter.java |  60 ++-
 .../spout/trident/KafkaTridentSpoutManager.java |  58 +--
 .../spout/trident/KafkaTridentSpoutOpaque.java  |  12 +-
 .../storm/kafka/trident/TridentKafkaState.java  | 115 +++++
 .../kafka/trident/TridentKafkaStateFactory.java |  63 +++
 .../FieldNameBasedTupleToKafkaMapper.java       |  41 ++
 .../mapper/TridentTupleToKafkaMapper.java       |  28 ++
 .../trident/selector/DefaultTopicSelector.java  |  34 ++
 .../trident/selector/KafkaTopicSelector.java    |  26 +
 .../apache/storm/kafka/bolt/KafkaBoltTest.java  |  91 ++++
 .../spout/ByTopicRecordTranslatorTest.java      |  72 +++
 .../spout/DefaultRecordTranslatorTest.java      |  37 ++
 .../storm/kafka/spout/KafkaSpoutConfigTest.java |  40 ++
 .../kafka/spout/KafkaSpoutRebalanceTest.java    |  35 +-
 .../spout/KafkaSpoutStreamsNamedTopicsTest.java |  38 --
 .../kafka/spout/SingleTopicKafkaSpoutTest.java  |   2 +-
 .../SingleTopicKafkaSpoutConfiguration.java     |  61 +--
 .../builders/TopicKeyValueTupleBuilder.java     |  40 --
 .../test/KafkaSpoutTopologyMainNamedTopics.java |  70 +--
 .../KafkaSpoutTopologyMainWildcardTopics.java   |  40 +-
 .../spout/test/TopicTest2TupleBuilder.java      |  40 --
 .../test/TopicsTest0Test1TupleBuilder.java      |  42 --
 .../kafka/DynamicPartitionConnections.java      |   2 +-
 .../jvm/org/apache/storm/kafka/KafkaSpout.java  |   2 +-
 .../jvm/org/apache/storm/kafka/KafkaUtils.java  |   4 +-
 .../apache/storm/kafka/PartitionManager.java    |   2 +-
 .../apache/storm/kafka/StaticCoordinator.java   |   4 +-
 .../org/apache/storm/kafka/bolt/KafkaBolt.java  |   2 +
 .../storm/kafka/DynamicBrokersReaderTest.java   |   6 +-
 .../apache/storm/kafka/TridentKafkaTest.java    |   2 +-
 .../apache/storm/kafka/ZkCoordinatorTest.java   |   2 +-
 .../apache/storm/kafka/bolt/KafkaBoltTest.java  |   4 +-
 .../src/jvm/org/apache/storm/tuple/Fields.java  |  18 +-
 70 files changed, 2781 insertions(+), 1547 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/docs/storm-kafka-client.md
----------------------------------------------------------------------
diff --git a/docs/storm-kafka-client.md b/docs/storm-kafka-client.md
index c8e038f..ec5056f 100644
--- a/docs/storm-kafka-client.md
+++ b/docs/storm-kafka-client.md
@@ -1,90 +1,254 @@
-#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`, which specifies
+that 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 guarantee type safety through the use
+of Java generics.  The defaults are `StringDeserializer`s and can be overwritten by calling `setKeyDeserializer` and/or `setValueDeserializer`.
+If these are set to null the code will fall back to what is set in the kafka properties, but it is preferable to be explicit here, again to maintain 
+type safety with the generics.
+
+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 Kafka Consumer Record into a Tuple, and which stream that tuple will be published into.
+By default the "topic", "partition", "offset", "key", and "value" will be emitted to the "default" stream.  If you want to output entries to different
+streams based on the topic, storm provides `ByTopicRecordTranslator`.  See below for more examples on how to use these.
+
+`setProp` can be used to set kafka properties that do not have a convenience method.
+
+`setGroupId` lets you set the id of the kafka consumer group property "group.id'
+
+`setSSLKeystore` and `setSSLTruststore` allow you to configure SSL authentication.
+
+### Usage Examples
+
+#### Create a Simple Insecure Spout
+The following will consume all events published to "topic" and send them to MyBolt with the fields "topic", "partition", "offset", "key", "value".
 ```java
-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 brokers list and match the pattern.  So in the following example
+"topic", "topic_foo" and "topic_bar" will all match the pattern "topic.*", but "not_my_topic" would not match. 
+```java
+
+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, but consumed by the spout 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, "topic_1", "topic_2", "topic_3").build()), 1);
+tp.setBolt("bolt", new myBolt()).shuffleGrouping("kafka_spout", "STREAM_1");
+tp.setBolt("another", new myOtherBolt()).shuffleGrouping("kafka_spout", "STREAM_2");
+...
+
+```
+
+#### 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 identical for each
+output topic it will throw an exception and not continue.
+
+### Custom RecordTranslators (ADVANCED)
+
+In most cases the built in SimpleRecordTranslator and ByTopicRecordTranslator should cover your use case.  If you do run into a situation where you need a custom one
+then this documentation will describe how to do this properly, and some of the less than obvious classes involved.
+
+The point of apply is to take a ConsumerRecord and turn it into a `List<Object>` that can be emitted.  What is not obvious is how to tell the spout to emit it to a
+specific stream.  To do this you will need to return an instance of `org.apache.storm.kafka.spout.KafkaTuple`.  This provides a method `routedTo` that will say which
+specific stream the tuple should go to.
+
+For Example:
+```java
+return new KafkaTuple(1, 2, 3, 4).routedTo("bar");
+```
+
+Will cause the tuple to be emitted on the "bar" stream.
+
+Be careful when writing custom record translators because just like in a storm spout it needs to be self consistent.  The `streams` method should return
+a full set of streams that this translator will ever try to emit on.  Additionally `getFieldsFor` should return a valid Fields object for each of those
+streams.  If you are doing this for Trident a value must be in the List returned by apply for every field in the Fields object for that stream,
+otherwise trident can throw exceptions.
+
+
+### Manual Partition Control (ADVANCED)
+
+By default Kafka will automatically assign partitions to the current set of spouts.  It handles lots of things, but in some cases you may want to manually assign the partitions.
+This can cause less churn in the assignments when spouts go down and come back up, but it can result in a lot of issues if not done right.  This can all be handled by subclassing
+Subscription and we have a few implementations that you can look at for examples on how to do this.  ManualPartitionNamedSubscription and ManualPartitionPatternSubscription.  Again
+please be careful when using these or implementing your own.
 
 ## Use the Maven Shade Plugin to Build the Uber Jar
 
@@ -112,7 +276,7 @@ Add the following to `REPO_HOME/storm/external/storm-kafka-client/pom.xml`
 </plugin>
 ```
 
-create the uber jar by running the commmand:
+create the uber jar by running the command:
 
 `mvn package -f REPO_HOME/storm/external/storm-kafka-client/pom.xml`
 
@@ -122,11 +286,11 @@ This will create the uber jar file with the name and location matching the follo
 
 ### Run Storm Topology
 
-Copy the file `REPO_HOME/storm/external/storm-kafka-client/target/storm-kafka-client-1.0.x.jar` to `STORM_HOME/extlib`
+Copy the file `REPO_HOME/storm/external/storm-kafka-client/target/storm-kafka-client-*.jar` to `STORM_HOME/extlib`
 
 Using the Kafka command line tools create three topics [test, test1, test2] and use the Kafka console producer to populate the topics with some data 
 
-Execute the command `STORM_HOME/bin/storm jar REPO_HOME/storm/external/storm/target/storm-kafka-client-1.0.x.jar org.apache.storm.kafka.spout.test.KafkaSpoutTopologyMain`
+Execute the command `STORM_HOME/bin/storm jar REPO_HOME/storm/external/storm/target/storm-kafka-client-*.jar org.apache.storm.kafka.spout.test.KafkaSpoutTopologyMain`
 
 With the debug level logs enabled it is possible to see the messages of each topic being redirected to the appropriate Bolt as defined 
 by the streams defined and choice of shuffle grouping.   
@@ -181,8 +345,3 @@ Currently the Kafka spout has has the following default values, which have shown
 * 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.

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java b/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java
index 2482eae..7d0785e 100644
--- a/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java
+++ b/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java
@@ -18,7 +18,11 @@
 
 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;
@@ -26,79 +30,36 @@ 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(KAFKA_LOCAL_BROKER, TOPIC_1, TOPIC_2)
+                .setGroupId("kafkaSpoutTestGroup")
+                .setMaxPartitionFectchBytes(200)
+                .setRecordTranslator((r) -> new Values(r.value()), new Fields("str"))
+                .setRetry(newRetryService())
                 .setOffsetCommitPeriodMs(10_000)
                 .setFirstPollOffsetStrategy(EARLIEST)
                 .setMaxUncommittedOffsets(250)
                 .build();
     }
 
-    protected Map<String,Object> newKafkaConsumerProps() {
-        Map<String, Object> props = new HashMap<>();
-        props.put(KafkaSpoutConfig.Consumer.BOOTSTRAP_SERVERS, "127.0.0.1:9092");
-        props.put(KafkaSpoutConfig.Consumer.GROUP_ID, "kafkaSpoutTestGroup");
-        props.put(KafkaSpoutConfig.Consumer.KEY_DESERIALIZER, "org.apache.kafka.common.serialization.StringDeserializer");
-        props.put(KafkaSpoutConfig.Consumer.VALUE_DESERIALIZER, "org.apache.kafka.common.serialization.StringDeserializer");
-        props.put("max.partition.fetch.bytes", 200);
-        return props;
-    }
-
-    protected KafkaSpoutTuplesBuilder<String, String> newTuplesBuilder() {
-        return new KafkaSpoutTuplesBuilderNamedTopics.Builder<>(
-                new TopicsTupleBuilder<String, String>(TOPIC_1, TOPIC_2))
-                .build();
-    }
-
     protected KafkaSpoutRetryService newRetryService() {
-        return new KafkaSpoutRetryExponentialBackoff(new KafkaSpoutRetryExponentialBackoff.TimeInterval(500L, TimeUnit.MICROSECONDS),
-                KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(2),
-                Integer.MAX_VALUE, KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(10));
-    }
-
-    protected KafkaSpoutStreams newKafkaSpoutStreams() {
-        return new KafkaSpoutStreamsNamedTopics.Builder(new Fields("str"), new String[]{"test-trident","test-trident-1"}).build();
-    }
-
-    protected static class TopicsTupleBuilder<K, V> extends KafkaSpoutTupleBuilder<K,V> {
-        public TopicsTupleBuilder(String... topics) {
-            super(topics);
-        }
-        @Override
-        public List<Object> buildTuple(ConsumerRecord<K, V> consumerRecord) {
-            return new Values(consumerRecord.value());
-        }
+        return new KafkaSpoutRetryExponentialBackoff(new TimeInterval(500L, TimeUnit.MICROSECONDS),
+                TimeInterval.milliSeconds(2), Integer.MAX_VALUE, TimeInterval.seconds(10));
     }
 
     public static void main(String[] args) throws Exception {

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountWildcardTopics.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountWildcardTopics.java b/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountWildcardTopics.java
index e33f19d..62ccef0 100644
--- a/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountWildcardTopics.java
+++ b/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountWildcardTopics.java
@@ -18,26 +18,27 @@
 
 package org.apache.storm.kafka.trident;
 
-import org.apache.storm.kafka.spout.KafkaSpoutStream;
-import org.apache.storm.kafka.spout.KafkaSpoutStreams;
-import org.apache.storm.kafka.spout.KafkaSpoutStreamsWildcardTopics;
-import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilder;
-import org.apache.storm.kafka.spout.KafkaSpoutTuplesBuilderWildcardTopics;
-import org.apache.storm.tuple.Fields;
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
 
 import java.util.regex.Pattern;
 
-public class TridentKafkaClientWordCountWildcardTopics extends TridentKafkaClientWordCountNamedTopics {
-    private static final String TOPIC_WILDCARD_PATTERN = "test-trident(-1)?";
+import org.apache.storm.kafka.spout.KafkaSpoutConfig;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
 
-    protected KafkaSpoutTuplesBuilder<String, String> newTuplesBuilder() {
-        return new KafkaSpoutTuplesBuilderWildcardTopics<>(new TopicsTupleBuilder<>(TOPIC_WILDCARD_PATTERN));
-    }
+public class TridentKafkaClientWordCountWildcardTopics extends TridentKafkaClientWordCountNamedTopics {
+    private static final Pattern TOPIC_WILDCARD_PATTERN = Pattern.compile("test-trident(-1)?");
 
-    protected KafkaSpoutStreams newKafkaSpoutStreams() {
-        final Fields outputFields = new Fields("str");
-        final KafkaSpoutStream kafkaSpoutStream = new KafkaSpoutStream(outputFields, Pattern.compile(TOPIC_WILDCARD_PATTERN));
-        return new KafkaSpoutStreamsWildcardTopics(kafkaSpoutStream);
+    protected KafkaSpoutConfig<String,String> newKafkaSpoutConfig() {
+        return KafkaSpoutConfig.builder("127.0.0.1:9092", TOPIC_WILDCARD_PATTERN)
+                .setGroupId("kafkaSpoutTestGroup")
+                .setMaxPartitionFectchBytes(200)
+                .setRecordTranslator((r) -> new Values(r.value()), new Fields("str"))
+                .setRetry(newRetryService())
+                .setOffsetCommitPeriodMs(10_000)
+                .setFirstPollOffsetStrategy(EARLIEST)
+                .setMaxUncommittedOffsets(250)
+                .build();
     }
 
     public static void main(String[] args) throws Exception {

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/README.md
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/README.md b/external/storm-kafka-client/README.md
index 50fe871..465e466 100644
--- a/external/storm-kafka-client/README.md
+++ b/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 includes the new Apache Kafka consumer API)
 
-Apache Storm Spout implementation to consume data from Apache Kafka versions 0.10 onwards (please see [Apache Kafka Version Compatibility] (#compatibility)). 
+Spouts and Bolts that write to and read from Kafka through the kafka-client library.
 
-The Apache Storm Spout allows clients to consume data from Kafka starting at offsets as defined by the offset strategy specified in `FirstPollOffsetStrategy`. 
-In case of failure, the Kafka Spout will re-start consuming messages from the offset that matches the chosen `FirstPollOffsetStrategy`.
-
-The Kafka Spout implementation allows you to specify the stream (`KafkaSpoutStream`) associated with each topic or topic wildcard. `KafkaSpoutStream` represents the stream and output fields. For named topics use `KafkaSpoutStreamsNamedTopics`, and for topic wildcards use `KafkaSpoutStreamsWildcardTopics`. 
-
-The `KafkaSpoutTuplesBuilder` wraps all the logic that builds `Tuple`s from `ConsumerRecord`s. The logic is provided by the user through implementing the appropriate number of `KafkaSpoutTupleBuilder` instances. For named topics use `KafkaSpoutTuplesBuilderNamedTopics`, and for topic wildcard use `KafkaSpoutTuplesBuilderWildcardTopics`.
-
-Multiple topics and topic wildcards can use the same `KafkaSpoutTupleBuilder` implementation, as long as the logic to build `Tuple`s from `ConsumerRecord`s is identical.
-
-
-# Usage Examples
-
-### Create a Kafka Spout
-
-The code snippet bellow is extracted from the example in the module [test] (https://github.com/apache/storm/tree/master/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test). The code that is common for named topics and topic wildcards is in the first box. The specific implementations are in the appropriate section. 
-
-These snippets serve as a reference and do not compile. If you would like to reuse this code in your implementation, please obtain it from the test module, where it is complete.
-
-```java
-KafkaSpout<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) for details on how to use it.

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java
new file mode 100644
index 0000000..84d3334
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/KafkaBolt.java
@@ -0,0 +1,223 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.bolt;
+
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichBolt;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.utils.TupleUtils;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.clients.producer.Callback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.storm.kafka.bolt.mapper.FieldNameBasedTupleToKafkaMapper;
+import org.apache.storm.kafka.bolt.mapper.TupleToKafkaMapper;
+import org.apache.storm.kafka.bolt.selector.DefaultTopicSelector;
+import org.apache.storm.kafka.bolt.selector.KafkaTopicSelector;
+import java.util.concurrent.Future;
+import java.util.concurrent.ExecutionException;
+import java.util.Map;
+import java.util.Properties;
+
+
+/**
+ * Bolt implementation that can send Tuple data to Kafka.
+ * <p/>
+ * Most configuration for this bolt should be through the various 
+ * setter methods in the bolt.
+ * For backwards compatibility it supports the producer
+ * configuration and topic to be placed in the storm config under
+ * <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 boltSpecifiedProperties = new Properties();
+    /**
+     * {@see KafkaBolt#setFireAndForget(boolean)} for more details on this. 
+     */
+    private boolean fireAndForget = false;
+    /**
+     * {@see KafkaBolt#setAsync(boolean)} for more details on this. 
+     */
+    private boolean async = true;
+
+    public KafkaBolt() {}
+
+    public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
+        this.mapper = mapper;
+        return this;
+    }
+
+    /**
+     * Set the messages to be published to a single topic
+     * @param topic the topic to publish to
+     * @return this
+     */
+    public KafkaBolt<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.boltSpecifiedProperties = producerProperties;
+        return this;
+    }
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        LOG.info("Preparing bolt with configuration {}", this);
+        //for backward compatibility.
+        if (mapper == null) {
+            LOG.info("Mapper not specified. Setting default mapper to {}", FieldNameBasedTupleToKafkaMapper.class.getSimpleName());
+            this.mapper = new FieldNameBasedTupleToKafkaMapper<K,V>();
+        }
+
+        //for backward compatibility.
+        if (topicSelector == null) {
+            if (stormConf.containsKey(TOPIC)) {
+                LOG.info("TopicSelector not specified. Using [{}] for topic [{}] specified in bolt configuration,",
+                        DefaultTopicSelector.class.getSimpleName(), stormConf.get(TOPIC));
+                this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
+            } else {
+                throw new IllegalStateException("topic should be specified in bolt's configuration");
+            }
+        }
+
+        producer = mkProducer(boltSpecifiedProperties);
+        this.collector = collector;
+    }
+    
+    /**
+     * Intended to be overridden for tests.  Make the producer with the given props
+     */
+    protected KafkaProducer<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 bolt will assume that sending a message to kafka will succeed and will ack
+     * the tuple as soon as it has handed the message off to the producer API
+     * if false (the default) the message will be acked after it was successfully sent to kafka or
+     * failed if it was not successfully sent.
+     * @param fireAndForget
+     */
+    public void setFireAndForget(boolean fireAndForget) {
+        this.fireAndForget = fireAndForget;
+    }
+
+    /**
+     * If set to true(the default) the bolt will not wait for the message
+     * to be fully sent to Kafka before getting another tuple to send.
+     * @param async true to have multiple tuples in flight to kafka, else false.
+     */
+    public void setAsync(boolean async) {
+        this.async = async;
+    }
+    
+    @Override
+    public String toString() {
+        return "KafkaBolt: {mapper: " + mapper +
+                " topicSelector: " + topicSelector +
+                " fireAndForget: " + fireAndForget +
+                " async: " + async +
+                " proerties: " + boltSpecifiedProperties;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/FieldNameBasedTupleToKafkaMapper.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/FieldNameBasedTupleToKafkaMapper.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/FieldNameBasedTupleToKafkaMapper.java
new file mode 100644
index 0000000..f7638aa
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/FieldNameBasedTupleToKafkaMapper.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.bolt.mapper;
+
+import org.apache.storm.tuple.Tuple;
+
+public class FieldNameBasedTupleToKafkaMapper<K,V> implements TupleToKafkaMapper<K, V> {
+    private static final long serialVersionUID = -8794262989021702349L;
+    public static final String BOLT_KEY = "key";
+    public static final String BOLT_MESSAGE = "message";
+    public String boltKeyField;
+    public String boltMessageField;
+
+    public FieldNameBasedTupleToKafkaMapper() {
+        this(BOLT_KEY, BOLT_MESSAGE);
+    }
+
+    public FieldNameBasedTupleToKafkaMapper(String boltKeyField, String boltMessageField) {
+        this.boltKeyField = boltKeyField;
+        this.boltMessageField = boltMessageField;
+    }
+
+    @Override
+    public K getKeyFromTuple(Tuple tuple) {
+        //for backward compatibility, we return null when key is not present.
+        return tuple.contains(boltKeyField) ? (K) tuple.getValueByField(boltKeyField) : null;
+    }
+
+    @Override
+    public V getMessageFromTuple(Tuple tuple) {
+        return (V) tuple.getValueByField(boltMessageField);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/TupleToKafkaMapper.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/TupleToKafkaMapper.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/TupleToKafkaMapper.java
new file mode 100644
index 0000000..9f11fc9
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/mapper/TupleToKafkaMapper.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.bolt.mapper;
+
+import org.apache.storm.tuple.Tuple;
+
+import java.io.Serializable;
+
+/**
+ * Interface defining a mapping from storm tuple to kafka key and message.
+ * @param <K> type of key.
+ * @param <V> type of value.
+ */
+public interface TupleToKafkaMapper<K,V> extends Serializable {
+    K getKeyFromTuple(Tuple tuple);
+    V getMessageFromTuple(Tuple tuple);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/DefaultTopicSelector.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/DefaultTopicSelector.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/DefaultTopicSelector.java
new file mode 100644
index 0000000..3d00fc1
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/DefaultTopicSelector.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.bolt.selector;
+
+import org.apache.storm.tuple.Tuple;
+
+public class DefaultTopicSelector implements KafkaTopicSelector {
+    private static final long serialVersionUID = 4601118062437851265L;
+    private final String topicName;
+
+    public DefaultTopicSelector(final String topicName) {
+        this.topicName = topicName;
+    }
+
+    @Override
+    public String getTopic(Tuple tuple) {
+        return topicName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java
new file mode 100644
index 0000000..ffe0b35a
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.bolt.selector;
+
+import org.apache.storm.tuple.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Uses field with a given index to select the topic name from a tuple .
+ */
+public class FieldIndexTopicSelector implements KafkaTopicSelector {
+    private static final long serialVersionUID = -3830575380208166367L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(FieldIndexTopicSelector.class);
+
+    private final int fieldIndex;
+    private final String defaultTopicName;
+
+    public FieldIndexTopicSelector(int fieldIndex, String defaultTopicName) {
+        this.fieldIndex = fieldIndex;
+        if (fieldIndex < 0) {
+            throw new IllegalArgumentException("fieldIndex cannot be negative");
+        }
+        this.defaultTopicName = defaultTopicName;
+    }
+
+    @Override
+    public String getTopic(Tuple tuple) {
+        if (fieldIndex < tuple.size()) {
+            return tuple.getString(fieldIndex);
+        } else {
+            LOG.warn("Field index {} is out of bounds. Using default topic {}", fieldIndex, defaultTopicName);
+            return defaultTopicName;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java
new file mode 100644
index 0000000..e90b26f
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.bolt.selector;
+
+import org.apache.storm.tuple.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Uses field name to select topic name from tuple .
+ */
+public class FieldNameTopicSelector implements KafkaTopicSelector {
+    private static final long serialVersionUID = -3903708904533396833L;
+    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
+
+    private final String fieldName;
+    private final String defaultTopicName;
+
+
+    public FieldNameTopicSelector(String fieldName, String defaultTopicName) {
+        this.fieldName = fieldName;
+        this.defaultTopicName = defaultTopicName;
+    }
+
+    @Override
+    public String getTopic(Tuple tuple) {
+        if (tuple.contains(fieldName)) {
+            return tuple.getStringByField(fieldName);
+        } else {
+            LOG.warn("Field {} Not Found. Returning default topic {}", fieldName, defaultTopicName);
+            return defaultTopicName;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/KafkaTopicSelector.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/KafkaTopicSelector.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/KafkaTopicSelector.java
new file mode 100644
index 0000000..cb7fb44
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/bolt/selector/KafkaTopicSelector.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.kafka.bolt.selector;
+
+import org.apache.storm.tuple.Tuple;
+
+import java.io.Serializable;
+
+public interface KafkaTopicSelector extends Serializable {
+    String getTopic(Tuple tuple);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ByTopicRecordTranslator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ByTopicRecordTranslator.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ByTopicRecordTranslator.java
new file mode 100644
index 0000000..855ab30
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ByTopicRecordTranslator.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+package org.apache.storm.kafka.spout;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.storm.tuple.Fields;
+
+/**
+ * Based off of a given Kafka topic a ConsumerRecord came from it will be translated to a Storm tuple
+ * and emitted to a given stream
+ * @param <K> the key of the incoming Records
+ * @param <V> the value of the incoming Records
+ */
+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<>();
+    
+    /**
+     * Create a simple record translator that will use func to extract the fields of the tuple,
+     * named by fields, and emit them to stream. This will handle all topics not explicitly set
+     * elsewhere.
+     * @param func extracts and turns them into a list of objects to be emitted
+     * @param fields the names of the fields extracted
+     * @param stream the stream to emit these fields on.
+     */
+    public ByTopicRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
+        this(new SimpleRecordTranslator<>(func, fields, stream));
+    }
+    
+    /**
+     * Create a simple record translator that will use func to extract the fields of the tuple,
+     * named by fields, and emit them to the default stream. This will handle all topics not explicitly set
+     * elsewhere.
+     * @param func extracts and turns them into a list of objects to be emitted
+     * @param fields the names of the fields extracted
+     */
+    public ByTopicRecordTranslator(Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
+        this(new SimpleRecordTranslator<>(func, fields));
+    }
+    
+    /**
+     * @param defaultTranslator a translator that will be used for all topics not explicitly set
+     * elsewhere.
+     */
+    public ByTopicRecordTranslator(RecordTranslator<K,V> defaultTranslator) {
+        this.defaultTranslator = defaultTranslator;
+        //This shouldn't throw on a Check, because nothing is configured yet
+        cacheNCheckFields(defaultTranslator);
+    }
+    
+    /**
+     * Configure a translator for a given topic with tuples to be emitted to the default stream.
+     * @param topic the topic this should be used for
+     * @param func extracts and turns them into a list of objects to be emitted
+     * @param fields the names of the fields extracted
+     * @return this to be able to chain configuration
+     * @throws IllegalStateException if the topic is already registered to another translator
+     * @throws IllegalArgumentException if the Fields for the stream this emits to do not match any already configured Fields for the same stream
+     */
+    public ByTopicRecordTranslator<K, V> forTopic(String topic, Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields) {
+        return forTopic(topic, new SimpleRecordTranslator<>(func, fields));
+    }
+    
+    /**
+     * Configure a translator for a given topic.
+     * @param topic the topic this should be used for
+     * @param func extracts and turns them into a list of objects to be emitted
+     * @param fields the names of the fields extracted
+     * @param stream the stream to emit the tuples to.
+     * @return this to be able to chain configuration
+     * @throws IllegalStateException if the topic is already registered to another translator
+     * @throws IllegalArgumentException if the Fields for the stream this emits to do not match any already configured Fields for the same stream
+     */
+    public ByTopicRecordTranslator<K, V> forTopic(String topic, Func<ConsumerRecord<K, V>, List<Object>> func, Fields fields, String stream) {
+        return forTopic(topic, new SimpleRecordTranslator<>(func, fields, stream));
+    }
+    
+    /**
+     * Configure a translator for a given kafka topic
+     * @param topic the topic this translator should handle
+     * @param translator the translator itself
+     * @return this to be able to chain configuration
+     * @throws IllegalStateException if the topic is already registered to another translator
+     * @throws IllegalArgumentException if the Fields for the stream this emits to do not match any already configured Fields for the same stream
+     */
+    public ByTopicRecordTranslator<K, V> forTopic(String topic, RecordTranslator<K,V> translator) {
+        if (topicToTranslator.containsKey(topic)) {
+            throw new IllegalStateException("Topic " + topic + " is already registered");
+        }
+        cacheNCheckFields(translator);
+        topicToTranslator.put(topic, translator);
+        return this;
+    }
+    
+    private void cacheNCheckFields(RecordTranslator<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);
+            }
+            
+            if (cached == null) {
+                streamToFields.put(stream, fromTrans);
+            }
+        }
+    }
+
+    @Override
+    public List<Object> apply(ConsumerRecord<K, V> record) {
+        RecordTranslator<K, V> trans = topicToTranslator.getOrDefault(record.topic(), defaultTranslator);
+        return trans.apply(record);
+    }
+
+    @Override
+    public Fields getFieldsFor(String stream) {
+        return streamToFields.get(stream);
+    }
+    
+    @Override
+    public List<String> streams() {
+        return new ArrayList<>(streamToFields.keySet());
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/DefaultRecordTranslator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/DefaultRecordTranslator.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/DefaultRecordTranslator.java
new file mode 100644
index 0000000..f40ab18
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/DefaultRecordTranslator.java
@@ -0,0 +1,42 @@
+/*
+ * 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.List;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+
+public class DefaultRecordTranslator<K, V> implements RecordTranslator<K, V> {
+    private static final long serialVersionUID = -5782462870112305750L;
+    public static final Fields FIELDS = new Fields("topic", "partition", "offset", "key", "value");
+    @Override
+    public List<Object> apply(ConsumerRecord<K, V> record) {
+        return new Values(record.topic(),
+                record.partition(),
+                record.offset(),
+                record.key(),
+                record.value());
+    }
+
+    @Override
+    public Fields getFieldsFor(String stream) {
+        return FIELDS;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/c9f9348e/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Func.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Func.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Func.java
new file mode 100644
index 0000000..a631d96
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/Func.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.spout;
+
+/**
+ * A simple interface to allow compatibility with non java 8
+ * code bases 
+ */
+public interface Func<V, R> {
+    R apply(V record);
+}